nitsanw commented on code in PR #4402: URL: https://github.com/apache/cassandra/pull/4402#discussion_r2517628574
########## src/java/org/apache/cassandra/db/compaction/CompactionCursor.java: ########## @@ -0,0 +1,1573 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.db.compaction; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Comparator; +import java.util.List; +import java.util.function.LongPredicate; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.UnmodifiableIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.io.sstable.ElementDescriptor; +import org.apache.cassandra.io.sstable.PartitionDescriptor; +import org.apache.cassandra.io.sstable.ReusableLivenessInfo; +import org.apache.cassandra.io.sstable.SSTableCursorReader; +import org.apache.cassandra.io.sstable.SSTableCursorWriter; +import org.apache.cassandra.io.util.ReusableDecoratedKey; +import org.apache.cassandra.io.util.ReusableLongToken; +import org.apache.cassandra.db.AbstractCompactionController; +import org.apache.cassandra.db.ClusteringComparator; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DeletionPurger; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.LivenessInfo; +import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.db.SystemKeyspace; +import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators; +import org.apache.cassandra.db.rows.BTreeRow; +import org.apache.cassandra.db.rows.Cell; +import org.apache.cassandra.db.rows.Cells; +import org.apache.cassandra.db.rows.RangeTombstoneMarker; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.UnfilteredRowIterators; +import org.apache.cassandra.db.rows.UnfilteredSerializer; +import org.apache.cassandra.db.transform.DuplicateRowChecker; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.io.sstable.ISSTableScanner; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.format.SSTableWriter; +import org.apache.cassandra.io.sstable.format.SortedTableWriter; +import org.apache.cassandra.io.sstable.format.Version; +import org.apache.cassandra.io.sstable.format.big.BigFormat; +import org.apache.cassandra.io.util.DataOutputBuffer; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.metrics.TopPartitionTracker; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.CompactionParams; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.TimeUUID; + +import static org.apache.cassandra.db.compaction.CompactionCursor.CellRosolution.COMPARE; +import static org.apache.cassandra.db.compaction.CompactionCursor.CellRosolution.LEFT; +import static org.apache.cassandra.db.compaction.CompactionCursor.CellRosolution.RIGHT; +import static org.apache.cassandra.io.sstable.SSTableCursorReader.State.CELL_END; +import static org.apache.cassandra.io.sstable.SSTableCursorReader.State.CELL_HEADER_START; +import static org.apache.cassandra.io.sstable.SSTableCursorReader.State.CELL_VALUE_START; +import static org.apache.cassandra.io.sstable.SSTableCursorReader.State.DONE; +import static org.apache.cassandra.io.sstable.SSTableCursorReader.State.ELEMENT_END; +import static org.apache.cassandra.io.sstable.SSTableCursorReader.State.PARTITION_END; +import static org.apache.cassandra.io.sstable.SSTableCursorReader.State.PARTITION_START; +import static org.apache.cassandra.io.sstable.SSTableCursorReader.State.ROW_START; +import static org.apache.cassandra.io.sstable.SSTableCursorReader.State.STATIC_ROW_START; +import static org.apache.cassandra.io.sstable.SSTableCursorReader.State.TOMBSTONE_START; +import static org.apache.cassandra.io.sstable.SSTableCursorReader.State.isState; +import static org.apache.cassandra.db.ClusteringPrefix.Kind.EXCL_END_BOUND; +import static org.apache.cassandra.db.ClusteringPrefix.Kind.EXCL_END_INCL_START_BOUNDARY; +import static org.apache.cassandra.db.ClusteringPrefix.Kind.EXCL_START_BOUND; +import static org.apache.cassandra.db.ClusteringPrefix.Kind.INCL_END_BOUND; +import static org.apache.cassandra.db.ClusteringPrefix.Kind.INCL_END_EXCL_START_BOUNDARY; +import static org.apache.cassandra.db.ClusteringPrefix.Kind.INCL_START_BOUND; + +/** + * Merge multiple iterators over the content of sstable into a "compacted" iterator. + * <p> + * On top of the actual merging the source iterators, this class: + * <ul> + * <li>purge gc-able tombstones if possible (see PurgeFunction below).</li> + * <li>invalidate cached partitions that are empty post-compaction. This avoids keeping partitions with + * only purgable tombstones in the row cache.</li> + * <li>keep tracks of the compaction progress.</li> + * <li>TODO:update 2ndary indexes if necessary (as we don't read-before-write on index updates, index entries are + * not deleted on deletion of the base table data, which is ok because we'll fix index inconsistency + * on reads. This however mean that potentially obsolete index entries could be kept a long time for + * data that is not read often, so compaction "pro-actively" fix such index entries. This is mainly + * an optimization).</li> + * </ul> + */ +public class CompactionCursor extends CompactionInfo.Holder +{ + private static final Logger LOGGER = LoggerFactory.getLogger(CompactionCursor.class.getName()); + + private final OperationType type; + private final AbstractCompactionController controller; + private final ActiveCompactionsTracker activeCompactions; + private final ImmutableSet<SSTableReader> sstables; + private final long nowInSec; + private final TimeUUID compactionId; + private final long totalInputBytes; + private final StatefulCursor[] sstableCursors; + private final boolean[] sstableCursorsEqualsNext; + private final boolean hasStaticColumns; + private final boolean enforceStrictLiveness; + + // Keep targetDirectory for compactions, needed for `nodetool compactionstats` + private volatile String targetDirectory; + + private SSTableCursorWriter ssTableCursorWriter; + private boolean finished = false; + + /* + * counters for merged partitions/rows/cells. + * array index represents (number of merged rows - 1), so index 0 is counter for no merge (1 row), + * index 1 is counter for 2 rows merged, and so on. + */ + private final long[] partitionMergeCounters; + private final long[] staticRowMergeCounters; + private final long[] rowMergeCounters; + private final long[] rangeTombstonesMergeCounters; + private final long[] cellMergeCounters; + + // Progress accounting + private long totalBytesRead = 0; + private long totalSourceCQLRows; + private long totalDataBytesWritten; + + // state + final Purger purger; + + private ReusableDecoratedKey prevKey = null; + // Partition state. Writes can be delayed if the deletion is purged, or live and partition is empty -> LIVE deletion. + ReusableDecoratedKey partitionKey; + PartitionDescriptor partitionDescriptor; + DeletionTime partitionDeletion; + // This will be 0 if we haven't written partition header. + int partitionHeaderLength = 0; + private CompactionAwareWriter compactionAwareWriter; + + public CompactionCursor(OperationType type, List<ISSTableScanner> scanners, AbstractCompactionController controller, long nowInSec, TimeUUID compactionId) + { + this(type, scanners, controller, nowInSec, compactionId, ActiveCompactionsTracker.NOOP, null); + } + + public CompactionCursor(OperationType type, + List<ISSTableScanner> scanners, + AbstractCompactionController controller, + long nowInSec, + TimeUUID compactionId, + ActiveCompactionsTracker activeCompactions, + TopPartitionTracker.Collector topPartitionCollector) + { + this.controller = controller; + this.type = type; + this.nowInSec = nowInSec; + this.compactionId = compactionId; + + long inputBytes = 0; + for (ISSTableScanner scanner : scanners) + inputBytes += scanner.getLengthInBytes(); + this.totalInputBytes = inputBytes; + this.partitionMergeCounters = new long[scanners.size()]; + this.staticRowMergeCounters = new long[partitionMergeCounters.length]; + this.rowMergeCounters = new long[partitionMergeCounters.length]; + this.rangeTombstonesMergeCounters = new long[partitionMergeCounters.length]; + this.cellMergeCounters = new long[partitionMergeCounters.length]; + // note that we leak `this` from the constructor when calling beginCompaction below, this means we have to get the sstables before + // calling that to avoid a NPE. + this.sstables = scanners.stream().map(ISSTableScanner::getBackingSSTables).flatMap(Collection::stream).collect(ImmutableSet.toImmutableSet()); + this.activeCompactions = activeCompactions == null ? ActiveCompactionsTracker.NOOP : activeCompactions; + this.activeCompactions.beginCompaction(this); // note that CompactionTask also calls this, but CT only creates CompactionIterator with a NOOP ActiveCompactions + + TableMetadata metadata = metadata(); + if (!(metadata.partitioner instanceof Murmur3Partitioner)) + throw new IllegalArgumentException("SSTableReader is not a murmur3 partitioner:" + metadata.partitioner.getClass().getCanonicalName() +" cursor compactions are only supported for Murmur3Partitioner."); + + if (metadata.indexes.size() != 0) + throw new IllegalArgumentException("SAI is not supported for cursor compactions: " + metadata.indexes +"."); + + for (ColumnMetadata column : metadata.columns()) { + if (column.isComplex()) { + throw new IllegalArgumentException("Complex column: " + column + " cursor compactions are not supported for complex types."); + } + else if (column.isCounterColumn()) { + throw new IllegalArgumentException("Counter column: " + column + " cursor compactions are not supported for counter types."); + } + } + + this.hasStaticColumns = metadata.hasStaticColumns(); + /** + * Pipeline should end up being: + * [MERGED -> ?TopPartitionTracker -> GarbageSkipper -> Purger -> DuplicateRowChecker -> Abortable] -> next() + * V - Merge - This is drawing on code all over the place to iterate through the data and merge partitions/rows/cells + * * Transactions, applied to above iterator: + * X - TODO: We can leave for now? - {@link TopPartitionTracker.TombstoneCounter} - Hooked into CFS metadata, tracks tombstone counts per pk. + * X - TODO: We can leave for now? - {@link CompactionIterator.GarbageSkipper} - filters out, or "skips" data shadowed by the provided "tombstone source". + * V * {@link CompactionIterator.Purger} - filters out, or "purges" gc-able tombstones. Also updates bytes read on every row % 100. + * X - TODO: We can leave for now? - {@link DuplicateRowChecker} - reports duplicate rows across replicas. + * X - TODO: We can leave for now? - Abortable - aborts the compaction if the user has requested it (at a certain granularity). + * {@link CompactionIterator#CompactionIterator(OperationType, List, AbstractCompactionController, long, TimeUUID, ActiveCompactionsTracker, TopPartitionTracker.Collector)} + */ + + // validation + // TODO: these can be views on range limited readers, not necessarily full readers. + for (ISSTableScanner scanner : scanners) + { + if (!scanner.isFullRange()) throw new IllegalArgumentException("Scanner is not full range:" + scanner + " cursor compactions are not supported"); + } + + // TODO: Implement CompactionIterator.GarbageSkipper like functionality + if (controller.tombstoneOption != CompactionParams.TombstoneOption.NONE) + throw new IllegalArgumentException("tombstoneOption: " + controller.tombstoneOption + " cursor compactions are not supported"); + + // Convert Readers to Cursors + this.sstableCursors = new StatefulCursor[sstables.size()]; + this.sstableCursorsEqualsNext = new boolean[sstables.size()]; + UnmodifiableIterator<SSTableReader> iterator = sstables.iterator(); + for (int i = 0; i < this.sstableCursors.length; i++) + { + SSTableReader ssTableReader = iterator.next(); + if (ssTableReader.getFilename().contains("system")) + throw new IllegalArgumentException("Cursor compactions on system tables are not supported."); + Version version = ssTableReader.descriptor.version; + if (!(version.format instanceof BigFormat)) + throw new IllegalArgumentException("Cursor compactions only supported on BIG format: " + version); + if (!version.isLatestVersion()) + throw new IllegalArgumentException("Cursor compactions only supported on latest version: " + version); + + this.sstableCursors[i] = new StatefulCursor(ssTableReader); + } + this.enforceStrictLiveness = controller.cfs.metadata.get().enforceStrictLiveness(); + + purger = new Purger(type, controller, nowInSec); + } + + /** + * @return false if finished, true if partition is written (which might require multiple partition reads) + */ + public boolean writeNextPartition(CompactionAwareWriter compactionAwareWriter) throws IOException { + while (!finished) { + if (tryWriteNextPartition(compactionAwareWriter)) { + return true; + } + } + return false; + } + + /** + * @return true if a partition was written + */ + private boolean tryWriteNextPartition(CompactionAwareWriter compactionAwareWriter) throws IOException + { + if (sortForPartitionMerge()) + { + finish(); + return false; + } + + // Top reader is on the current key/header + partitionDescriptor = sstableCursors[0].pHeader; + partitionKey = sstableCursors[0].currentDecoratedKey; + + // possibly reached boundary of the current writer + try + { + // TODO: Potentially redundant validation... Can be done on the writer level? + if (prevKey != null && prevKey.compareTo(partitionKey) >= 0) Review Comment: The implementation is BIG only at this time. I think splitting for BTI can come as a further PR. Let me know if it needs to happen in this PR. ########## src/java/org/apache/cassandra/io/sstable/UnfilteredDescriptor.java: ########## @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.io.sstable; + +import java.io.IOException; +import java.util.Arrays; + +import org.apache.cassandra.db.Columns; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.rows.DeserializationHelper; +import org.apache.cassandra.db.rows.UnfilteredSerializer; +import org.apache.cassandra.io.util.RandomAccessReader; + +import static org.apache.cassandra.io.sstable.SSTableCursorReader.readUnfilteredDeletionTime; + +public class ElementDescriptor extends ClusteringDescriptor +{ + private final ReusableLivenessInfo rowLivenessInfo = new ReusableLivenessInfo(); + private final DeletionTime deletionTime = DeletionTime.build(0, 0); + private final DeletionTime deletionTime2 = DeletionTime.build(0, 0); + + private long position; + private int flags; + private int extendedFlags; + + private long unfilteredSize; + private long unfilteredDataStart; +// private long prevUnfilteredSize; + private long unfilteredCellStart; + Columns rowColumns; + + void loadTombstone(RandomAccessReader dataReader, + SerializationHeader serializationHeader, + DeserializationHelper deserializationHelper, + AbstractType<?>[] clusteringColumnTypes, + int flags) throws IOException + { + this.flags = flags; + this.extendedFlags = 0; + rowColumns = null; + byte clusteringKind = dataReader.readByte(); + if (clusteringKind == STATIC_CLUSTERING_TYPE || clusteringKind == ROW_CLUSTERING_TYPE) { + // STATIC_CLUSTERING or CLUSTERING -> no deletion info, should not happen + throw new IllegalStateException(); + } + + int columnsBound = dataReader.readUnsignedShort(); + loadClustering(dataReader, clusteringColumnTypes, clusteringKind, columnsBound); + this.unfilteredSize = dataReader.readUnsignedVInt(); + dataReader.readUnsignedVInt(); // Unused: prevUnfilteredSize + if (clusteringKind == EXCL_END_INCL_START_BOUNDARY_CLUSTERING_TYPE || clusteringKind == INCL_END_EXCL_START_BOUNDARY_CLUSTERING_TYPE) + { + // boundary + readUnfilteredDeletionTime(dataReader, serializationHeader, deletionTime); // CLOSE + readUnfilteredDeletionTime(dataReader, serializationHeader, deletionTime2); // OPEN + } + else + { + // bound + readUnfilteredDeletionTime(dataReader, serializationHeader, deletionTime); // CLOSE|OPEN + } + } + + void loadRow(RandomAccessReader dataReader, + SerializationHeader serializationHeader, + DeserializationHelper deserializationHelper, + AbstractType<?>[] clusteringTypes, + int flags) throws IOException { + // body = whatever is covered by size, so inclusive of the prev_row_size inclusive of flags + position = dataReader.getPosition() - 1; + this.flags = flags; + this.extendedFlags = 0; + + loadClustering(dataReader, clusteringTypes, ROW_CLUSTERING_TYPE, clusteringTypes.length); + + rowColumns = serializationHeader.columns(false); + + loadCommonRowFields(dataReader, serializationHeader, deserializationHelper, flags); + } + + void loadStaticRow(RandomAccessReader dataReader, + SerializationHeader serializationHeader, + DeserializationHelper deserializationHelper, + int flags, + int extendedFlags) throws IOException { + // body = whatever is covered by size, so inclusive of the prev_row_size inclusive of flags + position = dataReader.getPosition() - 2; + this.flags = flags; + this.extendedFlags = extendedFlags; + // no clustering + loadClustering(dataReader, null, STATIC_CLUSTERING_TYPE, 0); + rowColumns = serializationHeader.columns(true); + + loadCommonRowFields(dataReader, serializationHeader, deserializationHelper, flags); + } + + private void loadCommonRowFields(RandomAccessReader dataReader, SerializationHeader serializationHeader, DeserializationHelper deserializationHelper, int flags) throws IOException + { + unfilteredSize = dataReader.readUnsignedVInt(); + unfilteredDataStart = dataReader.getPosition(); + // prevUnfilteredSize = ; + dataReader.readUnsignedVInt(); // unused + + SSTableCursorReader.readLivenessInfo(dataReader, serializationHeader, deserializationHelper, flags, rowLivenessInfo); + if (UnfilteredSerializer.hasDeletion(flags)) + { + // struct delta_deletion_time { + // varint delta_marked_for_delete_at; + // varint delta_local_deletion_time; + //}; + readUnfilteredDeletionTime(dataReader, serializationHeader, deletionTime); + } + else + { + deletionTime.resetLive(); + } + if (!UnfilteredSerializer.hasAllColumns(flags)) + { + // TODO: re-implement GC free Review Comment: I think it's an important optimization, but it should not block this PR IMO. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]

