public <T extends ColumnFamily> T cloneMeShallow(ColumnFamily.Factory<T> factory, boolean reversedInsertOrder) { T cf = factory.create(metadata, reversedInsertOrder); cf.delete(this); return cf; }
/** * If `columns` has any tombstones (top-level or range tombstones), they will be applied to this set of columns. */ public void delete(ColumnFamily columns) { delete(columns.deletionInfo()); }
public void addAtom(OnDiskAtom atom) { if (atom instanceof Cell) { addColumn((Cell)atom); } else { assert atom instanceof RangeTombstone; delete((RangeTombstone)atom); } }
public void delete(String cfName, long timestamp) { int localDeleteTime = (int) (System.currentTimeMillis() / 1000); addOrGet(cfName).delete(new DeletionInfo(timestamp, localDeleteTime)); }
public static void deserializerSuperColumnFamily(DataInput in, ColumnFamily cf, ColumnSerializer.Flag flag, int version) throws IOException { // Note that there was no way to insert a range tombstone in a SCF in 1.2 cf.delete(cf.getComparator().deletionInfoSerializer().deserialize(in, version)); assert !cf.deletionInfo().rangeIterator().hasNext(); Iterator<OnDiskAtom> iter = onDiskIterator(in, in.readInt(), flag, Integer.MIN_VALUE, cf.getComparator()); while (iter.hasNext()) cf.addAtom(iter.next()); }
this.sstable = sstable; columnFamily = ArrayBackedSortedColumns.factory.create(metadata); columnFamily.delete(deletion); this.atomIterator = atomIterator;
public ColumnFamily trim(ColumnFamily cf, int trimTo, long now) { // each cell can increment the count by at most one, so if we have fewer cells than trimTo, we can skip trimming if (cf.getColumnCount() < trimTo) return cf; ColumnCounter counter = columnCounter(cf.getComparator(), now); ColumnFamily trimmedCf = cf.getFactory().create(cf.metadata(), reversed, trimTo); trimmedCf.delete(cf); Collection<Cell> cells = reversed ? cf.getReverseSortedColumns() : cf.getSortedColumns(); DeletionInfo.InOrderTester tester = cf.deletionInfo().inOrderTester(reversed); for (Iterator<Cell> iter = cells.iterator(); iter.hasNext(); ) { Cell cell = iter.next(); counter.count(cell, tester); if (counter.live() > trimTo) { break; } else { trimmedCf.addColumn(cell); } } return trimmedCf; }
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); }
public ColumnFamily deserialize(DataInput in, ColumnFamily.Factory factory, ColumnSerializer.Flag flag, int version) throws IOException { if (!in.readBoolean()) return null; ColumnFamily cf = factory.create(Schema.instance.getCFMetaData(deserializeCfId(in, version))); if (cf.metadata().isSuper() && version < MessagingService.VERSION_20) { SuperColumns.deserializerSuperColumnFamily(in, cf, flag, version); } else { cf.delete(cf.getComparator().deletionInfoSerializer().deserialize(in, version)); ColumnSerializer columnSerializer = cf.getComparator().columnSerializer(); int size = in.readInt(); for (int i = 0; i < size; ++i) cf.addColumn(columnSerializer.deserialize(in, flag)); } return cf; }
public void addUpdateForKey(ColumnFamily cf, ByteBuffer key, Composite prefix, UpdateParameters params) throws InvalidRequestException { List<Operation> deletions = getOperations(); if (deletions.isEmpty()) { // We delete the slice selected by the prefix. // However, for performance reasons, we distinguish 2 cases: // - It's a full internal row delete // - It's a full cell name (i.e it's a dense layout and the prefix is full) if (prefix.isEmpty()) { // No columns specified, delete the row cf.delete(new DeletionInfo(params.timestamp, params.localDeletionTime)); } else if (cfm.comparator.isDense() && prefix.size() == cfm.clusteringColumns().size()) { cf.addAtom(params.makeTombstone(cfm.comparator.create(prefix, null))); } else { cf.addAtom(params.makeRangeTombstone(prefix.slice())); } } else { for (Operation op : deletions) op.execute(key, cf, prefix, params); } }
public LazilyCompactedRow(CompactionController controller, List<? extends OnDiskAtomIterator> rows) { super(rows.get(0).getKey()); this.rows = rows; this.controller = controller; indexer = controller.cfs.indexManager.gcUpdaterFor(key); // Combine top-level tombstones, keeping the one with the highest markedForDeleteAt timestamp. This may be // purged (depending on gcBefore), but we need to remember it to properly delete columns during the merge maxRowTombstone = DeletionTime.LIVE; for (OnDiskAtomIterator row : rows) { DeletionTime rowTombstone = row.getColumnFamily().deletionInfo().getTopLevelDeletion(); if (maxRowTombstone.compareTo(rowTombstone) < 0) maxRowTombstone = rowTombstone; } emptyColumnFamily = ArrayBackedSortedColumns.factory.create(controller.cfs.metadata); emptyColumnFamily.delete(maxRowTombstone); if (!maxRowTombstone.isLive() && maxRowTombstone.markedForDeleteAt < getMaxPurgeableTimestamp()) emptyColumnFamily.purgeTombstones(controller.gcBefore); reducer = new Reducer(); merger = Iterators.filter(MergeIterator.get(rows, emptyColumnFamily.getComparator().onDiskAtomComparator(), reducer), Predicates.notNull()); }
cfDiff.delete(cfComposite.deletionInfo());
public SimpleSliceReader(SSTableReader sstable, RowIndexEntry indexEntry, FileDataInput input, Composite finishColumn) { Tracing.trace("Seeking to partition beginning in data file"); this.finishColumn = finishColumn; this.comparator = sstable.metadata.comparator; try { if (input == null) { this.file = sstable.getFileDataInput(indexEntry.position); this.needsClosing = true; } else { this.file = input; input.seek(indexEntry.position); this.needsClosing = false; } // Skip key and data size ByteBufferUtil.skipShortLength(file); emptyColumnFamily = ArrayBackedSortedColumns.factory.create(sstable.metadata); emptyColumnFamily.delete(DeletionTime.serializer.deserialize(file)); atomIterator = emptyColumnFamily.metadata().getOnDiskIterator(file, sstable.descriptor.version); } catch (IOException e) { sstable.markSuspect(); throw new CorruptSSTableException(e, sstable.getFilename()); } }
if (iter.getColumnFamily() != null) container.delete(iter.getColumnFamily()); sstablesIterated++; while (iter.hasNext())
container.delete(maxRowTombstone); Iterator<Cell> iter = container.iterator(); Cell c = iter.next();
cf.delete(DeletionTime.serializer.deserialize(in));