/** * return a QueryFilter object that includes every column in the row. * This is dangerous on large rows; avoid except for test code. */ public static QueryFilter getIdentityFilter(DecoratedKey key, String cfName, long timestamp) { return new QueryFilter(key, cfName, new IdentityQueryFilter(), timestamp); }
public static void collateOnDiskAtom(ColumnFamily returnCF, List<? extends Iterator<? extends OnDiskAtom>> toCollate, IDiskAtomFilter filter, DecoratedKey key, int gcBefore, long timestamp) { List<Iterator<Cell>> filteredIterators = new ArrayList<>(toCollate.size()); for (Iterator<? extends OnDiskAtom> iter : toCollate) filteredIterators.add(gatherTombstones(returnCF, iter)); collateColumns(returnCF, filteredIterators, filter, key, gcBefore, timestamp); }
public Row getRow(QueryFilter filter) { ColumnFamilyStore cfStore = getColumnFamilyStore(filter.getColumnFamilyName()); ColumnFamily columnFamily = cfStore.getColumnFamily(filter); return new Row(filter.key, columnFamily); }
public static void collateColumns(ColumnFamily returnCF, List<? extends Iterator<Cell>> toCollate, IDiskAtomFilter filter, DecoratedKey key, int gcBefore, long timestamp) { Comparator<Cell> comparator = filter.getColumnComparator(returnCF.getComparator()); Iterator<Cell> reduced = toCollate.size() == 1 ? toCollate.get(0) : MergeIterator.get(toCollate, comparator, getReducer(comparator)); filter.collectReducedColumns(returnCF, reduced, key, gcBefore, timestamp); }
/** * Filter a cached row, which will not be modified by the filter, but may be modified by throwing out * tombstones that are no longer relevant. * The returned column family won't be thread safe. */ ColumnFamily filterColumnFamily(ColumnFamily cached, QueryFilter filter) { if (cached == null) return null; ColumnFamily cf = cached.cloneMeShallow(ArrayBackedSortedColumns.factory, filter.filter.isReversed()); int gcBefore = gcBefore(filter.timestamp); filter.collateOnDiskAtom(cf, filter.getIterator(cached), gcBefore); return removeDeletedCF(cf, gcBefore); }
if (cf != null) filter.delete(container.deletionInfo(), cf); isEmpty = false; Iterator<Cell> iter = filter.getIterator(cf); while (iter.hasNext()) QueryFilter reducedFilter = new QueryFilter(filter.key, filter.cfName, namesFilter.withUpdatedColumns(filterColumns), filter.timestamp); Tracing.trace("Merging data from sstable {}", sstable.descriptor.generation); sstable.incrementReadCount(); OnDiskAtomIterator iter = reducedFilter.getSSTableColumnIterator(sstable); iterators.add(iter); isEmpty = false; filter.collateOnDiskAtom(returnCF, container.iterator(), gcBefore);
if (cf != null) filter.delete(returnDeletionInfo, cf); Iterator<Cell> iter = filter.getIterator(cf); if (copyOnHeap) break; if (!filter.shouldInclude(sstable)) OnDiskAtomIterator iter = filter.getSSTableColumnIterator(sstable); iterators.add(iter); if (iter.getColumnFamily() != null) OnDiskAtomIterator iter = filter.getSSTableColumnIterator(sstable); ColumnFamily cf = iter.getColumnFamily(); filter.collateOnDiskAtom(returnCF, iterators, gcBefore);
((AbstractSimplePerColumnSecondaryIndex)index).expressionString(primary), index.getBaseCfs().metadata.getKeyValidator().getString(startKey.toByteBuffer())); QueryFilter indexFilter = QueryFilter.getSliceFilter(indexKey, index.getIndexCfs().name, lastSeenKey, ColumnFamily data = baseCfs.getColumnFamily(new QueryFilter(dk, baseCfs.name, filter.columnFilter(lastSeenKey.toByteBuffer()), filter.timestamp)); if (extraFilter != null) ColumnFamily cf = baseCfs.getColumnFamily(new QueryFilter(dk, baseCfs.name, extraFilter, filter.timestamp)); if (cf != null) data.addAll(cf);
static ColumnFamily resolveSuperset(Iterable<ColumnFamily> versions, long now) { assert Iterables.size(versions) > 0; ColumnFamily resolved = null; for (ColumnFamily cf : versions) { if (cf == null) continue; if (resolved == null) resolved = cf.cloneMeShallow(); else resolved.delete(cf); } if (resolved == null) return null; // mimic the collectCollatedColumn + removeDeleted path that getColumnFamily takes. // this will handle removing columns and subcolumns that are suppressed by a row or // supercolumn tombstone. QueryFilter filter = new QueryFilter(null, resolved.metadata().cfName, new IdentityQueryFilter(), now); List<CloseableIterator<Cell>> iters = new ArrayList<>(Iterables.size(versions)); for (ColumnFamily version : versions) if (version != null) iters.add(FBUtilities.closeableIterator(version.iterator())); filter.collateColumns(resolved, iters, Integer.MIN_VALUE); return ColumnFamilyStore.removeDeleted(resolved, Integer.MIN_VALUE); }
data = getTopLevelColumns(QueryFilter.getIdentityFilter(filter.key, name, filter.timestamp), Integer.MIN_VALUE); toCache = data; Tracing.trace("Populating row cache with the whole partition"); QueryFilter cacheFilter = new QueryFilter(filter.key, name, cacheSlice, filter.timestamp);
private static Row serializedColumnFamilies(DecoratedKey ksNameKey) { ColumnFamilyStore cfsStore = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF); return new Row(ksNameKey, cfsStore.getColumnFamily(QueryFilter.getIdentityFilter(ksNameKey, SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF, System.currentTimeMillis()))); }
public void collateOnDiskAtom(ColumnFamily returnCF, Iterator<? extends OnDiskAtom> toCollate, int gcBefore) { filter.collectReducedColumns(returnCF, gatherTombstones(returnCF, toCollate), this.key, gcBefore, timestamp); }
public static boolean isIndexBuilt(String keyspaceName, String indexName) { ColumnFamilyStore cfs = Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(INDEX_CF); QueryFilter filter = QueryFilter.getNamesFilter(decorate(ByteBufferUtil.bytes(keyspaceName)), INDEX_CF, FBUtilities.singleton(cfs.getComparator().makeCellName(indexName), cfs.getComparator()), System.currentTimeMillis()); return ColumnFamilyStore.removeDeleted(cfs.getColumnFamily(filter), Integer.MAX_VALUE) != null; }
index.expressionString(primary), indexComparator.getString(startPrefix)); QueryFilter indexFilter = QueryFilter.getSliceFilter(indexKey, index.getIndexCfs().name, lastSeenPrefix, : new ColumnSlice[]{ dataSlice }; SliceQueryFilter dataFilter = new SliceQueryFilter(slices, false, Integer.MAX_VALUE, baseCfs.metadata.clusteringColumns().size()); ColumnFamily newData = baseCfs.getColumnFamily(new QueryFilter(dk, baseCfs.name, dataFilter, filter.timestamp)); if (newData == null || index.isStale(entry, newData, filter.timestamp))
private static Row serializedUserTypes(DecoratedKey ksNameKey) { ColumnFamilyStore cfsStore = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_USER_TYPES_CF); return new Row(ksNameKey, cfsStore.getColumnFamily(QueryFilter.getIdentityFilter(ksNameKey, SystemKeyspace.SCHEMA_USER_TYPES_CF, System.currentTimeMillis()))); }
public ColumnFamily prune(DecoratedKey rowKey, ColumnFamily data) { if (optimizedFilter == null) return data; ColumnFamily pruned = data.cloneMeShallow(); IDiskAtomFilter filter = dataRange.columnFilter(rowKey.getKey()); Iterator<Cell> iter = filter.getColumnIterator(data); filter.collectReducedColumns(pruned, QueryFilter.gatherTombstones(pruned, iter), rowKey, cfs.gcBefore(timestamp), timestamp); return pruned; }
public Pair<CounterCacheKey, ClockAndCount> call() throws Exception { DecoratedKey key = cfs.partitioner.decorateKey(partitionKey); QueryFilter filter = QueryFilter.getNamesFilter(key, cfs.metadata.cfName, FBUtilities.singleton(cellName, cfs.metadata.comparator), Long.MIN_VALUE); ColumnFamily cf = cfs.getTopLevelColumns(filter, Integer.MIN_VALUE); if (cf == null) return null; Cell cell = cf.getColumn(cellName); if (cell == null || !cell.isLive(Long.MIN_VALUE)) return null; ClockAndCount clockAndCount = CounterContext.instance().getLocalClockAndCount(cell.value()); return Pair.create(CounterCacheKey.create(cfs.metadata.ksAndCFName, partitionKey, cellName), clockAndCount); } });
/** * @return a QueryFilter object that will return columns matching the given names * @param key the row to slice * @param cfName column family to query * @param columns the column names to restrict the results to, sorted in comparator order */ public static QueryFilter getNamesFilter(DecoratedKey key, String cfName, SortedSet<CellName> columns, long timestamp) { return new QueryFilter(key, cfName, new NamesQueryFilter(columns), timestamp); }
/** * Fetches a subset of schema (table data, columns metadata or triggers) for the keyspace. * * @param schemaCfName the schema table to get the data from (schema_keyspaces, schema_columnfamilies, schema_columns or schema_triggers) * @param ksName the keyspace of the tables we are interested in * @return a Row containing the schema data of a particular type for the keyspace */ public static Row readSchemaRow(String schemaCfName, String ksName) { DecoratedKey key = StorageService.getPartitioner().decorateKey(getSchemaKSKey(ksName)); ColumnFamilyStore schemaCFS = SystemKeyspace.schemaCFS(schemaCfName); ColumnFamily result = schemaCFS.getColumnFamily(QueryFilter.getIdentityFilter(key, schemaCfName, System.currentTimeMillis())); return new Row(key, result); }
assert name.equals(filter.getColumnFamilyName()) : filter.getColumnFamilyName();