/** * Should only be called by ColumnFamilyStore.apply via Keyspace.apply, which supplies the * appropriate OpOrdering. * * <p>replayPosition should only be null if this is a secondary index, in which case it is * *expected* to be null */ long put(PartitionUpdate update, UpdateTransaction indexer, OpOrder.Group opGroup) { AtomicBTreePartition previous = partitions.get(update.partitionKey()); long initialSize = 0; if (previous == null) { final DecoratedKey cloneKey = allocator.clone(update.partitionKey(), opGroup); AtomicBTreePartition empty = new AtomicBTreePartition(cfs.metadata, cloneKey, allocator); // We'll add the columns later. This avoids wasting works if we get beaten in the putIfAbsent previous = partitions.putIfAbsent(cloneKey, empty); if (previous == null) { previous = empty; // allocate the row overhead after the fact; this saves over allocating and having to free // after, but // means we can overshoot our declared limit. int overhead = (int) (cloneKey.getToken().getHeapSize() + ROW_OVERHEAD_HEAP_SIZE); allocator.onHeap().allocate(overhead, opGroup); initialSize = 8; } else { allocator.reclaimer().reclaimImmediately(cloneKey); } } long[] pair = previous.addAllWithSizeDelta(update, opGroup, indexer); minTimestamp = Math.min(minTimestamp, previous.stats().minTimestamp); liveDataSize.addAndGet(initialSize + pair[0]); columnsCollector.update(update.columns()); statsCollector.update(update.stats()); currentOperations.addAndGet(update.operationCount()); return pair[1]; }
private Collection<SSTableReader> writeSortedContents(File sstableDirectory) { boolean isBatchLogTable = cfs.name.equals(SystemKeyspace.BATCHES) && cfs.keyspace.getName().equals(SystemKeyspace.NAME); logger.debug("Writing {}", Memtable.this.toString()); Collection<SSTableReader> ssTables; try (SSTableTxnWriter writer = createFlushWriter( cfs.getSSTablePath(sstableDirectory), columnsCollector.get(), statsCollector.get())) { boolean trackContention = logger.isTraceEnabled(); int heavilyContendedRowCount = 0; // (we can't clear out the map as-we-go to free up memory, // since the memtable is being used for queries in the "pending flush" category) for (AtomicBTreePartition partition : partitions.values()) { // Each batchlog partition is a separate entry in the log. And for an entry, we only do 2 // operations: 1) we insert the entry and 2) we delete it. Further, BL data is strictly // local, // we don't need to preserve tombstones for repair. So if both operation are in this // memtable (which will almost always be the case if there is no ongoing failure), we can // just skip the entry (CASSANDRA-4667). if (isBatchLogTable && !partition.partitionLevelDeletion().isLive() && partition.hasRows()) continue; if (trackContention && partition.usePessimisticLocking()) heavilyContendedRowCount++; if (!partition.isEmpty()) { try (UnfilteredRowIterator iter = partition.unfilteredIterator()) { writer.append(iter); } } } if (writer.getFilePointer() > 0) { logger.debug( String.format( "Completed flushing %s (%s) for commitlog position %s", writer.getFilename(), FBUtilities.prettyPrintMemory(writer.getFilePointer()), commitLogUpperBound)); // sstables should contain non-repaired data. ssTables = writer.finish(true); } else { logger.debug( "Completed flushing {}; nothing needed to be retained. Commitlog position was {}", writer.getFilename(), commitLogUpperBound); writer.abort(); ssTables = Collections.emptyList(); } if (heavilyContendedRowCount > 0) logger.trace( String.format( "High update contention in %d/%d partitions of %s ", heavilyContendedRowCount, partitions.size(), Memtable.this.toString())); return ssTables; } }