public Collection<Mutation> augment(ByteBuffer key, ColumnFamily update) { List<Mutation> mutations = new ArrayList<Mutation>(update.getColumnCount()); for (Cell cell : update) { if (cell.value().remaining() > 0) { Mutation mutation = new Mutation(properties.getProperty("keyspace"), key); mutation.add(properties.getProperty("columnfamily"), cell.name(), cell.value(), System.currentTimeMillis()); mutations.add(mutation); } } return mutations; }
@VisibleForTesting List<Row> discardFirst(List<Row> rows, int toDiscard) { if (toDiscard == 0 || rows.isEmpty()) return rows; int i = 0; DecoratedKey firstKey = null; ColumnFamily firstCf = null; while (toDiscard > 0 && i < rows.size()) { Row first = rows.get(i++); firstKey = first.key; firstCf = first.cf.cloneMeShallow(isReversed()); toDiscard -= isReversed() ? discardLast(first.cf, toDiscard, firstCf) : discardFirst(first.cf, toDiscard, firstCf); } // If there is less live data than to discard, all is discarded if (toDiscard > 0) return Collections.<Row>emptyList(); // i is the index of the first row that we are sure to keep. On top of that, // we also keep firstCf is it hasn't been fully emptied by the last iteration above. int count = firstCf.getColumnCount(); int newSize = rows.size() - (count == 0 ? i : i - 1); List<Row> newRows = new ArrayList<Row>(newSize); if (count != 0) newRows.add(new Row(firstKey, firstCf)); newRows.addAll(rows.subList(i, rows.size())); return newRows; }
@VisibleForTesting List<Row> discardLast(List<Row> rows, int toDiscard) { if (toDiscard == 0 || rows.isEmpty()) return rows; int i = rows.size()-1; DecoratedKey lastKey = null; ColumnFamily lastCf = null; while (toDiscard > 0 && i >= 0) { Row last = rows.get(i--); lastKey = last.key; lastCf = last.cf.cloneMeShallow(isReversed()); toDiscard -= isReversed() ? discardFirst(last.cf, toDiscard, lastCf) : discardLast(last.cf, toDiscard, lastCf); } // If there is less live data than to discard, all is discarded if (toDiscard > 0) return Collections.<Row>emptyList(); // i is the index of the last row that we are sure to keep. On top of that, // we also keep lastCf is it hasn't been fully emptied by the last iteration above. int count = lastCf.getColumnCount(); int newSize = count == 0 ? i+1 : i+2; List<Row> newRows = new ArrayList<Row>(newSize); newRows.addAll(rows.subList(0, i+1)); if (count != 0) newRows.add(new Row(lastKey, lastCf)); return newRows; }
public static ArrayBackedSortedColumns localCopy(ColumnFamily original, AbstractAllocator allocator) { ArrayBackedSortedColumns copy = new ArrayBackedSortedColumns(original.metadata, false, new Cell[original.getColumnCount()], 0, 0); for (Cell cell : original) copy.internalAdd(cell.localCopy(original.metadata, allocator)); copy.sortedSize = copy.size; // internalAdd doesn't update sortedSize. copy.delete(original); return copy; }
public void run() { SSTableWriter writer = null; try { while (true) { Buffer b = writeQueue.take(); if (b == SENTINEL) return; writer = getWriter(); for (Map.Entry<DecoratedKey, ColumnFamily> entry : b.entrySet()) { if (entry.getValue().getColumnCount() > 0) writer.append(entry.getKey(), entry.getValue()); else if (!entry.getKey().equals(b.getFirstInsertedKey())) throw new AssertionError("Empty partition"); } writer.close(); } } catch (Throwable e) { JVMStabilityInspector.inspectThrowable(e); if (writer != null) writer.abort(); exception = e; } } }
@Override public Collection<Mutation> augment(ByteBuffer key, ColumnFamily update) { List<Mutation> mutations = new ArrayList<>(update.getColumnCount()); String indexKeySpace = properties.getProperty("keyspace"); String indexColumnFamily = properties.getProperty("table"); if (!update.deletionInfo().isLive()) { CFMetaData cfMetaData = Schema.instance.getCFMetaData(indexKeySpace, indexColumnFamily); for (ColumnDefinition columnDefinition : cfMetaData.allColumns()) { Mutation mutation = new Mutation(indexKeySpace, key); mutation.delete(indexColumnFamily, CellNames.simpleSparse(columnDefinition.name), System.currentTimeMillis()); mutations.add(mutation); } } return mutations; }
private ColumnFamily processModifications(ColumnFamily changesCF) { ColumnFamilyStore cfs = Keyspace.open(getKeyspaceName()).getColumnFamilyStore(changesCF.id()); ColumnFamily resultCF = changesCF.cloneMeShallow(); List<CounterUpdateCell> counterUpdateCells = new ArrayList<>(changesCF.getColumnCount()); for (Cell cell : changesCF) { if (cell instanceof CounterUpdateCell) counterUpdateCells.add((CounterUpdateCell)cell); else resultCF.addColumn(cell); } if (counterUpdateCells.isEmpty()) return resultCF; // only DELETEs ClockAndCount[] currentValues = getCurrentValues(counterUpdateCells, cfs); for (int i = 0; i < counterUpdateCells.size(); i++) { ClockAndCount currentValue = currentValues[i]; CounterUpdateCell update = counterUpdateCells.get(i); long clock = currentValue.clock + 1L; long count = currentValue.count + update.delta(); resultCF.addColumn(new BufferCounterCell(update.name(), CounterContext.instance().createGlobal(CounterId.getLocalId(), clock, count), update.timestamp())); } return resultCF; }
public long contentSerializedSize(ColumnFamily cf, TypeSizes typeSizes, int version) { long size = cf.getComparator().deletionInfoSerializer().serializedSize(cf.deletionInfo(), typeSizes, version); size += typeSizes.sizeof(cf.getColumnCount()); ColumnSerializer columnSerializer = cf.getComparator().columnSerializer(); for (Cell cell : cf) size += columnSerializer.serializedSize(cell, typeSizes); return size; }
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; }
private ByteBuffer extractDataValue(ColumnDefinition def, ByteBuffer rowKey, ColumnFamily data, Composite prefix) { switch (def.kind) { case PARTITION_KEY: return def.isOnAllComponents() ? rowKey : ((CompositeType)data.metadata().getKeyValidator()).split(rowKey)[def.position()]; case CLUSTERING_COLUMN: return prefix.get(def.position()); case REGULAR: CellName cname = prefix == null ? data.getComparator().cellFromByteBuffer(def.name.bytes) : data.getComparator().create(prefix, def); Cell cell = data.getColumn(cname); return cell == null ? null : cell.value(); case COMPACT_VALUE: assert data.getColumnCount() == 1; return data.getSortedColumns().iterator().next().value(); } throw new AssertionError(); }
public void serialize(ColumnFamily cf, DataOutputPlus out, int version) { try { if (cf == null) { out.writeBoolean(false); return; } out.writeBoolean(true); serializeCfId(cf.id(), out, version); cf.getComparator().deletionInfoSerializer().serialize(cf.deletionInfo(), out, version); ColumnSerializer columnSerializer = cf.getComparator().columnSerializer(); int count = cf.getColumnCount(); out.writeInt(count); int written = 0; for (Cell cell : cf) { columnSerializer.serialize(cell, out); written++; } assert count == written: "Column family had " + count + " columns, but " + written + " written"; } catch (IOException e) { throw new RuntimeException(e); } }
currentOperations.addAndGet(cf.getColumnCount() + (cf.isMarkedForDelete() ? 1 : 0) + cf.deletionInfo().rangeCount()); return pair.right;
Object[] tree = BTree.update(current.tree, metadata.comparator.columnComparator(Memtable.MEMORY_POOL instanceof NativePool), cm, cm.getColumnCount(), true, updater);
hasLegacyCounterShards = hasLegacyCounterShards || ((CounterCell) cell).hasLegacyShards(); return new ColumnStats(getColumnCount(), minTimestampTracker.get(), maxTimestampTracker.get(),