@Override public ByteBuffer getColumn(ColumnOrSuperColumn element) { return element.getColumn().bufferForName(); }
@Override public ByteBuffer getValue(ColumnOrSuperColumn element) { return element.getColumn().bufferForValue(); }
@Override public Object getMetaData(ColumnOrSuperColumn element, EntryMetaData meta) { switch(meta) { case TIMESTAMP: return element.getColumn().getTimestamp(); case TTL: return element.getColumn().getTtl(); default: throw new UnsupportedOperationException("Unsupported meta data: " + meta); } } }
private List<KeySlice> getRangeSlices(org.apache.cassandra.thrift.KeyRange keyRange, @Nullable SliceQuery sliceQuery) throws BackendException { SliceRange sliceRange = new SliceRange(); sliceRange.setStart(ArrayUtils.EMPTY_BYTE_ARRAY) .setFinish(ArrayUtils.EMPTY_BYTE_ARRAY) .setCount(5); } else { sliceRange.setStart(sliceQuery.getSliceStart().asByteBuffer()) .setFinish(sliceQuery.getSliceEnd().asByteBuffer()) .setCount((sliceQuery.hasLimit()) ? sliceQuery.getLimit() : Integer.MAX_VALUE); connection.getClient().get_range_slices(new ColumnParent(columnFamily), new SlicePredicate() .setSlice_range(sliceRange), keyRange, ConsistencyLevel.QUORUM);
client.set_keyspace(keyspaceName); // Don't remove client.set_keyspace(SYSTEM_KS); log.debug("Found existing keyspace {}", keyspaceName); } catch (InvalidRequestException e) { KsDef ksdef = new KsDef().setName(keyspaceName) .setCf_defs(new LinkedList<CfDef>()) // cannot be null but can be empty .setStrategy_class(storageConfig.get(REPLICATION_STRATEGY)) .setStrategy_options(strategyOptions); client.set_keyspace(SYSTEM_KS); try { client.system_add_keyspace(ksdef); retrySetKeyspace(keyspaceName, client); log.debug("Created keyspace {}", keyspaceName); return client.describe_keyspace(keyspaceName); } catch (Exception e) { throw new TemporaryBackendException(e);
IPartitioner partitioner = StorageService.getPartitioner(); SliceRange columnSlice = new SliceRange(); if (sliceQuery == null) { columnSlice.setStart(ArrayUtils.EMPTY_BYTE_ARRAY) .setFinish(ArrayUtils.EMPTY_BYTE_ARRAY) .setCount(5); } else { columnSlice.setStart(sliceQuery.getSliceStart().asByteBuffer()) .setFinish(sliceQuery.getSliceEnd().asByteBuffer()) .setCount(sliceQuery.hasLimit() ? sliceQuery.getLimit() : Integer.MAX_VALUE); SlicePredicate predicate = new SlicePredicate().setSlice_range(columnSlice); IDiskAtomFilter filter = ThriftValidation.asIFilter(predicate, cfm, null);
private void createColumnFamily(Cassandra.Client client, String ksName, String cfName, String comparator) throws BackendException { CfDef createColumnFamily = new CfDef(); createColumnFamily.setName(cfName); createColumnFamily.setKeyspace(ksName); createColumnFamily.setComparator_type(comparator); ImmutableMap.Builder<String, String> compressionOptions = new ImmutableMap.Builder<String, String>(); if (compressionEnabled) { compressionOptions.put("sstable_compression", compressionClass) .put("chunk_length_kb", Integer.toString(compressionChunkSizeKB)); } createColumnFamily.setCompression_options(compressionOptions.build()); // Hard-coded caching settings if (cfName.startsWith(Backend.EDGESTORE_NAME)) { createColumnFamily.setCaching("keys_only"); } else if (cfName.startsWith(Backend.INDEXSTORE_NAME)) { createColumnFamily.setCaching("rows_only"); } log.debug("Adding column family {} to keyspace {}...", cfName, ksName); try { client.system_add_column_family(createColumnFamily); } catch (SchemaDisagreementException e) { throw new TemporaryBackendException("Error in setting up column family", e); } catch (Exception e) { throw new PermanentBackendException(e); } log.debug("Added column family {} to keyspace {}.", cfName, ksName); }
private SliceRange getSliceRange(final SliceQuery slice, final int limit) { final SliceRange sliceRange = new SliceRange(); sliceRange.setStart(slice.getSliceStart().asByteBuffer()); sliceRange.setFinish(slice.getSliceEnd().asByteBuffer()); sliceRange.setCount(Math.min(limit, slice.getLimit())); return sliceRange; } }
private List<KeySlice> getKeySlice(ByteBuffer startKey, ByteBuffer endKey, SliceQuery columnSlice, int count) throws BackendException { return getRangeSlices(new org.apache.cassandra.thrift.KeyRange().setStart_key(startKey).setEnd_key(endKey).setCount(count), columnSlice); }
@Override public Map<String, String> getCompressionOptions(String cf) throws BackendException { CTConnection conn = null; Map<String, String> result = null; try { conn = pool.borrowObject(keySpaceName); Cassandra.Client client = conn.getClient(); KsDef ksDef = client.describe_keyspace(keySpaceName); for (CfDef cfDef : ksDef.getCf_defs()) { if (null != cfDef && cfDef.getName().equals(cf)) { result = cfDef.getCompression_options(); break; } } return result; } catch (InvalidRequestException e) { log.debug("Keyspace {} does not exist", keySpaceName); return null; } catch (Exception e) { throw new TemporaryBackendException(e); } finally { pool.returnObjectUnsafe(keySpaceName, conn); } }
private <T extends Token> List<KeySlice> getTokenSlice(T startToken, T endToken, SliceQuery sliceQuery, int count) throws BackendException { String st = sanitizeBrokenByteToken(startToken); String et = sanitizeBrokenByteToken(endToken); org.apache.cassandra.thrift.KeyRange kr = new org.apache.cassandra.thrift.KeyRange().setStart_token(st).setEnd_token(et).setCount(count); return getRangeSlices(kr, sliceQuery); }
private void ensureColumnFamilyExists(String ksName, String cfName, String comparator) throws BackendException { CTConnection conn = null; try { KsDef keyspaceDef = ensureKeyspaceExists(ksName); conn = pool.borrowObject(ksName); Cassandra.Client client = conn.getClient(); log.debug("Looking up metadata on keyspace {}...", ksName); boolean foundColumnFamily = false; for (CfDef cfDef : keyspaceDef.getCf_defs()) { String curCfName = cfDef.getName(); if (curCfName.equals(cfName)) foundColumnFamily = true; } if (!foundColumnFamily) { createColumnFamily(client, ksName, cfName, comparator); } else { log.debug("Keyspace {} and ColumnFamily {} were found.", ksName, cfName); } } catch (SchemaDisagreementException e) { throw new TemporaryBackendException(e); } catch (Exception e) { throw new PermanentBackendException(e); } finally { pool.returnObjectUnsafe(ksName, conn); } }
@Override public CTConnection makeObject(String key) throws Exception { CTConnection conn = makeRawConnection(); Cassandra.Client client = conn.getClient(); client.set_keyspace(key); return conn; }
@Override public boolean apply(@Nullable KeySlice row) { return (row != null) && row.getColumns().size() > 0; } }
@Override @SuppressWarnings("unchecked") public IPartitioner getCassandraPartitioner() throws BackendException { CTConnection conn = null; try { conn = pool.borrowObject(SYSTEM_KS); return FBUtilities.newPartitioner(conn.getClient().describe_partitioner()); } catch (Exception e) { throw new TemporaryBackendException(e); } finally { pool.returnObjectUnsafe(SYSTEM_KS, conn); } }
private CLevel() { db = org.apache.cassandra.db.ConsistencyLevel.valueOf(toString()); thrift = org.apache.cassandra.thrift.ConsistencyLevel.valueOf(toString()); astyanax = com.netflix.astyanax.model.ConsistencyLevel.valueOf("CL_" + toString()); }
protected List<KeySlice> checkFreshSlices(List<KeySlice> ks) { if (0 == ks.size()) { seenEnd = true; return Collections.emptyList(); } nextStartKey = ks.get(ks.size() - 1).bufferForKey(); nextStartToken = partitioner.getToken(nextStartKey); if (nextStartToken.equals(endToken)) { seenEnd = true; if (omitEndToken) ks.remove(ks.size() - 1); } return ks; }
public void start() { super.start(); startedLatch.countDown(); }
private void retrySetKeyspace(String ksName, Cassandra.Client client) throws BackendException { final long end = System.currentTimeMillis() + (60L * 1000L); while (System.currentTimeMillis() <= end) { try { client.set_keyspace(ksName); return; } catch (Exception e) { log.warn("Exception when changing to keyspace {} after creating it", ksName, e); try { Thread.sleep(1000L); } catch (InterruptedException ie) { throw new PermanentBackendException("Unexpected interrupt (shutting down?)", ie); } } } throw new PermanentBackendException("Could change to keyspace " + ksName + " after creating it"); }
@Override public StaticBuffer next() { ensureOpen(); if (!hasNext()) throw new NoSuchElementException(); mostRecentRow = ksIter.next(); Preconditions.checkNotNull(mostRecentRow); return StaticArrayBuffer.of(mostRecentRow.bufferForKey()); }