diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index 80dfa68cf7d9..8dcc01a0e007 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -97,6 +97,7 @@ import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.repair.CassandraTableRepairManager; import org.apache.cassandra.db.rows.CellPath; +import org.apache.cassandra.db.rows.UnfilteredSource; import org.apache.cassandra.db.streaming.CassandraStreamManager; import org.apache.cassandra.db.view.TableViews; import org.apache.cassandra.dht.AbstractBounds; @@ -3002,7 +3003,7 @@ public int getLevelFanoutSize() return compactionStrategyManager.getLevelFanoutSize(); } - public static class ViewFragment + public static class ViewFragment implements ReadableView { public final List sstables; public final Iterable memtables; @@ -3012,6 +3013,18 @@ public ViewFragment(List sstables, Iterable memtables) this.sstables = sstables; this.memtables = memtables; } + + @Override + public Iterable memtables() + { + return memtables; + } + + @Override + public List sstables() + { + return sstables; + } } public static class RefViewFragment extends ViewFragment implements AutoCloseable diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java index 2e816a2581cf..f228c2bdaa62 100644 --- a/src/java/org/apache/cassandra/db/ReadCommand.java +++ b/src/java/org/apache/cassandra/db/ReadCommand.java @@ -126,7 +126,42 @@ public abstract class ReadCommand extends AbstractReadQuery { private interface ReadCompleter { - T complete(UnfilteredPartitionIterator iterator, ReadExecutionController executionController, Index.Searcher searcher, ColumnFamilyStore cfs, long startTimeNanos); + T complete(ReadCommand command, UnfilteredPartitionIterator iterator, ReadExecutionController executionController, ColumnFamilyStore cfs, long startTimeNanos); + T complete(ReadCommand command, Index.Searcher searcher, ReadExecutionController executionController, ColumnFamilyStore cfs, long startTimeNanos); + + ReadCompleter IMMEDIATE = new ReadCompleter<>() + { + @Override + public UnfilteredPartitionIterator complete(ReadCommand command, UnfilteredPartitionIterator iterator, ReadExecutionController executionController, ColumnFamilyStore cfs, long startTimeNanos) + { + return command.completeRead(iterator, executionController, null, cfs, startTimeNanos); + } + + @Override + public UnfilteredPartitionIterator complete(ReadCommand command, Index.Searcher searcher, ReadExecutionController executionController, ColumnFamilyStore cfs, long startTimeNanos) + { + UnfilteredPartitionIterator iterator = searcher.search(executionController); + return command.completeRead(iterator, executionController, searcher, cfs, startTimeNanos); + } + }; + + ReadCompleter TRACKED = new ReadCompleter<>() + { + @Override + public PartialTrackedRead complete(ReadCommand command, UnfilteredPartitionIterator iterator, ReadExecutionController executionController, ColumnFamilyStore cfs, long startTimeNanos) + { + return command.createInProgressRead(iterator, executionController, null, cfs, startTimeNanos); + } + + @Override + public PartialTrackedRead complete(ReadCommand command, Index.Searcher searcher, ReadExecutionController executionController, ColumnFamilyStore cfs, long startTimeNanos) + { + if (!searcher.isMultiStep()) + throw new IllegalStateException("Cannot use " + searcher.getClass().getName() + " with tracked reads"); + + return searcher.asMultiStep().beginRead(executionController, cfs, startTimeNanos); + } + }; } private static final int TEST_ITERATION_DELAY_MILLIS = CassandraRelevantProperties.TEST_READ_ITERATION_DELAY_MS.getInt(); @@ -479,28 +514,26 @@ private T beginRead(ReadExecutionController executionController, @Nullable C ConsensusRequestRouter.validateSafeToReadNonTransactionally(this, cm); Index.QueryPlan indexQueryPlan = indexQueryPlan(); - Index.Searcher searcher = null; if (indexQueryPlan != null) { cfs.indexManager.checkQueryability(indexQueryPlan); - searcher = indexQueryPlan.searcherFor(this); - Tracing.trace("Executing read on {}.{} using index{} {}", - cfs.metadata.keyspace, - cfs.metadata.name, - indexQueryPlan.getIndexes().size() == 1 ? "" : "es", - indexQueryPlan.getIndexes() - .stream() - .map(i -> i.getIndexMetadata().name) - .collect(Collectors.joining(","))); + if (Tracing.isTracing()) + { + Tracing.trace("Executing read on {}.{} using index{} {}", + cfs.metadata.keyspace, + cfs.metadata.name, + indexQueryPlan.getIndexes().size() == 1 ? "" : "es", + indexQueryPlan.getIndexes() + .stream() + .map(i -> i.getIndexMetadata().name) + .collect(Collectors.joining(","))); + } + Index.Searcher searcher = indexQueryPlan.searcherFor(this); + return completer.complete(this, searcher, executionController, cfs, startTimeNanos); } - - if (searcher != null && metadata().replicationType().isTracked()) - throw new UnsupportedOperationException("TODO: support tracked index reads"); - - UnfilteredPartitionIterator iterator = (null == searcher) ? queryStorage(cfs, executionController) : searcher.search(executionController); - - return completer.complete(iterator, executionController, searcher, cfs, startTimeNanos); + UnfilteredPartitionIterator iterator = queryStorage(cfs, executionController); + return completer.complete(this, iterator, executionController, cfs, startTimeNanos); } finally { @@ -586,7 +619,7 @@ protected abstract PartialTrackedRead createInProgressRead(UnfilteredPartitionIt public PartialTrackedRead beginTrackedRead(ReadExecutionController executionController) { - return beginRead(executionController, null, this::createInProgressRead); + return beginRead(executionController, null, ReadCompleter.TRACKED); } public UnfilteredPartitionIterator completeTrackedRead(UnfilteredPartitionIterator iterator, PartialTrackedRead read) @@ -604,12 +637,12 @@ public UnfilteredPartitionIterator completeTrackedRead(UnfilteredPartitionIterat // iterators created inside the try as long as we do close the original resultIterator), or by closing the result. public UnfilteredPartitionIterator executeLocally(ReadExecutionController executionController) { - return beginRead(executionController, null, this::completeRead); + return beginRead(executionController, null, ReadCompleter.IMMEDIATE); } public UnfilteredPartitionIterator executeLocally(ReadExecutionController executionController, @Nullable ClusterMetadata cm) { - return beginRead(executionController, cm, this::completeRead); + return beginRead(executionController, cm, ReadCompleter.IMMEDIATE); } protected abstract void recordLatency(TableMetrics metric, long latencyNanos); @@ -1144,7 +1177,7 @@ public String name() return toCQLString(); } - InputCollector iteratorsForPartition(ColumnFamilyStore.ViewFragment view, ReadExecutionController controller) + InputCollector iteratorsForPartition(ReadableView view, ReadExecutionController controller) { final BiFunction, RepairedDataInfo, UnfilteredRowIterator> merge = (unfilteredRowIterators, repairedDataInfo) -> { @@ -1194,7 +1227,7 @@ static class InputCollector List repairedIters; List unrepairedIters; - InputCollector(ColumnFamilyStore.ViewFragment view, + InputCollector(ReadableView view, ReadExecutionController controller, BiFunction, RepairedDataInfo, T> repairedMerger, Function postLimitAdditionalPartitions) @@ -1204,12 +1237,12 @@ static class InputCollector if (isTrackingRepairedStatus) { - for (SSTableReader sstable : view.sstables) + for (SSTableReader sstable : view.sstables()) { if (considerRepairedForTracking(sstable)) { if (repairedSSTables == null) - repairedSSTables = Sets.newHashSetWithExpectedSize(view.sstables.size()); + repairedSSTables = Sets.newHashSetWithExpectedSize(view.sstables().size()); repairedSSTables.add(sstable); } } @@ -1217,14 +1250,14 @@ static class InputCollector if (repairedSSTables == null) { repairedIters = Collections.emptyList(); - unrepairedIters = new ArrayList<>(view.sstables.size()); + unrepairedIters = new ArrayList<>(view.sstables().size()); } else { repairedIters = new ArrayList<>(repairedSSTables.size()); // when we're done collating, we'll merge the repaired iters and add the // result to the unrepaired list, so size that list accordingly - unrepairedIters = new ArrayList<>((view.sstables.size() - repairedSSTables.size()) + Iterables.size(view.memtables) + 1); + unrepairedIters = new ArrayList<>((view.sstables().size() - repairedSSTables.size()) + Iterables.size(view.memtables()) + 1); } this.repairedMerger = repairedMerger; this.postLimitAdditionalPartitions = postLimitAdditionalPartitions; diff --git a/src/java/org/apache/cassandra/db/ReadableView.java b/src/java/org/apache/cassandra/db/ReadableView.java new file mode 100644 index 000000000000..d68335a2d4fd --- /dev/null +++ b/src/java/org/apache/cassandra/db/ReadableView.java @@ -0,0 +1,30 @@ +/* + * 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; + +import java.util.List; + +import org.apache.cassandra.db.rows.UnfilteredSource; +import org.apache.cassandra.io.sstable.format.SSTableReader; + +public interface ReadableView +{ + Iterable memtables(); + List sstables(); +} diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java index a1f312b62614..d4b0bd4d3ad0 100644 --- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java +++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java @@ -56,6 +56,7 @@ import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.db.rows.UnfilteredRowIteratorWithLowerBound; import org.apache.cassandra.db.rows.UnfilteredRowIterators; +import org.apache.cassandra.db.rows.UnfilteredSource; import org.apache.cassandra.db.rows.WrappingUnfilteredRowIterator; import org.apache.cassandra.db.transform.RTBoundValidator; import org.apache.cassandra.db.transform.Transformation; @@ -715,15 +716,20 @@ public Unfiltered next() * Also note that one must have created a {@code ReadExecutionController} on the queried table and we require it as * a parameter to enforce that fact, even though it's not explicitlly used by the method. */ - public UnfilteredRowIterator queryMemtableAndDisk(ColumnFamilyStore cfs, ReadExecutionController executionController) + public UnfilteredRowIterator queryMemtableAndDisk(ReadableView view, ColumnFamilyStore cfs, ReadExecutionController executionController) { assert executionController != null && executionController.validForReadOn(cfs); Tracing.trace("Executing single-partition query on {}", cfs.name); - return queryMemtableAndDiskInternal(cfs, executionController); + return queryMemtableAndDiskInternal(view, cfs, executionController); + } + + public UnfilteredRowIterator queryMemtableAndDisk(ColumnFamilyStore cfs, ReadExecutionController executionController) + { + return queryMemtableAndDisk(cfs.select(View.select(SSTableSet.LIVE, partitionKey())), cfs, executionController); } - private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs, ReadExecutionController controller) + private UnfilteredRowIterator queryMemtableAndDiskInternal(ReadableView view, ColumnFamilyStore cfs, ReadExecutionController controller) { /* * We have 2 main strategies: @@ -747,12 +753,12 @@ private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs && !queriesMulticellType() && !controller.isTrackingRepairedStatus()) { - return queryMemtableAndSSTablesInTimestampOrder(cfs, (ClusteringIndexNamesFilter)clusteringIndexFilter(), controller); + return queryMemtableAndSSTablesInTimestampOrder(view, cfs, (ClusteringIndexNamesFilter)clusteringIndexFilter(), controller); } Tracing.trace("Acquiring sstable references"); - ColumnFamilyStore.ViewFragment view = cfs.select(View.select(SSTableSet.LIVE, partitionKey())); - view.sstables.sort(SSTableReader.maxTimestampDescending); + List sstables = view.sstables(); + sstables.sort(SSTableReader.maxTimestampDescending); ClusteringIndexFilter filter = clusteringIndexFilter(); long minTimestamp = Long.MAX_VALUE; long mostRecentPartitionTombstone = Long.MIN_VALUE; @@ -761,7 +767,7 @@ private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs { SSTableReadMetricsCollector metricsCollector = new SSTableReadMetricsCollector(); - for (Memtable memtable : view.memtables) + for (UnfilteredSource memtable : view.memtables()) { UnfilteredRowIterator iter = memtable.rowIterator(partitionKey(), filter.getSlices(metadata()), columnFilter(), filter.isReversed(), metricsCollector); if (iter == null) @@ -790,14 +796,14 @@ private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs * In other words, iterating in descending maxTimestamp order allow to do our mostRecentPartitionTombstone * elimination in one pass, and minimize the number of sstables for which we read a partition tombstone. */ - view.sstables.sort(SSTableReader.maxTimestampDescending); + sstables.sort(SSTableReader.maxTimestampDescending); int nonIntersectingSSTables = 0; int includedDueToTombstones = 0; if (controller.isTrackingRepairedStatus()) Tracing.trace("Collecting data from sstables and tracking repaired status"); - for (SSTableReader sstable : view.sstables) + for (SSTableReader sstable : sstables) { // if we've already seen a partition tombstone with a timestamp greater // than the most recent update to this sstable, we can skip it @@ -865,7 +871,7 @@ private UnfilteredRowIterator queryMemtableAndDiskInternal(ColumnFamilyStore cfs if (Tracing.isTracing()) Tracing.trace("Skipped {}/{} non-slice-intersecting sstables, included {} due to tombstones", - nonIntersectingSSTables, view.sstables.size(), includedDueToTombstones); + nonIntersectingSSTables, sstables.size(), includedDueToTombstones); if (inputCollector.isEmpty()) return EmptyIterators.unfilteredRow(cfs.metadata(), partitionKey(), filter.isReversed()); @@ -989,16 +995,15 @@ private boolean queriesMulticellType() * no collection or counters are included). * This method assumes the filter is a {@code ClusteringIndexNamesFilter}. */ - private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ColumnFamilyStore cfs, ClusteringIndexNamesFilter filter, ReadExecutionController controller) + private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ReadableView view, ColumnFamilyStore cfs, ClusteringIndexNamesFilter filter, ReadExecutionController controller) { Tracing.trace("Acquiring sstable references"); - ColumnFamilyStore.ViewFragment view = cfs.select(View.select(SSTableSet.LIVE, partitionKey())); ImmutableBTreePartition result = null; SSTableReadMetricsCollector metricsCollector = new SSTableReadMetricsCollector(); Tracing.trace("Merging memtable contents"); - for (Memtable memtable : view.memtables) + for (UnfilteredSource memtable : view.memtables()) { try (UnfilteredRowIterator iter = memtable.rowIterator(partitionKey, filter.getSlices(metadata()), columnFilter(), isReversed(), metricsCollector)) { @@ -1014,9 +1019,10 @@ private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ColumnFam } /* add the SSTables on disk */ - view.sstables.sort(SSTableReader.maxTimestampDescending); + List sstables = view.sstables(); + sstables.sort(SSTableReader.maxTimestampDescending); // read sorted sstables - for (SSTableReader sstable : view.sstables) + for (SSTableReader sstable : sstables) { // if we've already seen a partition tombstone with a timestamp greater // than the most recent update to this sstable, we're done, since the rest of the sstables diff --git a/src/java/org/apache/cassandra/db/memtable/Memtable.java b/src/java/org/apache/cassandra/db/memtable/Memtable.java index 6e6767568025..44d3d25250ee 100644 --- a/src/java/org/apache/cassandra/db/memtable/Memtable.java +++ b/src/java/org/apache/cassandra/db/memtable/Memtable.java @@ -24,6 +24,7 @@ import javax.annotation.concurrent.NotThreadSafe; import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.db.RegularAndStaticColumns; import org.apache.cassandra.db.commitlog.CommitLogPosition; @@ -205,6 +206,7 @@ default long put(MutationId mutationId, PartitionUpdate update, UpdateTransactio long put(MutationId mutationId, PartitionUpdate update, UpdateTransaction indexer, OpOrder.Group opGroup, boolean assumeMissing); // Read operations are provided by the UnfilteredSource interface. + Partition snapshotPartition(DecoratedKey partitionKey); // Statistics diff --git a/src/java/org/apache/cassandra/db/memtable/ShardedSkipListMemtable.java b/src/java/org/apache/cassandra/db/memtable/ShardedSkipListMemtable.java index 1cc96aeacfb4..bb545cee15a4 100644 --- a/src/java/org/apache/cassandra/db/memtable/ShardedSkipListMemtable.java +++ b/src/java/org/apache/cassandra/db/memtable/ShardedSkipListMemtable.java @@ -284,6 +284,12 @@ public UnfilteredRowIterator rowIterator(DecoratedKey key) return p != null ? p.unfilteredIterator() : null; } + @Override + public Partition snapshotPartition(DecoratedKey partitionKey) + { + return getPartition(partitionKey); + } + public FlushablePartitionSet getFlushSet(PartitionPosition from, PartitionPosition to) { long keySize = 0; diff --git a/src/java/org/apache/cassandra/db/memtable/SkipListMemtable.java b/src/java/org/apache/cassandra/db/memtable/SkipListMemtable.java index 7176294e8d35..688930fbfdec 100644 --- a/src/java/org/apache/cassandra/db/memtable/SkipListMemtable.java +++ b/src/java/org/apache/cassandra/db/memtable/SkipListMemtable.java @@ -228,6 +228,12 @@ public UnfilteredRowIterator rowIterator(DecoratedKey key) return p != null ? p.unfilteredIterator() : null; } + @Override + public Partition snapshotPartition(DecoratedKey partitionKey) + { + return getPartition(partitionKey); + } + private static int estimateRowOverhead(final int count) { // calculate row overhead diff --git a/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java index fd50fffe61d7..4c38cfa70dff 100644 --- a/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java +++ b/src/java/org/apache/cassandra/db/memtable/TrieMemtable.java @@ -327,6 +327,12 @@ private Partition getPartition(DecoratedKey key) return null; } + @Override + public Partition snapshotPartition(DecoratedKey partitionKey) + { + return getPartition(partitionKey); + } + @Override public UnfilteredRowIterator rowIterator(DecoratedKey key, Slices slices, ColumnFilter selectedColumns, boolean reversed, SSTableReadsListener listener) { diff --git a/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java b/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java index fd7880e367e4..cc7e5b120c2a 100644 --- a/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java +++ b/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java @@ -427,4 +427,16 @@ public void close() }; } } + + public static void consume(UnfilteredPartitionIterator iterator) + { + while (iterator.hasNext()) + { + try (UnfilteredRowIterator partition = iterator.next()) + { + while (partition.hasNext()) + partition.next(); + } + } + } } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/index/Index.java b/src/java/org/apache/cassandra/index/Index.java index 5225b82dee53..8b8556a80e6f 100644 --- a/src/java/org/apache/cassandra/index/Index.java +++ b/src/java/org/apache/cassandra/index/Index.java @@ -29,12 +29,15 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.Callable; +import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Predicate; import java.util.function.Supplier; import javax.annotation.Nonnull; import javax.annotation.Nullable; +import com.google.common.collect.PeekingIterator; + import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.cql3.QueryOptions; import org.apache.cassandra.cql3.restrictions.Restriction; @@ -44,6 +47,7 @@ import org.apache.cassandra.db.RangeTombstone; import org.apache.cassandra.db.ReadCommand; import org.apache.cassandra.db.ReadExecutionController; +import org.apache.cassandra.db.ReadableView; import org.apache.cassandra.db.RegularAndStaticColumns; import org.apache.cassandra.db.WriteContext; import org.apache.cassandra.db.filter.RowFilter; @@ -54,6 +58,7 @@ import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.index.internal.CollatedViewIndexBuilder; import org.apache.cassandra.index.transactions.IndexTransaction; @@ -67,6 +72,8 @@ import org.apache.cassandra.schema.IndexMetadata; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.reads.tracked.PartialTrackedRead; +import org.apache.cassandra.utils.CloseablePeekingIterator; /** * Consisting of a top level Index interface and two sub-interfaces which handle read and write operations, @@ -706,6 +713,65 @@ default PartitionIterator filterReplicaFilteringProtection(PartitionIterator ful { return command().rowFilter().filter(fullResponse, command().metadata(), command().nowInSec()); } + + default boolean isMultiStep() + { + return false; + } + + default MultiStepSearcher asMultiStep() + { + throw new IllegalStateException(getClass().getSimpleName() + " is not a multi-step searcher"); + } + } + + default boolean supportsMutationTracking() + { + return false; + } + + interface IndexMatch + { + DecoratedKey key(); + } + + interface MatchIndexer + { + void index(PartitionUpdate update, Consumer indexTo); + } + + /** + * Extended searcher capable of participating in tracked reads + */ + interface MultiStepSearcher extends Searcher + { + + PartialTrackedRead beginRead(ReadExecutionController executionController, ColumnFamilyStore cfs, long startTimeNanos); + + CloseablePeekingIterator matchIterator(ReadExecutionController executionController); + MatchIndexer matchIndexer(); + + UnfilteredRowIterator queryNextMatches(ReadExecutionController executionController, DecoratedKey partitionKey, ReadableView view, PeekingIterator matches); + + /** + * Since partition updates may not contain all the info the index query needs to know if it will create a hit + * it may return false positives. This filter is meant to catch and remove them from the augmented result + */ + UnfilteredPartitionIterator filterCompletedRead(UnfilteredPartitionIterator iterator); + + Comparator matchComparator(); + + @Override + default boolean isMultiStep() + { + return true; + } + + @Override + default MultiStepSearcher asMultiStep() + { + return this; + } } /** diff --git a/src/java/org/apache/cassandra/index/internal/CassandraIndex.java b/src/java/org/apache/cassandra/index/internal/CassandraIndex.java index d362a86e12a7..e38e05a9cd8e 100644 --- a/src/java/org/apache/cassandra/index/internal/CassandraIndex.java +++ b/src/java/org/apache/cassandra/index/internal/CassandraIndex.java @@ -332,160 +332,217 @@ public void validate(PartitionUpdate update, ClientState state) throws InvalidRe } } - public Indexer indexerFor(final DecoratedKey key, - final RegularAndStaticColumns columns, - final long nowInSec, - final WriteContext ctx, - final IndexTransaction.Type transactionType, - Memtable memtable) + static abstract class AbstractIndexer implements Indexer { - /* - * Indexes on regular and static columns (the non primary-key ones) only care about updates with live - * data for the column they index. In particular, they don't care about having just row or range deletions - * as they don't know how to update the index table unless they know exactly the value that is deleted. - * - * Note that in practice this means that those indexes are only purged of stale entries on compaction, - * when we resolve both the deletion and the prior data it deletes. Of course, such stale entries are also - * filtered on read. - */ - if (!isPrimaryKeyIndex() && !columns.contains(indexedColumn)) - return null; + public void begin() + { + } + + public void partitionDelete(DeletionTime deletionTime) + { + } + + public void rangeTombstone(RangeTombstone tombstone) + { + } + + abstract CassandraIndex index(); + + ColumnMetadata indexedColumn() + { + return index().indexedColumn; + } + + boolean isPrimaryKeyIndex() + { + return index().isPrimaryKeyIndex(); + } - return new Indexer() + abstract long nowInSec(); + abstract DecoratedKey key(); + + abstract void insert(DecoratedKey rowKey, Clustering clustering, Cell cell, LivenessInfo info); + abstract void delete(DecoratedKey rowKey, Clustering clustering, Cell cell, long nowInSec); + abstract void delete(DecoratedKey rowKey, Clustering clustering, DeletionTime deletion); + + public void insertRow(Row row) { - public void begin() + if (row.isStatic() && !indexedColumn().isStatic() && !indexedColumn().isPartitionKey()) + return; + + if (isPrimaryKeyIndex()) { + indexPrimaryKey(row.clustering(), + getPrimaryKeyIndexLiveness(row), + row.deletion()); } - - public void partitionDelete(DeletionTime deletionTime) + else { + if (indexedColumn().isComplex()) + indexCells(row.clustering(), row.getComplexColumnData(indexedColumn())); + else + indexCell(row.clustering(), row.getCell(indexedColumn())); } + } + + public void removeRow(Row row) + { + if (isPrimaryKeyIndex()) + return; + + if (indexedColumn().isComplex()) + removeCells(row.clustering(), row.getComplexColumnData(indexedColumn())); + else + removeCell(row.clustering(), row.getCell(indexedColumn())); + } + + public void updateRow(Row oldRow, Row newRow) + { + assert oldRow.isStatic() == newRow.isStatic(); + if (newRow.isStatic() != indexedColumn().isStatic()) + return; + + if (isPrimaryKeyIndex()) + indexPrimaryKey(newRow.clustering(), + getPrimaryKeyIndexLiveness(newRow), + newRow.deletion()); - public void rangeTombstone(RangeTombstone tombstone) + if (indexedColumn().isComplex()) { + indexCells(newRow.clustering(), newRow.getComplexColumnData(indexedColumn())); + removeCells(oldRow.clustering(), oldRow.getComplexColumnData(indexedColumn())); } - - public void insertRow(Row row) + else { - if (row.isStatic() && !indexedColumn.isStatic() && !indexedColumn.isPartitionKey()) - return; - - if (isPrimaryKeyIndex()) - { - indexPrimaryKey(row.clustering(), - getPrimaryKeyIndexLiveness(row), - row.deletion()); - } - else - { - if (indexedColumn.isComplex()) - indexCells(row.clustering(), row.getComplexColumnData(indexedColumn)); - else - indexCell(row.clustering(), row.getCell(indexedColumn)); - } + indexCell(newRow.clustering(), newRow.getCell(indexedColumn())); + removeCell(oldRow.clustering(), oldRow.getCell(indexedColumn())); } + } - public void removeRow(Row row) - { - if (isPrimaryKeyIndex()) - return; + public void finish() + { + } - if (indexedColumn.isComplex()) - removeCells(row.clustering(), row.getComplexColumnData(indexedColumn)); - else - removeCell(row.clustering(), row.getCell(indexedColumn)); - } + private void indexCells(Clustering clustering, Iterable> cells) + { + if (cells == null) + return; - public void updateRow(Row oldRow, Row newRow) - { - assert oldRow.isStatic() == newRow.isStatic(); - if (newRow.isStatic() != indexedColumn.isStatic()) - return; + for (Cell cell : cells) + indexCell(clustering, cell); + } - if (isPrimaryKeyIndex()) - indexPrimaryKey(newRow.clustering(), - getPrimaryKeyIndexLiveness(newRow), - newRow.deletion()); + private void indexCell(Clustering clustering, Cell cell) + { + if (cell == null || !cell.isLive(nowInSec())) + return; - if (indexedColumn.isComplex()) - { - indexCells(newRow.clustering(), newRow.getComplexColumnData(indexedColumn)); - removeCells(oldRow.clustering(), oldRow.getComplexColumnData(indexedColumn)); - } - else + insert(key(), + clustering, + cell, + LivenessInfo.withExpirationTime(cell.timestamp(), cell.ttl(), cell.localDeletionTime())); + } + + private void removeCells(Clustering clustering, Iterable> cells) + { + if (cells == null) + return; + + for (Cell cell : cells) + removeCell(clustering, cell); + } + + private void removeCell(Clustering clustering, Cell cell) + { + if (cell == null || !cell.isLive(nowInSec())) + return; + + delete(key(), clustering, cell, nowInSec()); + } + + private void indexPrimaryKey(final Clustering clustering, + final LivenessInfo liveness, + final Row.Deletion deletion) + { + if (liveness.timestamp() != LivenessInfo.NO_TIMESTAMP) + insert(key(), clustering, null, liveness); + + if (!deletion.isLive()) + delete(key(), clustering, deletion.time()); + } + + private LivenessInfo getPrimaryKeyIndexLiveness(Row row) + { + long timestamp = row.primaryKeyLivenessInfo().timestamp(); + int ttl = row.primaryKeyLivenessInfo().ttl(); + for (Cell cell : row.cells()) + { + long cellTimestamp = cell.timestamp(); + if (cell.isLive(nowInSec())) { - indexCell(newRow.clustering(), newRow.getCell(indexedColumn)); - removeCell(oldRow.clustering(), oldRow.getCell(indexedColumn)); + if (cellTimestamp > timestamp) + timestamp = cellTimestamp; } } + return LivenessInfo.create(timestamp, ttl, nowInSec()); + } + } - public void finish() - { - } + public Indexer indexerFor(final DecoratedKey key, + final RegularAndStaticColumns columns, + final long nowInSec, + final WriteContext ctx, + final IndexTransaction.Type transactionType, + Memtable memtable) + { + /* + * Indexes on regular and static columns (the non primary-key ones) only care about updates with live + * data for the column they index. In particular, they don't care about having just row or range deletions + * as they don't know how to update the index table unless they know exactly the value that is deleted. + * + * Note that in practice this means that those indexes are only purged of stale entries on compaction, + * when we resolve both the deletion and the prior data it deletes. Of course, such stale entries are also + * filtered on read. + */ + if (!isPrimaryKeyIndex() && !columns.contains(indexedColumn)) + return null; - private void indexCells(Clustering clustering, Iterable> cells) + return new AbstractIndexer() + { + @Override + CassandraIndex index() { - if (cells == null) - return; - - for (Cell cell : cells) - indexCell(clustering, cell); + return CassandraIndex.this; } - private void indexCell(Clustering clustering, Cell cell) + @Override + long nowInSec() { - if (cell == null || !cell.isLive(nowInSec)) - return; - - insert(key.getKey(), - clustering, - cell, - LivenessInfo.withExpirationTime(cell.timestamp(), cell.ttl(), cell.localDeletionTime()), - ctx); + return nowInSec; } - private void removeCells(Clustering clustering, Iterable> cells) + @Override + DecoratedKey key() { - if (cells == null) - return; - - for (Cell cell : cells) - removeCell(clustering, cell); + return key; } - private void removeCell(Clustering clustering, Cell cell) + @Override + void insert(DecoratedKey rowKey, Clustering clustering, Cell cell, LivenessInfo info) { - if (cell == null || !cell.isLive(nowInSec)) - return; - - delete(key.getKey(), clustering, cell, ctx, nowInSec); + CassandraIndex.this.insert(rowKey.getKey(), clustering, cell, info, ctx); } - private void indexPrimaryKey(final Clustering clustering, - final LivenessInfo liveness, - final Row.Deletion deletion) + @Override + void delete(DecoratedKey rowKey, Clustering clustering, Cell cell, long nowInSec) { - if (liveness.timestamp() != LivenessInfo.NO_TIMESTAMP) - insert(key.getKey(), clustering, null, liveness, ctx); - - if (!deletion.isLive()) - delete(key.getKey(), clustering, deletion.time(), ctx); + CassandraIndex.this.delete(rowKey.getKey(), clustering, cell, ctx, nowInSec); } - private LivenessInfo getPrimaryKeyIndexLiveness(Row row) + @Override + void delete(DecoratedKey rowKey, Clustering clustering, DeletionTime deletion) { - long timestamp = row.primaryKeyLivenessInfo().timestamp(); - int ttl = row.primaryKeyLivenessInfo().ttl(); - for (Cell cell : row.cells()) - { - long cellTimestamp = cell.timestamp(); - if (cell.isLive(nowInSec)) - { - if (cellTimestamp > timestamp) - timestamp = cellTimestamp; - } - } - return LivenessInfo.create(timestamp, ttl, nowInSec); + CassandraIndex.this.delete(rowKey.getKey(), clustering, deletion, ctx); } }; } @@ -507,6 +564,15 @@ public void deleteStaleEntry(DecoratedKey indexKey, logger.trace("Removed index entry for stale value {}", indexKey); } + public IndexEntry createIndexEntry(DecoratedKey rowKey, Clustering clustering, Cell cell, LivenessInfo info) + { + DecoratedKey indexKey = getIndexKeyFor(getIndexedValue(rowKey.getKey(), + clustering, + cell)); + Clustering indexClustering = buildIndexClustering(rowKey.getKey(), clustering, cell); + return new IndexEntry(indexKey, indexClustering, info.timestamp(), rowKey, clustering); + } + /** * Called when adding a new entry to the index */ diff --git a/src/java/org/apache/cassandra/index/internal/CassandraIndexSearcher.java b/src/java/org/apache/cassandra/index/internal/CassandraIndexSearcher.java index 61d446674ec7..ee7b58539bd4 100644 --- a/src/java/org/apache/cassandra/index/internal/CassandraIndexSearcher.java +++ b/src/java/org/apache/cassandra/index/internal/CassandraIndexSearcher.java @@ -21,8 +21,11 @@ package org.apache.cassandra.index.internal; import java.nio.ByteBuffer; +import java.util.Optional; import java.util.SortedSet; +import java.util.function.Consumer; +import com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,9 +35,12 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.DataRange; import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.LivenessInfo; import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.db.ReadCommand; import org.apache.cassandra.db.ReadExecutionController; +import org.apache.cassandra.db.ReadableView; import org.apache.cassandra.db.SinglePartitionReadCommand; import org.apache.cassandra.db.Slice; import org.apache.cassandra.db.Slices; @@ -43,23 +49,109 @@ import org.apache.cassandra.db.filter.ClusteringIndexSliceFilter; import org.apache.cassandra.db.filter.ColumnFilter; import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.lifecycle.SSTableSet; +import org.apache.cassandra.db.lifecycle.View; +import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.rows.Cell; +import org.apache.cassandra.db.rows.Row; import org.apache.cassandra.db.rows.RowIterator; +import org.apache.cassandra.db.rows.Rows; import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.db.rows.UnfilteredRowIterators; +import org.apache.cassandra.db.transform.Transformation; import org.apache.cassandra.dht.AbstractBounds; import org.apache.cassandra.index.Index; import org.apache.cassandra.index.internal.composites.CollectionValueIndex; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.reads.tracked.PartialTrackedIndexRead; +import org.apache.cassandra.service.reads.tracked.PartialTrackedRead; +import org.apache.cassandra.utils.AbstractIterator; +import org.apache.cassandra.utils.CloseablePeekingIterator; import org.apache.cassandra.utils.btree.BTreeSet; -public abstract class CassandraIndexSearcher implements Index.Searcher +public abstract class CassandraIndexSearcher implements Index.MultiStepSearcher { + protected abstract class AbstractMatchIndexer extends CassandraIndex.AbstractIndexer implements Index.MatchIndexer + { + protected DecoratedKey key; + protected Consumer indexTo; + + @Override + long nowInSec() + { + return command.nowInSec(); + } + + @Override + CassandraIndex index() + { + return index; + } + + @Override + DecoratedKey key() + { + return key; + } + + protected abstract M createMatch(DecoratedKey rowKey, Clustering clustering, Cell cell, LivenessInfo info); + + @Override + void insert(DecoratedKey rowKey, Clustering clustering, Cell cell, LivenessInfo info) + { + indexTo.accept(createMatch(rowKey, clustering, cell, info)); + } + + @Override + void delete(DecoratedKey rowKey, Clustering clustering, Cell cell, long nowInSec) + { + + } + + @Override + void delete(DecoratedKey rowKey, Clustering clustering, DeletionTime deletion) + { + + } + + @Override + public void insertRow(Row row) + { + if (!expression.isSatisfiedBy(command.metadata(), key, row, nowInSec())) + return; + super.insertRow(row); + } + + @Override + public void index(PartitionUpdate update, Consumer indexTo) + { + // FIXME: this is messy + this.key = update.partitionKey(); + this.indexTo = indexTo; + + try + { + Row staticRow = update.staticRow(); + if (staticRow != Rows.EMPTY_STATIC_ROW) + insertRow(staticRow); + + update.forEach(this::insertRow); + } + finally + { + this.key = null; + this.indexTo = null; + } + } + } + private static final Logger logger = LoggerFactory.getLogger(CassandraIndexSearcher.class); private final RowFilter.Expression expression; protected final CassandraIndex index; protected final ReadCommand command; + protected final DecoratedKey indexedKey; public CassandraIndexSearcher(ReadCommand command, RowFilter.Expression expression, @@ -68,6 +160,9 @@ public CassandraIndexSearcher(ReadCommand command, this.command = command; this.expression = expression; this.index = index; + Optional backingTable = index.getBackingTable(); + Preconditions.checkState(backingTable.isPresent()); + this.indexedKey = backingTable.get().decorateKey(expression.getIndexValue()); } @Override @@ -76,19 +171,96 @@ public ReadCommand command() return command; } - // of this method. - public UnfilteredPartitionIterator search(ReadExecutionController executionController) + @Override + public PartialTrackedRead beginRead(ReadExecutionController executionController, ColumnFamilyStore cfs, long startTimeNanos) + { + return PartialTrackedIndexRead.create(executionController, cfs, startTimeNanos, command, this); + } + + @Override + public UnfilteredPartitionIterator filterCompletedRead(UnfilteredPartitionIterator iterator) + { + return Transformation.apply(iterator, new Transformation() + { + DecoratedKey key = null; + @Override + protected DecoratedKey applyToPartitionKey(DecoratedKey key) + { + this.key = key; + return super.applyToPartitionKey(key); + } + + @Override + protected Row applyToRow(Row row) + { + if (!expression.isSatisfiedBy(command.metadata(), key, row, command.nowInSec())) + return null; + return row; + } + }); + } + + protected RowIterator queryIndex(DecoratedKey indexKey, ReadExecutionController executionController) { - // the value of the index expression is the partition key in the index table - DecoratedKey indexKey = index.getBackingTable().get().decorateKey(expression.getIndexValue()); UnfilteredRowIterator indexIter = queryIndex(indexKey, command, executionController); + return UnfilteredRowIterators.filter(indexIter, command.nowInSec()); + } + + protected class ResultIterator extends AbstractIterator implements UnfilteredPartitionIterator + { + private final CloseablePeekingIterator matchIterator; + private final ReadExecutionController executionController; + + public ResultIterator(CloseablePeekingIterator matchIterator, ReadExecutionController executionController) + { + this.matchIterator = matchIterator; + this.executionController = executionController; + } + + @Override + protected UnfilteredRowIterator computeNext() + { + while (matchIterator.hasNext()) + { + DecoratedKey key = matchIterator.peek().key(); + ReadableView view = index.baseCfs.select(View.select(SSTableSet.LIVE, key)); + UnfilteredRowIterator partition = queryNextMatches(executionController, key, view, matchIterator); + + if (partition == null) + continue; + + if (!partition.isEmpty()) + return partition; + + partition.close(); + } + return endOfData(); + } + + @Override + public TableMetadata metadata() + { + return command.metadata(); + } + + @Override + public void close() + { + matchIterator.close(); + } + } + + @Override + public UnfilteredPartitionIterator search(ReadExecutionController executionController) + { + CloseablePeekingIterator matchIterator = matchIterator(executionController); try { - return queryDataFromIndex(indexKey, UnfilteredRowIterators.filter(indexIter, command.nowInSec()), command, executionController); + return new ResultIterator(matchIterator, executionController); } - catch (RuntimeException | Error e) + catch (Throwable e) { - indexIter.close(); + matchIterator.close(); throw e; } } diff --git a/src/java/org/apache/cassandra/index/internal/IndexEntry.java b/src/java/org/apache/cassandra/index/internal/IndexEntry.java index c8e9955d57be..b641a3fa2625 100644 --- a/src/java/org/apache/cassandra/index/internal/IndexEntry.java +++ b/src/java/org/apache/cassandra/index/internal/IndexEntry.java @@ -20,29 +20,29 @@ */ package org.apache.cassandra.index.internal; -import java.nio.ByteBuffer; - import org.apache.cassandra.db.Clustering; import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.schema.TableMetadata; /** * Entries in indexes on non-compact tables (tables with composite comparators) * can be encapsulated as IndexedEntry instances. These are not used when dealing * with indexes on static/compact tables (i.e. KEYS indexes). */ -public final class IndexEntry +public final class IndexEntry implements Index.IndexMatch { public final DecoratedKey indexValue; public final Clustering indexClustering; public final long timestamp; - public final ByteBuffer indexedKey; + public final DecoratedKey indexedKey; public final Clustering indexedEntryClustering; public IndexEntry(DecoratedKey indexValue, Clustering indexClustering, long timestamp, - ByteBuffer indexedKey, + DecoratedKey indexedKey, Clustering indexedEntryClustering) { this.indexValue = indexValue; @@ -51,4 +51,31 @@ public IndexEntry(DecoratedKey indexValue, this.indexedKey = indexedKey; this.indexedEntryClustering = indexedEntryClustering; } + + @Override + public DecoratedKey key() + { + return indexedKey; + } + + public static int compare(TableMetadata indexMetadata, TableMetadata baseMetadata, IndexEntry left, IndexEntry right) + { + int cmp = left.indexValue.compareTo(right.indexValue); + if (cmp != 0) + return cmp; + + cmp = indexMetadata.comparator.compare(left.indexClustering, right.indexClustering); + if (cmp != 0) + return cmp; + + cmp = left.indexedKey.compareTo(right.indexedKey); + if (cmp != 0) + return cmp; + + cmp = baseMetadata.comparator.compare(left.indexedEntryClustering, right.indexedEntryClustering); + if (cmp != 0) + return cmp; + + return Long.compare(left.timestamp, right.timestamp); + } } diff --git a/src/java/org/apache/cassandra/index/internal/composites/ClusteringColumnIndex.java b/src/java/org/apache/cassandra/index/internal/composites/ClusteringColumnIndex.java index 23cff3f8468c..e600c5aeef2e 100644 --- a/src/java/org/apache/cassandra/index/internal/composites/ClusteringColumnIndex.java +++ b/src/java/org/apache/cassandra/index/internal/composites/ClusteringColumnIndex.java @@ -94,7 +94,7 @@ public IndexEntry decodeEntry(DecoratedKey indexedValue, return new IndexEntry(indexedValue, clustering, indexEntry.primaryKeyLivenessInfo().timestamp(), - clustering.bufferAt(0), + baseCfs.decorateKey(clustering.bufferAt(0)), builder.build()); } diff --git a/src/java/org/apache/cassandra/index/internal/composites/CollectionKeyIndexBase.java b/src/java/org/apache/cassandra/index/internal/composites/CollectionKeyIndexBase.java index f0201e1effec..1ca5da256f73 100644 --- a/src/java/org/apache/cassandra/index/internal/composites/CollectionKeyIndexBase.java +++ b/src/java/org/apache/cassandra/index/internal/composites/CollectionKeyIndexBase.java @@ -83,7 +83,7 @@ public IndexEntry decodeEntry(DecoratedKey indexedValue, return new IndexEntry(indexedValue, clustering, indexEntry.primaryKeyLivenessInfo().timestamp(), - clustering.bufferAt(0), + baseCfs.decorateKey(clustering.bufferAt(0)), indexedEntryClustering); } } diff --git a/src/java/org/apache/cassandra/index/internal/composites/CollectionValueIndex.java b/src/java/org/apache/cassandra/index/internal/composites/CollectionValueIndex.java index ed929f22b3c5..5768a5ce3caf 100644 --- a/src/java/org/apache/cassandra/index/internal/composites/CollectionValueIndex.java +++ b/src/java/org/apache/cassandra/index/internal/composites/CollectionValueIndex.java @@ -90,7 +90,7 @@ public IndexEntry decodeEntry(DecoratedKey indexedValue, Row indexEntry) return new IndexEntry(indexedValue, clustering, indexEntry.primaryKeyLivenessInfo().timestamp(), - clustering.bufferAt(0), + baseCfs.decorateKey(clustering.bufferAt(0)), indexedEntryClustering); } diff --git a/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java b/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java index 6629c0625640..3c5c13f9f491 100644 --- a/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java +++ b/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java @@ -19,20 +19,27 @@ import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Comparator; import java.util.List; +import com.google.common.base.Preconditions; +import com.google.common.collect.PeekingIterator; + import org.apache.cassandra.db.Clustering; import org.apache.cassandra.db.ClusteringComparator; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.LivenessInfo; import org.apache.cassandra.db.ReadCommand; import org.apache.cassandra.db.ReadExecutionController; +import org.apache.cassandra.db.ReadableView; import org.apache.cassandra.db.SinglePartitionReadCommand; import org.apache.cassandra.db.WriteContext; import org.apache.cassandra.db.filter.ClusteringIndexNamesFilter; import org.apache.cassandra.db.filter.DataLimits; import org.apache.cassandra.db.filter.RowFilter; import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.rows.Cell; import org.apache.cassandra.db.rows.Row; import org.apache.cassandra.db.rows.RowIterator; import org.apache.cassandra.db.rows.Rows; @@ -44,10 +51,12 @@ import org.apache.cassandra.index.internal.CassandraIndexSearcher; import org.apache.cassandra.index.internal.IndexEntry; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.AbstractIterator; +import org.apache.cassandra.utils.CloseablePeekingIterator; import org.apache.cassandra.utils.btree.BTreeSet; -public class CompositesSearcher extends CassandraIndexSearcher +public class CompositesSearcher extends CassandraIndexSearcher { public CompositesSearcher(ReadCommand command, RowFilter.Expression expression, @@ -56,6 +65,25 @@ public CompositesSearcher(ReadCommand command, super(command, expression, index); } + @Override + public Index.MatchIndexer matchIndexer() + { + return new AbstractMatchIndexer() + { + @Override + protected IndexEntry createMatch(DecoratedKey rowKey, Clustering clustering, Cell cell, LivenessInfo info) + { + return index.createIndexEntry(rowKey, clustering, cell, info); + } + }; + } + + @Override + public Comparator matchComparator() + { + return (left, right) -> IndexEntry.compare(index.getIndexCfs().metadata(), command.metadata(), left, right); + } + private boolean isMatchingEntry(DecoratedKey partitionKey, IndexEntry entry, ReadCommand command) { return command.selectsKey(partitionKey) && command.selectsClustering(partitionKey, entry.indexedEntryClustering); @@ -66,6 +94,107 @@ private boolean isStaticColumn() return index.getIndexedColumn().isStatic(); } + @Override + public CloseablePeekingIterator matchIterator(ReadExecutionController executionController) + { + RowIterator indexHits = queryIndex(indexedKey, executionController); + try + { + Preconditions.checkState(indexHits.staticRow() == Rows.EMPTY_STATIC_ROW); + return new AbstractIterator() + { + @Override + protected IndexEntry computeNext() + { + while (indexHits.hasNext()) + { + IndexEntry nextEntry = index.decodeEntry(indexedKey, indexHits.next()); + DecoratedKey partitionKey = nextEntry.indexedKey; + if (!isMatchingEntry(partitionKey, nextEntry, command)) + continue; + + return nextEntry; + } + return endOfData(); + } + + @Override + public void close() + { + if (indexHits != null) + indexHits.close(); + } + }; + } + catch (Throwable e) + { + if (indexHits != null) + indexHits.close(); + throw e; + } + } + + @Override + public UnfilteredRowIterator queryNextMatches(ReadExecutionController executionController, DecoratedKey partitionKey, ReadableView view, PeekingIterator matches) + { + Preconditions.checkArgument(matches.hasNext()); + SinglePartitionReadCommand dataCmd; + List entries = new ArrayList<>(); + if (isStaticColumn()) + { + + // If the index is on a static column, we just need to do a full read on the partition. + // Note that we want to re-use the command.columnFilter() in case of future change. + dataCmd = SinglePartitionReadCommand.create(index.baseCfs.metadata(), + command.nowInSec(), + command.columnFilter(), + RowFilter.none(), + DataLimits.NONE, + partitionKey, + command.clusteringIndexFilter(partitionKey)); + entries.add(matches.next()); + } + else + { + // Gather all index hits belonging to the same partition and query the data for those hits. + // TODO: it's much more efficient to do 1 read for all hits to the same partition than doing + // 1 read per index hit. However, this basically mean materializing all hits for a partition + // in memory so we should consider adding some paging mechanism. However, index hits should + // be relatively small so it's much better than the previous code that was materializing all + // *data* for a given partition. + BTreeSet.Builder> clusterings = BTreeSet.builder(index.baseCfs.getComparator()); + while (matches.hasNext() && partitionKey.equals(matches.peek().indexedKey)) + { + // We're queried a slice of the index, and some hits may not match some of the clustering column constraints, + // but they will have been filtered out upstream + IndexEntry nextEntry = matches.next(); + clusterings.add(nextEntry.indexedEntryClustering); + entries.add(nextEntry); + } + + // since non-matching entries will have been filtered out by matchIterator, it should not be possible to have empty clusterings + Preconditions.checkArgument(!clusterings.isEmpty()); + + // Query the gathered index hits. We still need to filter stale hits from the resulting query. + ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(clusterings.build(), false); + dataCmd = SinglePartitionReadCommand.create(index.baseCfs.metadata(), + command.nowInSec(), + command.columnFilter(), + command.rowFilter(), + DataLimits.NONE, + partitionKey, + filter, + (Index.QueryPlan) null); + } + + // by the next caller of next, or through closing this iterator is this come before. + return filterStaleEntries(dataCmd.queryMemtableAndDisk(view, index.baseCfs, executionController), + indexedKey.getKey(), + entries, + executionController.getWriteContext(), + command.nowInSec()); + } + protected UnfilteredPartitionIterator queryDataFromIndex(final DecoratedKey indexKey, final RowIterator indexHits, final ReadCommand command, @@ -115,12 +244,13 @@ private boolean prepareNext() } SinglePartitionReadCommand dataCmd; - DecoratedKey partitionKey = index.baseCfs.decorateKey(nextEntry.indexedKey); + DecoratedKey partitionKey = nextEntry.indexedKey; List entries = new ArrayList<>(); if (isStaticColumn()) { // The index hit may not match the commad key constraint - if (!isMatchingEntry(partitionKey, nextEntry, command)) { + if (!isMatchingEntry(partitionKey, nextEntry, command)) + { nextEntry = indexHits.hasNext() ? index.decodeEntry(indexKey, indexHits.next()) : null; continue; } @@ -146,7 +276,7 @@ private boolean prepareNext() // be relatively small so it's much better than the previous code that was materializing all // *data* for a given partition. BTreeSet.Builder> clusterings = BTreeSet.builder(index.baseCfs.getComparator()); - while (nextEntry != null && partitionKey.getKey().equals(nextEntry.indexedKey)) + while (nextEntry != null && partitionKey.equals(nextEntry.indexedKey)) { // We're queried a slice of the index, but some hits may not match some of the clustering column constraints if (isMatchingEntry(partitionKey, nextEntry, command)) @@ -297,8 +427,8 @@ private IndexEntry findEntry(Clustering clustering) // those tables do not support static columns. By consequence if a table // has some static columns and all its clustering key elements are null // it means that the partition exists and contains only static data - if (!dataIter.metadata().hasStaticColumns() || !containsOnlyNullValues(indexedEntryClustering)) - staleEntries.add(entry); + if (!dataIter.metadata().hasStaticColumns() || !containsOnlyNullValues(indexedEntryClustering)) + staleEntries.add(entry); } // entries correspond to the rows we've queried, so we shouldn't have a row that has no corresponding entry. throw new AssertionError(); diff --git a/src/java/org/apache/cassandra/index/internal/composites/PartitionKeyIndex.java b/src/java/org/apache/cassandra/index/internal/composites/PartitionKeyIndex.java index b8235370b70c..dd862837b8cd 100644 --- a/src/java/org/apache/cassandra/index/internal/composites/PartitionKeyIndex.java +++ b/src/java/org/apache/cassandra/index/internal/composites/PartitionKeyIndex.java @@ -86,7 +86,7 @@ public IndexEntry decodeEntry(DecoratedKey indexedValue, Row indexEntry) return new IndexEntry(indexedValue, clustering, indexEntry.primaryKeyLivenessInfo().timestamp(), - clustering.bufferAt(0), + baseCfs.decorateKey(clustering.bufferAt(0)), builder.build()); } diff --git a/src/java/org/apache/cassandra/index/internal/composites/RegularColumnIndex.java b/src/java/org/apache/cassandra/index/internal/composites/RegularColumnIndex.java index 9dcb8dfcc4d9..384b37532189 100644 --- a/src/java/org/apache/cassandra/index/internal/composites/RegularColumnIndex.java +++ b/src/java/org/apache/cassandra/index/internal/composites/RegularColumnIndex.java @@ -96,7 +96,7 @@ public IndexEntry decodeEntry(DecoratedKey indexedValue, Row indexEntry) return new IndexEntry(indexedValue, clustering, indexEntry.primaryKeyLivenessInfo().timestamp(), - clustering.bufferAt(0), + baseCfs.decorateKey(clustering.bufferAt(0)), indexedEntryClustering); } diff --git a/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java b/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java index 98e4e0216040..8370dbf42e09 100644 --- a/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java +++ b/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java @@ -18,21 +18,25 @@ package org.apache.cassandra.index.internal.keys; import java.nio.ByteBuffer; +import java.util.Comparator; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import com.google.common.base.Preconditions; +import com.google.common.collect.PeekingIterator; import org.apache.cassandra.db.Clustering; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.LivenessInfo; import org.apache.cassandra.db.ReadCommand; import org.apache.cassandra.db.ReadExecutionController; +import org.apache.cassandra.db.ReadableView; import org.apache.cassandra.db.SinglePartitionReadCommand; import org.apache.cassandra.db.WriteContext; import org.apache.cassandra.db.filter.ColumnFilter; import org.apache.cassandra.db.filter.DataLimits; import org.apache.cassandra.db.filter.RowFilter; import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.rows.Cell; import org.apache.cassandra.db.rows.Row; import org.apache.cassandra.db.rows.RowIterator; import org.apache.cassandra.db.rows.Rows; @@ -40,12 +44,13 @@ import org.apache.cassandra.index.Index; import org.apache.cassandra.index.internal.CassandraIndex; import org.apache.cassandra.index.internal.CassandraIndexSearcher; +import org.apache.cassandra.index.internal.IndexEntry; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.AbstractIterator; +import org.apache.cassandra.utils.CloseablePeekingIterator; -public class KeysSearcher extends CassandraIndexSearcher +public class KeysSearcher extends CassandraIndexSearcher { - private static final Logger logger = LoggerFactory.getLogger(KeysSearcher.class); - public KeysSearcher(ReadCommand command, RowFilter.Expression expression, CassandraIndex indexer) @@ -53,6 +58,91 @@ public KeysSearcher(ReadCommand command, super(command, expression, indexer); } + @Override + public Index.MatchIndexer matchIndexer() + { + return new AbstractMatchIndexer() + { + @Override + protected IndexEntry createMatch(DecoratedKey rowKey, Clustering clustering, Cell cell, LivenessInfo info) + { + return index.createIndexEntry(rowKey, clustering, cell, info); + } + }; + } + + @Override + public Comparator matchComparator() + { + return (left, right) -> IndexEntry.compare(index.getIndexCfs().metadata(), command.metadata(), left, right); + } + + @Override + public CloseablePeekingIterator matchIterator(ReadExecutionController executionController) + { + RowIterator indexHits = queryIndex(indexedKey, executionController); + try + { + Preconditions.checkState(indexHits.staticRow() == Rows.EMPTY_STATIC_ROW); + return new AbstractIterator() + { + @Override + protected IndexEntry computeNext() + { + while (indexHits.hasNext()) + { + Row hit = indexHits.next(); + DecoratedKey key = index.baseCfs.decorateKey(hit.clustering().bufferAt(0)); + if (!command.selectsKey(key)) + continue; + + return new IndexEntry(indexedKey, hit.clustering(), hit.primaryKeyLivenessInfo().timestamp(), key, Clustering.EMPTY); + } + return endOfData(); + } + + @Override + public void close() + { + if (indexHits != null) + indexHits.close(); + } + }; + + } + catch (Throwable e) + { + if (indexHits != null) + indexHits.close(); + throw e; + } + } + + @Override + public UnfilteredRowIterator queryNextMatches(ReadExecutionController executionController, DecoratedKey key, ReadableView view, PeekingIterator matches) + { + Preconditions.checkArgument(matches.hasNext()); + + IndexEntry entry = matches.next(); + + ColumnFilter extendedFilter = getExtendedFilter(command.columnFilter()); + SinglePartitionReadCommand dataCmd = SinglePartitionReadCommand.create(index.baseCfs.metadata(), + command.nowInSec(), + extendedFilter, + command.rowFilter(), + DataLimits.NONE, + key, + command.clusteringIndexFilter(key)); + + // Otherwise, we close right away if empty, and if it's assigned to next it will be called either + // by the next caller of next, or through closing this iterator is this come before. + return filterIfStale(dataCmd.queryMemtableAndDisk(index.baseCfs, executionController), + entry.timestamp, + indexedKey.getKey(), + executionController.getWriteContext(), + command.nowInSec()); + } + protected UnfilteredPartitionIterator queryDataFromIndex(final DecoratedKey indexKey, final RowIterator indexHits, final ReadCommand command, @@ -103,10 +193,10 @@ private boolean prepareNext() command.clusteringIndexFilter(key), (Index.QueryPlan) null); - // Otherwise, we close right away if empty, and if it's assigned to next it will be called either - // by the next caller of next, or through closing this iterator is this come before. + // Otherwise, we close right away if empty, and if it's assigned to next it will be called either + // by the next caller of next, or through closing this iterator is this come before. UnfilteredRowIterator dataIter = filterIfStale(dataCmd.queryMemtableAndDisk(index.baseCfs, executionController), - hit, + hit.primaryKeyLivenessInfo().timestamp(), indexKey.getKey(), executionController.getWriteContext(), command.nowInSec()); @@ -148,7 +238,7 @@ private ColumnFilter getExtendedFilter(ColumnFilter initialFilter) } private UnfilteredRowIterator filterIfStale(UnfilteredRowIterator iterator, - Row indexHit, + long timestamp, ByteBuffer indexedValue, WriteContext ctx, long nowInSec) @@ -159,7 +249,7 @@ private UnfilteredRowIterator filterIfStale(UnfilteredRowIterator iterator, // Index is stale, remove the index entry and ignore index.deleteStaleEntry(index.getIndexCfs().decorateKey(indexedValue), makeIndexClustering(iterator.partitionKey().getKey(), Clustering.EMPTY), - DeletionTime.build(indexHit.primaryKeyLivenessInfo().timestamp(), nowInSec), + DeletionTime.build(timestamp, nowInSec), ctx); iterator.close(); return null; diff --git a/src/java/org/apache/cassandra/index/sai/plan/QueryController.java b/src/java/org/apache/cassandra/index/sai/plan/QueryController.java index 49e2aeab6d88..1964f1df91d9 100644 --- a/src/java/org/apache/cassandra/index/sai/plan/QueryController.java +++ b/src/java/org/apache/cassandra/index/sai/plan/QueryController.java @@ -42,6 +42,7 @@ import org.apache.cassandra.db.PartitionRangeReadCommand; import org.apache.cassandra.db.ReadCommand; import org.apache.cassandra.db.ReadExecutionController; +import org.apache.cassandra.db.ReadableView; import org.apache.cassandra.db.SinglePartitionReadCommand; import org.apache.cassandra.db.filter.ClusteringIndexFilter; import org.apache.cassandra.db.filter.ClusteringIndexNamesFilter; @@ -156,7 +157,7 @@ public boolean hasAnalyzer(RowFilter.Expression expression) return index != null && index.hasAnalyzer(); } - public UnfilteredRowIterator queryStorage(List keys, ReadExecutionController executionController) + public UnfilteredRowIterator queryStorage(ReadableView view, List keys, ReadExecutionController executionController) { if (keys.isEmpty()) throw new IllegalArgumentException("At least one primary key is required!"); @@ -169,7 +170,7 @@ public UnfilteredRowIterator queryStorage(List keys, ReadExecutionCo keys.get(0).partitionKey(), makeFilter(keys)); - return partition.queryMemtableAndDisk(cfs, executionController); + return partition.queryMemtableAndDisk(view, cfs, executionController); } private static Runnable getIndexReleaser(Set referencedIndexes) @@ -260,7 +261,10 @@ public KeyRangeIterator.Builder getIndexQueryResults(Collection expr if (unrepairedIterator.getMaxKeys() > 0) { builder.add(unrepairedIterator); - queryContext.hasUnrepairedMatches = true; + + // mutation tracking internally repairs all partitions ranges/keys as part of a read + if (!command.metadata().replicationType().isTracked()) + queryContext.hasUnrepairedMatches = true; } else { diff --git a/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java b/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java index 20a9cad58c45..a347bb20fc0c 100644 --- a/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java +++ b/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java @@ -21,16 +21,21 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; import java.util.function.Supplier; import javax.annotation.Nonnull; import javax.annotation.Nullable; +import com.google.common.base.Preconditions; +import com.google.common.collect.PeekingIterator; + import io.netty.util.concurrent.FastThreadLocal; import org.apache.cassandra.db.Clustering; import org.apache.cassandra.db.ClusteringBound; @@ -41,19 +46,24 @@ import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.db.ReadCommand; import org.apache.cassandra.db.ReadExecutionController; +import org.apache.cassandra.db.ReadableView; import org.apache.cassandra.db.RegularAndStaticColumns; import org.apache.cassandra.db.Slices; import org.apache.cassandra.db.filter.ClusteringIndexFilter; import org.apache.cassandra.db.filter.ClusteringIndexNamesFilter; import org.apache.cassandra.db.filter.ClusteringIndexSliceFilter; import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.lifecycle.SSTableSet; +import org.apache.cassandra.db.lifecycle.View; import org.apache.cassandra.db.partitions.PartitionIterator; +import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; import org.apache.cassandra.db.rows.AbstractUnfilteredRowIterator; import org.apache.cassandra.db.rows.Row; import org.apache.cassandra.db.rows.RowIterator; import org.apache.cassandra.db.rows.Unfiltered; import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.transform.Transformation; import org.apache.cassandra.dht.AbstractBounds; import org.apache.cassandra.dht.Token; import org.apache.cassandra.exceptions.RequestTimeoutException; @@ -64,17 +74,25 @@ import org.apache.cassandra.index.sai.utils.PrimaryKey; import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.reads.tracked.PartialTrackedIndexRead; +import org.apache.cassandra.service.reads.tracked.PartialTrackedRead; import org.apache.cassandra.utils.AbstractIterator; import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.utils.CloseablePeekingIterator; -public class StorageAttachedIndexSearcher implements Index.Searcher +public class StorageAttachedIndexSearcher implements Index.MultiStepSearcher { private static final int PARTITION_ROW_BATCH_SIZE = 100; + private final ColumnFamilyStore cfs; private final ReadCommand command; private final QueryController queryController; private final QueryContext queryContext; private final TableQueryMetrics tableQueryMetrics; + private final int partitionRowBatchSize; + private final FilterTree filterTree; + private final PrimaryKey.Factory keyFactory; + private final boolean topK; private static final FastThreadLocal> nextKeys = new FastThreadLocal<>() { @@ -91,10 +109,15 @@ public StorageAttachedIndexSearcher(ColumnFamilyStore cfs, RowFilter indexFilter, long executionQuotaMs) { + this.cfs = cfs; this.command = command; this.queryContext = new QueryContext(command, executionQuotaMs); this.queryController = new QueryController(cfs, command, indexFilter, queryContext); this.tableQueryMetrics = tableQueryMetrics; + this.partitionRowBatchSize = Math.min(PARTITION_ROW_BATCH_SIZE, command.limits().count()); + this.filterTree = Operation.buildFilter(queryController, queryController.usesStrictFiltering()); + this.keyFactory = queryController.primaryKeyFactory(); + this.topK = command.isTopK(); } @Override @@ -113,34 +136,131 @@ public PartitionIterator filterReplicaFilteringProtection(PartitionIterator full } // if no analyzer does transformation - return Index.Searcher.super.filterReplicaFilteringProtection(fullResponse); + return Index.MultiStepSearcher.super.filterReplicaFilteringProtection(fullResponse); + } + + protected class ResultIterator extends AbstractIterator implements UnfilteredPartitionIterator + { + private final CloseablePeekingIterator matchIterator; + private final ReadExecutionController executionController; + + public ResultIterator(CloseablePeekingIterator matchIterator, ReadExecutionController executionController) + { + this.matchIterator = matchIterator; + this.executionController = executionController; + } + + @Override + protected UnfilteredRowIterator computeNext() + { + while (matchIterator.hasNext()) + { + DecoratedKey key = matchIterator.peek().partitionKey(); + ReadableView view = cfs.select(View.select(SSTableSet.LIVE, key)); + UnfilteredRowIterator partition = queryNextMatches(executionController, key, view, matchIterator); + + if (partition == null) + continue; + + if (!partition.isEmpty()) + return partition; + + partition.close(); + } + return endOfData(); + } + + @Override + public TableMetadata metadata() + { + return command.metadata(); + } + + @Override + public void close() + { + matchIterator.close(); + } + } + + private ResultIterator searchInternal(ReadExecutionController executionController) + { + CloseablePeekingIterator matchIterator = matchIterator(executionController); + try + { + return new ResultIterator(matchIterator, executionController); + } + catch (Throwable e) + { + matchIterator.close(); + throw e; + } } @Override public UnfilteredPartitionIterator search(ReadExecutionController executionController) throws RequestTimeoutException { if (!command.isTopK()) - return new ResultRetriever(executionController, false); - else { - Supplier resultSupplier = () -> new ResultRetriever(executionController, true); + return searchInternal(executionController); + } - // VSTODO performance: if there is shadowed primary keys, we have to at least query twice. - // First time to find out there are shadow keys, second time to find out there are no more shadow keys. - while (true) + Supplier resultSupplier = () -> searchInternal(executionController); + // VSTODO performance: if there is shadowed primary keys, we have to at least query twice. + // First time to find out there are shadow keys, second time to find out there are no more shadow keys. + while (true) + { + long lastShadowedKeysCount = queryContext.vectorContext().getShadowedPrimaryKeys().size(); + ResultIterator result = resultSupplier.get(); + UnfilteredPartitionIterator topK = (UnfilteredPartitionIterator) new VectorTopKProcessor(command).filter(result); + + long currentShadowedKeysCount = queryContext.vectorContext().getShadowedPrimaryKeys().size(); + if (lastShadowedKeysCount == currentShadowedKeysCount) + return topK; + } + } + + private class MatchIndexer implements Index.MatchIndexer + { + private final PrimaryKey.Factory keyFactory; + + public MatchIndexer() + { + this.keyFactory = queryController.primaryKeyFactory(); + } + + @Override + public void index(PartitionUpdate update, Consumer indexTo) + { + DecoratedKey key = update.partitionKey(); + Row staticRow = update.staticRow(); + + boolean hasClustering = cfs.getComparator().size() > 0; + + if (!filterTree.restrictsNonStaticRow()) { - long lastShadowedKeysCount = queryContext.vectorContext().getShadowedPrimaryKeys().size(); - ResultRetriever result = resultSupplier.get(); - UnfilteredPartitionIterator topK = (UnfilteredPartitionIterator) new VectorTopKProcessor(command).filter(result); + if (filterTree.isSatisfiedBy(key, staticRow, staticRow)) + { + indexTo.accept(keyFactory.create(key)); + } - long currentShadowedKeysCount = queryContext.vectorContext().getShadowedPrimaryKeys().size(); - if (lastShadowedKeysCount == currentShadowedKeysCount) - return topK; + } + else + { + for (Row row : update) + { + if (filterTree.isSatisfiedBy(key, row, staticRow)) + { + PrimaryKey primaryKey = hasClustering ? keyFactory.create(key, row.clustering()) + : keyFactory.create(key); + indexTo.accept(primaryKey); + } + } } } } - private class ResultRetriever extends AbstractIterator implements UnfilteredPartitionIterator + private class MatchIterator extends AbstractIterator { private final PrimaryKey firstPrimaryKey; private final PrimaryKey lastPrimaryKey; @@ -149,162 +269,91 @@ private class ResultRetriever extends AbstractIterator im private AbstractBounds currentKeyRange; private final KeyRangeIterator resultKeyIterator; - private final FilterTree filterTree; - private final ReadExecutionController executionController; - private final PrimaryKey.Factory keyFactory; - private final boolean topK; - private final int partitionRowBatchSize; private PrimaryKey lastKey; - private ResultRetriever(ReadExecutionController executionController, boolean topK) + private MatchIterator() { this.keyRanges = queryController.dataRanges().iterator(); this.firstDataRange = keyRanges.next(); this.currentKeyRange = firstDataRange.keyRange(); this.resultKeyIterator = Operation.buildIterator(queryController); - this.filterTree = Operation.buildFilter(queryController, queryController.usesStrictFiltering()); - this.executionController = executionController; - this.keyFactory = queryController.primaryKeyFactory(); this.firstPrimaryKey = queryController.firstPrimaryKeyInRange(); this.lastPrimaryKey = queryController.lastPrimaryKeyInRange(); - this.topK = topK; - - // Ensure we don't fetch larger batches than the provided LIMIT to avoid fetching keys we won't use: - this.partitionRowBatchSize = Math.min(PARTITION_ROW_BATCH_SIZE, command.limits().count()); } @Override - public UnfilteredRowIterator computeNext() + public PrimaryKey computeNext() { - if (resultKeyIterator == null) - return endOfData(); - - // If being called for the first time, skip to the beginning of the range. - // We can't put this code in the constructor because it may throw and the caller - // may not be prepared for that. - if (lastKey == null) + while (true) { - PrimaryKey skipTarget = firstPrimaryKey; - ClusteringComparator comparator = command.metadata().comparator; + if (resultKeyIterator == null) + return endOfData(); - // If there are no clusterings, the first data range selects an entire partitions, or we have static - // expressions, don't bother trying to skip forward within the partition. - if (comparator.size() > 0 && !firstDataRange.selectsAllPartition() && !command.rowFilter().hasStaticExpression()) + // If being called for the first time, skip to the beginning of the range. + // We can't put this code in the constructor because it may throw and the caller + // may not be prepared for that. + if (lastKey == null) { - // Only attempt to skip if the first data range covers a single partition. - if (currentKeyRange.left.equals(currentKeyRange.right) && currentKeyRange.left instanceof DecoratedKey) - { - DecoratedKey decoratedKey = (DecoratedKey) currentKeyRange.left; - ClusteringIndexFilter filter = firstDataRange.clusteringIndexFilter(decoratedKey); + PrimaryKey skipTarget = firstPrimaryKey; + ClusteringComparator comparator = command.metadata().comparator; - if (filter instanceof ClusteringIndexSliceFilter) + // If there are no clusterings, the first data range selects an entire partitions, or we have static + // expressions, don't bother trying to skip forward within the partition. + if (comparator.size() > 0 && !firstDataRange.selectsAllPartition() && !command.rowFilter().hasStaticExpression()) + { + // Only attempt to skip if the first data range covers a single partition. + if (currentKeyRange.left.equals(currentKeyRange.right) && currentKeyRange.left instanceof DecoratedKey) { - Slices slices = ((ClusteringIndexSliceFilter) filter).requestedSlices(); + DecoratedKey decoratedKey = (DecoratedKey) currentKeyRange.left; + ClusteringIndexFilter filter = firstDataRange.clusteringIndexFilter(decoratedKey); - if (!slices.isEmpty()) + if (filter instanceof ClusteringIndexSliceFilter) { - ClusteringBound startBound = slices.get(0).start(); + Slices slices = ((ClusteringIndexSliceFilter) filter).requestedSlices(); - if (!startBound.isEmpty()) + if (!slices.isEmpty()) { - ByteBuffer[] rawValues = startBound.getBufferArray(); + ClusteringBound startBound = slices.get(0).start(); - if (rawValues.length == comparator.size()) - skipTarget = keyFactory.create(decoratedKey, Clustering.make(rawValues)); + if (!startBound.isEmpty()) + { + ByteBuffer[] rawValues = startBound.getBufferArray(); + + if (rawValues.length == comparator.size()) + skipTarget = keyFactory.create(decoratedKey, Clustering.make(rawValues)); + } } } - } - else if (filter instanceof ClusteringIndexNamesFilter) - { - ClusteringIndexNamesFilter namesFilter = (ClusteringIndexNamesFilter) filter; - - if (!namesFilter.requestedRows().isEmpty()) + else if (filter instanceof ClusteringIndexNamesFilter) { - Clustering skipClustering = namesFilter.requestedRows().iterator().next(); - skipTarget = keyFactory.create(decoratedKey, skipClustering); + ClusteringIndexNamesFilter namesFilter = (ClusteringIndexNamesFilter) filter; + + if (!namesFilter.requestedRows().isEmpty()) + { + Clustering skipClustering = namesFilter.requestedRows().iterator().next(); + skipTarget = keyFactory.create(decoratedKey, skipClustering); + } } } } - } - resultKeyIterator.skipTo(skipTarget); - } + resultKeyIterator.skipTo(skipTarget); + } - // Theoretically we wouldn't need this if the caller of computeNext always ran the - // returned iterators to the completion. Unfortunately, we have no control over the caller behavior here. - // Hence, we skip to the next partition in order to comply to the unwritten partition iterator contract - // saying this iterator must not return the same partition twice. - skipToNextPartition(); + PrimaryKey nextKey = nextKeyInRange(); + if (nextKey == null) + return endOfData(); - UnfilteredRowIterator iterator = nextRowIterator(this::nextSelectedKeysInRange); - return iterator != null ? iteratePartition(iterator) : endOfData(); - } + if (queryController.doesNotSelect(nextKey) || nextKey.equals(lastKey, false)) + continue; - /** - * Tries to obtain a row iterator for the supplied keys by repeatedly calling - * {@link ResultRetriever#queryStorageAndFilter} until it gives a non-null result. - * The keysSupplier should return the next batch of keys with every call to get() - * and null when there are no more keys to try. - * - * @return an iterator or null if all keys were tried with no success - */ - private @Nullable UnfilteredRowIterator nextRowIterator(@Nonnull Supplier> keysSupplier) - { - UnfilteredRowIterator iterator = null; - while (iterator == null) - { - List keys = keysSupplier.get(); - if (keys.isEmpty()) - return null; - iterator = queryStorageAndFilter(keys); + lastKey = nextKey; + return nextKey; } - return iterator; } - /** - * Retrieves the next batch of primary keys (i.e. up to {@link #partitionRowBatchSize} of them) that are - * contained by one of the query key ranges and selected by the {@link QueryController}. If the next key falls - * out of the current key range, it skips to the next key range, and so on. If no more keys accepted by - * the controller are available, and empty list is returned. - * - * @return a list of up to {@link #partitionRowBatchSize} primary keys - */ - private List nextSelectedKeysInRange() - { - List threadLocalNextKeys = nextKeys.get(); - threadLocalNextKeys.clear(); - PrimaryKey firstKey; - - do - { - firstKey = nextKeyInRange(); - if (firstKey == null) - return Collections.emptyList(); - } - while (queryController.doesNotSelect(firstKey) || firstKey.equals(lastKey, false)); - - lastKey = firstKey; - threadLocalNextKeys.add(firstKey); - fillNextSelectedKeysInPartition(firstKey.partitionKey(), threadLocalNextKeys); - return threadLocalNextKeys; - } - - /** - * Retrieves the next batch of primary keys (i.e. up to {@link #partitionRowBatchSize} of them) that belong to - * the given partition and are selected by the query controller, advancing the underlying iterator only while - * the next key belongs to that partition. - * - * @return a list of up to {@link #partitionRowBatchSize} primary keys within the given partition - */ - private List nextSelectedKeysInPartition(DecoratedKey partitionKey) - { - List threadLocalNextKeys = nextKeys.get(); - threadLocalNextKeys.clear(); - fillNextSelectedKeysInPartition(partitionKey, threadLocalNextKeys); - return threadLocalNextKeys; - } /** * Returns the next available key contained by one of the keyRanges. @@ -334,25 +383,6 @@ private List nextSelectedKeysInPartition(DecoratedKey partitionKey) return key; } - private void fillNextSelectedKeysInPartition(DecoratedKey partitionKey, List nextPrimaryKeys) - { - while (resultKeyIterator.hasNext() - && resultKeyIterator.peek().partitionKey().equals(partitionKey) - && nextPrimaryKeys.size() < partitionRowBatchSize) - { - PrimaryKey key = nextKey(); - - if (key == null) - break; - - if (queryController.doesNotSelect(key) || key.equals(lastKey, false)) - continue; - - nextPrimaryKeys.add(key); - lastKey = key; - } - } - /** * Gets the next key from the underlying operation. * Returns null if there are no more keys <= lastPrimaryKey. @@ -389,189 +419,164 @@ private void skipTo(@Nonnull Token token) resultKeyIterator.skipTo(keyFactory.create(token)); } - /** - * Skips to the key that belongs to a different partition than the last key we fetched. - */ - private void skipToNextPartition() + @Override + public void close() { - if (lastKey == null) - return; - DecoratedKey lastPartitionKey = lastKey.partitionKey(); - while (resultKeyIterator.hasNext() && resultKeyIterator.peek().partitionKey().equals(lastPartitionKey)) - resultKeyIterator.next(); + FileUtils.closeQuietly(resultKeyIterator); + if (tableQueryMetrics != null) tableQueryMetrics.record(queryContext); } + } + @Override + public CloseablePeekingIterator matchIterator(ReadExecutionController executionController) + { + return new MatchIterator(); + } - /** - * Returns an iterator over the rows in the partition associated with the given iterator. - * Initially, it retrieves the rows from the given iterator until it runs out of data. - * Then it iterates the remaining primary keys obtained from the index in batches until the end of the - * partition, lazily constructing an itertor for each batch. Only one row iterator is open at a time. - *

- * The rows are retrieved in the order of primary keys provided by the underlying index. - * The iterator is complete when the next key to be fetched belongs to different partition - * (but the iterator does not consume that key). - * - * @param startIter an iterator positioned at the first row in the partition that we want to return - */ - private @Nonnull UnfilteredRowIterator iteratePartition(@Nonnull UnfilteredRowIterator startIter) - { - return new AbstractUnfilteredRowIterator(startIter.metadata(), - startIter.partitionKey(), - startIter.partitionLevelDeletion(), - startIter.columns(), - startIter.staticRow(), - startIter.isReverseOrder(), - startIter.stats()) - { - private UnfilteredRowIterator currentIter = startIter; - private final DecoratedKey partitionKey = startIter.partitionKey(); + @Override + public Comparator matchComparator() + { + return Comparator.naturalOrder(); + } - @Override - protected Unfiltered computeNext() - { - while (!currentIter.hasNext()) - { - currentIter.close(); - currentIter = nextRowIterator(() -> nextSelectedKeysInPartition(partitionKey)); - if (currentIter == null) - return endOfData(); - } - return currentIter.next(); - } + @Override + public Index.MatchIndexer matchIndexer() + { + return new MatchIndexer(); + } - @Override - public void close() - { - FileUtils.closeQuietly(currentIter); - super.close(); - } - }; - } + @Override + public PartialTrackedRead beginRead(ReadExecutionController executionController, ColumnFamilyStore cfs, long startTimeNanos) + { + return PartialTrackedIndexRead.create(executionController, cfs, startTimeNanos, command, this); + } - private UnfilteredRowIterator queryStorageAndFilter(List keys) - { - long startTimeNanos = Clock.Global.nanoTime(); + private UnfilteredRowIterator queryStorageAndFilter(ReadableView view, ReadExecutionController executionController, List keys) + { + long startTimeNanos = Clock.Global.nanoTime(); - try (UnfilteredRowIterator partition = queryController.queryStorage(keys, executionController)) - { - queryContext.partitionsRead++; - queryContext.checkpoint(); + try (UnfilteredRowIterator partition = queryController.queryStorage(view, keys, executionController)) + { + queryContext.partitionsRead++; + queryContext.checkpoint(); - UnfilteredRowIterator filtered = filterPartition(keys, partition, filterTree); + UnfilteredRowIterator filtered = filterPartition(keys, partition, filterTree); - // Note that we record the duration of the read after post-filtering, which actually - // materializes the rows from disk. - tableQueryMetrics.postFilteringReadLatency.update(Clock.Global.nanoTime() - startTimeNanos, TimeUnit.NANOSECONDS); + // Note that we record the duration of the read after post-filtering, which actually + // materializes the rows from disk. + tableQueryMetrics.postFilteringReadLatency.update(Clock.Global.nanoTime() - startTimeNanos, TimeUnit.NANOSECONDS); - return filtered; - } + return filtered; } + } - private UnfilteredRowIterator filterPartition(List keys, UnfilteredRowIterator partition, FilterTree tree) + /** + * Returns an iterator over the rows in the partition associated with the given iterator. + * Initially, it retrieves the rows from the given iterator until it runs out of data. + * Then it iterates the remaining primary keys obtained from the index in batches until the end of the + * partition, lazily constructing an itertor for each batch. Only one row iterator is open at a time. + *

+ * The rows are retrieved in the order of primary keys provided by the underlying index. + * The iterator is complete when the next key to be fetched belongs to different partition + * (but the iterator does not consume that key). + * + * @param startIter an iterator positioned at the first row in the partition that we want to return + */ + private @Nonnull UnfilteredRowIterator iteratePartition(ReadExecutionController executionController, ReadableView view, PeekingIterator matchIter, @Nonnull UnfilteredRowIterator startIter) + { + return new AbstractUnfilteredRowIterator(startIter.metadata(), + startIter.partitionKey(), + startIter.partitionLevelDeletion(), + startIter.columns(), + startIter.staticRow(), + startIter.isReverseOrder(), + startIter.stats()) { - Row staticRow = partition.staticRow(); - DecoratedKey partitionKey = partition.partitionKey(); - List matches = new ArrayList<>(); - boolean hasMatch = false; - Set keysToShadow = topK ? new HashSet<>(keys) : Collections.emptySet(); - - while (partition.hasNext()) - { - Unfiltered unfiltered = partition.next(); - - if (unfiltered.isRow()) - { - queryContext.rowsFiltered++; - - if (tree.isSatisfiedBy(partitionKey, (Row) unfiltered, staticRow)) - { - matches.add(unfiltered); - hasMatch = true; - - if (topK) - { - PrimaryKey shadowed = keyFactory.hasClusteringColumns() - ? keyFactory.create(partitionKey, ((Row) unfiltered).clustering()) - : keyFactory.create(partitionKey); - keysToShadow.remove(shadowed); - } - } - } - } + private UnfilteredRowIterator currentIter = startIter; + private final DecoratedKey partitionKey = startIter.partitionKey(); - // If any non-static rows match the filter, there should be no need to shadow the static primary key: - if (topK && hasMatch && keyFactory.hasClusteringColumns()) - keysToShadow.remove(keyFactory.create(partitionKey, Clustering.STATIC_CLUSTERING)); - - // We may not have any non-static row data to filter... - if (!hasMatch) + @Override + protected Unfiltered computeNext() { - queryContext.rowsFiltered++; - - if (tree.isSatisfiedBy(partitionKey, staticRow, staticRow)) + while (!currentIter.hasNext()) { - hasMatch = true; - - if (topK) - keysToShadow.clear(); + currentIter.close(); + currentIter = nextRowIterator(executionController, partitionKey, view, matchIter); + if (currentIter == null) + return endOfData(); } + return currentIter.next(); } - if (topK && !keysToShadow.isEmpty()) + @Override + public void close() { - // Record primary keys shadowed by expired TTLs, row tombstones, or range tombstones: - queryContext.vectorContext().recordShadowedPrimaryKeys(keysToShadow); - } + // skip to the next partition key if the matchIterator hasn't been exhausted + while (matchIter.hasNext() && matchIter.peek().partitionKey().equals(partitionKey)) + matchIter.next(); - if (!hasMatch) - { - // If there are no matches, return an empty partition. If reconciliation is required at the - // coordinator, replica filtering protection may make a second round trip to complete its view - // of the partition. - return null; + FileUtils.closeQuietly(currentIter); + super.close(); } + }; + } - // Return all matches found, along with the static row... - return new PartitionIterator(partition, staticRow, matches.iterator()); - } - - private class PartitionIterator extends AbstractUnfilteredRowIterator + private void fillNextSelectedKeysInPartition(DecoratedKey partitionKey, List nextPrimaryKeys, PeekingIterator resultKeyIterator) + { + while (resultKeyIterator.hasNext() + && resultKeyIterator.peek().partitionKey().equals(partitionKey) + && nextPrimaryKeys.size() < partitionRowBatchSize) { - private final Iterator rows; - - public PartitionIterator(UnfilteredRowIterator partition, Row staticRow, Iterator rows) - { - super(partition.metadata(), - partition.partitionKey(), - partition.partitionLevelDeletion(), - partition.columns(), - staticRow, - partition.isReverseOrder(), - partition.stats()); - - this.rows = rows; - } - - @Override - protected Unfiltered computeNext() - { - return rows.hasNext() ? rows.next() : endOfData(); - } + nextPrimaryKeys.add(resultKeyIterator.next()); } + } - @Override - public TableMetadata metadata() - { - return queryController.metadata(); - } + /** + * Retrieves the next batch of primary keys (i.e. up to {@link #partitionRowBatchSize} of them) that belong to + * the given partition and are selected by the query controller, advancing the underlying iterator only while + * the next key belongs to that partition. + * + * @return a list of up to {@link #partitionRowBatchSize} primary keys within the given partition + */ + private List nextSelectedKeysInPartition(DecoratedKey partitionKey, PeekingIterator matches) + { + List threadLocalNextKeys = nextKeys.get(); + threadLocalNextKeys.clear(); + fillNextSelectedKeysInPartition(partitionKey, threadLocalNextKeys, matches); + return threadLocalNextKeys; + } - @Override - public void close() + /** + * Tries to obtain a row iterator for the supplied keys by repeatedly calling + * {@link StorageAttachedIndexSearcher#queryStorageAndFilter} until it gives a non-null result. + * The keysSupplier should return the next batch of keys with every call to get() + * and null when there are no more keys to try. + * + * @return an iterator or null if all keys were tried with no success + */ + private @Nullable UnfilteredRowIterator nextRowIterator(ReadExecutionController executionController, DecoratedKey partitionKey, ReadableView view, PeekingIterator matches) + { + UnfilteredRowIterator iterator = null; + while (iterator == null) { - FileUtils.closeQuietly(resultKeyIterator); - if (tableQueryMetrics != null) tableQueryMetrics.record(queryContext); + List keys = nextSelectedKeysInPartition(partitionKey, matches); + if (keys.isEmpty()) + return null; + iterator = queryStorageAndFilter(view, executionController, keys); } + return iterator; + } + + @Override + public UnfilteredRowIterator queryNextMatches(ReadExecutionController executionController, DecoratedKey partitionKey, ReadableView view, PeekingIterator matchIter) + { + Preconditions.checkArgument(matchIter.hasNext()); + Preconditions.checkArgument(matchIter.peek().partitionKey().equals(partitionKey)); + + UnfilteredRowIterator iterator = nextRowIterator(executionController, partitionKey, view, matchIter); + if (iterator == null) + return null; + return iteratePartition(executionController, view, matchIter, iterator); } /** @@ -685,4 +690,122 @@ public Row next() } }; } + + private UnfilteredRowIterator filterPartition(List keys, UnfilteredRowIterator partition, FilterTree tree) + { + Row staticRow = partition.staticRow(); + DecoratedKey partitionKey = partition.partitionKey(); + List matches = new ArrayList<>(); + boolean hasMatch = false; + Set keysToShadow = topK ? new HashSet<>(keys) : Collections.emptySet(); + + while (partition.hasNext()) + { + Unfiltered unfiltered = partition.next(); + + if (unfiltered.isRow()) + { + queryContext.rowsFiltered++; + + if (tree.isSatisfiedBy(partitionKey, (Row) unfiltered, staticRow)) + { + matches.add(unfiltered); + hasMatch = true; + + if (topK) + { + PrimaryKey shadowed = keyFactory.hasClusteringColumns() + ? keyFactory.create(partitionKey, ((Row) unfiltered).clustering()) + : keyFactory.create(partitionKey); + keysToShadow.remove(shadowed); + } + } + } + } + + // If any non-static rows match the filter, there should be no need to shadow the static primary key: + if (topK && hasMatch && keyFactory.hasClusteringColumns()) + keysToShadow.remove(keyFactory.create(partitionKey, Clustering.STATIC_CLUSTERING)); + + // We may not have any non-static row data to filter... + if (!hasMatch) + { + queryContext.rowsFiltered++; + + if (tree.isSatisfiedBy(partitionKey, staticRow, staticRow)) + { + hasMatch = true; + + if (topK) + keysToShadow.clear(); + } + } + + if (topK && !keysToShadow.isEmpty()) + { + // Record primary keys shadowed by expired TTLs, row tombstones, or range tombstones: + queryContext.vectorContext().recordShadowedPrimaryKeys(keysToShadow); + } + + if (!hasMatch) + { + // If there are no matches, return an empty partition. If reconciliation is required at the + // coordinator, replica filtering protection may make a second round trip to complete its view + // of the partition. + return null; + } + + // Return all matches found, along with the static row... + return new AbstractUnfilteredRowIterator(partition.metadata(), + partition.partitionKey(), + partition.partitionLevelDeletion(), + partition.columns(), + staticRow, + partition.isReverseOrder(), + partition.stats()) + { + private final Iterator rows = matches.iterator(); + + @Override + protected Unfiltered computeNext() + { + return rows.hasNext() ? rows.next() : endOfData(); + } + }; + } + + @Override + public UnfilteredPartitionIterator filterCompletedRead(UnfilteredPartitionIterator iterator) + { + return Transformation.apply(iterator, new Transformation() + { + DecoratedKey key = null; + Row staticRow = null; + + @Override + protected DecoratedKey applyToPartitionKey(DecoratedKey key) + { + this.key = key; + return super.applyToPartitionKey(key); + } + + @Override + protected UnfilteredRowIterator applyToPartition(UnfilteredRowIterator partition) + { + this.staticRow = partition.staticRow(); + if (!filterTree.restrictsNonStaticRow()) + return filterTree.isSatisfiedBy(partition.partitionKey(), staticRow, staticRow) ? partition : null; + + return Transformation.apply(partition, this); + } + + @Override + protected Row applyToRow(Row row) + { + if (!filterTree.isSatisfiedBy(key, row, staticRow)) + return null; + return super.applyToRow(row); + } + }); + } } diff --git a/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java b/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java index 6de7a6c88462..0dcef7c7158b 100644 --- a/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java +++ b/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java @@ -29,6 +29,7 @@ import org.apache.cassandra.db.marshal.ByteBufferAccessor; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.Token; +import org.apache.cassandra.index.Index; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.bytecomparable.ByteComparable; import org.apache.cassandra.utils.bytecomparable.ByteSource; @@ -40,7 +41,7 @@ * The {@link Factory.TokenOnlyPrimaryKey} is used by the {@link org.apache.cassandra.index.sai.plan.StorageAttachedIndexSearcher} to * position the search within the query range. */ -public interface PrimaryKey extends Comparable, ByteComparable +public interface PrimaryKey extends Comparable, ByteComparable, Index.IndexMatch { /** * See the javadoc for {@link #kind()} for how this enum is used. @@ -474,6 +475,12 @@ public PrimaryKey toStatic() */ DecoratedKey partitionKey(); + @Override + default DecoratedKey key() + { + return partitionKey(); + } + /** * Returns the {@link Clustering} representing the clustering component of the {@link PrimaryKey}. *

diff --git a/src/java/org/apache/cassandra/replication/BroadcastLogOffsets.java b/src/java/org/apache/cassandra/replication/BroadcastLogOffsets.java index ce1d4b672fa2..953d408b8f1f 100644 --- a/src/java/org/apache/cassandra/replication/BroadcastLogOffsets.java +++ b/src/java/org/apache/cassandra/replication/BroadcastLogOffsets.java @@ -21,9 +21,6 @@ import java.util.ArrayList; import java.util.List; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.dht.AbstractBounds; import org.apache.cassandra.dht.IPartitioner; @@ -36,8 +33,6 @@ public class BroadcastLogOffsets { - private static final Logger logger = LoggerFactory.getLogger(BroadcastLogOffsets.class); - private final String keyspace; private final Range range; private final List replicatedOffsets; @@ -73,7 +68,6 @@ public String toString() public static final IVerbHandler verbHandler = message -> { BroadcastLogOffsets replicatedOffsets = message.payload; - logger.trace("Received replicated offsets {} from {}", replicatedOffsets, message.from()); MutationTrackingService.instance.updateReplicatedOffsets(replicatedOffsets.keyspace, replicatedOffsets.range, replicatedOffsets.replicatedOffsets, diff --git a/src/java/org/apache/cassandra/service/reads/tracked/AbstractPartialTrackedRead.java b/src/java/org/apache/cassandra/service/reads/tracked/AbstractPartialTrackedRead.java deleted file mode 100644 index 2819068a4344..000000000000 --- a/src/java/org/apache/cassandra/service/reads/tracked/AbstractPartialTrackedRead.java +++ /dev/null @@ -1,266 +0,0 @@ -/* - * 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.service.reads.tracked; - -import com.google.common.base.Preconditions; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.db.Mutation; -import org.apache.cassandra.db.ReadExecutionController; -import org.apache.cassandra.db.partitions.PartitionUpdate; -import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; -import org.apache.cassandra.db.transform.RTBoundValidator; - - -public abstract class AbstractPartialTrackedRead implements PartialTrackedRead -{ - private static final Logger logger = LoggerFactory.getLogger(AbstractPartialTrackedRead.class); - - protected interface Augmentable - { - State augment(PartitionUpdate update); - } - - protected static abstract class State - { - protected static final State CLOSED = new State() - { - @Override - String name() - { - return "closed"; - } - - @Override - boolean isClosed() - { - return true; - } - }; - - abstract String name(); - - boolean isInitialized() - { - return false; - } - - Initialized asInitialized() - { - throw new IllegalStateException("State is " + name() + ", not " + Initialized.NAME); - } - - boolean isPrepared() - { - return false; - } - - Prepared asPrepared() - { - throw new IllegalStateException("State is " + name() + ", not " + Prepared.NAME); - } - - boolean isCompleted() - { - return false; - } - - Completed asCompleted() - { - throw new IllegalStateException("State is " + name() + ", not " + Completed.NAME); - } - - boolean isAugmentable() - { - return isPrepared() || isInitialized(); - } - - Augmentable asAugmentable() - { - if (isPrepared()) return asPrepared(); - throw new IllegalStateException("State is " + name() + ", not augmentable"); - } - - boolean isClosed() - { - return false; - } - - void close() - { - } - } - - // TODO (expected): this is a redundant state, never exposed - protected final class Initialized extends State - { - static final String NAME = "initialized"; - - @Override - String name() - { - return NAME; - } - - @Override - boolean isInitialized() - { - return true; - } - - @Override - Initialized asInitialized() - { - return this; - } - - Prepared prepare(UnfilteredPartitionIterator initialData) - { - return prepareInternal(initialData); - } - } - - protected abstract Prepared prepareInternal(UnfilteredPartitionIterator initialData); - - protected abstract class Prepared extends State implements Augmentable - { - private static final String NAME = "prepared"; - - @Override - String name() - { - return NAME; - } - - @Override - boolean isPrepared() - { - return true; - } - - @Override - Prepared asPrepared() - { - return this; - } - - abstract Completed complete(); - - } - - protected abstract class Completed extends State - { - private static final String NAME = "completed"; - - @Override - String name() - { - return NAME; - } - - protected abstract UnfilteredPartitionIterator iterator(); - protected abstract CompletedRead createResult(UnfilteredPartitionIterator iterator); - - protected CompletedRead getResult() - { - UnfilteredPartitionIterator result = command().completeTrackedRead(iterator(), AbstractPartialTrackedRead.this); - // validate that the sequence of RT markers is correct: open is followed by close, deletion times for both - // ends equal, and there are no dangling RT bound in any partition. - result = RTBoundValidator.validate(result, RTBoundValidator.Stage.PROCESSED, true); - return createResult(result); - } - } - - final ReadExecutionController executionController; - final ColumnFamilyStore cfs; - final long startTimeNanos; - private State state = new Initialized(); - - public AbstractPartialTrackedRead(ReadExecutionController executionController, ColumnFamilyStore cfs, long startTimeNanos) - { - this.executionController = executionController; - this.cfs = cfs; - this.startTimeNanos = startTimeNanos; - } - - @Override - public ReadExecutionController executionController() - { - return executionController; - } - - @Override - public ColumnFamilyStore cfs() - { - return cfs; - } - - @Override - public long startTimeNanos() - { - return startTimeNanos; - } - - protected synchronized State state() - { - return state; - } - - /** - * Implementors need to call this before returning this from createInProgressRead - * TODO (expected): this is a redundant transition from a redundant state (INITIALIZED) - */ - synchronized void prepare(UnfilteredPartitionIterator initialData) - { - logger.trace("Preparing read {}", this); - state = state.asInitialized().prepare(initialData); - } - - @Override - public synchronized void augment(Mutation mutation) - { - PartitionUpdate update = mutation.getPartitionUpdate(command().metadata()); - if (update != null) - state = state.asAugmentable().augment(update); - } - - @Override - public synchronized CompletedRead complete() - { - Preconditions.checkState(state.isPrepared()); - Completed completed = state.asPrepared().complete(); - state = completed; - return completed.getResult(); - } - - @Override - public synchronized void close() - { - if (state.isClosed()) - return; - - logger.trace("Closing read {}", this); - state.close(); - executionController.close(); - state = State.CLOSED; - } -} diff --git a/src/java/org/apache/cassandra/service/reads/tracked/ExtendingCompletedRead.java b/src/java/org/apache/cassandra/service/reads/tracked/ExtendingCompletedRead.java index 88f4b496e805..21a552e677da 100644 --- a/src/java/org/apache/cassandra/service/reads/tracked/ExtendingCompletedRead.java +++ b/src/java/org/apache/cassandra/service/reads/tracked/ExtendingCompletedRead.java @@ -18,7 +18,7 @@ package org.apache.cassandra.service.reads.tracked; -import org.apache.cassandra.transport.Dispatcher; +import com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,48 +35,31 @@ import org.apache.cassandra.db.transform.Transformation; import org.apache.cassandra.dht.AbstractBounds; import org.apache.cassandra.tracing.Tracing; +import org.apache.cassandra.transport.Dispatcher; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.concurrent.AsyncPromise; import org.apache.cassandra.utils.concurrent.Future; -class ExtendingCompletedRead implements PartialTrackedRead.CompletedRead +public abstract class ExtendingCompletedRead implements PartialTrackedRead.CompletedRead { private static final Logger logger = LoggerFactory.getLogger(ExtendingCompletedRead.class); - final PartitionRangeReadCommand command; - final UnfilteredPartitionIterator iterator; // merged end-result counter final DataLimits.Counter mergedResultCounter; - private final boolean partitionsFetched; private final boolean initialIteratorExhausted; - protected final AbstractBounds followUpBounds; - public ExtendingCompletedRead(PartitionRangeReadCommand command, - UnfilteredPartitionIterator iterator, - boolean partitionsFetched, - boolean initialIteratorExhausted, - AbstractBounds followUpBounds) + public ExtendingCompletedRead(ReadCommand command, boolean partitionsFetched, boolean initialIteratorExhausted) { - this.command = command; - this.iterator = iterator; - mergedResultCounter = command.limits().newCounter(command.nowInSec(), - true, - command.selectsFullPartition(), - command.metadata().enforceStrictLiveness()); + this.mergedResultCounter = command.limits().newCounter(command.nowInSec(), + true, + command.selectsFullPartition(), + command.metadata().enforceStrictLiveness()); this.partitionsFetched = partitionsFetched; this.initialIteratorExhausted = initialIteratorExhausted; - this.followUpBounds = followUpBounds; } - @Override - public TrackedDataResponse response() - { - PartitionIterator filtered = UnfilteredPartitionIterators.filter(iterator, command.nowInSec()); - PartitionIterator counted = Transformation.apply(filtered, mergedResultCounter); - PartitionIterator result = Transformation.apply(counted, new EmptyPartitionsDiscarder()); - return TrackedDataResponse.create(result, command.columnFilter()); - } + abstract ReadCommand command(); static boolean followUpReadRequired(ReadCommand command, DataLimits.Counter mergedResultCounter, boolean initialIteratorExhausted, boolean partitionsFetched) { @@ -116,7 +99,7 @@ static boolean followUpReadRequired(ReadCommand command, DataLimits.Counter merg protected boolean followUpRequired() { - return followUpReadRequired(command, mergedResultCounter, initialIteratorExhausted, partitionsFetched); + return followUpReadRequired(command(), mergedResultCounter, initialIteratorExhausted, partitionsFetched); } static int toQuery(ReadCommand command, DataLimits.Counter mergedResultCounter) @@ -145,18 +128,21 @@ public Future followupRead(TrackedDataResponse initialRespo * the total # of rows remaining - if it has some. If we don't grab enough rows in some of the partitions, * then future ShortReadRowsProtection.moreContents() calls will fetch the missing ones. */ - int toQuery = toQuery(command, mergedResultCounter); + int toQuery = toQuery(command(), mergedResultCounter); - ColumnFamilyStore.metricsFor(command.metadata().id).shortReadProtectionRequests.mark(); + ColumnFamilyStore.metricsFor(command().metadata().id).shortReadProtectionRequests.mark(); Tracing.trace("Requesting {} extra rows from {} for short read protection", toQuery, FBUtilities.getBroadcastAddressAndPort()); logger.info("Requesting {} extra rows from {} for short read protection", toQuery, FBUtilities.getBroadcastAddressAndPort()); return makeFollowupRead(initialResponse, toQuery, consistencyLevel, requestTime); } + protected abstract AbstractBounds followUpBounds(); + protected Future makeFollowupRead(TrackedDataResponse initialResponse, int toQuery, ConsistencyLevel consistencyLevel, Dispatcher.RequestTime requestTime) { - TrackedRead.Range followUpRead = PartialTrackedRangeRead.makeFollowUpRead(command, followUpBounds, toQuery, consistencyLevel, requestTime); + Preconditions.checkState(command() instanceof PartitionRangeReadCommand); + TrackedRead.Range followUpRead = PartialTrackedRangeRead.makeFollowUpRead((PartitionRangeReadCommand) command(), followUpBounds(), toQuery, consistencyLevel, requestTime); logger.trace("Short read detected, starting followup read {}", followUpRead); followUpRead.start(requestTime); AsyncPromise combinedRead = new AsyncPromise<>(); @@ -180,9 +166,49 @@ protected Future makeFollowupRead(TrackedDataResponse initi return combinedRead; } - @Override - public void close() + static class RangeRead extends ExtendingCompletedRead { - iterator.close(); + final PartitionRangeReadCommand command; + final UnfilteredPartitionIterator iterator; + protected final AbstractBounds followUpBounds; + + public RangeRead(PartitionRangeReadCommand command, + UnfilteredPartitionIterator iterator, + boolean partitionsFetched, + boolean initialIteratorExhausted, + AbstractBounds followUpBounds) + { + super(command, partitionsFetched, initialIteratorExhausted); + this.command = command; + this.iterator = iterator; + this.followUpBounds = followUpBounds; + } + + @Override + ReadCommand command() + { + return command; + } + + @Override + protected AbstractBounds followUpBounds() + { + return followUpBounds; + } + + @Override + public TrackedDataResponse response() + { + PartitionIterator filtered = UnfilteredPartitionIterators.filter(iterator, command.nowInSec()); + PartitionIterator counted = Transformation.apply(filtered, mergedResultCounter); + PartitionIterator result = Transformation.apply(counted, new EmptyPartitionsDiscarder()); + return TrackedDataResponse.create(result, command.columnFilter()); + } + + @Override + public void close() + { + iterator.close(); + } } } diff --git a/src/java/org/apache/cassandra/service/reads/tracked/FilteredFollowupRead.java b/src/java/org/apache/cassandra/service/reads/tracked/FilteredFollowupRead.java index fe87dbad0f99..d07539a460b6 100644 --- a/src/java/org/apache/cassandra/service/reads/tracked/FilteredFollowupRead.java +++ b/src/java/org/apache/cassandra/service/reads/tracked/FilteredFollowupRead.java @@ -109,7 +109,7 @@ public void start() { partialRead = new AtomicReference<>(); TrackedRead.Range rangeRead = makeFollowUpRead(command, followUpBounds, remaining, consistencyLevel, requestTime); - rangeRead.startLocal(requestTime, partialRead::set); + rangeRead.startLocal(requestTime, partialRead::set, TrackedLocalReads.Completer.DEFAULT); futures.add(rangeRead.future()); } else diff --git a/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedIndexRead.java b/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedIndexRead.java new file mode 100644 index 000000000000..c112dca015c6 --- /dev/null +++ b/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedIndexRead.java @@ -0,0 +1,861 @@ +/* + * 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.service.reads.tracked; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicBoolean; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; +import com.google.common.collect.PeekingIterator; + +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.PartitionRangeReadCommand; +import org.apache.cassandra.db.ReadCommand; +import org.apache.cassandra.db.ReadExecutionController; +import org.apache.cassandra.db.ReadableView; +import org.apache.cassandra.db.SinglePartitionReadCommand; +import org.apache.cassandra.db.Slices; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.lifecycle.SSTableSet; +import org.apache.cassandra.db.lifecycle.View; +import org.apache.cassandra.db.memtable.Memtable; +import org.apache.cassandra.db.partitions.Partition; +import org.apache.cassandra.db.partitions.PartitionIterator; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.partitions.SimpleBTreePartition; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.rows.UnfilteredSource; +import org.apache.cassandra.db.transform.EmptyPartitionsDiscarder; +import org.apache.cassandra.db.transform.Transformation; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.ExcludingBounds; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.index.Index; +import org.apache.cassandra.index.Index.IndexMatch; +import org.apache.cassandra.index.transactions.UpdateTransaction; +import org.apache.cassandra.io.sstable.SSTableReadsListener; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.transport.Dispatcher; +import org.apache.cassandra.utils.AbstractIterator; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.CloseablePeekingIterator; +import org.apache.cassandra.utils.concurrent.AsyncPromise; +import org.apache.cassandra.utils.concurrent.Future; +import org.apache.cassandra.utils.concurrent.FutureCombiner; +import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException; + +public class PartialTrackedIndexRead> extends PartialTrackedRead +{ + private final ReadCommand command; + private final Searcher searcher; + + private ConsistencyLevel consistencyLevel; + private Dispatcher.RequestTime requestTime; + + PartialTrackedIndexRead(ReadExecutionController executionController, ColumnFamilyStore cfs, long startTimeNanos, ReadCommand command, Searcher searcher) + { + super(executionController, cfs, startTimeNanos); + this.command = command; + this.searcher = searcher; + } + + public static > PartialTrackedIndexRead create(ReadExecutionController executionController, ColumnFamilyStore cfs, long startTimeNanos, ReadCommand command, Searcher searcher) + { + PartialTrackedIndexRead read = new PartialTrackedIndexRead<>(executionController, cfs, startTimeNanos, command, searcher); + read.prepare(null); + return read; + } + + @Override + public ReadCommand command() + { + return command; + } + + @Override + public Searcher searcher() + { + return searcher; + } + + @Override + public void setFollowUpReadContext(ConsistencyLevel consistencyLevel, Dispatcher.RequestTime requestTime) + { + this.consistencyLevel = consistencyLevel; + this.requestTime = requestTime; + } + + public interface CompletedIndexPartitionRead + { + UnfilteredRowIterator readHit(CloseablePeekingIterator matchIterator); + } + + public interface CompletedIndexRead extends CompletedRead + { + CompletedIndexPartitionRead partitionRead(DecoratedKey key); + CloseablePeekingIterator matchIterator(); + } + + private static DecoratedKey maxKey(DecoratedKey left, DecoratedKey right) + { + if (left == null) + return right; + if (right == null) + return left; + + return right.compareTo(left) > 0 ? right : left; + } + + private static class FollowUpRead> implements CompletedIndexPartitionRead, AutoCloseable + { + private final DecoratedKey key; + private final PartialTrackedIndexRead read; + private final CompletedIndexRead completedRead; + private final CompletedIndexPartitionRead partitionRead; + + public FollowUpRead(DecoratedKey key, PartialTrackedIndexRead read) + { + Preconditions.checkArgument(!read.command.isRangeRequest()); + this.key = key; + this.read = read; + this.completedRead = (CompletedIndexRead) read.complete(); + this.partitionRead = Preconditions.checkNotNull(completedRead.partitionRead(key)); + } + + static > Future> start(ReadCommand command, DecoratedKey key, ConsistencyLevel consistencyLevel, Dispatcher.RequestTime requestTime) + { + ClusterMetadata metadata = ClusterMetadata.current(); + Preconditions.checkState(command instanceof PartitionRangeReadCommand, "additional reads can only be made with range reads"); + + PartitionRangeReadCommand rangeReadCommand = (PartitionRangeReadCommand) command; + SinglePartitionReadCommand partitionReadCommand = SinglePartitionReadCommand.fromRangeRead(key, rangeReadCommand, rangeReadCommand.limits()); + + AsyncPromise> followUpPromise = new AsyncPromise<>(); + TrackedRead.Partition trackedRead = TrackedRead.Partition.create(metadata, partitionReadCommand, consistencyLevel, requestTime); + + trackedRead.startLocal(requestTime, null, ((promise1, read, consistencyLevel1, rt) -> { + try + { + followUpPromise.trySuccess(new FollowUpRead<>(key, (PartialTrackedIndexRead) read)); + } + catch (Exception e) + { + followUpPromise.tryFailure(e); + } + })); + return followUpPromise; + } + + @Override + public UnfilteredRowIterator readHit(CloseablePeekingIterator matchIterator) + { + Preconditions.checkState(matchIterator.hasNext()); + Preconditions.checkState(matchIterator.peek().key().equals(key)); + return partitionRead.readHit(matchIterator); + } + + @Override + public void close() + { + read.close(); + } + + static > void close(Map>> followUpReads) + { + for (Future> future : followUpReads.values()) + { + future.addCallback((followup, failure) -> { + if (failure != null) + followup.close(); + }); + } + } + + static > Map> getResults(Map>> futures, List> matchIterators) + { + Map> followupReads = new HashMap<>(); + for (Future> future : futures.values()) + { + try + { + FollowUpRead followUpRead = future.get(); + matchIterators.add(followUpRead.completedRead.matchIterator()); + followupReads.put(followUpRead.key, followUpRead); + } + catch (ExecutionException e) + { + throw new RuntimeException(e); + } + catch (InterruptedException e) + { + throw new UncheckedInterruptedException(e); + } + } + return followupReads; + } + + } + + private static class SnapshotView implements ReadableView + { + final List snapshots; + final List sstables; + private AugmentedPartition augmentedPartition = null; + + public SnapshotView(List snapshots, List sstables) + { + this.snapshots = snapshots; + this.sstables = sstables; + } + + public static SnapshotView create(DecoratedKey key, ColumnFamilyStore cfs) + { + ColumnFamilyStore.ViewFragment view = cfs.select(View.select(SSTableSet.LIVE, key)); + return new SnapshotView(MemtableSnapshot.create(key, view.memtables), view.sstables()); + } + + @Override + public Iterable memtables() + { + return snapshots; + } + + @Override + public List sstables() + { + return sstables; + } + + public void augment(PartitionUpdate update) + { + if (augmentedPartition == null) + { + augmentedPartition = new AugmentedPartition(update.partitionKey(), update.metadata()); + snapshots.add(augmentedPartition); + } + + augmentedPartition.augment(update); + } + } + + private static abstract class SinglePartitionSource implements UnfilteredSource + { + abstract Partition partition(); + + @Override + public UnfilteredRowIterator rowIterator(DecoratedKey key, Slices slices, ColumnFilter columnFilter, boolean reversed, SSTableReadsListener listener) + { + Partition partition = partition(); + Preconditions.checkState(key.equals(partition.partitionKey())); + return partition.unfilteredIterator(columnFilter, slices, reversed); + } + + @Override + public UnfilteredPartitionIterator partitionIterator(ColumnFilter columnFilter, DataRange dataRange, SSTableReadsListener listener) + { + throw new IllegalStateException("Range scans not supported"); + } + + @Override + public long getMinTimestamp() + { + return partition().stats().minTimestamp; + } + + @Override + public long getMinLocalDeletionTime() + { + return partition().stats().minLocalDeletionTime; + } + } + + private static class MemtableSnapshot extends SinglePartitionSource + { + private final Partition partition; + + public MemtableSnapshot(Partition partition) + { + this.partition = partition; + } + + static List create(DecoratedKey key, Iterable memtables) + { + List snapshots = new ArrayList<>(); + for (Memtable memtable : memtables) + { + Partition partition = memtable.snapshotPartition(key); + if (partition != null) + snapshots.add(new MemtableSnapshot(partition)); + } + return snapshots; + } + + @Override + Partition partition() + { + return partition; + } + } + + private static class AugmentedPartition extends SinglePartitionSource + { + private final SimpleBTreePartition data; + + AugmentedPartition(DecoratedKey key, TableMetadata metadata) + { + this.data = new SimpleBTreePartition(key, metadata, UpdateTransaction.NO_OP); + } + + void augment(PartitionUpdate update) + { + data.update(update); + } + + @Override + Partition partition() + { + return data; + } + } + + class IndexPartitionRead implements CompletedIndexPartitionRead + { + private final DecoratedKey partitionKey; + private final SnapshotView view; + + IndexPartitionRead(DecoratedKey partitionKey, SnapshotView view) + { + this.partitionKey = partitionKey; + this.view = view; + } + + void augment(PartitionUpdate update) + { + Preconditions.checkArgument(update.partitionKey().equals(partitionKey)); + view.augment(update); + } + + @Override + public UnfilteredRowIterator readHit(CloseablePeekingIterator matchIterator) + { + Preconditions.checkArgument(matchIterator.hasNext()); + Preconditions.checkArgument(matchIterator.peek().key().equals(partitionKey)); + return searcher.queryNextMatches(executionController, partitionKey, view, matchIterator); + } + } + + IndexPartitionRead createRead(DecoratedKey key, ColumnFamilyStore cfs) + { + SnapshotView view = SnapshotView.create(key, cfs); + return new IndexPartitionRead(key, view); + } + + @Override + protected Prepared prepareInternal(UnfilteredPartitionIterator initialData) + { + SortedMap reads = new TreeMap<>(); + if (command instanceof SinglePartitionReadCommand) + { + SinglePartitionReadCommand cmd = (SinglePartitionReadCommand) command; + DecoratedKey key = cmd.partitionKey(); + IndexPartitionRead partitionRead = createRead(key, cfs); + reads.put(key, partitionRead); + } + + DecoratedKey maxKey = null; + CloseablePeekingIterator matchIterator = searcher.matchIterator(executionController); + try + { + SortedSet materializedMatches = new TreeSet<>(searcher.matchComparator()); + while (matchIterator.hasNext() && materializedMatches.size() < command.limits().count()) + { + Match match = matchIterator.next(); + materializedMatches.add(match); + if (!reads.containsKey(match.key())) + { + // TODO (now): make decorated key part of IndexEntry interface + DecoratedKey key = match.key(); + maxKey = maxKey(maxKey, key); + + IndexPartitionRead partitionRead = createRead(key, cfs); + + reads.put(key, partitionRead); + } + } + return new IndexPrepared(maxKey, materializedMatches, matchIterator, reads); + } + catch (Throwable t) + { + FileUtils.closeQuietly(matchIterator); + throw t; + } + } + + @Override + public synchronized void complete(AsyncPromise promise, ConsistencyLevel consistencyLevel, Dispatcher.RequestTime requestTime) + { + Preconditions.checkState(state().isPrepared()); + IndexPrepared prepared = (IndexPrepared) state(); + + if (prepared.isCompletable()) + { + super.complete(promise, consistencyLevel, requestTime); + return; + } + + IndexPreComplete preComplete = prepared.preComplete(); + state = preComplete; + + // simple listener - completion will handle any failed futures + preComplete.future().addListener(() -> super.complete(promise, consistencyLevel, requestTime)); + } + + private abstract class AbstractIndexPrepared extends Prepared + { + protected DecoratedKey maxKey; + protected final SortedSet materializedMatches; + // there may be additional matches for keys we've already scanned, this allows us to read them before + // starting a short read + protected final CloseablePeekingIterator additionalMatches; + + protected final SortedMap reads; + + // for range scans, if we learn of new keys with matching contents as part of reconciliation, we need + // to do follow up reads against them since we didn't snapshot memtable contents for the keys during + // the prepare phase of the read. Futures for those reads are kept here + protected final Map>> followUpReads; + + public AbstractIndexPrepared(DecoratedKey maxKey, + SortedSet materializedMatches, + CloseablePeekingIterator additionalMatches, + SortedMap reads, + Map>> followUpReads) + { + this.maxKey = maxKey; + this.materializedMatches = materializedMatches; + this.reads = reads; + this.additionalMatches = additionalMatches; + this.followUpReads = followUpReads; + } + + boolean isCompletable() + { + return Iterables.all(followUpReads.values(), Future::isDone); + } + + @Override + Completed complete() + { + Preconditions.checkState(isCompletable()); + List> matchIterators = new ArrayList<>(followUpReads.size() + 1); + matchIterators.add(CloseablePeekingIterator.wrap(materializedMatches.iterator())); + Map> followUpResults = FollowUpRead.getResults(followUpReads, matchIterators); + return new IndexCompleted(maxKey, new MergingMatchIterator(matchIterators), additionalMatches, reads, followUpResults); + } + + @Override + void close() + { + FollowUpRead.close(followUpReads); + super.close(); + } + } + + private class IndexPrepared extends AbstractIndexPrepared + { + private Index.MatchIndexer matchIndexer = null; + + public IndexPrepared(DecoratedKey maxKey, SortedSet materializedMatches, CloseablePeekingIterator additionalMatches, SortedMap reads) + { + super(maxKey, materializedMatches, additionalMatches, reads, new HashMap<>()); + } + + private Index.MatchIndexer matchIndexer() + { + if (matchIndexer == null) + matchIndexer = searcher.matchIndexer(); + return matchIndexer; + } + + private boolean indexNewKey(PartitionUpdate update) + { + AtomicBoolean hasMatches = new AtomicBoolean(false); + matchIndexer().index(update, e -> hasMatches.set(true)); + return hasMatches.get(); + } + + private boolean indexUpdate(PartitionUpdate update) + { + int startingSize = materializedMatches.size(); + matchIndexer().index(update, materializedMatches::add); + return materializedMatches.size() > startingSize; + } + + @Override + public State augment(PartitionUpdate update) + { + Preconditions.checkState(consistencyLevel != null, + "PartialTrackedRead#setFollowUpReadContext needs to be called before making reads available for augmenting mutation"); + DecoratedKey key = update.partitionKey(); + IndexPartitionRead read = reads.get(key); + if (read == null) + { + // TODO: maybe we should immediately start a follow up read if it's likely this key will be included in the response + if (!followUpReads.containsKey(key) && indexNewKey(update)) + { + maxKey = maxKey(maxKey, update.partitionKey()); + Future> followUpRead = FollowUpRead.start(command, update.partitionKey(), consistencyLevel, requestTime); + followUpReads.put(key, followUpRead); + } + return this; + } + + read.augment(update); + indexUpdate(update); + + // TODO: calling this method mever results in a state change, remove return? + return this; + } + + IndexPreComplete preComplete() + { + return new IndexPreComplete(maxKey, materializedMatches, additionalMatches, reads, followUpReads); + } + } + + private class IndexPreComplete extends AbstractIndexPrepared + { + public IndexPreComplete(DecoratedKey maxKey, SortedSet materializedMatches, CloseablePeekingIterator additionalMatches, SortedMap reads, Map>> followUpReads) + { + super(maxKey, materializedMatches, additionalMatches, reads, followUpReads); + } + + @Override + public State augment(PartitionUpdate update) + { + throw new IllegalStateException("cannot augment reads pending completion"); + } + + Future>> future() + { + return FutureCombiner.allOf(followUpReads.values()); + } + } + + private class IndexCompleted extends Completed + { + private final DecoratedKey maxKey; + private final CloseablePeekingIterator materializedMatchIterator; + private final CloseablePeekingIterator additionalMatchIterator; + private final SortedMap reads; + private final Map> followUpReads; + + public IndexCompleted(DecoratedKey maxKey, CloseablePeekingIterator materializedMatchIterator, CloseablePeekingIterator additionalMatchIterator, SortedMap reads, Map> followUpReads) + { + this.maxKey = maxKey; + this.materializedMatchIterator = materializedMatchIterator; + this.additionalMatchIterator = additionalMatchIterator; + this.reads = reads; + this.followUpReads = followUpReads; + } + + @Override + protected CompletedRead getResult() + { + return new IndexCompletedRead(maxKey, materializedMatchIterator, additionalMatchIterator, reads, followUpReads); + } + } + + protected class MergingMatchIterator extends AbstractIterator + { + private final List> iterators; + + public MergingMatchIterator(List> iterators) + { + this.iterators = iterators; + } + + @Override + protected Match computeNext() + { + int minIdx = -1; + Match minMatch = null; + for (int i=0,mi=iterators.size(); i iterator = iterators.get(i); + if (!iterator.hasNext()) + continue; + + if (minMatch == null) + { + minMatch = iterator.peek(); + minIdx = i; + continue; + } + + Match thisMatch = iterator.peek(); + int cmp = searcher.matchComparator().compare(thisMatch, minMatch); + if (cmp < 0) + { + minMatch = thisMatch; + minIdx = i; + } + else if (cmp == 0) + { + // if this iterator equals the current minimum, advance the iterator - we don't merge equal matches + iterator.next(); + } + } + + if (minMatch != null) + { + Preconditions.checkArgument(minIdx >= 0); + iterators.get(minIdx).next(); + return minMatch; + } + + return endOfData(); + } + + @Override + public void close() + { + FileUtils.closeQuietly(iterators); + } + } + + /** + * Merges a materialized iterator and an additional iterator. The additional iterator is meant to be the initial + * match iterator from the searcher. If we encounter previously unseen keys from the initial match iterator, it + * means that we're in a short read and need to start a follow-up read, which this iterator signals to the caller + */ + private class MergingStoppingMatchIterator extends AbstractIterator + { + private final DecoratedKey maxKey; + private final PeekingIterator materializedIterator; + private final CloseablePeekingIterator additionalIterator; + private boolean followUpRequired = false; + + public MergingStoppingMatchIterator(DecoratedKey maxKey, Iterator materializedIterator, CloseablePeekingIterator additionalIterator) + { + this.maxKey = maxKey; + this.materializedIterator = Iterators.peekingIterator(materializedIterator); + this.additionalIterator = additionalIterator; + } + + @Override + protected Match computeNext() + { + if (materializedIterator.hasNext() && additionalIterator.hasNext()) + { + int cmp = searcher.matchComparator().compare(materializedIterator.peek(), additionalIterator.peek()); + if (cmp == 0) + { + additionalIterator.next(); + return materializedIterator.next(); + } + else if (cmp < 0) + { + return materializedIterator.next(); + } + else + { + Match match = additionalIterator.next(); + DecoratedKey key = match.key(); + Preconditions.checkArgument(key.compareTo(maxKey) <= 0); + return match; + } + } + + if (materializedIterator.hasNext()) + return materializedIterator.next(); + + if (additionalIterator.hasNext()) + { + Match match = additionalIterator.next(); + DecoratedKey key = match.key(); + if (key.compareTo(maxKey) > 0) + { + Preconditions.checkArgument(command.isRangeRequest()); + followUpRequired = true; + return endOfData(); + } + return match; + } + + return endOfData(); + } + + @Override + public void close() + { + additionalIterator.close(); + } + } + + private class IndexCompletedRead extends ExtendingCompletedRead implements CompletedIndexRead + { + private final DecoratedKey maxKey; + private final MergingStoppingMatchIterator matchIterator; + private final SortedMap reads; + + private final Map> followupReads; + + public IndexCompletedRead(DecoratedKey maxKey, CloseablePeekingIterator materializedMatches, CloseablePeekingIterator additionalMatches, SortedMap reads, Map> followupReads) + { + super(command, materializedMatches.hasNext(), true); + this.maxKey = maxKey; + this.matchIterator = new MergingStoppingMatchIterator(maxKey, materializedMatches, additionalMatches); + this.reads = reads; + this.followupReads = followupReads; + } + + @Override + public CloseablePeekingIterator matchIterator() + { + return matchIterator; + } + + @Override + ReadCommand command() + { + return command; + } + + @Override + protected AbstractBounds followUpBounds() + { + Preconditions.checkState(command.isRangeRequest()); + Preconditions.checkNotNull(maxKey); + AbstractBounds bounds = command.dataRange().keyRange(); + return bounds.inclusiveRight() + ? new Range<>(maxKey, bounds.right) + : new ExcludingBounds<>(maxKey, bounds.right); + } + + private class UnfilteredResultIterator extends AbstractIterator implements UnfilteredPartitionIterator + { + private final Map> followUpReads; + private final CloseablePeekingIterator matchIter; + + public UnfilteredResultIterator(CloseablePeekingIterator matchIter, Map> followUpReads) + { + this.matchIter = matchIter; + this.followUpReads = followUpReads; + } + + @Override + public TableMetadata metadata() + { + return command.metadata(); + } + + @Override + protected UnfilteredRowIterator computeNext() + { + for (;;) + { + if (!matchIter.hasNext()) + return endOfData(); + + DecoratedKey nextKey = matchIter.peek().key(); + IndexPartitionRead read = reads.get(nextKey); + if (read != null) + return read.readHit(matchIter); + + FollowUpRead followUpRead = followUpReads.get(nextKey); + if (followUpRead == null) + throw new IllegalStateException("Received match for key without initial or followup read: " + ByteBufferUtil.bytesToHex(nextKey.getKey())); + + UnfilteredRowIterator next = followUpRead.readHit(matchIter); + if (next != null) + return next; + } + } + + @Override + public void close() + { + matchIter.close(); + } + } + + private PartitionIterator filter(UnfilteredPartitionIterator iterator) + { + iterator = searcher.filterCompletedRead(iterator); + iterator = command.completeTrackedRead(iterator, PartialTrackedIndexRead.this); + PartitionIterator filtered = UnfilteredPartitionIterators.filter(iterator, command.nowInSec()); + PartitionIterator counted = Transformation.apply(filtered, mergedResultCounter); + PartitionIterator result = Transformation.apply(counted, new EmptyPartitionsDiscarder()); + return result; + } + + @Override + public TrackedDataResponse response() + { + try (UnfilteredResultIterator iterator = new UnfilteredResultIterator(matchIterator, followupReads)) + { + PartitionIterator filtered = filter(iterator); + return TrackedDataResponse.create(filtered, command.columnFilter()); + } + } + + @Override + protected boolean followUpRequired() + { + if (!command.isRangeRequest()) + return false; + return matchIterator.followUpRequired || super.followUpRequired(); + } + + @Override + public void close() + { + FileUtils.closeQuietly(matchIterator); + FileUtils.closeQuietly(followupReads.values()); + } + + @Override + public CompletedIndexPartitionRead partitionRead(DecoratedKey key) + { + return reads.get(key); + } + } + +} diff --git a/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedRangeRead.java b/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedRangeRead.java index 2f3b58dfdc2b..0bad668ff1ce 100644 --- a/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedRangeRead.java +++ b/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedRangeRead.java @@ -25,7 +25,6 @@ import java.util.TreeMap; import com.google.common.base.Preconditions; -import org.apache.cassandra.transport.Dispatcher; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,6 +44,7 @@ import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.partitions.SimpleBTreePartition; import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators; import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.db.transform.Transformation; import org.apache.cassandra.dht.AbstractBounds; @@ -55,9 +55,10 @@ import org.apache.cassandra.locator.ReplicaPlan; import org.apache.cassandra.locator.ReplicaPlans; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.transport.Dispatcher; import org.apache.cassandra.utils.concurrent.Future; -public abstract class PartialTrackedRangeRead extends AbstractPartialTrackedRead +public abstract class PartialTrackedRangeRead extends PartialTrackedRead { private static final Logger logger = LoggerFactory.getLogger(PartialTrackedRangeRead.class); @@ -111,19 +112,15 @@ public ReadCommand command() protected static class ShortReadSupport { - final DecoratedKey lastPartitionKey; // key of the last observed partition final boolean partitionsFetched; // whether we've seen any new partitions since iteration start or last moreContents() call final boolean initialIteratorExhausted; final AbstractBounds followUpBounds; - boolean wasAugmented; ShortReadSupport(Builder builder, boolean initialIteratorExhausted, AbstractBounds followUpBounds) { - this.lastPartitionKey = builder.lastPartitionKey; this.partitionsFetched = builder.partitionsFetched; this.initialIteratorExhausted = initialIteratorExhausted; this.followUpBounds = followUpBounds; - this.wasAugmented = false; } protected static class Builder @@ -175,34 +172,19 @@ private Materializer(ReadCommand command) RangePrepared materialize(UnfilteredPartitionIterator inputIterator) { - try + try (inputIterator) { - UnfilteredPartitionIterator materialized = Transformation.apply(inputIterator, new Transformation() - { - @Override - protected UnfilteredRowIterator applyToPartition(UnfilteredRowIterator partition) - { - SimpleBTreePartition materialized = data.computeIfAbsent(partition.partitionKey(), key -> new SimpleBTreePartition(key, partition.metadata(), UpdateTransaction.NO_OP)); - materialized.update(PartitionUpdate.fromIterator(partition, command.columnFilter())); - shortReadSupport.lastPartitionKey = partition.partitionKey(); - shortReadSupport.partitionsFetched = true; - return queryPartition(materialized); - } - }); + UnfilteredPartitionIterator materialized = Transformation.apply(inputIterator, this); UnfilteredPartitionIterator filtered = filter(materialized); try (UnfilteredPartitionIterator iterator = shortReadSupport.counter.applyTo(filtered)) { - consume(iterator); + UnfilteredPartitionIterators.consume(iterator); } return createRangePrepared(); } - finally - { - inputIterator.close(); - } } @Override @@ -260,7 +242,7 @@ public State augment(PartitionUpdate update) } } - protected abstract class RangeCompleted extends Completed + protected abstract class RangeCompleted extends AbstractCompleted { protected final SortedMap data; protected final ShortReadSupport shortReadSupport; @@ -337,18 +319,6 @@ UnfilteredRowIterator queryPartition(AbstractBTreePartition partition) command.clusteringIndexFilter(partition.partitionKey()).isReversed()); } - private static void consume(UnfilteredPartitionIterator iterator) - { - while (iterator.hasNext()) - { - try (UnfilteredRowIterator partition = iterator.next()) - { - while (partition.hasNext()) - partition.next(); - } - } - } - public AbstractBounds followUpBounds() { RangeCompleted completed = (RangeCompleted) state().asCompleted(); @@ -401,7 +371,7 @@ public SimpleCompleted(SortedMap data, Short @Override protected CompletedRead extendRead(UnfilteredPartitionIterator iterator) { - return new ExtendingCompletedRead(command, iterator, shortReadSupport.partitionsFetched, shortReadSupport.initialIteratorExhausted, shortReadSupport.followUpBounds); + return new ExtendingCompletedRead.RangeRead(command, iterator, shortReadSupport.partitionsFetched, shortReadSupport.initialIteratorExhausted, shortReadSupport.followUpBounds); } } @@ -523,7 +493,7 @@ RangePrepared createRangePrepared() } } - static class FilteredCompletedRead extends ExtendingCompletedRead + static class FilteredCompletedRead extends ExtendingCompletedRead.RangeRead { private final DecoratedKey lastMatchingKey; private final SortedMap followUpReadInfo; @@ -537,7 +507,6 @@ public FilteredCompletedRead(PartitionRangeReadCommand command, UnfilteredPartit /** * Even if we reached the limit during materialization, if there are keys ahead of the first materialized key * or interleaved with them, then we need to read them - * @return */ private boolean hasInterleavedFollowupKeys() { diff --git a/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedRead.java b/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedRead.java index 32409e59d033..764d9dc6a23a 100644 --- a/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedRead.java +++ b/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedRead.java @@ -15,10 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.cassandra.service.reads.tracked; import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.ConsistencyLevel; @@ -27,22 +28,257 @@ import org.apache.cassandra.db.ReadExecutionController; import org.apache.cassandra.db.filter.DataLimits; import org.apache.cassandra.db.partitions.PartitionIterator; +import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators; +import org.apache.cassandra.db.transform.RTBoundValidator; import org.apache.cassandra.index.Index; +import org.apache.cassandra.utils.concurrent.AsyncPromise; import org.apache.cassandra.utils.concurrent.Future; import org.apache.cassandra.replication.Log2OffsetsMap; import org.apache.cassandra.replication.MutationJournal; import org.apache.cassandra.replication.ShortMutationId; import org.apache.cassandra.transport.Dispatcher; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -public interface PartialTrackedRead +public abstract class PartialTrackedRead { - Logger logger = LoggerFactory.getLogger(PartialTrackedRead.class); - - interface CompletedRead extends AutoCloseable + private static final Logger logger = LoggerFactory.getLogger(PartialTrackedRead.class); + + final ReadExecutionController executionController; + final ColumnFamilyStore cfs; + final long startTimeNanos; + + public PartialTrackedRead(ReadExecutionController executionController, ColumnFamilyStore cfs, long startTimeNanos) + { + this.executionController = executionController; + this.cfs = cfs; + this.startTimeNanos = startTimeNanos; + } + + public ReadExecutionController executionController() + { + return executionController; + } + + public ColumnFamilyStore cfs() + { + return cfs; + } + + public long startTimeNanos() + { + return startTimeNanos; + } + + abstract ReadCommand command(); + + public abstract Index.Searcher searcher(); + + protected interface Augmentable + { + State augment(PartitionUpdate update); + } + + protected static abstract class State + { + protected static final State CLOSED = new State() + { + @Override + String name() + { + return "closed"; + } + + @Override + boolean isClosed() + { + return true; + } + }; + + abstract String name(); + + Initialized asInitialized() + { + throw new IllegalStateException("State is " + name() + ", not " + Initialized.NAME); + } + + boolean isPrepared() + { + return false; + } + + Prepared asPrepared() + { + throw new IllegalStateException("State is " + name() + ", not " + Prepared.NAME); + } + + Completed asCompleted() + { + throw new IllegalStateException("State is " + name() + ", not " + Completed.NAME); + } + + Augmentable asAugmentable() + { + if (isPrepared()) return asPrepared(); + throw new IllegalStateException("State is " + name() + ", not augmentable"); + } + + boolean isClosed() + { + return false; + } + + void close() + { + } + } + + // TODO (expected): this is a redundant state, never exposed + protected final class Initialized extends State + { + static final String NAME = "initialized"; + + @Override + String name() + { + return NAME; + } + + @Override + Initialized asInitialized() + { + return this; + } + + Prepared prepare(UnfilteredPartitionIterator initialData) + { + return prepareInternal(initialData); + } + } + + protected abstract Prepared prepareInternal(UnfilteredPartitionIterator initialData); + + protected static abstract class Prepared extends State implements Augmentable + { + private static final String NAME = "prepared"; + + @Override + String name() + { + return NAME; + } + + @Override + boolean isPrepared() + { + return true; + } + + @Override + Prepared asPrepared() + { + return this; + } + + abstract Completed complete(); + } + + protected static abstract class Completed extends State + { + private static final String NAME = "completed"; + + @Override + String name() + { + return NAME; + } + + protected abstract CompletedRead getResult(); + } + + protected abstract class AbstractCompleted extends Completed + { + protected abstract UnfilteredPartitionIterator iterator(); + protected abstract CompletedRead createResult(UnfilteredPartitionIterator iterator); + + @Override + protected CompletedRead getResult() + { + UnfilteredPartitionIterator result = command().completeTrackedRead(iterator(), PartialTrackedRead.this); + // validate that the sequence of RT markers is correct: open is followed by close, deletion times for both + // ends equal, and there are no dangling RT bound in any partition. + result = RTBoundValidator.validate(result, RTBoundValidator.Stage.PROCESSED, true); + return createResult(result); + } + } + + protected State state = new Initialized(); + + protected synchronized State state() + { + return state; + } + + /** + * Implementors need to call this before returning this from createInProgressRead + * TODO (expected): this is a redundant transition from a redundant state (INITIALIZED) + */ + synchronized void prepare(UnfilteredPartitionIterator initialData) + { + logger.trace("Preparing read {}", this); + state = state.asInitialized().prepare(initialData); + } + + void augment(PartitionUpdate update) + { + state = state.asAugmentable().augment(update); + } + + public synchronized void augment(Mutation mutation) + { + PartitionUpdate update = mutation.getPartitionUpdate(command().metadata()); + if (update != null) + augment(update); + } + + void augment(Log2OffsetsMap augmentingOffsets) + { + augmentingOffsets.forEach(this::augment); + } + + void augment(ShortMutationId mutationId) + { + Mutation mutation = MutationJournal.instance.read(mutationId); + Preconditions.checkNotNull(mutation, "Missing mutation %s", mutationId); + if (!command().selectsKey(mutation.key())) + { + logger.trace("Skipping mutation {} - {} not in read range", mutationId, mutation.key()); + return; + } + augment(mutation); + } + + public synchronized CompletedRead complete() + { + Preconditions.checkState(state.isPrepared()); + Completed completed = state.asPrepared().complete(); + state = completed; + return completed.getResult(); + } + + public synchronized void close() + { + if (state.isClosed()) + return; + + logger.trace("Closing read {}", this); + state.close(); + executionController.close(); + state = State.CLOSED; + } + + public interface CompletedRead extends AutoCloseable { TrackedDataResponse response(); // must be called from the read stage Future followupRead(TrackedDataResponse initialResponse, ConsistencyLevel consistencyLevel, Dispatcher.RequestTime requestTime); @@ -86,36 +322,48 @@ public void close() } } - CompletedRead complete(); + /** + * Sets consistency level and expiration info to be used for follow up reads. Needs to be called before making the + * read available for receiving augmenting mutations + */ + void setFollowUpReadContext(ConsistencyLevel consistencyLevel, Dispatcher.RequestTime requestTime) {} - void augment(Mutation mutation); - - default void augment(Log2OffsetsMap augmentingOffsets) + void complete(AsyncPromise promise, ConsistencyLevel consistencyLevel, Dispatcher.RequestTime requestTime) { - augmentingOffsets.forEach(this::augment); + complete(promise, this, consistencyLevel, requestTime); } - default void augment(ShortMutationId mutationId) + static void complete(AsyncPromise promise, PartialTrackedRead read, ConsistencyLevel consistencyLevel, Dispatcher.RequestTime requestTime) { - Mutation mutation = MutationJournal.instance.read(mutationId); - Preconditions.checkNotNull(mutation, "Missing mutation %s", mutationId); - if (!command().selectsKey(mutation.key())) + try (CompletedRead completedRead = read.complete()) { - logger.trace("Skipping mutation {} - {} not in read range", mutationId, mutation.key()); - return; + TrackedDataResponse response = completedRead.response(); + Future followUp = completedRead.followupRead(response, consistencyLevel, requestTime); + + if (followUp != null) + { + followUp.addCallback((newResponse, error) -> { + if (error != null) + { + promise.tryFailure(error); + return; + } + promise.trySuccess(newResponse); + }); + } + else + { + promise.trySuccess(response); + } + } + catch (Exception e) + { + promise.tryFailure(e); + throw e; + } + finally + { + read.close(); } - augment(mutation); } - - ReadExecutionController executionController(); - - Index.Searcher searcher(); - - ColumnFamilyStore cfs(); - - long startTimeNanos(); - - ReadCommand command(); - - void close(); } diff --git a/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedSinglePartitionRead.java b/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedSinglePartitionRead.java index 895706f05674..262415192b3f 100644 --- a/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedSinglePartitionRead.java +++ b/src/java/org/apache/cassandra/service/reads/tracked/PartialTrackedSinglePartitionRead.java @@ -36,7 +36,7 @@ import static org.apache.cassandra.db.partitions.UnfilteredPartitionIterators.MergeListener.NOOP; -public class PartialTrackedSinglePartitionRead extends AbstractPartialTrackedRead +public class PartialTrackedSinglePartitionRead extends PartialTrackedRead { private final Index.Searcher searcher; private final SinglePartitionReadCommand command; @@ -95,7 +95,7 @@ Completed complete() } } - private class SinglePartitionCompleted extends Completed + private class SinglePartitionCompleted extends AbstractCompleted { private final UnfilteredPartitionIterator initialData; private final SimpleBTreePartition augmentedData; diff --git a/src/java/org/apache/cassandra/service/reads/tracked/TrackedLocalReads.java b/src/java/org/apache/cassandra/service/reads/tracked/TrackedLocalReads.java index fa6c3193b377..aa58dc331910 100644 --- a/src/java/org/apache/cassandra/service/reads/tracked/TrackedLocalReads.java +++ b/src/java/org/apache/cassandra/service/reads/tracked/TrackedLocalReads.java @@ -22,6 +22,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.cassandra.concurrent.Stage; import org.apache.cassandra.db.*; @@ -37,12 +39,8 @@ import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.transport.Dispatcher; import org.apache.cassandra.utils.concurrent.AsyncPromise; -import org.apache.cassandra.utils.concurrent.Future; import org.jctools.maps.NonBlockingHashMap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * Since the read reconciliations don't use 2 way callbacks, maps of active reads and reconciliations * are maintained and expired here. @@ -53,6 +51,12 @@ public class TrackedLocalReads implements ExpiredStatePurger.Expireable { private static final Logger logger = LoggerFactory.getLogger(TrackedLocalReads.class); + public interface Completer + { + void complete(AsyncPromise promise, PartialTrackedRead read, ConsistencyLevel consistencyLevel, Dispatcher.RequestTime requestTime); + TrackedLocalReads.Completer DEFAULT = (promise, read, consistencyLevel, requestTime) -> read.complete(promise, consistencyLevel, requestTime); + } + private final NonBlockingHashMap coordinators = new NonBlockingHashMap<>(); public TrackedLocalReads() @@ -66,7 +70,8 @@ public AsyncPromise beginRead( ReadCommand command, ConsistencyLevel consistencyLevel, int[] summaryNodes, - Dispatcher.RequestTime requestTime) + Dispatcher.RequestTime requestTime, + TrackedLocalReads.Completer completer) { Keyspace keyspace = Keyspace.open(command.metadata().keyspace); ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(command.metadata().id); @@ -96,18 +101,18 @@ public AsyncPromise beginRead( } // TODO: confirm all summaryNodes are present in the replica plan AsyncPromise promise = new AsyncPromise<>(); - beginReadInternal(readId, command, replicaPlan, summaryNodes, requestTime, promise); + beginReadInternal(readId, command, replicaPlan, summaryNodes, requestTime, promise, completer); return promise; } // TODO (expected): skip local summaries and reconcile when summaryNodes is empty (e.g. for CL.ONE) - private void beginReadInternal( - TrackedRead.Id readId, - ReadCommand command, - ReplicaPlan.AbstractForRead replicaPlan, - int[] summaryNodes, - Dispatcher.RequestTime requestTime, - AsyncPromise promise) + private void beginReadInternal(TrackedRead.Id readId, + ReadCommand command, + ReplicaPlan.AbstractForRead replicaPlan, + int[] summaryNodes, + Dispatcher.RequestTime requestTime, + AsyncPromise promise, + TrackedLocalReads.Completer completer) { PartialTrackedRead read = null; MutationSummary secondarySummary; @@ -118,6 +123,7 @@ private void beginReadInternal( try { read = command.beginTrackedRead(controller); + read.setFollowUpReadContext(replicaPlan.consistencyLevel(), requestTime); // Create another summary once initial data has been read fully. We do this to catch // any mutations that may have arrived during initial read execution. secondarySummary = command.createMutationSummary(true); @@ -131,8 +137,7 @@ private void beginReadInternal( throw e; } - Coordinator coordinator = - new Coordinator(readId, promise, read, replicaPlan.consistencyLevel(), requestTime); + Coordinator coordinator = new Coordinator(readId, promise, read, replicaPlan.consistencyLevel(), requestTime, completer); coordinators.put(readId, coordinator); // TODO (expected): reconsider the approach to tracked mutation metrics @@ -185,19 +190,21 @@ private static class Coordinator private final PartialTrackedRead read; private final ConsistencyLevel consistencyLevel; private final Dispatcher.RequestTime requestTime; - - Coordinator( - TrackedRead.Id readId, - AsyncPromise promise, - PartialTrackedRead read, - ConsistencyLevel consistencyLevel, - Dispatcher.RequestTime requestTime) + private final Completer completer; + + Coordinator(TrackedRead.Id readId, + AsyncPromise promise, + PartialTrackedRead read, + ConsistencyLevel consistencyLevel, + Dispatcher.RequestTime requestTime, + Completer completer) { this.readId = readId; this.promise = promise; this.read = Preconditions.checkNotNull(read); this.consistencyLevel = consistencyLevel; this.requestTime = requestTime; + this.completer = completer; } boolean isPurgeable(long nanoTime) @@ -215,52 +222,20 @@ void acknowledgeReconcile(Log2OffsetsMap augmentingOffsets) { logger.trace("Reconciliation completed for {}, missing {}", readId, augmentingOffsets); - Stage.READ.submit(() -> { - + Stage.READ.submit(() -> { try { read.augment(augmentingOffsets); - complete(); - } catch (Throwable t) { + // TODO: Does the implementation of this in FollowUpRead in PartialTrackedIndexRead need to be on a READ Stage thread? + completer.complete(promise, read, consistencyLevel, requestTime); + } + catch (Throwable t) + { logger.error("Exception thrown during read completion", t); promise.tryFailure(t); throw t; } }); } - - private void complete() - { - try (PartialTrackedRead.CompletedRead completedRead = read.complete()) - { - TrackedDataResponse response = completedRead.response(); - Future followUp = completedRead.followupRead(response, consistencyLevel, requestTime); - - if (followUp != null) - { - followUp.addCallback((newResponse, error) -> { - if (error != null) - { - promise.tryFailure(error); - return; - } - promise.trySuccess(newResponse); - }); - } - else - { - promise.trySuccess(response); - } - } - catch (Exception e) - { - promise.tryFailure(e); - throw e; - } - finally - { - read.close(); - } - } } } diff --git a/src/java/org/apache/cassandra/service/reads/tracked/TrackedRead.java b/src/java/org/apache/cassandra/service/reads/tracked/TrackedRead.java index 975c2742fc1d..27642583b246 100644 --- a/src/java/org/apache/cassandra/service/reads/tracked/TrackedRead.java +++ b/src/java/org/apache/cassandra/service/reads/tracked/TrackedRead.java @@ -18,8 +18,19 @@ package org.apache.cassandra.service.reads.tracked; +import java.io.IOException; +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; + import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.cassandra.concurrent.Stage; import org.apache.cassandra.db.*; @@ -47,18 +58,6 @@ import org.apache.cassandra.utils.concurrent.Future; import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException; -import java.io.IOException; -import java.util.Collections; -import java.util.Map; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Consumer; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import static java.util.concurrent.TimeUnit.NANOSECONDS; import static org.apache.cassandra.metrics.ClientRequestsMetricsHolder.readMetrics; @@ -249,7 +248,7 @@ private static int endpointToHostId(Replica replica) return metadata.directory.peerId(replica.endpoint()).id(); } - private void start(Dispatcher.RequestTime requestTime, Consumer partialReadConsumer) + private void start(Dispatcher.RequestTime requestTime, Consumer partialReadConsumer, TrackedLocalReads.Completer completer) { // TODO: skip local coordination if this node knows its recovering from an outage // TODO: read speculation @@ -279,7 +278,7 @@ private void start(Dispatcher.RequestTime requestTime, Consumer { AsyncPromise promise = - MutationTrackingService.instance.localReads().beginRead(readId, ClusterMetadata.current(), command, consistencyLevel, summaryNodes, requestTime); + MutationTrackingService.instance.localReads().beginRead(readId, ClusterMetadata.current(), command, consistencyLevel, summaryNodes, requestTime, completer); promise.addCallback((response, error) -> { if (error != null) { @@ -326,12 +325,12 @@ private void start(Dispatcher.RequestTime requestTime, Consumer partialReadConsumer) + public void startLocal(Dispatcher.RequestTime requestTime, Consumer partialReadConsumer, TrackedLocalReads.Completer completer) { - start(requestTime, partialReadConsumer); + start(requestTime, partialReadConsumer, completer); } private void onResponse(TrackedDataResponse response) @@ -450,7 +449,7 @@ public void executeLocally(Message message, ClusterMetadata m AsyncPromise promise = MutationTrackingService.instance .localReads() - .beginRead(readId, metadata, command, consistencyLevel, summaryNodes, requestTime); + .beginRead(readId, metadata, command, consistencyLevel, summaryNodes, requestTime, TrackedLocalReads.Completer.DEFAULT); promise.addCallback((response, error) -> { if (error != null) { diff --git a/src/java/org/apache/cassandra/utils/AbstractIterator.java b/src/java/org/apache/cassandra/utils/AbstractIterator.java index 7dd32b84907c..4a89001c19d1 100644 --- a/src/java/org/apache/cassandra/utils/AbstractIterator.java +++ b/src/java/org/apache/cassandra/utils/AbstractIterator.java @@ -18,12 +18,9 @@ */ package org.apache.cassandra.utils; -import java.util.Iterator; import java.util.NoSuchElementException; -import com.google.common.collect.PeekingIterator; - -public abstract class AbstractIterator implements Iterator, PeekingIterator, CloseableIterator +public abstract class AbstractIterator implements CloseablePeekingIterator { private static enum State { MUST_FETCH, HAS_NEXT, DONE, FAILED } diff --git a/src/java/org/apache/cassandra/utils/CloseablePeekingIterator.java b/src/java/org/apache/cassandra/utils/CloseablePeekingIterator.java new file mode 100644 index 000000000000..fa5d4dd7cb7b --- /dev/null +++ b/src/java/org/apache/cassandra/utils/CloseablePeekingIterator.java @@ -0,0 +1,40 @@ +/* + * 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.utils; + +import java.util.Iterator; + +import com.google.common.collect.PeekingIterator; + +public interface CloseablePeekingIterator extends PeekingIterator, CloseableIterator +{ + static CloseablePeekingIterator wrap(Iterator iterator) + { + return new AbstractIterator<>() + { + @Override + protected V computeNext() + { + if (!iterator.hasNext()) + return endOfData(); + return iterator.next(); + } + }; + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/ReadRepairIndexTest.java b/test/distributed/org/apache/cassandra/distributed/test/ReadRepairIndexTest.java new file mode 100644 index 000000000000..66be3cbc1ffa --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/ReadRepairIndexTest.java @@ -0,0 +1,219 @@ +/* + * 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.distributed.test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import org.junit.AfterClass; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.schema.ReplicationType; +import org.apache.cassandra.service.reads.repair.ReadRepairStrategy; + +import static org.apache.cassandra.distributed.shared.AssertUtils.row; + +@RunWith(Parameterized.class) +public class ReadRepairIndexTest extends TestBaseImpl +{ + private static final int NUM_NODES = 2; + enum IndexType + { + SECONDARY, SAI; + } + + /** + * The read repair strategy to be used + */ + @Parameterized.Parameter + public ReadRepairStrategy strategy; + + /** + * The node to be used as coordinator + */ + @Parameterized.Parameter(1) + public int coordinator; + + /** + * Whether to flush data after mutations + */ + @Parameterized.Parameter(2) + public boolean flush; + + /** + * Whether paging is used for the distributed queries + */ + @Parameterized.Parameter(3) + public boolean paging; + + @Parameterized.Parameter(4) + public ReplicationType replicationType; + + @Parameterized.Parameter(5) + public IndexType indexType; + + + @Parameterized.Parameters(name = "{index}: strategy={0} coordinator={1} flush={2} paging={3} replication={4} index={5}") + public static Collection data() + { + List result = new ArrayList<>(); + for (int coordinator = 1; coordinator <= NUM_NODES; coordinator++) + for (boolean flush : BOOLEANS) + for (boolean paging : BOOLEANS) + for (ReplicationType replication : ReplicationType.values()) + for (IndexType indexType : IndexType.values()) + result.add(new Object[]{ ReadRepairStrategy.BLOCKING, coordinator, flush, paging, replication, indexType}); + return result; + } + + private static Cluster cluster; + + @BeforeClass + public static void setupCluster() throws IOException + { + cluster = Cluster.build(NUM_NODES) + .withConfig(config -> config.set("read_request_timeout", "1m") + .set("write_request_timeout", "1m")) + .start(); + } + + @AfterClass + public static void teardownCluster() + { + if (cluster != null) + cluster.close(); + } + + protected Tester tester(String restriction) + { + return new Tester(restriction, cluster, strategy, coordinator, flush, paging, replicationType, indexType); + } + + protected static class Tester extends ReadRepairQueryTester.AbstractTester + { + private int nameSeq = 0; + private final IndexType indexType; + + public Tester(String restriction, Cluster cluster, ReadRepairStrategy strategy, int coordinator, boolean flush, boolean paging, ReplicationType replicationType, IndexType indexType) + { + super(restriction, cluster, strategy, coordinator, flush, paging, replicationType); + this.indexType = indexType; + } + + @Override + Tester self() + { + return this; + } + + Tester createIndex(String columns) + { + String query = String.format("CREATE INDEX %s_index_%d ON %s(v)", tableName, nameSeq++, qualifiedTableName); + switch (indexType) + { + case SECONDARY: + break; + + case SAI: + query = query + " USING 'sai'"; + break; + default: + throw new AssertionError(indexType); + } + cluster.schemaChange(query); + return this; + } + } + + /** + * A partition that would not be an index hit on one node would be on the other + */ + @Test + public void singlePartitionUpdatedPartition() + { + tester("WHERE k=1 AND v=2") + .createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c))") + .createIndex("v") + .mutate(2, "INSERT INTO %s (k, c, v) VALUES (1, 2, 2)") + .mutate(1, "INSERT INTO %s (k, c, v) VALUES (1, 1, 1)") + .queryColumns("k, c, v", 1, 0, + rows(row(1, 2, 2)), + rows(row(1, 2, 2)), + rows(row(1, 2, 2))) + .tearDown(1, + rows(row(1, 1, 1), row(1, 2, 2)), + rows(row(1, 2, 2))); + + } + @Test + public void rangeReadTest() + { + tester("WHERE v=2") + .createTable("CREATE TABLE %s (k int, v int, PRIMARY KEY (k))") + .createIndex("v") + .mutate(2, "INSERT INTO %s (k, v) VALUES (1, 2)") + .mutate(1, "INSERT INTO %s (k, v) VALUES (2, 1)") + .mutate(2, "INSERT INTO %s (k, v) VALUES (3, 1)") + .mutate(1, "INSERT INTO %s (k, v) VALUES (4, 2)") + .queryColumns("k, v", 2, 0, + rows(row(1, 2), row(4, 2)), + rows(row(1, 2), row(4, 2)), + rows(row(1, 2), row(4, 2))) + .tearDown(2, + rows(row(1, 2), row(2, 1), row(4, 2), row(3, 1)), + (replicationType.isTracked() + ? rows(row(1, 2), row(2, 1), row(4, 2), row(3, 1)) + : rows(row(1, 2), row(2, 1), row(4, 2))), + (replicationType.isTracked() + ? rows(row(1, 2), row(2, 1), row(4, 2), row(3, 1)) + : rows(row(1, 2), row(4, 2), row(3, 1)))); + } + + @Test + public void sortedRangeRead() + { + Assume.assumeTrue("CassandraIndex doesn't support numerical ranges", indexType == IndexType.SAI); + + tester("WHERE v>2") + .createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c))") + .createIndex("v") + .mutate(2, "INSERT INTO %s (k, c, v) VALUES (1, 2, 2)") + .mutate(1, "INSERT INTO %s (k, c, v) VALUES (1, 4, 4)") + .mutate(2, "INSERT INTO %s (k, c, v) VALUES (5, 2, 1)") + .mutate(1, "INSERT INTO %s (k, c, v) VALUES (8, 4, 3)") + .queryColumns("k, c, v", 2, 0, + rows(row(1, 4, 4), row(8, 4, 3)), + rows(row(1, 4, 4), row(8, 4, 3)), + rows(row(1, 4, 4), row(8, 4, 3))) + .tearDown(2, + rows(row(5, 2, 1), row(1, 2, 2), row(1, 4, 4), row(8, 4, 3)), + replicationType.isTracked() + ? rows(row(5, 2, 1), row(1, 2, 2), row(1, 4, 4), row(8, 4, 3)) + : rows(row(1, 4, 4), row(8, 4, 3)), + rows(row(5, 2, 1), row(1, 2, 2), row(1, 4, 4), row(8, 4, 3))); + + } +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/ReadRepairQueryTester.java b/test/distributed/org/apache/cassandra/distributed/test/ReadRepairQueryTester.java index 447bdbde88c0..8522b7710b00 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/ReadRepairQueryTester.java +++ b/test/distributed/org/apache/cassandra/distributed/test/ReadRepairQueryTester.java @@ -137,12 +137,12 @@ protected Tester tester(String restriction) return new Tester(restriction, cluster, strategy, coordinator, flush, paging, replicationType); } - protected static class Tester extends ReadRepairTester + static abstract class AbstractTester> extends ReadRepairTester { private final String restriction; // the tested CQL query WHERE restriction private final String allColumnsQuery; // a SELECT * query for the table using the tested restriction - Tester(String restriction, Cluster cluster, ReadRepairStrategy strategy, int coordinator, boolean flush, boolean paging, ReplicationType replicationType) + AbstractTester(String restriction, Cluster cluster, ReadRepairStrategy strategy, int coordinator, boolean flush, boolean paging, ReplicationType replicationType) { super(cluster, strategy, coordinator, flush, paging, false, replicationType); this.restriction = restriction; @@ -150,12 +150,6 @@ protected static class Tester extends ReadRepairTester allColumnsQuery = String.format("SELECT * FROM %s %s", qualifiedTableName, restriction); } - @Override - Tester self() - { - return this; - } - /** * Runs the tested query with CL=ALL selectig only the specified columns and verifies that it returns the * specified rows. Then, it runs the query again selecting all the columns, and verifies that the first query @@ -168,12 +162,12 @@ Tester self() * @param node1Rows the rows in the first node, which is the one with the most updated data * @param node2Rows the rows in the second node, which is the one meant to receive the RR writes */ - Tester queryColumns(String columns, - long columnsQueryRepairedRows, - long rowsQueryRepairedRows, - Object[][] columnsQueryResults, - Object[][] node1Rows, - Object[][] node2Rows) + T queryColumns(String columns, + long columnsQueryRepairedRows, + long rowsQueryRepairedRows, + Object[][] columnsQueryResults, + Object[][] node1Rows, + Object[][] node2Rows) { // query only the selected columns with CL=ALL to trigger partial read repair on that column String columnsQuery = String.format("SELECT %s FROM %s %s", columns, qualifiedTableName, restriction); @@ -206,7 +200,7 @@ Tester queryColumns(String columns, assertRowsDistributed(columnsQuery, columnsQueryRepairedRows, columnsQueryResults); // query entire rows to repair the rest of the columns, that might trigger new repairs for those columns - return verifyQuery(allColumnsQuery, rowsQueryRepairedRows, node1Rows, node2Rows); + return verifyQuery(allColumnsQuery, rowsQueryRepairedRows, node1Rows, node1Rows, node2Rows); } /** @@ -223,13 +217,13 @@ Tester queryColumns(String columns, * @param node1Rows the rows in the first node, which is the one with the most updated data * @param node2Rows the rows in the second node, which is the one meant to receive the RR writes */ - Tester deleteColumn(String columnDeletion, - String columns, - long columnsQueryRepairedRows, - long rowsQueryRepairedRows, - Object[][] columnsQueryResults, - Object[][] node1Rows, - Object[][] node2Rows) + T deleteColumn(String columnDeletion, + String columns, + long columnsQueryRepairedRows, + long rowsQueryRepairedRows, + Object[][] columnsQueryResults, + Object[][] node1Rows, + Object[][] node2Rows) { assert restriction != null; @@ -250,7 +244,7 @@ Tester deleteColumn(String columnDeletion, * Executes the specified row deletion on just one node and verifies the tested query, to ensure that the tested * query propagates the row deletion. */ - Tester deleteRows(String rowDeletion, long repairedRows, Object[][] node1Rows, Object[][] node2Rows) + T deleteRows(String rowDeletion, long repairedRows, Object[][] node1Rows, Object[][] node2Rows) { mutate(1, rowDeletion); @@ -259,28 +253,28 @@ Tester deleteRows(String rowDeletion, long repairedRows, Object[][] node1Rows, O if (replicationType.isTracked()) repairedRows = Math.min(repairedRows, 1); - return verifyQuery(allColumnsQuery, repairedRows, node1Rows, node2Rows); + return verifyQuery(allColumnsQuery, repairedRows, node1Rows, node1Rows, node2Rows); } - Tester mutate(String... queries) + T mutate(String... queries) { return mutate(1, queries); } - private Tester verifyQuery(String query, long expectedRepairedRows, Object[][] node1Rows, Object[][] node2Rows) + private T verifyQuery(String query, long expectedRepairedRows, Object[][] allRows, Object[][] node1Rows, Object[][] node2Rows) { // verify the per-replica status before running the query distributedly assertRows(cluster.get(1).executeInternal(query), node1Rows); assertRows(cluster.get(2).executeInternal(query), strategy == NONE ? EMPTY_ROWS : node2Rows); // now, run the query with CL=ALL to reconcile and repair the replicas - assertRowsDistributed(query, expectedRepairedRows, node1Rows); + assertRowsDistributed(query, expectedRepairedRows, allRows); // run the query locally again to verify that the distributed query has repaired everything - assertRows(cluster.get(1).executeInternal(query), node1Rows); - assertRows(cluster.get(2).executeInternal(query), strategy == NONE ? EMPTY_ROWS : node1Rows); + assertRows(cluster.get(1).executeInternal(query), allRows); + assertRows(cluster.get(2).executeInternal(query), strategy == NONE ? EMPTY_ROWS : allRows); - return this; + return self(); } /** @@ -295,11 +289,11 @@ void tearDown() * Verifies the final status of the nodes with an unrestricted query, to ensure that the main tested query * hasn't triggered any unexpected repairs. Then, it verifies that the node that hasn't been used as coordinator * hasn't triggered any unexpected repairs. Finally, it drops the table. - * + *

* The expectUnrepaired flag is meant for range query tests where logged replication table special casing * doesn't apply since we do expect the final query to find and repair missing mutations */ - void tearDown(long repairedRows, Object[][] node1Rows, Object[][] node2Rows, boolean expectUnrepaired) + void tearDown(long repairedRows, Object[][] allRows, Object[][] node1Rows, Object[][] node2Rows, boolean expectUnrepaired) { if (replicationType.isTracked() && !expectUnrepaired) { @@ -321,7 +315,7 @@ void tearDown(long repairedRows, Object[][] node1Rows, Object[][] node2Rows, boo // we also expect all pending mutations to be reconciled in the initial read, and none to be reconciled on the verification step repairedRows = 0; } - verifyQuery("SELECT * FROM " + qualifiedTableName, repairedRows, node1Rows, node2Rows); + verifyQuery("SELECT * FROM " + qualifiedTableName, repairedRows, allRows, node1Rows, node2Rows); for (int n = 1; n <= cluster.size(); n++) { if (n == coordinator) @@ -337,7 +331,37 @@ void tearDown(long repairedRows, Object[][] node1Rows, Object[][] node2Rows, boo void tearDown(long repairedRows, Object[][] node1Rows, Object[][] node2Rows) { - tearDown(repairedRows, node1Rows, node2Rows, false); + tearDown(repairedRows, node1Rows, node1Rows, node2Rows, false); + } + + void tearDown(long repairedRows, Object[][] allRows, Object[][] node1Rows, Object[][] node2Rows) + { + tearDown(repairedRows, allRows, node1Rows, node2Rows, false); + } + + void tearDown(long repairedRows, Object[][] node1Rows, Object[][] node2Rows, boolean expectUnrepaired) + { + tearDown(repairedRows, node1Rows, node1Rows, node2Rows, expectUnrepaired); + } + } + + void tearDown(long repairedRows, Object[][] node1Rows, Object[][] node2Rows, boolean expectUnrepaired) + { + + } + + + protected static class Tester extends AbstractTester + { + public Tester(String restriction, Cluster cluster, ReadRepairStrategy strategy, int coordinator, boolean flush, boolean paging, ReplicationType replicationType) + { + super(restriction, cluster, strategy, coordinator, flush, paging, replicationType); + } + + @Override + Tester self() + { + return this; } } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/ReadRepairTester.java b/test/distributed/org/apache/cassandra/distributed/test/ReadRepairTester.java index ad1d3b65aec0..f119be861046 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/ReadRepairTester.java +++ b/test/distributed/org/apache/cassandra/distributed/test/ReadRepairTester.java @@ -35,7 +35,6 @@ import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL; import static org.apache.cassandra.distributed.shared.AssertUtils.assertEquals; -import static org.apache.cassandra.distributed.test.TestBaseImpl.KEYSPACE; /** * Extensible helper class for read repair tests. @@ -45,7 +44,7 @@ public abstract class ReadRepairTester> private static final AtomicInteger seqNumber = new AtomicInteger(); private final String keyspaceName = "ks_" + seqNumber.getAndIncrement(); - private static final String tableName = "tbl"; + static final String tableName = "tbl"; final String qualifiedTableName = keyspaceName + '.' + tableName; protected final Cluster cluster; @@ -116,7 +115,7 @@ T mutate(int node, String... queries) // flush the update node to ensure reads come from sstables if (flush) - cluster.get(node).flush(KEYSPACE); + cluster.get(node).flush(keyspaceName); return self(); } diff --git a/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java b/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java index b23e3f45875a..92d2f8ee40e0 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/sai/StrictFilteringTest.java @@ -19,16 +19,23 @@ package org.apache.cassandra.distributed.test.sai; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; import java.util.Iterator; +import java.util.List; import org.junit.AfterClass; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.test.TestBaseImpl; +import org.apache.cassandra.schema.ReplicationType; import static org.junit.Assert.assertEquals; @@ -43,16 +50,38 @@ * * @see CASSANDRA-19018 */ +@RunWith(Parameterized.class) public class StrictFilteringTest extends TestBaseImpl { private static Cluster CLUSTER; + private static int keyspaceIdx; + + @Parameterized.Parameter + public ReplicationType replicationType; + @BeforeClass public static void setUpCluster() throws IOException { CLUSTER = init(Cluster.build(2).withConfig(config -> config.set("hinted_handoff_enabled", false).with(GOSSIP).with(NETWORK)).start()); } + @Parameterized.Parameters(name = "{index}: replication={0}") + public static Collection data() + { + List result = new ArrayList<>(); + for (ReplicationType replication : ReplicationType.values()) + result.add(new Object[]{replication}); + return result; + } + + @Before + public void setup() + { + KEYSPACE = "ks_" + keyspaceIdx++; + CLUSTER.schemaChange("CREATE KEYSPACE " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': " + CLUSTER.size() + "} AND replication_type='" + replicationType.toString() + "';"); + } + @Test public void shouldDegradeToUnionOnSingleStatic() { diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java index 7367d3967ca6..9ad82f7d31e4 100644 --- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java +++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java @@ -58,6 +58,7 @@ import org.apache.cassandra.db.memtable.AbstractMemtable; import org.apache.cassandra.db.memtable.Memtable; import org.apache.cassandra.db.partitions.FilteredPartition; +import org.apache.cassandra.db.partitions.Partition; import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; import org.apache.cassandra.db.rows.Cell; @@ -967,6 +968,12 @@ public void performSnapshot(String snapshotName) { } + @Override + public Partition snapshotPartition(DecoratedKey partitionKey) + { + return null; + } + @Override public UnfilteredRowIterator rowIterator(DecoratedKey key, Slices slices,