diff --git a/.gitmodules b/.gitmodules index 616dacf610a7..510c7c19e633 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,4 +1,4 @@ [submodule "modules/accord"] path = modules/accord - url = https://github.com/apache/cassandra-accord.git - branch = trunk + url = https://github.com/belliottsmith/cassandra-accord.git + branch = rebootstrap-sq diff --git a/modules/accord b/modules/accord index 520818a004a8..5b35a301d160 160000 --- a/modules/accord +++ b/modules/accord @@ -1 +1 @@ -Subproject commit 520818a004a89217cf86efa6c8fa2968401968ec +Subproject commit 5b35a301d160ede05e69ac1907cd10a4df1799c1 diff --git a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java index 3b40d7e4123a..8373f3eb8877 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java @@ -156,13 +156,13 @@ public static StatementRestrictions empty(StatementType type, TableMetadata tabl return new StatementRestrictions(type, table, IndexHints.NONE, false); } - private StatementRestrictions(StatementType type, TableMetadata table, IndexHints indexHints, boolean allowFiltering) + private StatementRestrictions(StatementType type, TableMetadata table, IndexHints indexHints, boolean allowFilteringOfPrimaryKeys) { this.type = type; this.table = table; this.indexHints = indexHints; this.partitionKeyRestrictions = new PartitionKeyRestrictions(table.partitionKeyAsClusteringComparator()); - this.clusteringColumnsRestrictions = new ClusteringColumnRestrictions(table, allowFiltering); + this.clusteringColumnsRestrictions = new ClusteringColumnRestrictions(table, allowFilteringOfPrimaryKeys); this.nonPrimaryKeyRestrictions = RestrictionSet.empty(); this.notNullColumns = new HashSet<>(); } @@ -370,7 +370,7 @@ else if (operator.requiresIndexing()) } else { - if (!allowFiltering && requiresAllowFilteringIfNotSpecified(table)) + if (!allowFiltering && requiresAllowFilteringIfNotSpecified(table, false)) throw invalidRequest(allowFilteringMessage(state)); } @@ -381,14 +381,14 @@ else if (operator.requiresIndexing()) validateSecondaryIndexSelections(); } - public static boolean requiresAllowFilteringIfNotSpecified(TableMetadata metadata) + public static boolean requiresAllowFilteringIfNotSpecified(TableMetadata metadata, boolean isPrimaryKey) { if (!metadata.isVirtual()) return true; VirtualTable tableNullable = VirtualKeyspaceRegistry.instance.getTableNullable(metadata.id); assert tableNullable != null; - return !tableNullable.allowFilteringImplicitly(); + return isPrimaryKey ? !tableNullable.allowFilteringPrimaryKeysImplicitly() : !tableNullable.allowFilteringImplicitly(); } private void addRestriction(Restriction restriction, IndexRegistry indexRegistry, IndexHints indexHints) @@ -593,7 +593,7 @@ private void processPartitionKeyRestrictions(ClientState state, boolean hasQueri // components must have a EQ. Only the last partition key component can be in IN relation. if (partitionKeyRestrictions.needFiltering()) { - if (!allowFiltering && !forView && !hasQueriableIndex && requiresAllowFilteringIfNotSpecified(table)) + if (!allowFiltering && !forView && !hasQueriableIndex && requiresAllowFilteringIfNotSpecified(table, true)) throw new InvalidRequestException(allowFilteringMessage(state)); isKeyRange = true; diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java index ecdce1437b1f..aa5500356d02 100644 --- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java @@ -1474,7 +1474,7 @@ private StatementRestrictions prepareRestrictions(ClientState state, boundNames, orderings, selectsOnlyStaticColumns, - parameters.allowFiltering || !requiresAllowFilteringIfNotSpecified(metadata), + parameters.allowFiltering || !requiresAllowFilteringIfNotSpecified(metadata, true), forView); } @@ -1700,7 +1700,7 @@ private void checkNeedsFiltering(TableMetadata table, StatementRestrictions rest { // We will potentially filter data if the row filter is not the identity and there isn't any index group // supporting all the expressions in the filter. - if (requiresAllowFilteringIfNotSpecified(table)) + if (requiresAllowFilteringIfNotSpecified(table, true)) checkFalse(restrictions.needFiltering(table), StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE); } } diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java index 25986314a278..f448bcf8724d 100644 --- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java +++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java @@ -647,7 +647,7 @@ public PartitionIterator execute(ConsistencyLevel consistency, ClientState state public UnfilteredPartitionIterator executeLocally(ReadExecutionController executionController) { VirtualTable view = VirtualKeyspaceRegistry.instance.getTableNullable(metadata().id); - UnfilteredPartitionIterator resultIterator = view.select(dataRange, columnFilter(), rowFilter()); + UnfilteredPartitionIterator resultIterator = view.select(dataRange, columnFilter(), rowFilter(), limits()); return limits().filter(rowFilter().filter(resultIterator, nowInSec()), nowInSec(), selectsFullPartition()); } diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java index 7c4864e6db2d..780a8c083f59 100644 --- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java +++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java @@ -1516,7 +1516,7 @@ public PartitionIterator execute(ConsistencyLevel consistency, ClientState state public UnfilteredPartitionIterator executeLocally(ReadExecutionController executionController) { VirtualTable view = VirtualKeyspaceRegistry.instance.getTableNullable(metadata().id); - UnfilteredPartitionIterator resultIterator = view.select(partitionKey, clusteringIndexFilter, columnFilter(), rowFilter()); + UnfilteredPartitionIterator resultIterator = view.select(partitionKey, clusteringIndexFilter, columnFilter(), rowFilter(), limits()); return limits().filter(rowFilter().filter(resultIterator, nowInSec()), nowInSec(), selectsFullPartition()); } diff --git a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java index bc17dd0158f1..83f14a10736a 100644 --- a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java +++ b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java @@ -33,6 +33,7 @@ import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.btree.BTree; /** * Represents which (non-PK) columns (and optionally which sub-part of a column for complex columns) are selected @@ -64,7 +65,6 @@ */ public abstract class ColumnFilter { - public static final ColumnFilter NONE = selection(RegularAndStaticColumns.NONE); public static final Serializer serializer = new Serializer(); @@ -305,6 +305,11 @@ public boolean isWildcard() return false; } + /** + * Rebinds matching columns into a new filter; ignores any missing but fails if any are a different type + */ + public abstract ColumnFilter rebind(TableMetadata newTable); + /** * Returns the CQL string corresponding to this {@code ColumnFilter}. * @@ -630,6 +635,12 @@ public boolean isWildcard() return true; } + @Override + public ColumnFilter rebind(TableMetadata newTable) + { + return new WildCardColumnFilter(ColumnFilter.rebind(newTable, fetchedAndQueried)); + } + @Override protected SortedSetMultimap subSelections() { @@ -779,6 +790,17 @@ public Tester newTester(ColumnMetadata column) return new Tester(fetchingStrategy.fetchesAllColumns(column.isStatic()), s.iterator()); } + @Override + public ColumnFilter rebind(TableMetadata newTable) + { + RegularAndStaticColumns queried = ColumnFilter.rebind(newTable, this.queried); + RegularAndStaticColumns fetched = this.queried == this.fetched ? queried : ColumnFilter.rebind(newTable, this.fetched); + SortedSetMultimap subSelections = this.subSelections; + if (subSelections != null) + subSelections = TreeMultimap.create(subSelections); + return new SelectionColumnFilter(fetchingStrategy, queried, fetched, subSelections); + } + @Override protected SortedSetMultimap subSelections() { @@ -1003,4 +1025,26 @@ private long subSelectionsSerializedSize(SortedSetMultimap builder = BTree.fastBuilder()) + { + for (ColumnMetadata in : columns) + { + ColumnMetadata out = newTable.getColumn(in.name); + if (out != null) + builder.add(out); + } + return Columns.from(builder); + } + } } diff --git a/src/java/org/apache/cassandra/db/filter/RowFilter.java b/src/java/org/apache/cassandra/db/filter/RowFilter.java index 12c98c72782a..16ba694e1424 100644 --- a/src/java/org/apache/cassandra/db/filter/RowFilter.java +++ b/src/java/org/apache/cassandra/db/filter/RowFilter.java @@ -141,7 +141,7 @@ public void addCustomIndexExpression(TableMetadata metadata, IndexMetadata targe add(new CustomExpression(metadata, targetIndex, value)); } - private void add(Expression expression) + public void add(Expression expression) { expression.validate(); expressions.add(expression); @@ -549,6 +549,28 @@ public void validateForIndexing() "Index expression values may not be larger than 64K"); } + /** + * Rebind this expression to a table metadata that is expected to have equivalent columns. + * If any referenced column is missing, returns null; + * if any referenced column has a different type throws an exception + */ + public Expression rebind(TableMetadata newTable) + { + throw new UnsupportedOperationException("Expression " + toString(true) + " does not support rebinding to another table definition"); + } + + protected static ColumnMetadata rebind(ColumnMetadata in, TableMetadata newTable) + { + ColumnMetadata out = newTable.getColumn(in.name); + if (out == null) + return null; + + if (!out.type.equals(in.type) && !out.type.isCompatibleWith(in.type) || !in.type.isCompatibleWith(out.type)) + throw new IllegalArgumentException("The provided TableMetadata is not compatible with the expression"); + + return out; + } + /** * Returns whether the provided row satisfied this expression or not. * @@ -734,6 +756,16 @@ public static class SimpleExpression extends Expression super(column, operator, value); } + @Override + public Expression rebind(TableMetadata newTable) + { + ColumnMetadata out = rebind(column, newTable); + if (out == null) + return null; + + return new SimpleExpression(out, operator, value); + } + @Override public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row, long nowInSec) { @@ -853,6 +885,16 @@ public void validate() throws InvalidRequestException checkBindValueSet(value, "Unsupported unset map value for column %s", column.name); } + @Override + public Expression rebind(TableMetadata newTable) + { + ColumnMetadata out = rebind(column, newTable); + if (out == null) + return null; + + return new MapElementExpression(out, key, operator, value); + } + @Override public ByteBuffer getIndexValue() { @@ -978,6 +1020,12 @@ protected Kind kind() return Kind.CUSTOM; } + @Override + public Expression rebind(TableMetadata newTable) + { + return new CustomExpression(table, targetIndex, value); + } + // Filtering by custom expressions isn't supported yet, so just accept any row @Override public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row, long nowInSec) diff --git a/src/java/org/apache/cassandra/db/marshal/TxnIdUtf8Type.java b/src/java/org/apache/cassandra/db/marshal/TxnIdUtf8Type.java index 784969ead703..17c62be40ad8 100644 --- a/src/java/org/apache/cassandra/db/marshal/TxnIdUtf8Type.java +++ b/src/java/org/apache/cassandra/db/marshal/TxnIdUtf8Type.java @@ -48,6 +48,12 @@ public void validate(V value, ValueAccessor accessor) throws MarshalExcep String describe() { return "TxnId"; } + @Override + public boolean isEmptyValueMeaningless() + { + return true; + } + @Override public TypeSerializer getSerializer() { diff --git a/src/java/org/apache/cassandra/db/virtual/AbstractLazyVirtualTable.java b/src/java/org/apache/cassandra/db/virtual/AbstractLazyVirtualTable.java new file mode 100644 index 000000000000..da7f37e1ed29 --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/AbstractLazyVirtualTable.java @@ -0,0 +1,753 @@ +/* + * 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.virtual; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.UnaryOperator; + +import javax.annotation.Nullable; + +import accord.utils.Invariants; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.Clusterable; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.ClusteringPrefix; +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.RegularAndStaticColumns; +import org.apache.cassandra.db.filter.ClusteringIndexFilter; +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.marshal.AbstractType; +import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.rows.BTreeRow; +import org.apache.cassandra.db.rows.BufferCell; +import org.apache.cassandra.db.rows.ColumnData; +import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.Bounds; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.ReadTimeoutException; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.ClientWarn; +import org.apache.cassandra.utils.BulkIterator; +import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.btree.BTree; +import org.apache.cassandra.utils.btree.UpdateFunction; + +import static org.apache.cassandra.db.ClusteringPrefix.Kind.STATIC_CLUSTERING; +import static org.apache.cassandra.db.ConsistencyLevel.ONE; +import static org.apache.cassandra.utils.Clock.Global.nanoTime; + +/** + * An abstract virtual table implementation that builds the resultset on demand. + */ +public abstract class AbstractLazyVirtualTable implements VirtualTable +{ + public enum OnTimeout { BEST_EFFORT, FAIL } + + // in the special case where we know we have enough rows in the collector, throw this exception to terminate early + public static class InternalDoneException extends RuntimeException {} + // in the special case where we have timed out, throw this exception to terminate early + public static class InternalTimeoutException extends RuntimeException {} + + public static class FilterRange + { + final V min, max; + public FilterRange(V min, V max) + { + this.min = min; + this.max = max; + } + } + + public interface PartitionsCollector + { + DataRange dataRange(); + RowFilter rowFilter(); + ColumnFilter columnFilter(); + DataLimits limits(); + long nowInSeconds(); + long timestampMicros(); + long deadlineNanos(); + boolean isEmpty(); + + RowCollector row(Object... primaryKeys); + PartitionCollector partition(Object... partitionKeys); + UnfilteredPartitionIterator finish(); + + @Nullable Object[] singlePartitionKey(); + FilterRange filters(String column, Function translate, UnaryOperator exclusiveStart, UnaryOperator exclusiveEnd); + } + + public interface PartitionCollector + { + void collect(Consumer addTo); + } + + public interface RowsCollector + { + RowCollector add(Object... clusteringKeys); + } + + public interface RowCollector + { + default void lazyCollect(Consumer addToIfNeeded) { eagerCollect(addToIfNeeded); } + void eagerCollect(Consumer addToNow); + } + + public interface ColumnsCollector + { + /** + * equivalent to + * {@code + * if (value == null) add(columnName, null); + * else if (f1.apply(value) == null) add(columnName, f1.apply(value)); + * else add(columnName, f2.apply(f1.apply(value))); + * } + */ + ColumnsCollector add(String columnName, V1 value, Function f1, Function f2); + + default ColumnsCollector add(String columnName, V value, Function transform) + { + return add(columnName, value, Function.identity(), transform); + } + default ColumnsCollector add(String columnName, Object value) + { + return add(columnName, value, Function.identity()); + } + } + + public static class SimplePartitionsCollector implements PartitionsCollector + { + final TableMetadata metadata; + final boolean isSorted; + final boolean isSortedByPartitionKey; + + final Map columnLookup = new HashMap<>(); + final NavigableMap partitions; + + final DataRange dataRange; + final ColumnFilter columnFilter; + final RowFilter rowFilter; + final DataLimits limits; + + final long startedAtNanos = Clock.Global.nanoTime(); + final long deadlineNanos; + + final long nowInSeconds = Clock.Global.nowInSeconds(); + final long timestampMicros; + + int totalRowCount; + int lastFilteredTotalRowCount, lastFilteredPartitionCount; + + @Override public DataRange dataRange() { return dataRange; } + @Override public RowFilter rowFilter() { return rowFilter; } + @Override public ColumnFilter columnFilter() { return columnFilter; } + @Override public DataLimits limits() { return limits; } + @Override public long nowInSeconds() { return nowInSeconds; } + @Override public long timestampMicros() { return timestampMicros; } + @Override public long deadlineNanos() { return deadlineNanos; } + @Override public boolean isEmpty() { return totalRowCount == 0; } + + public SimplePartitionsCollector(TableMetadata metadata, boolean isSorted, boolean isSortedByPartitionKey, + DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) + { + this.metadata = metadata; + this.isSorted = isSorted; + this.isSortedByPartitionKey = isSortedByPartitionKey; + this.dataRange = dataRange; + this.columnFilter = columnFilter; + this.rowFilter = rowFilter; + this.limits = limits; + this.timestampMicros = FBUtilities.timestampMicros(); + this.deadlineNanos = startedAtNanos + DatabaseDescriptor.getReadRpcTimeout(TimeUnit.NANOSECONDS); + this.partitions = new TreeMap<>(dataRange.isReversed() ? DecoratedKey.comparator.reversed() : DecoratedKey.comparator); + for (ColumnMetadata cm : metadata.columns()) + columnLookup.put(cm.name.toString(), cm); + } + + public Object[] singlePartitionKey() + { + AbstractBounds bounds = dataRange().keyRange(); + if (!bounds.isStartInclusive() || !bounds.isEndInclusive() || !bounds.left.equals(bounds.right) || !(bounds.left instanceof DecoratedKey)) + return null; + + return composePartitionKeys((DecoratedKey) bounds.left, metadata); + } + + @Override + public PartitionCollector partition(Object ... partitionKeys) + { + int pkSize = metadata.partitionKeyColumns().size(); + if (pkSize != partitionKeys.length) + throw new IllegalArgumentException(); + + DecoratedKey partitionKey = decomposePartitionKeys(metadata, partitionKeys); + if (!dataRange.contains(partitionKey)) + return dropCks -> {}; + + return partitions.computeIfAbsent(partitionKey, SimplePartition::new); + } + + @Override + public UnfilteredPartitionIterator finish() + { + final Iterator partitions = this.partitions.values().iterator(); + return new UnfilteredPartitionIterator() + { + @Override public TableMetadata metadata() { return metadata; } + @Override public void close() {} + + @Override + public boolean hasNext() + { + return partitions.hasNext(); + } + + @Override + public UnfilteredRowIterator next() + { + SimplePartition partition = partitions.next(); + Iterator rows = partition.rows(); + + return new UnfilteredRowIterator() + { + @Override public TableMetadata metadata() { return metadata; } + @Override public boolean isReverseOrder() { return dataRange.isReversed(); } + @Override public RegularAndStaticColumns columns() { return columnFilter.fetchedColumns(); } + @Override public DecoratedKey partitionKey() { return partition.key; } + + @Override public Row staticRow() { return partition.staticRow(); } + @Override public boolean hasNext() { return rows.hasNext(); } + @Override public Unfiltered next() { return rows.next(); } + + @Override public void close() {} + @Override public DeletionTime partitionLevelDeletion() { return DeletionTime.LIVE; } + @Override public EncodingStats stats() { return EncodingStats.NO_STATS; } + }; + } + }; + } + + @Override + @Nullable + public FilterRange filters(String columnName, Function translate, UnaryOperator exclusiveStart, UnaryOperator exclusiveEnd) + { + ColumnMetadata column = columnLookup.get(columnName); + O min = null, max = null; + for (RowFilter.Expression expression : rowFilter().getExpressions()) + { + if (!expression.column().equals(column)) + continue; + + O bound = translate.apply((I)column.type.compose(expression.getIndexValue())); + switch (expression.operator()) + { + default: throw new InvalidRequestException("Operator " + expression.operator() + " not supported for txn_id"); + case EQ: min = max = bound; break; + case LTE: max = bound; break; + case LT: max = exclusiveEnd.apply(bound); break; + case GTE: min = bound; break; + case GT: min = exclusiveStart.apply(bound); break; + } + } + + return new FilterRange<>(min, max); + } + + @Override + public RowCollector row(Object... primaryKeys) + { + int pkSize = metadata.partitionKeyColumns().size(); + int ckSize = metadata.clusteringColumns().size(); + if (pkSize + ckSize != primaryKeys.length) + throw new IllegalArgumentException(); + + Object[] partitionKeyValues = new Object[pkSize]; + Object[] clusteringValues = new Object[ckSize]; + + System.arraycopy(primaryKeys, 0, partitionKeyValues, 0, pkSize); + System.arraycopy(primaryKeys, pkSize, clusteringValues, 0, ckSize); + + DecoratedKey partitionKey = decomposePartitionKeys(metadata, partitionKeyValues); + Clustering clustering = decomposeClusterings(metadata, clusteringValues); + + if (!dataRange.contains(partitionKey) || !dataRange.clusteringIndexFilter(partitionKey).selects(clustering)) + return drop -> {}; + + return partitions.computeIfAbsent(partitionKey, SimplePartition::new).row(clustering); + } + + private final class SimplePartition implements PartitionCollector, RowsCollector + { + private final DecoratedKey key; + // we assume no duplicate rows, and impose the condition lazily + private SimpleRow[] rows; + private int rowCount; + private SimpleRow staticRow; + private boolean dropRows; + + private SimplePartition(DecoratedKey key) + { + this.key = key; + this.rows = new SimpleRow[1]; + } + + @Override + public void collect(Consumer addTo) + { + addTo.accept(this); + } + + @Override + public RowCollector add(Object... clusteringKeys) + { + int ckSize = metadata.clusteringColumns().size(); + if (ckSize != clusteringKeys.length) + throw new IllegalArgumentException(); + + return row(decomposeClusterings(metadata, clusteringKeys)); + } + + RowCollector row(Clustering clustering) + { + if (nanoTime() > deadlineNanos) + throw new InternalTimeoutException(); + + if (dropRows || !dataRange.clusteringIndexFilter(key).selects(clustering)) + return drop -> {}; + + if (totalRowCount >= limits.count()) + { + boolean filter; + if (!isSorted || !isSortedByPartitionKey || lastFilteredPartitionCount == partitions.size()) + { + filter = totalRowCount / 2 >= Math.max(1024, limits.count()); + } + else + { + int rowsAddedSinceLastFiltered = totalRowCount - lastFilteredTotalRowCount; + int threshold = Math.max(32, Math.min(1024, lastFilteredTotalRowCount / 2)); + filter = lastFilteredTotalRowCount == 0 || rowsAddedSinceLastFiltered >= threshold; + } + + if (filter) + { + // first filter within each partition + for (SimplePartition partition : partitions.values()) + { + int curCount = partition.rowCount; + int newCount = Math.min(curCount, limits.perPartitionCount()); + newCount = partition.filterAndSortAndTruncate(newCount); + totalRowCount -= curCount - newCount; + } + + // then drop any partitions that completely fall outside our limit + Iterator iter = partitions.descendingMap().values().iterator(); + SimplePartition last; + while (true) + { + SimplePartition next = last = iter.next(); + if (totalRowCount - next.rowCount < limits.count()) + break; + + iter.remove(); + totalRowCount -= next.rowCount; + if (next == this) + dropRows = true; + } + + // possibly truncate the last partition if it partially falls outside the limit + int overflow = Math.max(0, totalRowCount - limits.count()); + int curCount = last.rowCount; + int newCount = curCount - overflow; + newCount = last.filterAndSortAndTruncate(newCount); + totalRowCount -= curCount - newCount; + lastFilteredTotalRowCount = totalRowCount; + lastFilteredPartitionCount = partitions.size(); + + if (isSortedByPartitionKey && totalRowCount - newCount >= limits.count()) + throw new InternalDoneException(); + + if (isSorted && totalRowCount >= limits.count()) + throw new InternalDoneException(); + + if (dropRows) + return drop -> {}; + } + } + + SimpleRow result = new SimpleRow(clustering); + if (clustering.kind() == STATIC_CLUSTERING) + { + Invariants.require(staticRow == null); + staticRow = result; + } + else + { + totalRowCount++; + if (rowCount == rows.length) + rows = Arrays.copyOf(rows, Math.max(8, rowCount * 2)); + rows[rowCount++] = result; + } + return result; + } + + void filterAndSort() + { + int newCount = 0; + for (int i = 0 ; i < rowCount; ++i) + { + if (rows[i].rowFilterIncludes()) + { + if (newCount != i) + rows[newCount] = rows[i]; + newCount++; + } + } + totalRowCount -= (rowCount - newCount); + Arrays.fill(rows, newCount, rowCount, null); + rowCount = newCount; + Arrays.sort(rows, 0, newCount, rowComparator()); + } + + int filterAndSortAndTruncate(int newCount) + { + Invariants.requireArgument(newCount <= rowCount); + filterAndSort(); + if (rowCount < newCount) + return rowCount; + + Arrays.fill(rows, newCount, rowCount, null); + rowCount = newCount; + return newCount; + } + + private Comparator rowComparator() + { + Comparator cmp = dataRange.isReversed() ? metadata.comparator.reversed() : metadata.comparator; + return (a, b) -> cmp.compare(a.clustering, b.clustering); + } + + Row staticRow() + { + if (staticRow == null) + return null; + + return staticRow.materialiseAndFilter(); + } + + Iterator rows() + { + filterAndSort(); + return Arrays.stream(rows, 0, rowCount).map(SimpleRow::materialiseAndFilter).iterator(); + } + + private final class SimpleRow implements RowCollector + { + final Clustering clustering; + SomeColumns state; + + private SimpleRow(Clustering clustering) + { + this.clustering = clustering; + } + + @Override + public void lazyCollect(Consumer addToIfNeeded) + { + Invariants.require(state == null); + state = new LazyColumnsCollector(addToIfNeeded); + } + + @Override + public void eagerCollect(Consumer addToNow) + { + Invariants.require(state == null); + state = new EagerColumnsCollector(addToNow); + } + + boolean rowFilterIncludes() + { + return null != materialiseAndFilter(); + } + + Row materialiseAndFilter() + { + if (state == null) + return null; + + FilteredRow filtered = state.materialiseAndFilter(this); + state = filtered; + return filtered == null ? null : filtered.row; + } + + DecoratedKey partitionKey() + { + return SimplePartition.this.key; + } + + SimplePartitionsCollector collector() + { + return SimplePartitionsCollector.this; + } + } + } + + static abstract class SomeColumns + { + abstract FilteredRow materialiseAndFilter(SimplePartition.SimpleRow parent); + } + + static class LazyColumnsCollector extends SomeColumns + { + final Consumer lazy; + LazyColumnsCollector(Consumer lazy) + { + this.lazy = lazy; + } + + @Override + FilteredRow materialiseAndFilter(SimplePartition.SimpleRow parent) + { + return parent.collector().new EagerColumnsCollector(lazy).materialiseAndFilter(parent); + } + } + + class EagerColumnsCollector extends SomeColumns implements ColumnsCollector + { + Object[] columns = new Object[4]; + int columnCount; + + public EagerColumnsCollector(Consumer add) + { + add.accept(this); + } + + @Override + public ColumnsCollector add(String name, V1 v1, Function f1, Function f2) + { + ColumnMetadata cm = columnLookup.get(name); + if (!columnFilter.fetches(cm)) + return this; + + V2 v2 = f1.apply(v1); + if (v2 == null) + return this; + + Object result = f2.apply(v2); + if (result == null) + return this; + + if (columnCount * 2 == columns.length) + columns = Arrays.copyOf(columns, columnCount * 4); + + columns[columnCount * 2] = cm; + columns[columnCount * 2 + 1] = result; + ++columnCount; + return this; + } + + @Override + FilteredRow materialiseAndFilter(SimplePartition.SimpleRow parent) + { + for (int i = 0 ; i < columnCount ; i++) + { + ColumnMetadata cm = (ColumnMetadata) columns[i * 2]; + Object value = columns[i * 2 + 1]; + ByteBuffer bb = value instanceof ByteBuffer ? (ByteBuffer)value : decompose(cm.type, value); + columns[i] = BufferCell.live(cm, timestampMicros, bb); + } + Arrays.sort(columns, 0, columnCount, (a, b) -> ColumnData.comparator.compare((BufferCell)a, (BufferCell)b)); + Object[] btree = BTree.build(BulkIterator.of(columns), columnCount, UpdateFunction.noOp); + BTreeRow row = BTreeRow.create(parent.clustering, LivenessInfo.create(timestampMicros, nowInSeconds), Row.Deletion.LIVE, btree); + if (!rowFilter.isSatisfiedBy(metadata, parent.partitionKey(), row, nowInSeconds)) + return null; + return new FilteredRow(row); + } + } + + static class FilteredRow extends SomeColumns + { + final Row row; + FilteredRow(Row row) + { + this.row = row; + } + + @Override + FilteredRow materialiseAndFilter(SimplePartition.SimpleRow parent) + { + return this; + } + } + } + + protected final TableMetadata metadata; + private final OnTimeout onTimeout; + private final Sorted sorted, sortedByPartitionKey; + + protected AbstractLazyVirtualTable(TableMetadata metadata, OnTimeout onTimeout, Sorted sorted) + { + this(metadata, onTimeout, sorted, sorted); + } + + protected AbstractLazyVirtualTable(TableMetadata metadata, OnTimeout onTimeout, Sorted sorted, Sorted sortedByPartitionKey) + { + if (!metadata.isVirtual()) + throw new IllegalArgumentException("Cannot instantiate a non-virtual table"); + + this.metadata = metadata; + this.onTimeout = onTimeout; + this.sorted = sorted; + this.sortedByPartitionKey = sortedByPartitionKey; + } + + @Override + public TableMetadata metadata() + { + return metadata; + } + + public OnTimeout onTimeout() { return onTimeout; } + + protected PartitionsCollector collector(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) + { + boolean isSorted = isSorted(sorted, !dataRange.isReversed()); + boolean isSortedByPartitionKey = isSorted || isSorted(sortedByPartitionKey, !dataRange.isReversed()); + return new SimplePartitionsCollector(metadata, isSorted, isSortedByPartitionKey, dataRange, columnFilter, rowFilter, limits); + } + + + private static boolean isSorted(Sorted sorted, boolean asc) + { + return sorted == Sorted.SORTED || sorted == (asc ? Sorted.ASC : Sorted.DESC); + } + + protected abstract void collect(PartitionsCollector collector); + + @Override + public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) + { + return select(new DataRange(new Bounds<>(partitionKey, partitionKey), clusteringIndexFilter), columnFilter, rowFilter, limits); + } + + @Override + public final UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) + { + PartitionsCollector collector = collector(dataRange, columnFilter, rowFilter, limits); + try + { + collect(collector); + } + catch (InternalDoneException ignore) {} + catch (InternalTimeoutException ignore) + { + if (onTimeout != OnTimeout.BEST_EFFORT || collector.isEmpty()) + throw new ReadTimeoutException(ONE, 0, 1, false); + ClientWarn.instance.warn("Ran out of time. Returning best effort."); + } + return collector.finish(); + } + + @Override + public void apply(PartitionUpdate update) + { + throw new InvalidRequestException("Modification is not supported by table " + metadata); + } + + @Override + public void truncate() + { + throw new InvalidRequestException("Truncation is not supported by table " + metadata); + } + + @Override + public String toString() + { + return metadata().toString(); + } + + static Object[] composePartitionKeys(DecoratedKey decoratedKey, TableMetadata metadata) + { + if (metadata.partitionKeyColumns().size() == 1) + return new Object[] { metadata.partitionKeyType.compose(decoratedKey.getKey()) }; + + ByteBuffer[] split = ((CompositeType)metadata.partitionKeyType).split(decoratedKey.getKey()); + Object[] result = new Object[split.length]; + for (int i = 0 ; i < split.length ; ++i) + result[i] = metadata.partitionKeyColumns().get(i).type.compose(split[i]); + return result; + } + + static Object[] composeClusterings(ClusteringPrefix clustering, TableMetadata metadata) + { + Object[] result = new Object[clustering.size()]; + for (int i = 0 ; i < result.length ; ++i) + result[i] = metadata.clusteringColumns().get(i).type.compose(clustering.get(i), clustering.accessor()); + return result; + } + + private static ByteBuffer decompose(AbstractType type, Object value) + { + return type.decomposeUntyped(value); + } + + static DecoratedKey decomposePartitionKeys(TableMetadata metadata, Object... partitionKeys) + { + ByteBuffer partitionKey = partitionKeys.length == 1 + ? decompose(metadata.partitionKeyType, partitionKeys[0]) + : ((CompositeType) metadata.partitionKeyType).decompose(partitionKeys); + return metadata.partitioner.decorateKey(partitionKey); + } + + static Clustering decomposeClusterings(TableMetadata metadata, Object... clusteringKeys) + { + if (clusteringKeys.length == 0) + return Clustering.EMPTY; + + ByteBuffer[] clusteringByteBuffers = new ByteBuffer[clusteringKeys.length]; + for (int i = 0; i < clusteringKeys.length; i++) + { + if (clusteringKeys[i] instanceof ByteBuffer) clusteringByteBuffers[i] = (ByteBuffer) clusteringKeys[i]; + else clusteringByteBuffers[i] = decompose(metadata.clusteringColumns().get(i).type, clusteringKeys[i]); + } + return Clustering.make(clusteringByteBuffers); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/AbstractMutableLazyVirtualTable.java b/src/java/org/apache/cassandra/db/virtual/AbstractMutableLazyVirtualTable.java new file mode 100644 index 000000000000..be928c5f52f5 --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/AbstractMutableLazyVirtualTable.java @@ -0,0 +1,135 @@ +/* + * 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.virtual; + +import java.util.Iterator; + +import javax.annotation.Nullable; + +import org.apache.cassandra.db.DeletionInfo; +import org.apache.cassandra.db.RangeTombstone; +import org.apache.cassandra.db.Slice; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.rows.Cell; +import org.apache.cassandra.db.rows.ColumnData; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.TableMetadata; + +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; +import static org.apache.cassandra.db.ClusteringPrefix.Kind.STATIC_CLUSTERING; + +/** + * An abstract virtual table implementation that builds the resultset on demand and allows fine-grained source + * modification via INSERT/UPDATE, DELETE and TRUNCATE operations. + * + * Virtual table implementation need to be thread-safe has they can be called from different threads. + */ +public abstract class AbstractMutableLazyVirtualTable extends AbstractLazyVirtualTable +{ + protected AbstractMutableLazyVirtualTable(TableMetadata metadata, OnTimeout onTimeout, Sorted sorted) + { + super(metadata, onTimeout, sorted); + } + + protected AbstractMutableLazyVirtualTable(TableMetadata metadata, OnTimeout onTimeout, Sorted sorted, Sorted sortedByPartitionKey) + { + super(metadata, onTimeout, sorted, sortedByPartitionKey); + } + + protected void applyPartitionDeletion(Object[] partitionKeys) + { + throw invalidRequest("Partition deletion is not supported by table %s", metadata()); + } + + protected void applyRangeTombstone(Object[] partitionKey, Object[] start, boolean startInclusive, Object[] end, boolean endInclusive) + { + throw invalidRequest("Range deletion is not supported by table %s", metadata()); + } + + protected void applyRowDeletion(Object[] partitionKey, @Nullable Object[] clusteringKeys) + { + throw invalidRequest("Row deletion is not supported by table %s", metadata()); + } + + protected void applyRowUpdate(Object[] partitionKey, @Nullable Object[] clusteringColumns, ColumnMetadata[] columns, Object[] values) + { + throw invalidRequest("Column modification is not supported by table %s", metadata()); + } + + private void applyRangeTombstone(Object[] pks, RangeTombstone rt) + { + Slice slice = rt.deletedSlice(); + Object[] starts = composeClusterings(slice.start(), metadata()); + Object[] ends = composeClusterings(slice.end(), metadata()); + applyRangeTombstone(pks, starts, slice.start().isInclusive(), ends, slice.end().isInclusive()); + } + + private void applyRow(Object[] pks, Row row) + { + Object[] cks = row.clustering().kind() == STATIC_CLUSTERING ? null : composeClusterings(row.clustering(), metadata()); + if (!row.deletion().isLive()) + { + applyRowDeletion(pks, cks); + } + else + { + ColumnMetadata[] columns = new ColumnMetadata[row.columnCount()]; + Object[] values = new Object[row.columnCount()]; + int i = 0; + for (ColumnData cd : row) + { + ColumnMetadata cm = cd.column(); + if (cm.isComplex()) + throw new InvalidRequestException(metadata() + " does not support complex column updates"); + Cell cell = (Cell)cd; + columns[i] = cm; + if (!cell.isTombstone()) + values[i] = cm.type.compose(cell.value(), cell.accessor()); + ++i; + } + applyRowUpdate(pks, cks, columns, values); + } + } + + public void apply(PartitionUpdate update) + { + TableMetadata metadata = metadata(); + Object[] pks = composePartitionKeys(update.partitionKey(), metadata); + + DeletionInfo deletionInfo = update.deletionInfo(); + if (!deletionInfo.getPartitionDeletion().isLive()) + { + applyPartitionDeletion(pks); + } + else if (deletionInfo.hasRanges()) + { + Iterator iter = deletionInfo.rangeIterator(false); + while (iter.hasNext()) + applyRangeTombstone(pks, iter.next()); + } + else + { + for (Row row : update) + applyRow(pks, row); + if (!update.staticRow().isEmpty()) + applyRow(pks, update.staticRow()); + } + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/AbstractVirtualTable.java b/src/java/org/apache/cassandra/db/virtual/AbstractVirtualTable.java index a32ea67ab69f..2c4a04e5a0f3 100644 --- a/src/java/org/apache/cassandra/db/virtual/AbstractVirtualTable.java +++ b/src/java/org/apache/cassandra/db/virtual/AbstractVirtualTable.java @@ -29,6 +29,7 @@ import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.db.filter.ClusteringIndexFilter; 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.AbstractUnfilteredPartitionIterator; import org.apache.cassandra.db.partitions.PartitionUpdate; @@ -76,7 +77,7 @@ public DataSet data(DecoratedKey partitionKey) } @Override - public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter) + public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) { Partition partition = data(partitionKey).getPartition(partitionKey); @@ -89,7 +90,7 @@ public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringI } @Override - public final UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter) + public final UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) { DataSet data = data(); diff --git a/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java b/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java index 87029aa94fd2..1f796bdd151e 100644 --- a/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java +++ b/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java @@ -18,44 +18,36 @@ package org.apache.cassandra.db.virtual; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Comparator; import java.util.Date; -import java.util.HashSet; -import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.Optional; -import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.TimeoutException; import java.util.function.BiConsumer; import java.util.function.Function; +import java.util.function.UnaryOperator; import java.util.stream.Collectors; import javax.annotation.Nullable; -import com.google.common.collect.BoundType; -import com.google.common.collect.Range; -import com.google.common.collect.Sets; - import accord.coordinate.AbstractCoordination; import accord.coordinate.Coordination; import accord.coordinate.Coordinations; import accord.coordinate.PrepareRecovery; import accord.coordinate.tracking.AbstractTracker; +import accord.local.cfk.CommandsForKey.TxnInfo; +import accord.primitives.RoutingKeys; import accord.utils.SortedListMap; -import org.apache.cassandra.cql3.Operator; -import org.apache.cassandra.db.EmptyIterators; -import org.apache.cassandra.db.filter.ClusteringIndexFilter; -import org.apache.cassandra.db.filter.ColumnFilter; -import org.apache.cassandra.db.filter.RowFilter; -import org.apache.cassandra.db.partitions.SingletonUnfilteredPartitionIterator; -import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; -import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.db.marshal.TxnIdUtf8Type; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,8 +66,6 @@ import accord.local.CommandStores.LatentStoreSelector; import accord.local.Commands; import accord.local.DurableBefore; -import accord.local.LoadKeys; -import accord.local.LoadKeysFor; import accord.local.MaxConflicts; import accord.local.Node; import accord.local.PreLoadContext; @@ -92,10 +82,8 @@ import accord.primitives.ProgressToken; import accord.primitives.Route; import accord.primitives.SaveStatus; -import accord.primitives.Status; import accord.primitives.Timestamp; import accord.primitives.TxnId; -import accord.utils.Invariants; import accord.utils.UnhandledEnum; import accord.utils.async.AsyncChain; import accord.utils.async.AsyncChains; @@ -110,11 +98,11 @@ import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.Int32Type; import org.apache.cassandra.db.marshal.UTF8Type; -import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.LocalPartitioner; import org.apache.cassandra.dht.NormalizedRanges; import org.apache.cassandra.dht.Token; import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; @@ -126,7 +114,9 @@ import org.apache.cassandra.service.accord.AccordKeyspace; import org.apache.cassandra.service.accord.AccordService; import org.apache.cassandra.service.accord.AccordTracing; -import org.apache.cassandra.service.accord.CommandStoreTxnBlockedGraph; +import org.apache.cassandra.service.accord.DebugBlockedTxns; +import org.apache.cassandra.service.accord.IAccordService; +import org.apache.cassandra.service.accord.JournalKey; import org.apache.cassandra.service.accord.api.AccordAgent; import org.apache.cassandra.service.accord.api.TokenKey; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; @@ -143,15 +133,20 @@ import static accord.local.RedundantStatus.Property.LOCALLY_REDUNDANT; import static accord.local.RedundantStatus.Property.LOCALLY_SYNCED; import static accord.local.RedundantStatus.Property.LOCALLY_WITNESSED; +import static accord.local.RedundantStatus.Property.LOG_UNAVAILABLE; import static accord.local.RedundantStatus.Property.QUORUM_APPLIED; -import static accord.local.RedundantStatus.Property.PRE_BOOTSTRAP; +import static accord.local.RedundantStatus.Property.UNREADY; import static accord.local.RedundantStatus.Property.SHARD_APPLIED; import static com.google.common.collect.ImmutableList.toImmutableList; import static java.lang.String.format; import static java.util.concurrent.TimeUnit.NANOSECONDS; import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; +import static org.apache.cassandra.db.virtual.AbstractLazyVirtualTable.OnTimeout.BEST_EFFORT; +import static org.apache.cassandra.db.virtual.AbstractLazyVirtualTable.OnTimeout.FAIL; +import static org.apache.cassandra.db.virtual.VirtualTable.Sorted.ASC; +import static org.apache.cassandra.db.virtual.VirtualTable.Sorted.SORTED; +import static org.apache.cassandra.db.virtual.VirtualTable.Sorted.UNSORTED; import static org.apache.cassandra.schema.SchemaConstants.VIRTUAL_ACCORD_DEBUG; -import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; import static org.apache.cassandra.utils.MonotonicClock.Global.approxTime; public class AccordDebugKeyspace extends VirtualKeyspace @@ -175,6 +170,8 @@ public class AccordDebugKeyspace extends VirtualKeyspace public static final String TXN_TRACES = "txn_traces"; public static final String TXN_OPS = "txn_ops"; + private static final Function TO_STRING = AccordDebugKeyspace::toStringOrNull; + public static final AccordDebugKeyspace instance = new AccordDebugKeyspace(); private AccordDebugKeyspace() @@ -202,7 +199,7 @@ private AccordDebugKeyspace() } // TODO (desired): human readable packed key tracker (but requires loading Txn, so might be preferable to only do conditionally) - public static final class ExecutorsTable extends AbstractVirtualTable + public static final class ExecutorsTable extends AbstractLazyVirtualTable { private ExecutorsTable() { @@ -221,43 +218,44 @@ private ExecutorsTable() " keys_loading text,\n" + " keys_loading_for text,\n" + " PRIMARY KEY (executor_id, status, position, unique_position)" + - ')', UTF8Type.instance)); + ')', Int32Type.instance), FAIL, ASC); } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { AccordCommandStores commandStores = (AccordCommandStores) AccordService.instance().node().commandStores(); - SimpleDataSet ds = new SimpleDataSet(metadata()); - + // TODO (desired): we can easily also support sorted collection for DESC queries for (AccordExecutor executor : commandStores.executors()) { - int uniquePos = 0; int executorId = executor.executorId(); - AccordExecutor.TaskInfo prev = null; - for (AccordExecutor.TaskInfo info : executor.taskSnapshot()) - { - if (prev != null && info.status() == prev.status() && info.position() == prev.position()) ++uniquePos; - else uniquePos = 0; - prev = info; - PreLoadContext preLoadContext = info.preLoadContext(); - ds.row(executorId, Objects.toString(info.status()), info.position(), uniquePos) - .column("description", info.describe()) - .column("command_store_id", info.commandStoreId()) - .column("txn_id", preLoadContext == null ? null : toStringOrNull(preLoadContext.primaryTxnId())) - .column("txn_id_additional", preLoadContext == null ? null : toStringOrNull(preLoadContext.additionalTxnId())) - .column("keys", preLoadContext == null ? null : toStringOrNull(preLoadContext.keys())) - .column("keys_loading", preLoadContext == null ? null : toStringOrNull(preLoadContext.loadKeys())) - .column("keys_loading_for", preLoadContext == null ? null : toStringOrNull(preLoadContext.loadKeysFor())) - ; - } + collector.partition(executorId).collect(rows -> { + int uniquePos = 0; + AccordExecutor.TaskInfo prev = null; + for (AccordExecutor.TaskInfo info : executor.taskSnapshot()) + { + if (prev != null && info.status() == prev.status() && info.position() == prev.position()) ++uniquePos; + else uniquePos = 0; + prev = info; + PreLoadContext preLoadContext = info.preLoadContext(); + rows.add(info.status().name(), info.position(), uniquePos) + .lazyCollect(columns -> { + columns.add("description", info.describe()) + .add("command_store_id", info.commandStoreId()) + .add("txn_id", preLoadContext, PreLoadContext::primaryTxnId, TO_STRING) + .add("txn_id_additional", preLoadContext, PreLoadContext::additionalTxnId, TO_STRING) + .add("keys", preLoadContext, PreLoadContext::keys, TO_STRING) + .add("keys_loading", preLoadContext, PreLoadContext::loadKeys, TO_STRING) + .add("keys_loading_for", preLoadContext, PreLoadContext::loadKeysFor, TO_STRING); + }); + } + }); } - return ds; } } // TODO (desired): human readable packed key tracker (but requires loading Txn, so might be preferable to only do conditionally) - public static final class CoordinationsTable extends AbstractVirtualTable + public static final class CoordinationsTable extends AbstractLazyVirtualTable { private CoordinationsTable() { @@ -275,48 +273,37 @@ private CoordinationsTable() " replies text,\n" + " tracker text,\n" + " PRIMARY KEY (txn_id, kind, coordination_id)" + - ')', UTF8Type.instance)); + ')', TxnIdUtf8Type.instance), FAIL, UNSORTED); } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { Coordinations coordinations = AccordService.instance().node().coordinations(); - SimpleDataSet ds = new SimpleDataSet(metadata()); for (Coordination c : coordinations) { - ds.row(toStringOrNull(c.txnId()), c.kind().toString(), c.coordinationId()) - .column("nodes", toStringOrNull(c.nodes())) - .column("nodes_inflight", toStringOrNull(c.inflight())) - .column("nodes_contacted", toStringOrNull(c.contacted())) - .column("description", c.describe()) - .column("participants", toStringOrNull(c.scope())) - .column("replies", summarise(c.replies())) - .column("tracker", summarise(c.tracker())); + collector.row(toStringOrNull(c.txnId()), c.kind().toString(), c.coordinationId()) + .lazyCollect(columns -> { + columns.add("nodes", c, Coordination::nodes, TO_STRING) + .add("nodes_inflight", c, Coordination::inflight, TO_STRING) + .add("nodes_contacted", c, Coordination::contacted, TO_STRING) + .add("description", c, Coordination::describe, TO_STRING) + .add("participants", c, Coordination::scope, TO_STRING) + .add("replies", c, Coordination::replies, CoordinationsTable::summarise) + .add("tracker", c, Coordination::tracker, AbstractTracker::summariseTracker); + }); } - return ds; } - private static String summarise(@Nullable SortedListMap replies) + private static String summarise(SortedListMap replies) { - if (replies == null) - return null; return AbstractCoordination.summariseReplies(replies, 60); } - - private static String summarise(@Nullable AbstractTracker tracker) - { - if (tracker == null) - return null; - return tracker.summariseTracker(); - } } - - // TODO (desired): don't report null as "null" - public static final class CommandsForKeyTable extends AbstractVirtualTable implements AbstractVirtualTable.DataSet + private static abstract class AbstractCommandsForKeyTable extends AbstractLazyVirtualTable { - static class Entry + static class Entry implements Comparable { final int commandStoreId; final CommandsForKey cfk; @@ -326,48 +313,33 @@ static class Entry this.commandStoreId = commandStoreId; this.cfk = cfk; } - } - private CommandsForKeyTable() - { - super(parse(VIRTUAL_ACCORD_DEBUG, COMMANDS_FOR_KEY, - "Accord per-CommandStore CommandsForKey Managed Transaction State", - "CREATE TABLE %s (\n" + - " key text,\n" + - " command_store_id int,\n" + - " txn_id 'TxnIdUtf8Type',\n" + - " ballot text,\n" + - " deps_known_before text,\n" + - " execute_at text,\n" + - " flags text,\n" + - " missing text,\n" + - " status text,\n" + - " status_overrides text,\n" + - " PRIMARY KEY (key, command_store_id, txn_id)" + - ')', UTF8Type.instance)); - } - @Override - public DataSet data() - { - return this; + @Override + public int compareTo(Entry that) + { + return Integer.compare(this.commandStoreId, that.commandStoreId); + } } - @Override - public boolean isEmpty() + AbstractCommandsForKeyTable(TableMetadata metadata) { - return false; + super(metadata, BEST_EFFORT, SORTED); } + abstract void collect(PartitionCollector partition, int commandStoreId, CommandsForKey cfk); + @Override - public Partition getPartition(DecoratedKey partitionKey) + public void collect(PartitionsCollector collector) { - String keyStr = UTF8Type.instance.compose(partitionKey.getKey()); - TokenKey key = TokenKey.parse(keyStr, DatabaseDescriptor.getPartitioner()); + Object[] partitionKey = collector.singlePartitionKey(); + if (partitionKey == null) + throw new InvalidRequestException(metadata + " currently only supports querying single partitions"); + + TokenKey key = TokenKey.parse((String) partitionKey[0], DatabaseDescriptor.getPartitioner()); List cfks = new CopyOnWriteArrayList<>(); - PreLoadContext context = PreLoadContext.contextFor(key, LoadKeys.SYNC, LoadKeysFor.READ_WRITE, "commands_for_key table query"); CommandStores commandStores = AccordService.instance().node().commandStores(); - AccordService.getBlocking(commandStores.forEach(context, key, Long.MIN_VALUE, Long.MAX_VALUE, safeStore -> { + AccordService.getBlocking(commandStores.forEach("commands_for_key table query", RoutingKeys.of(key), Long.MIN_VALUE, Long.MAX_VALUE, safeStore -> { SafeCommandsForKey safeCfk = safeStore.get(key); CommandsForKey cfk = safeCfk.current(); if (cfk == null) @@ -377,36 +349,57 @@ public Partition getPartition(DecoratedKey partitionKey) })); if (cfks.isEmpty()) - return null; + return; - SimpleDataSet ds = new SimpleDataSet(metadata); - for (Entry e : cfks) - { - CommandsForKey cfk = e.cfk; - for (int i = 0 ; i < cfk.size() ; ++i) - { - CommandsForKey.TxnInfo txn = cfk.get(i); - ds.row(keyStr, e.commandStoreId, toStringOrNull(txn.plainTxnId())) - .column("ballot", toStringOrNull(txn.ballot())) - .column("deps_known_before", toStringOrNull(txn.depsKnownUntilExecuteAt())) - .column("flags", flags(txn)) - .column("execute_at", toStringOrNull(txn.plainExecuteAt())) - .column("missing", Arrays.toString(txn.missing())) - .column("status", toStringOrNull(txn.status())) - .column("status_overrides", txn.statusOverrides() == 0 ? null : ("0x" + Integer.toHexString(txn.statusOverrides()))); - } - } + cfks.sort(collector.dataRange().isReversed() ? Comparator.reverseOrder() : Comparator.naturalOrder()); + for (Entry entry : cfks) + collect(collector.partition(partitionKey[0]), entry.commandStoreId, entry.cfk); + } + } - return ds.getPartition(partitionKey); + public static final class CommandsForKeyTable extends AbstractCommandsForKeyTable + { + private CommandsForKeyTable() + { + super(parse(VIRTUAL_ACCORD_DEBUG, COMMANDS_FOR_KEY, + "Accord per-CommandStore CommandsForKey Managed Transaction State", + "CREATE TABLE %s (\n" + + " key text,\n" + + " command_store_id int,\n" + + " txn_id 'TxnIdUtf8Type',\n" + + " ballot text,\n" + + " deps_known_before text,\n" + + " execute_at text,\n" + + " flags text,\n" + + " missing text,\n" + + " status text,\n" + + " status_overrides text,\n" + + " PRIMARY KEY (key, command_store_id, txn_id)" + + ')', UTF8Type.instance)); } @Override - public Iterator getPartitions(DataRange range) + void collect(PartitionCollector partition, int commandStoreId, CommandsForKey cfk) { - throw new UnsupportedOperationException(); + partition.collect(rows -> { + for (int i = 0 ; i < cfk.size() ; ++i) + { + TxnInfo txn = cfk.get(i); + rows.add(commandStoreId, txn.plainTxnId().toString()) + .lazyCollect(columns -> { + columns.add("ballot", txn.ballot(), AccordDebugKeyspace::toStringOrNull) + .add("deps_known_before", txn, TxnInfo::depsKnownUntilExecuteAt, TO_STRING) + .add("flags", txn, CommandsForKeyTable::flags) + .add("execute_at", txn, TxnInfo::plainExecuteAt, TO_STRING) + .add("missing", txn, TxnInfo::missing, Arrays::toString) + .add("status", txn, TxnInfo::status, TO_STRING) + .add("status_overrides", txn.statusOverrides() == 0 ? null : ("0x" + Integer.toHexString(txn.statusOverrides()))); + }); + } + }); } - private static String flags(CommandsForKey.TxnInfo txn) + private static String flags(TxnInfo txn) { StringBuilder sb = new StringBuilder(); if (!txn.mayExecute()) @@ -427,21 +420,8 @@ private static String flags(CommandsForKey.TxnInfo txn) } } - - // TODO (expected): test this table - public static final class CommandsForKeyUnmanagedTable extends AbstractVirtualTable implements AbstractVirtualTable.DataSet + public static final class CommandsForKeyUnmanagedTable extends AbstractCommandsForKeyTable { - static class Entry - { - final int commandStoreId; - final CommandsForKey cfk; - - Entry(int commandStoreId, CommandsForKey cfk) - { - this.commandStoreId = commandStoreId; - this.cfk = cfk; - } - } private CommandsForKeyUnmanagedTable() { super(parse(VIRTUAL_ACCORD_DEBUG, COMMANDS_FOR_KEY_UNMANAGED, @@ -457,71 +437,30 @@ private CommandsForKeyUnmanagedTable() } @Override - public DataSet data() + void collect(PartitionCollector partition, int commandStoreId, CommandsForKey cfk) { - return this; - } - - @Override - public boolean isEmpty() - { - return false; - } - - @Override - public Partition getPartition(DecoratedKey partitionKey) - { - String keyStr = UTF8Type.instance.compose(partitionKey.getKey()); - TokenKey key = TokenKey.parse(keyStr, DatabaseDescriptor.getPartitioner()); - - List cfks = new CopyOnWriteArrayList<>(); - PreLoadContext context = PreLoadContext.contextFor(key, LoadKeys.SYNC, LoadKeysFor.READ_WRITE, "commands_for_key_unmanaged table query"); - CommandStores commandStores = AccordService.instance().node().commandStores(); - AccordService.getBlocking(commandStores.forEach(context, key, Long.MIN_VALUE, Long.MAX_VALUE, safeStore -> { - SafeCommandsForKey safeCfk = safeStore.get(key); - CommandsForKey cfk = safeCfk.current(); - if (cfk == null) - return; - - cfks.add(new Entry(safeStore.commandStore().id(), cfk)); - })); - - if (cfks.isEmpty()) - return null; - - SimpleDataSet ds = new SimpleDataSet(metadata); - for (Entry e : cfks) - { - CommandsForKey cfk = e.cfk; + partition.collect(rows -> { for (int i = 0 ; i < cfk.unmanagedCount() ; ++i) { CommandsForKey.Unmanaged txn = cfk.getUnmanaged(i); - ds.row(keyStr, e.commandStoreId, toStringOrNull(txn.txnId)) - .column("waiting_until", toStringOrNull(txn.waitingUntil)) - .column("waiting_until_status", toStringOrNull(txn.pending)); + rows.add(commandStoreId, toStringOrNull(txn.txnId)) + .lazyCollect(columns -> { + columns.add("waiting_until", txn.waitingUntil, TO_STRING) + .add("waiting_until_status", txn.pending, TO_STRING); + }); } - } - - return ds.getPartition(partitionKey); - } - - @Override - public Iterator getPartitions(DataRange range) - { - throw new UnsupportedOperationException(); + }); } } - - public static final class DurabilityServiceTable extends AbstractVirtualTable + public static final class DurabilityServiceTable extends AbstractLazyVirtualTable { private DurabilityServiceTable() { super(parse(VIRTUAL_ACCORD_DEBUG, DURABILITY_SERVICE, "Accord per-Range Durability Service State", "CREATE TABLE %s (\n" + - " keyspace_name text,\n" + - " table_name text,\n" + + " table_id text,\n" + " token_start 'TokenUtf8Type',\n" + " token_end 'TokenUtf8Type',\n" + " last_started_at bigint,\n" + @@ -540,82 +479,77 @@ private DurabilityServiceTable() " current_splits int,\n" + " stopping boolean,\n" + " stopped boolean,\n" + - " PRIMARY KEY (keyspace_name, table_name, token_start)" + - ')', UTF8Type.instance)); + " PRIMARY KEY (table_id, token_start)" + + ')', UTF8Type.instance), FAIL, UNSORTED); } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { ShardDurability.ImmutableView view = ((AccordService) AccordService.instance()).shardDurability(); - SimpleDataSet ds = new SimpleDataSet(metadata()); while (view.advance()) { TableId tableId = (TableId) view.shard().range.start().prefix(); - TableMetadata tableMetadata = tableMetadata(tableId); - ds.row(keyspace(tableMetadata), table(tableId, tableMetadata), printToken(view.shard().range.start())) - .column("token_end", printToken(view.shard().range.end())) - .column("last_started_at", approxTime.translate().toMillisSinceEpoch(view.lastStartedAtMicros() * 1000)) - .column("cycle_started_at", approxTime.translate().toMillisSinceEpoch(view.cycleStartedAtMicros() * 1000)) - .column("retries", view.retries()) - .column("min", Objects.toString(view.min())) - .column("requested_by", Objects.toString(view.requestedBy())) - .column("active", Objects.toString(view.active())) - .column("waiting", Objects.toString(view.waiting())) - .column("node_offset", view.nodeOffset()) - .column("cycle_offset", view.cycleOffset()) - .column("active_index", view.activeIndex()) - .column("next_index", view.nextIndex()) - .column("next_to_index", view.toIndex()) - .column("end_index", view.cycleLength()) - .column("current_splits", view.currentSplits()) - .column("stopping", view.stopping()) - .column("stopped", view.stopped()) - ; + collector.row(tableId.toString(), printToken(view.shard().range.start())) + .eagerCollect(columns -> { + columns.add("token_end", printToken(view.shard().range.end())) + .add("last_started_at", approxTime.translate().toMillisSinceEpoch(view.lastStartedAtMicros() * 1000)) + .add("cycle_started_at", approxTime.translate().toMillisSinceEpoch(view.cycleStartedAtMicros() * 1000)) + .add("retries", view.retries()) + .add("min", Objects.toString(view.min())) + .add("requested_by", Objects.toString(view.requestedBy())) + .add("active", Objects.toString(view.active())) + .add("waiting", Objects.toString(view.waiting())) + .add("node_offset", view.nodeOffset()) + .add("cycle_offset", view.cycleOffset()) + .add("active_index", view.activeIndex()) + .add("next_index", view.nextIndex()) + .add("next_to_index", view.toIndex()) + .add("end_index", view.cycleLength()) + .add("current_splits", view.currentSplits()) + .add("stopping", view.stopping()) + .add("stopped", view.stopped()); + }); } - return ds; } } - public static final class DurableBeforeTable extends AbstractVirtualTable + public static final class DurableBeforeTable extends AbstractLazyVirtualTable { private DurableBeforeTable() { super(parse(VIRTUAL_ACCORD_DEBUG, DURABLE_BEFORE, "Accord Node's DurableBefore State", "CREATE TABLE %s (\n" + - " keyspace_name text,\n" + - " table_name text,\n" + + " table_id text,\n" + " token_start 'TokenUtf8Type',\n" + " token_end 'TokenUtf8Type',\n" + " quorum 'TxnIdUtf8Type',\n" + " universal 'TxnIdUtf8Type',\n" + - " PRIMARY KEY (keyspace_name, table_name, token_start)" + - ')', UTF8Type.instance)); + " PRIMARY KEY (table_id, token_start)" + + ')', UTF8Type.instance), FAIL, UNSORTED); } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { DurableBefore durableBefore = AccordService.instance().node().durableBefore(); - return durableBefore.foldlWithBounds( - (entry, ds, start, end) -> { + durableBefore.foldlWithBounds( + (entry, ignore, start, end) -> { TableId tableId = (TableId) start.prefix(); - TableMetadata tableMetadata = tableMetadata(tableId); - ds.row(keyspace(tableMetadata), table(tableId, tableMetadata), printToken(start)) - .column("token_end", printToken(end)) - .column("quorum", entry.quorumBefore.toString()) - .column("universal", entry.universalBefore.toString()); - return ds; - }, - new SimpleDataSet(metadata()), - ignore -> false - ); + collector.row(tableId.toString(), printToken(start)) + .lazyCollect(columns -> { + columns.add("token_end", end, AccordDebugKeyspace::printToken) + .add("quorum", entry.quorumBefore, TO_STRING) + .add("universal", entry.universalBefore, TO_STRING); + }); + return null; + }, null, ignore -> false); } } - public static final class ExecutorCacheTable extends AbstractVirtualTable + public static final class ExecutorCacheTable extends AbstractLazyVirtualTable { private ExecutorCacheTable() { @@ -628,77 +562,76 @@ private ExecutorCacheTable() " hits bigint,\n" + " misses bigint,\n" + " PRIMARY KEY (executor_id, scope)" + - ')', Int32Type.instance)); + ')', Int32Type.instance), FAIL, UNSORTED); } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { AccordCommandStores stores = (AccordCommandStores) AccordService.instance().node().commandStores(); - SimpleDataSet ds = new SimpleDataSet(metadata()); for (AccordExecutor executor : stores.executors()) { try (AccordExecutor.ExclusiveGlobalCaches cache = executor.lockCaches()) { - addRow(ds, executor.executorId(), "commands", cache.commands.statsSnapshot()); - addRow(ds, executor.executorId(), AccordKeyspace.COMMANDS_FOR_KEY, cache.commandsForKey.statsSnapshot()); + addRow(collector, executor.executorId(), "commands", cache.commands.statsSnapshot()); + addRow(collector, executor.executorId(), AccordKeyspace.COMMANDS_FOR_KEY, cache.commandsForKey.statsSnapshot()); } } - return ds; } - private static void addRow(SimpleDataSet ds, int executorId, String scope, AccordCache.ImmutableStats stats) + private static void addRow(PartitionsCollector collector, int executorId, String scope, AccordCache.ImmutableStats stats) { - ds.row(executorId, scope) - .column("queries", stats.hits + stats.misses) - .column("hits", stats.hits) - .column("misses", stats.misses); + collector.row(executorId, scope) + .eagerCollect(columns -> { + columns.add("queries", stats.hits + stats.misses) + .add("hits", stats.hits) + .add("misses", stats.misses); + }); } } - - public static final class MaxConflictsTable extends AbstractVirtualTable + public static final class MaxConflictsTable extends AbstractLazyVirtualTable { private MaxConflictsTable() { super(parse(VIRTUAL_ACCORD_DEBUG, MAX_CONFLICTS, "Accord per-CommandStore MaxConflicts State", "CREATE TABLE %s (\n" + - " keyspace_name text,\n" + - " table_name text,\n" + " command_store_id bigint,\n" + " token_start 'TokenUtf8Type',\n" + + " table_id text,\n" + " token_end 'TokenUtf8Type',\n" + " timestamp text,\n" + - " PRIMARY KEY (keyspace_name, table_name, command_store_id, token_start)" + - ')', UTF8Type.instance)); + " PRIMARY KEY (command_store_id, token_start)" + + ')', Int32Type.instance), FAIL, ASC); } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { CommandStores commandStores = AccordService.instance().node().commandStores(); - SimpleDataSet dataSet = new SimpleDataSet(metadata()); for (CommandStore commandStore : commandStores.all()) { int commandStoreId = commandStore.id(); MaxConflicts maxConflicts = commandStore.unsafeGetMaxConflicts(); TableId tableId = ((AccordCommandStore) commandStore).tableId(); - TableMetadata tableMetadata = tableMetadata(tableId); - - maxConflicts.foldlWithBounds( - (timestamp, ds, start, end) -> { - return ds.row(keyspace(tableMetadata), table(tableId, tableMetadata), commandStoreId, printToken(start)) - .column("token_end", printToken(end)) - .column("timestamp", timestamp.toString()) - ; - }, - dataSet, - ignore -> false - ); + String tableIdStr = tableId.toString(); + + collector.partition(commandStoreId).collect(rows -> { + maxConflicts.foldlWithBounds( + (timestamp, rs, start, end) -> { + rows.add(printToken(start)) + .lazyCollect(columns -> { + columns.add("token_end", end, AccordDebugKeyspace::printToken) + .add("table_id", tableIdStr) + .add("timestamp", timestamp, TO_STRING); + }); + return rows; + }, rows, ignore -> false + ); + }); } - return dataSet; } } @@ -791,18 +724,16 @@ private SimpleDataSet data(Collection tableStates) } // TODO (desired): human readable packed key tracker (but requires loading Txn, so might be preferable to only do conditionally) - public static final class ProgressLogTable extends AbstractVirtualTable + public static final class ProgressLogTable extends AbstractLazyVirtualTable { private ProgressLogTable() { super(parse(VIRTUAL_ACCORD_DEBUG, PROGRESS_LOG, "Accord per-CommandStore ProgressLog State", "CREATE TABLE %s (\n" + - " keyspace_name text,\n" + - " table_name text,\n" + - " table_id text,\n" + " command_store_id int,\n" + " txn_id 'TxnIdUtf8Type',\n" + + " table_id text,\n" + // Timer + BaseTxnState " contact_everyone boolean,\n" + // WaitingState @@ -818,43 +749,46 @@ private ProgressLogTable() " home_progress text,\n" + " home_retry_counter int,\n" + " home_scheduled_at timestamp,\n" + - " PRIMARY KEY (keyspace_name, table_name, table_id, command_store_id, txn_id)" + - ')', UTF8Type.instance)); + " PRIMARY KEY (command_store_id, txn_id)" + + ')', Int32Type.instance), FAIL, ASC); } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { CommandStores commandStores = AccordService.instance().node().commandStores(); - SimpleDataSet ds = new SimpleDataSet(metadata()); for (CommandStore commandStore : commandStores.all()) { DefaultProgressLog.ImmutableView view = ((DefaultProgressLog) commandStore.unsafeProgressLog()).immutableView(); TableId tableId = ((AccordCommandStore)commandStore).tableId(); String tableIdStr = tableId.toString(); - TableMetadata tableMetadata = tableMetadata(tableId); - while (view.advance()) - { - ds.row(keyspace(tableMetadata), table(tableId, tableMetadata), tableIdStr, view.commandStoreId(), view.txnId().toString()) - .column("contact_everyone", view.contactEveryone()) - .column("waiting_is_uninitialised", view.isWaitingUninitialised()) - .column("waiting_blocked_until", view.waitingIsBlockedUntil().name()) - .column("waiting_home_satisfies", view.waitingHomeSatisfies().name()) - .column("waiting_progress", view.waitingProgress().name()) - .column("waiting_retry_counter", view.waitingRetryCounter()) - .column("waiting_packed_key_tracker_bits", Long.toBinaryString(view.waitingPackedKeyTrackerBits())) - .column("waiting_scheduled_at", toTimestamp(view.timerScheduledAt(TxnStateKind.Waiting))) - .column("home_phase", view.homePhase().name()) - .column("home_progress", view.homeProgress().name()) - .column("home_retry_counter", view.homeRetryCounter()) - .column("home_scheduled_at", toTimestamp(view.timerScheduledAt(TxnStateKind.Home))) - ; - } + collector.partition(commandStore.id()).collect(collect -> { + while (view.advance()) + { + // TODO (required): view should return an immutable per-row view so that we can call lazyAdd + collect.add(view.txnId().toString()) + .eagerCollect(columns -> { + columns.add("table_id", tableIdStr) + .add("contact_everyone", view.contactEveryone()) + .add("waiting_is_uninitialised", view.isWaitingUninitialised()) + .add("waiting_blocked_until", view.waitingIsBlockedUntil().name()) + .add("waiting_home_satisfies", view.waitingHomeSatisfies().name()) + .add("waiting_progress", view.waitingProgress().name()) + .add("waiting_retry_counter", view.waitingRetryCounter()) + .add("waiting_packed_key_tracker_bits", Long.toBinaryString(view.waitingPackedKeyTrackerBits())) + .add("waiting_scheduled_at", view.timerScheduledAt(TxnStateKind.Waiting), ProgressLogTable::toTimestamp) + .add("home_phase", view.homePhase().name()) + .add("home_progress", view.homeProgress().name()) + .add("home_retry_counter", view.homeRetryCounter()) + .add("home_scheduled_at", view.timerScheduledAt(TxnStateKind.Home), ProgressLogTable::toTimestamp); + }); + } + + }); } - return ds; } - private Date toTimestamp(Long deadline) + private static Date toTimestamp(Long deadline) { if (deadline == null) return null; @@ -864,19 +798,17 @@ private Date toTimestamp(Long deadline) } } - public static final class RedundantBeforeTable extends AbstractVirtualTable + public static final class RedundantBeforeTable extends AbstractLazyVirtualTable { private RedundantBeforeTable() { super(parse(VIRTUAL_ACCORD_DEBUG, REDUNDANT_BEFORE, "Accord per-CommandStore RedundantBefore State", "CREATE TABLE %s (\n" + - " keyspace_name text,\n" + - " table_name text,\n" + - " table_id text,\n" + + " command_store_id int,\n" + " token_start 'TokenUtf8Type',\n" + + " table_id text,\n" + " token_end 'TokenUtf8Type',\n" + - " command_store_id int,\n" + " start_epoch bigint,\n" + " end_epoch bigint,\n" + " gc_before 'TxnIdUtf8Type',\n" + @@ -888,97 +820,90 @@ private RedundantBeforeTable() " locally_redundant 'TxnIdUtf8Type',\n" + " locally_synced 'TxnIdUtf8Type',\n" + " locally_witnessed 'TxnIdUtf8Type',\n" + - " pre_bootstrap 'TxnIdUtf8Type',\n" + + " log_unavailable 'TxnIdUtf8Type',\n" + + " unready 'TxnIdUtf8Type',\n" + " stale_until_at_least 'TxnIdUtf8Type',\n" + - " PRIMARY KEY (keyspace_name, table_name, table_id, command_store_id, token_start)" + - ')', UTF8Type.instance)); + " PRIMARY KEY (command_store_id, token_start)" + + ')', Int32Type.instance), FAIL, ASC); } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { CommandStores commandStores = AccordService.instance().node().commandStores(); - SimpleDataSet dataSet = new SimpleDataSet(metadata()); for (CommandStore commandStore : commandStores.all()) { int commandStoreId = commandStore.id(); - TableId tableId = ((AccordCommandStore)commandStore).tableId(); - String tableIdStr = tableId.toString(); - TableMetadata tableMetadata = tableMetadata(tableId); - String keyspace = keyspace(tableMetadata); - String table = table(tableId, tableMetadata); - commandStore.unsafeGetRedundantBefore().foldl( - (entry, ds) -> { - ds.row(keyspace, table, tableIdStr, commandStoreId, printToken(entry.range.start())) - .column("token_end", printToken(entry.range.end())) - .column("start_epoch", entry.startEpoch) - .column("end_epoch", entry.endEpoch) - .column("gc_before", entry.maxBound(GC_BEFORE).toString()) - .column("shard_applied", entry.maxBound(SHARD_APPLIED).toString()) - .column("quorum_applied", entry.maxBound(QUORUM_APPLIED).toString()) - .column("locally_applied", entry.maxBound(LOCALLY_APPLIED).toString()) - .column("locally_durable_to_command_store", entry.maxBound(LOCALLY_DURABLE_TO_COMMAND_STORE).toString()) - .column("locally_durable_to_data_store", entry.maxBound(LOCALLY_DURABLE_TO_DATA_STORE).toString()) - .column("locally_redundant", entry.maxBound(LOCALLY_REDUNDANT).toString()) - .column("locally_synced", entry.maxBound(LOCALLY_SYNCED).toString()) - .column("locally_witnessed", entry.maxBound(LOCALLY_WITNESSED).toString()) - .column("pre_bootstrap", entry.maxBound(PRE_BOOTSTRAP).toString()) - .column("stale_until_at_least", entry.staleUntilAtLeast != null ? entry.staleUntilAtLeast.toString() : null); - return ds; - }, - dataSet, - ignore -> false - ); + collector.partition(commandStoreId).collect(rows -> { + TableId tableId = ((AccordCommandStore)commandStore).tableId(); + String tableIdStr = tableId.toString(); + commandStore.unsafeGetRedundantBefore().foldl( + (entry, rs) -> { + rs.add(printToken(entry.range.start())).lazyCollect(columns -> { + columns.add("table_id", tableIdStr) + .add("token_end", entry.range.end(), AccordDebugKeyspace::printToken) + .add("start_epoch", entry.startEpoch) + .add("end_epoch", entry.endEpoch) + .add("gc_before", entry, e -> e.maxBound(GC_BEFORE), TO_STRING) + .add("shard_applied", entry, e -> e.maxBound(SHARD_APPLIED), TO_STRING) + .add("quorum_applied", entry, e -> e.maxBound(QUORUM_APPLIED), TO_STRING) + .add("locally_applied", entry, e -> e.maxBound(LOCALLY_APPLIED), TO_STRING) + .add("locally_durable_to_command_store", entry, e -> e.maxBound(LOCALLY_DURABLE_TO_COMMAND_STORE), TO_STRING) + .add("locally_durable_to_data_store", entry, e -> e.maxBound(LOCALLY_DURABLE_TO_DATA_STORE), TO_STRING) + .add("locally_redundant", entry, e -> e.maxBound(LOCALLY_REDUNDANT), TO_STRING) + .add("locally_synced", entry, e -> e.maxBound(LOCALLY_SYNCED), TO_STRING) + .add("locally_witnessed", entry, e -> e.maxBound(LOCALLY_WITNESSED), TO_STRING) + .add("log_unavailable", entry, e -> e.maxBound(LOG_UNAVAILABLE), TO_STRING) + .add("unready", entry, e -> e.maxBound(UNREADY), TO_STRING) + .add("stale_until_at_least", entry.staleUntilAtLeast, TO_STRING); + }); + return rs; + }, rows, ignore -> false + ); + }); } - return dataSet; } } - public static final class RejectBeforeTable extends AbstractVirtualTable + public static final class RejectBeforeTable extends AbstractLazyVirtualTable { private RejectBeforeTable() { super(parse(VIRTUAL_ACCORD_DEBUG, REJECT_BEFORE, "Accord per-CommandStore RejectBefore State", "CREATE TABLE %s (\n" + - " keyspace_name text,\n" + - " table_name text,\n" + - " table_id text,\n" + " command_store_id int,\n" + " token_start 'TokenUtf8Type',\n" + + " table_id text,\n" + " token_end 'TokenUtf8Type',\n" + " timestamp text,\n" + - " PRIMARY KEY (keyspace_name, table_name, table_id, command_store_id, token_start)" + - ')', UTF8Type.instance)); + " PRIMARY KEY (command_store_id, token_start)" + + ')', UTF8Type.instance), FAIL, ASC); } @Override - public DataSet data() + protected void collect(PartitionsCollector collector) { CommandStores commandStores = AccordService.instance().node().commandStores(); - SimpleDataSet dataSet = new SimpleDataSet(metadata()); for (CommandStore commandStore : commandStores.all()) { RejectBefore rejectBefore = commandStore.unsafeGetRejectBefore(); if (rejectBefore == null) continue; - TableId tableId = ((AccordCommandStore)commandStore).tableId(); - String tableIdStr = tableId.toString(); - TableMetadata tableMetadata = tableMetadata(tableId); - String keyspace = keyspace(tableMetadata); - String table = table(tableId, tableMetadata); - rejectBefore.foldlWithBounds( - (timestamp, ds, start, end) -> ds.row(keyspace, table, tableIdStr, commandStore.id(), printToken(start)) - .column("token_end", printToken(end)) - .column("timestamp", timestamp.toString()) - , - dataSet, - ignore -> false - ); + collector.partition(commandStore.id()).collect(rows -> { + TableId tableId = ((AccordCommandStore)commandStore).tableId(); + String tableIdStr = tableId.toString(); + rejectBefore.foldlWithBounds((timestamp, rs, start, end) -> { + rs.add(printToken(start)) + .lazyCollect(columns -> columns.add("table_id", tableIdStr) + .add("token_end", end, AccordDebugKeyspace::printToken) + .add("timestamp", timestamp, AccordDebugKeyspace::toStringOrNull)); + return rs; + }, rows, ignore -> false); + }); } - return dataSet; } } @@ -995,11 +920,11 @@ private TxnTraceTable() super(parse(VIRTUAL_ACCORD_DEBUG, TXN_TRACE, "Accord Transaction Trace Configuration", "CREATE TABLE %s (\n" + - " txn_id text,\n" + + " txn_id 'TxnIdUtf8Type',\n" + " event_type text,\n" + " permits int,\n" + " PRIMARY KEY (txn_id, event_type)" + - ')', UTF8Type.instance)); + ')', TxnIdUtf8Type.instance)); } @Override @@ -1056,21 +981,21 @@ public void truncate() } } - public static final class TxnTracesTable extends AbstractMutableVirtualTable + public static final class TxnTracesTable extends AbstractMutableLazyVirtualTable { private TxnTracesTable() { super(parse(VIRTUAL_ACCORD_DEBUG, TXN_TRACES, "Accord Transaction Traces", "CREATE TABLE %s (\n" + - " txn_id text,\n" + + " txn_id 'TxnIdUtf8Type',\n" + " event_type text,\n" + " id_micros bigint,\n" + " at_micros bigint,\n" + " command_store_id int,\n" + " message text,\n" + " PRIMARY KEY (txn_id, event_type, id_micros, at_micros)" + - ')', UTF8Type.instance)); + ')', TxnIdUtf8Type.instance), FAIL, UNSORTED, UNSORTED); } private AccordTracing tracing() @@ -1079,29 +1004,29 @@ private AccordTracing tracing() } @Override - protected void applyPartitionDeletion(ColumnValues partitionKey) + protected void applyPartitionDeletion(Object[] partitionKeys) { - TxnId txnId = TxnId.parse(partitionKey.value(0)); + TxnId txnId = TxnId.parse((String)partitionKeys[0]); tracing().eraseEvents(txnId); } @Override - protected void applyRangeTombstone(ColumnValues partitionKey, Range range) - { - TxnId txnId = TxnId.parse(partitionKey.value(0)); - if (!range.hasLowerBound() || range.lowerBoundType() != BoundType.CLOSED) throw invalidRequest("May restrict deletion by at most one event_type"); - if (range.lowerEndpoint().size() != 1) throw invalidRequest("Deletion restricted by lower bound on id_micros or at_micros is unsupported"); - if (!range.hasUpperBound() || (range.upperBoundType() != BoundType.CLOSED && range.upperEndpoint().size() == 1)) throw invalidRequest("Range deletion must specify one event_type"); - if (!range.upperEndpoint().value(0).equals(range.lowerEndpoint().value(0))) throw invalidRequest("May restrict deletion by at most one event_type"); - if (range.upperEndpoint().size() > 2) throw invalidRequest("Deletion restricted by upper bound on at_micros is unsupported"); - TraceEventType eventType = parseEventType(range.lowerEndpoint().value(0)); - if (range.upperEndpoint().size() == 1) + protected void applyRangeTombstone(Object[] partitionKeys, Object[] starts, boolean startInclusive, Object[] ends, boolean endInclusive) + { + TxnId txnId = TxnId.parse((String) partitionKeys[0]); + if (!startInclusive) throw invalidRequest("May restrict deletion by at most one event_type"); + if (starts.length != 1) throw invalidRequest("Deletion restricted by lower bound on id_micros or at_micros is unsupported"); + if (ends.length == 0 || (ends.length == 1 && !endInclusive)) throw invalidRequest("Range deletion must specify one event_type"); + if (!ends[0].equals(starts[0])) throw invalidRequest("May restrict deletion by at most one event_type"); + if (ends.length > 2) throw invalidRequest("Deletion restricted by upper bound on at_micros is unsupported"); + TraceEventType eventType = parseEventType((String) starts[0]); + if (ends.length == 1) { tracing().eraseEvents(txnId, eventType); } else { - long before = range.upperEndpoint().value(1); + long before = (Long)ends[1]; tracing().eraseEventsBefore(txnId, eventType, before); } } @@ -1113,36 +1038,118 @@ public void truncate() } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { - SimpleDataSet dataSet = new SimpleDataSet(metadata()); tracing().forEach(id -> true, (txnId, eventType, permits, events) -> { events.forEach(e -> { - e.messages().forEach(m -> { - dataSet.row(txnId.toString(), eventType.name(), e.idMicros, NANOSECONDS.toMicros(m.atNanos - e.atNanos)) - .column("command_store_id", m.commandStoreId) - .column("message", m.message); - }); + if (e.messages().isEmpty()) + { + collector.row(txnId.toString(), eventType.name(), e.idMicros, 0L) + .eagerCollect(columns -> { + columns.add("message", ""); + }); + } + else + { + e.messages().forEach(m -> { + collector.row(txnId.toString(), eventType.name(), e.idMicros, NANOSECONDS.toMicros(m.atNanos - e.atNanos)) + .eagerCollect(columns -> { + columns.add("command_store_id", m.commandStoreId) + .add("message", m.message); + }); + }); + } }); }); - return dataSet; } } // TODO (desired): don't report null as "null" - public static final class TxnTable extends AbstractVirtualTable implements AbstractVirtualTable.DataSet + abstract static class AbstractJournalTable extends AbstractLazyVirtualTable { - static class Entry + static final CompositeType PK = CompositeType.getInstance(Int32Type.instance, UTF8Type.instance); + + AbstractJournalTable(TableMetadata metadata) { - final int commandStoreId; - final Command command; + super(metadata, FAIL, ASC); + } + + @Override + public boolean allowFilteringImplicitly() + { + return false; + } + + @Override + public boolean allowFilteringPrimaryKeysImplicitly() + { + return true; + } + + @Override + public void collect(PartitionsCollector collector) + { + AccordService accord; + { + IAccordService iaccord = AccordService.instance(); + if (!iaccord.isEnabled()) + return; + + accord = (AccordService) iaccord; + } + + DataRange dataRange = collector.dataRange(); + JournalKey min = toJournalKey(dataRange.startKey()), + max = toJournalKey(dataRange.stopKey()); - Entry(int commandStoreId, Command command) + if (min == null && max == null) { - this.commandStoreId = commandStoreId; - this.command = command; + FilterRange filterTxnId = collector.filters("txn_id", TxnId::parse, UnaryOperator.identity(), UnaryOperator.identity()); + FilterRange filterCommandStoreId = collector.filters("command_store_id", UnaryOperator.identity(), i -> i + 1, i -> i - 1); + + int minCommandStoreId = filterCommandStoreId.min == null ? -1 : filterCommandStoreId.min; + int maxCommandStoreId = filterCommandStoreId.max == null ? Integer.MAX_VALUE : filterCommandStoreId.max; + + if (filterTxnId.min != null && filterTxnId.max != null && filterTxnId.min.equals(filterTxnId.max)) + { + TxnId txnId = filterTxnId.min; + accord.node().commandStores().forAllUnsafe(commandStore -> { + if (commandStore.id() < minCommandStoreId || commandStore.id() > maxCommandStoreId) + return; + + collect(collector, accord, new JournalKey(txnId, JournalKey.Type.COMMAND_DIFF, commandStore.id())); + }); + return; + } + + if (filterTxnId.min != null || filterTxnId.max != null || minCommandStoreId >= 0 || maxCommandStoreId < Integer.MAX_VALUE) + { + min = new JournalKey(filterTxnId.min == null ? TxnId.NONE : filterTxnId.min, JournalKey.Type.COMMAND_DIFF, Math.max(0, minCommandStoreId)); + max = new JournalKey(filterTxnId.max == null ? TxnId.MAX.withoutNonIdentityFlags() : filterTxnId.max, JournalKey.Type.COMMAND_DIFF, maxCommandStoreId); + } } + + accord.journal().forEach(key -> collect(collector, accord, key), min, max, true); } + + abstract void collect(PartitionsCollector collector, AccordService accord, JournalKey key); + + private static JournalKey toJournalKey(PartitionPosition position) + { + if (position.isMinimum()) + return null; + + if (!(position instanceof DecoratedKey)) + throw new InvalidRequestException("Cannot filter this table by partial partition key"); + + ByteBuffer[] keys = PK.split(((DecoratedKey) position).getKey()); + return new JournalKey(TxnId.parse(UTF8Type.instance.compose(keys[1])), JournalKey.Type.COMMAND_DIFF, Int32Type.instance.compose(keys[0])); + } + } + + // TODO (desired): don't report null as "null" + public static final class TxnTable extends AbstractJournalTable + { private TxnTable() { super(parse(VIRTUAL_ACCORD_DEBUG, TXN, @@ -1165,88 +1172,51 @@ private TxnTable() " participants_has_touched text,\n" + " participants_executes text,\n" + " participants_waits_on text,\n" + - " PRIMARY KEY (txn_id, command_store_id)" + - ')', UTF8Type.instance)); + " PRIMARY KEY ((command_store_id, txn_id))" + + ')', PK)); } @Override - public DataSet data() + void collect(PartitionsCollector collector, AccordService accord, JournalKey key) { - return this; - } - - @Override - public boolean isEmpty() - { - return false; - } - - @Override - public Partition getPartition(DecoratedKey partitionKey) - { - String txnIdStr = UTF8Type.instance.compose(partitionKey.getKey()); - TxnId txnId = TxnId.parse(txnIdStr); - - List commands = new CopyOnWriteArrayList<>(); - AccordService.instance().node().commandStores().forEachCommandStore(store -> { - Command command = ((AccordCommandStore)store).loadCommand(txnId); - if (command != null) - commands.add(new Entry(store.id(), command)); - }); - - if (commands.isEmpty()) - return null; + if (key.type != JournalKey.Type.COMMAND_DIFF) + return; - SimpleDataSet ds = new SimpleDataSet(metadata); - for (Entry e : commands) - { - Command command = e.command; - ds.row(txnIdStr, e.commandStoreId) - .column("save_status", toStringOrNull(command.saveStatus())) - .column("route", toStringOrNull(command.route())) - .column("participants_owns", toStr(command, StoreParticipants::owns, StoreParticipants::stillOwns)) - .column("participants_touches", toStr(command, StoreParticipants::touches, StoreParticipants::stillTouches)) - .column("participants_has_touched", toStringOrNull(command.participants().hasTouched())) - .column("participants_executes", toStr(command, StoreParticipants::executes, StoreParticipants::stillExecutes)) - .column("participants_waits_on", toStr(command, StoreParticipants::waitsOn, StoreParticipants::stillWaitsOn)) - .column("durability", toStringOrNull(command.durability())) - .column("execute_at", toStringOrNull(command.executeAt())) - .column("executes_at_least", toStringOrNull(command.executesAtLeast())) - .column("txn", toStringOrNull(command.partialTxn())) - .column("deps", toStringOrNull(command.partialDeps())) - .column("waiting_on", toStringOrNull(command.waitingOn())) - .column("writes", toStringOrNull(command.writes())) - .column("result", toStringOrNull(command.result())); - } + AccordCommandStore commandStore = (AccordCommandStore) accord.node().commandStores().forId(key.commandStoreId); + if (commandStore == null) + return; - return ds.getPartition(partitionKey); - } + Command command = commandStore.loadCommand(key.id); + if (command == null) + return; - @Override - public Iterator getPartitions(DataRange range) - { - throw new UnsupportedOperationException(); + collector.row(key.commandStoreId, key.id.toString()) + .lazyCollect(columns -> addColumns(command, columns)); + } + + private static void addColumns(Command command, ColumnsCollector columns) + { + StoreParticipants participants = command.participants(); + columns.add("save_status", command.saveStatus(), TO_STRING) + .add("route", participants, StoreParticipants::route, TO_STRING) + .add("participants_owns", participants, p -> toStr(p, StoreParticipants::owns, StoreParticipants::stillOwns)) + .add("participants_touches", participants, p -> toStr(p, StoreParticipants::touches, StoreParticipants::stillTouches)) + .add("participants_has_touched", participants, StoreParticipants::hasTouched, TO_STRING) + .add("participants_executes", participants, p -> toStr(p, StoreParticipants::executes, StoreParticipants::stillExecutes)) + .add("participants_waits_on", participants, p -> toStr(p, StoreParticipants::waitsOn, StoreParticipants::stillWaitsOn)) + .add("durability", command, Command::durability, TO_STRING) + .add("execute_at", command, Command::executeAt, TO_STRING) + .add("executes_at_least", command, Command::executesAtLeast, TO_STRING) + .add("txn", command, Command::partialTxn, TO_STRING) + .add("deps", command, Command::partialDeps, TO_STRING) + .add("waiting_on", command, Command::waitingOn, TO_STRING) + .add("writes", command, Command::writes, TO_STRING) + .add("result", command, Command::result, TO_STRING); } } - public static final class JournalTable extends AbstractVirtualTable implements AbstractVirtualTable.DataSet + public static final class JournalTable extends AbstractJournalTable { - static class Entry - { - final int commandStoreId; - final long segment; - final int position; - final CommandChange.Builder builder; - - Entry(int commandStoreId, long segment, int position, CommandChange.Builder builder) - { - this.commandStoreId = commandStoreId; - this.segment = segment; - this.position = position; - this.builder = builder; - } - } - private JournalTable() { super(parse(VIRTUAL_ACCORD_DEBUG, JOURNAL, @@ -1270,68 +1240,40 @@ private JournalTable() " participants_has_touched text,\n" + " participants_executes text,\n" + " participants_waits_on text,\n" + - " PRIMARY KEY (txn_id, command_store_id, segment, segment_position)" + - ')', UTF8Type.instance)); - } - - @Override - public DataSet data() - { - return this; - } - - @Override - public boolean isEmpty() - { - return false; + " PRIMARY KEY ((command_store_id, txn_id), segment, segment_position)" + + ')', PK)); } @Override - public Partition getPartition(DecoratedKey partitionKey) + void collect(PartitionsCollector collector, AccordService accord, JournalKey key) { - String txnIdStr = UTF8Type.instance.compose(partitionKey.getKey()); - TxnId txnId = TxnId.parse(txnIdStr); - - List entries = new ArrayList<>(); - AccordService.instance().node().commandStores().forEachCommandStore(store -> { - for (AccordJournal.DebugEntry e : ((AccordCommandStore)store).debugCommand(txnId)) - entries.add(new Entry(store.id(), e.segment, e.position, e.builder)); + AccordCommandStore commandStore = (AccordCommandStore) accord.node().commandStores().forId(key.commandStoreId); + collector.partition(key.commandStoreId, key.id.toString()).collect(rows -> { + for (AccordJournal.DebugEntry e : commandStore.debugCommand(key.id)) + { + CommandChange.Builder b = e.builder; + StoreParticipants participants = b.participants() != null ? b.participants() : StoreParticipants.empty(key.id); + rows.add(e.segment, e.position) + .lazyCollect(columns -> { + columns.add("save_status", b.saveStatus(), TO_STRING) + .add("route", participants, StoreParticipants::route, TO_STRING) + .add("participants_owns", participants, p -> toStr(p, StoreParticipants::owns, StoreParticipants::stillOwns)) + .add("participants_touches", participants, p -> toStr(p, StoreParticipants::touches, StoreParticipants::stillTouches)) + .add("participants_has_touched", participants, StoreParticipants::hasTouched, TO_STRING) + .add("participants_executes", participants, p -> toStr(p, StoreParticipants::executes, StoreParticipants::stillExecutes)) + .add("participants_waits_on", participants, p -> toStr(p, StoreParticipants::waitsOn, StoreParticipants::stillWaitsOn)) + .add("durability", b.durability(), TO_STRING) + .add("execute_at", b.executeAt(), TO_STRING) + .add("executes_at_least", b.executesAtLeast(), TO_STRING) + .add("txn", b.partialTxn(), TO_STRING) + .add("deps", b.partialDeps(), TO_STRING) + .add("writes", b.writes(), TO_STRING) + .add("result", b.result(), TO_STRING); + }); + } }); - - if (entries.isEmpty()) - return null; - - SimpleDataSet ds = new SimpleDataSet(metadata); - for (Entry e : entries) - { - CommandChange.Builder b = e.builder; - StoreParticipants participants = b.participants(); - if (participants == null) participants = StoreParticipants.empty(txnId); - ds.row(txnIdStr, e.commandStoreId, e.segment, e.position) - .column("save_status", toStringOrNull(b.saveStatus())) - .column("route", toStringOrNull(participants.route())) - .column("participants_owns", toStr(participants, StoreParticipants::owns, StoreParticipants::stillOwns)) - .column("participants_touches", toStr(participants, StoreParticipants::touches, StoreParticipants::stillTouches)) - .column("participants_has_touched", toStringOrNull(participants.hasTouched())) - .column("participants_executes", toStr(participants, StoreParticipants::executes, StoreParticipants::stillExecutes)) - .column("participants_waits_on", toStr(participants, StoreParticipants::waitsOn, StoreParticipants::stillWaitsOn)) - .column("durability", toStringOrNull(b.durability())) - .column("execute_at", toStringOrNull(b.executeAt())) - .column("executes_at_least", toStringOrNull(b.executesAtLeast())) - .column("txn", toStringOrNull(b.partialTxn())) - .column("deps", toStringOrNull(b.partialDeps())) - .column("writes", toStringOrNull(b.writes())) - .column("result", toStringOrNull(b.result())); - } - - return ds.getPartition(partitionKey); } - @Override - public Iterator getPartitions(DataRange range) - { - throw new UnsupportedOperationException(); - } } /** @@ -1346,7 +1288,7 @@ public Iterator getPartitions(DataRange range) */ // Had to be separate from the "regular" journal table since it does not have segment and position, and command store id is inferred // TODO (required): add access control - public static final class TxnOpsTable extends AbstractMutableVirtualTable implements AbstractVirtualTable.DataSet + public static final class TxnOpsTable extends AbstractMutableLazyVirtualTable { // TODO (expected): test each of these operations enum Op { ERASE_VESTIGIAL, INVALIDATE, TRY_EXECUTE, FORCE_APPLY, FORCE_UPDATE, RECOVER, FETCH, RESET_PROGRESS_LOG } @@ -1359,41 +1301,21 @@ private TxnOpsTable() " command_store_id int,\n" + " op text," + " PRIMARY KEY (txn_id, command_store_id)" + - ')', UTF8Type.instance)); - } - - @Override - public DataSet data() - { - throw new UnsupportedOperationException(TXN_OPS + " is a write-only table"); + ')', UTF8Type.instance), FAIL, UNSORTED); } @Override - public boolean isEmpty() - { - return true; - } - - @Override - public Partition getPartition(DecoratedKey partitionKey) - { - throw new UnsupportedOperationException(TXN_OPS + " is a write-only table"); - } - - @Override - public Iterator getPartitions(DataRange range) + protected void collect(PartitionsCollector collector) { throw new UnsupportedOperationException(TXN_OPS + " is a write-only table"); } - @Override - protected void applyColumnUpdate(ColumnValues partitionKey, ColumnValues clusteringColumns, Optional columnValue) + protected void applyRowUpdate(Object[] partitionKeys, Object[] clusteringKeys, ColumnMetadata[] columns, Object[] values) { - TxnId txnId = TxnId.parse(partitionKey.value(0)); - int commandStoreId = clusteringColumns.value(0); - Invariants.require(columnValue.isPresent()); - Op op = Op.valueOf(columnValue.get().value()); + TxnId txnId = TxnId.parse((String) partitionKeys[0]); + int commandStoreId = (Integer) clusteringKeys[0]; + Op op = Op.valueOf((String)values[0]); switch (op) { default: throw new UnhandledEnum(op); @@ -1521,117 +1443,57 @@ private void cleanup(TxnId txnId, int commandStoreId, Cleanup cleanup) } } - public static class TxnBlockedByTable extends AbstractVirtualTable + public static class TxnBlockedByTable extends AbstractLazyVirtualTable { - enum Reason { Self, Txn, Key } + enum Reason + {Self, Txn, Key} protected TxnBlockedByTable() { super(parse(VIRTUAL_ACCORD_DEBUG, TXN_BLOCKED_BY, - "Accord Transactions Blocked By Table" , + "Accord Transactions Blocked By Table", "CREATE TABLE %s (\n" + - " txn_id text,\n" + - " keyspace_name text,\n" + - " table_name text,\n" + + " txn_id 'TxnIdUtf8Type',\n" + " command_store_id int,\n" + " depth int,\n" + - " blocked_by text,\n" + - " reason text,\n" + + " blocked_by_key text,\n" + + " blocked_by_txn_id 'TxnIdUtf8Type',\n" + " save_status text,\n" + " execute_at text,\n" + - " key text,\n" + - " PRIMARY KEY (txn_id, keyspace_name, table_name, command_store_id, depth, blocked_by, reason)" + - ')', UTF8Type.instance)); + " PRIMARY KEY (txn_id, command_store_id, depth, blocked_by_key, blocked_by_txn_id)" + + ')', TxnIdUtf8Type.instance), BEST_EFFORT, ASC); } @Override - public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter) + protected void collect(PartitionsCollector collector) { - Partition partition = data(partitionKey, rowFilter).getPartition(partitionKey); - - if (null == partition) - return EmptyIterators.unfilteredPartition(metadata); + Object[] pks = collector.singlePartitionKey(); + if (pks == null) + throw new InvalidRequestException(metadata + " only supports single partition key queries"); - long now = currentTimeMillis(); - UnfilteredRowIterator rowIterator = partition.toRowIterator(metadata(), clusteringIndexFilter, columnFilter, now); - return new SingletonUnfilteredPartitionIterator(rowIterator); - } + FilterRange depthRange = collector.filters("depth", Function.identity(), i -> i + 1, i -> i - 1); + int maxDepth = depthRange.max == null ? Integer.MAX_VALUE : depthRange.max; - public DataSet data(DecoratedKey partitionKey, RowFilter rowFilter) - { - int maxDepth = Integer.MAX_VALUE; - if (rowFilter != null && rowFilter.getExpressions().size() > 0) - { - Invariants.require(rowFilter.getExpressions().size() == 1, "Only depth filter is supported"); - RowFilter.Expression expression = rowFilter.getExpressions().get(0); - Invariants.require(expression.column().name.toString().equals("depth"), "Only depth filter is supported, but got: %s", expression.column().name); - Invariants.require(expression.operator() == Operator.LT || expression.operator() == Operator.LTE, "Only < and <= queries are supported"); - if (expression.operator() == Operator.LT) - maxDepth = expression.getIndexValue().getInt(0); - else - maxDepth = expression.getIndexValue().getInt(0) + 1; - } - - TxnId id = TxnId.parse(UTF8Type.instance.compose(partitionKey.getKey())); - List shards = AccordService.instance().debugTxnBlockedGraph(id); - - SimpleDataSet ds = new SimpleDataSet(metadata()); - CommandStores commandStores = AccordService.instance().node().commandStores(); - for (CommandStoreTxnBlockedGraph shard : shards) - { - Set processed = new HashSet<>(); - process(ds, commandStores, shard, processed, id, 0, maxDepth, id, Reason.Self, null); - // everything was processed right? - if (!shard.txns.isEmpty() && !shard.txns.keySet().containsAll(processed)) - Invariants.expect(false, "Skipped txns: " + Sets.difference(shard.txns.keySet(), processed)); - } - - return ds; - } - - private void process(SimpleDataSet ds, CommandStores commandStores, CommandStoreTxnBlockedGraph shard, Set processed, TxnId userTxn, int depth, int maxDepth, TxnId txnId, Reason reason, Runnable onDone) - { - if (!processed.add(txnId)) - throw new IllegalStateException("Double processed " + txnId); - CommandStoreTxnBlockedGraph.TxnState txn = shard.txns.get(txnId); - if (txn == null) - { - Invariants.require(reason == Reason.Self, "Txn %s unknown for reason %s", txnId, reason); - return; - } - // was it applied? If so ignore it - if (reason != Reason.Self && txn.saveStatus.hasBeen(Status.Applied)) - return; - TableId tableId = tableId(shard.commandStoreId, commandStores); - TableMetadata tableMetadata = tableMetadata(tableId); - ds.row(userTxn.toString(), keyspace(tableMetadata), table(tableId, tableMetadata), - shard.commandStoreId, depth, reason == Reason.Self ? "" : txn.txnId.toString(), reason.name()); - ds.column("save_status", txn.saveStatus.name()); - if (txn.executeAt != null) - ds.column("execute_at", txn.executeAt.toString()); - if (onDone != null) - onDone.run(); - if (txn.isBlocked()) - { - for (TxnId blockedBy : txn.blockedBy) + TxnId txnId = TxnId.parse((String) pks[0]); + PartitionCollector partition = collector.partition(pks[0]); + partition.collect(rows -> { + try { - if (!processed.contains(blockedBy) && depth < maxDepth) - process(ds, commandStores, shard, processed, userTxn, depth + 1, maxDepth, blockedBy, Reason.Txn, null); + DebugBlockedTxns.visit(AccordService.instance(), txnId, maxDepth, collector.deadlineNanos(), txn -> { + String keyStr = txn.blockedViaKey == null ? "" : txn.blockedViaKey.toString(); + String txnIdStr = txn.txnId == null || txn.txnId.equals(txnId) ? "" : txn.txnId.toString(); + rows.add(txn.commandStoreId, txn.depth, keyStr, txnIdStr) + .eagerCollect(columns -> { + columns.add("save_status", txn.saveStatus, TO_STRING) + .add("execute_at", txn.executeAt, TO_STRING); + }); + }); } - - for (TokenKey blockedBy : txn.blockedByKey) + catch (TimeoutException e) { - TxnId blocking = shard.keys.get(blockedBy); - if (!processed.contains(blocking) && depth < maxDepth) - process(ds, commandStores, shard, processed, userTxn, depth + 1, maxDepth, blocking, Reason.Key, () -> ds.column("key", printToken(blockedBy))); + throw new InternalTimeoutException(); } - } - } - - @Override - public DataSet data() - { - throw new InvalidRequestException("Must select a single txn_id"); + }); } } @@ -1650,33 +1512,12 @@ private static TableMetadata tableMetadata(TableId tableId) return Schema.instance.getTableMetadata(tableId); } - private static String keyspace(TableMetadata metadata) - { - return metadata == null ? "Unknown" : metadata.keyspace; - } - - private static String table(TableId tableId, TableMetadata metadata) - { - return metadata == null ? tableId.toString() : metadata.name; - } - private static String printToken(RoutingKey routingKey) { TokenKey key = (TokenKey) routingKey; return key.token().getPartitioner().getTokenFactory().toString(key.token()); } - private static ByteBuffer sortToken(RoutingKey routingKey) - { - TokenKey key = (TokenKey) routingKey; - Token token = key.token(); - IPartitioner partitioner = token.getPartitioner(); - ByteBuffer out = ByteBuffer.allocate(partitioner.accordSerializedSize(token)); - partitioner.accordSerialize(token, out); - out.flip(); - return out; - } - private static TableMetadata parse(String keyspace, String table, String comment, String schema, AbstractType partitionKeyType) { return CreateTableStatement.parse(format(schema, table), keyspace) @@ -1686,13 +1527,11 @@ private static TableMetadata parse(String keyspace, String table, String comment .build(); } - private static String toStr(Command command, Function> a, Function> b) - { - return toStr(command.participants(), a, b); - } - private static String toStr(StoreParticipants participants, Function> a, Function> b) { + if (participants == null) + return null; + Participants av = a.apply(participants); Participants bv = b.apply(participants); if (av == bv || av.equals(bv)) @@ -1710,6 +1549,14 @@ private static String toStringOrNull(Object o) { if (o == null) return null; - return Objects.toString(o); + + try + { + return Objects.toString(o); + } + catch (Throwable t) + { + return "'; + } } } diff --git a/src/java/org/apache/cassandra/db/virtual/AccordDebugRemoteKeyspace.java b/src/java/org/apache/cassandra/db/virtual/AccordDebugRemoteKeyspace.java new file mode 100644 index 000000000000..8698b3ccd821 --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/AccordDebugRemoteKeyspace.java @@ -0,0 +1,31 @@ +/* + * 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.virtual; + +import org.apache.cassandra.schema.SchemaConstants; + +public class AccordDebugRemoteKeyspace extends RemoteToLocalVirtualKeyspace +{ + public static final AccordDebugRemoteKeyspace instance = new AccordDebugRemoteKeyspace(SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.instance); + + public AccordDebugRemoteKeyspace(String name, VirtualKeyspace wrap) + { + super(name, wrap); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java b/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java index cc311e1a2653..5bff0fe3be43 100644 --- a/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java +++ b/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java @@ -50,6 +50,7 @@ import org.apache.cassandra.db.EmptyIterators; import org.apache.cassandra.db.filter.ClusteringIndexFilter; 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.marshal.AbstractType; import org.apache.cassandra.db.marshal.BooleanType; @@ -308,7 +309,7 @@ else if (partitionKeyTypes.size() > 1) public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringFilter, ColumnFilter columnFilter, - RowFilter rowFilter) + RowFilter rowFilter, DataLimits limits) { if (!data.iterator().hasNext()) return EmptyIterators.unfilteredPartition(metadata); @@ -349,7 +350,7 @@ public UnfilteredPartitionIterator select(DecoratedKey partitionKey, } @Override - public UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter) + public UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) { return createPartitionIterator(metadata, new AbstractIterator<>() { diff --git a/src/java/org/apache/cassandra/db/virtual/PartitionKeyStatsTable.java b/src/java/org/apache/cassandra/db/virtual/PartitionKeyStatsTable.java index 550743c6a734..d63ae194b484 100644 --- a/src/java/org/apache/cassandra/db/virtual/PartitionKeyStatsTable.java +++ b/src/java/org/apache/cassandra/db/virtual/PartitionKeyStatsTable.java @@ -42,6 +42,7 @@ import org.apache.cassandra.db.context.CounterContext; import org.apache.cassandra.db.filter.ClusteringIndexFilter; 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.marshal.CompositeType; import org.apache.cassandra.db.marshal.CounterColumnType; @@ -146,7 +147,7 @@ public PartitionKeyStatsTable(String keyspace) } @Override - public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter) + public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) { if (clusteringIndexFilter.isReversed()) throw new InvalidRequestException(REVERSED_QUERY_ERROR); @@ -345,7 +346,7 @@ public TableMetadata metadata() } @Override - public UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter) + public UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) { throw new InvalidRequestException(UNSUPPORTED_RANGE_QUERY_ERROR); } diff --git a/src/java/org/apache/cassandra/db/virtual/RemoteToLocalVirtualKeyspace.java b/src/java/org/apache/cassandra/db/virtual/RemoteToLocalVirtualKeyspace.java new file mode 100644 index 000000000000..1f395045bb9c --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/RemoteToLocalVirtualKeyspace.java @@ -0,0 +1,29 @@ +/* + * 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.virtual; + +import java.util.stream.Collectors; + +public class RemoteToLocalVirtualKeyspace extends VirtualKeyspace +{ + public RemoteToLocalVirtualKeyspace(String name, VirtualKeyspace wrap) + { + super(name, wrap.tables().stream().map(vt -> new RemoteToLocalVirtualTable(name, vt)).collect(Collectors.toList())); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/RemoteToLocalVirtualTable.java b/src/java/org/apache/cassandra/db/virtual/RemoteToLocalVirtualTable.java new file mode 100644 index 000000000000..c2a1c6a9ad20 --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/RemoteToLocalVirtualTable.java @@ -0,0 +1,609 @@ +/* + * 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.virtual; + +import java.nio.ByteBuffer; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NavigableSet; +import java.util.function.Function; + +import accord.utils.Invariants; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.db.BufferClusteringBound; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.ClusteringBound; +import org.apache.cassandra.db.ClusteringPrefix; +import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionInfo; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.PartitionRangeReadCommand; +import org.apache.cassandra.db.RangeTombstone; +import org.apache.cassandra.db.ReadCommand; +import org.apache.cassandra.db.ReadResponse; +import org.apache.cassandra.db.SinglePartitionReadCommand; +import org.apache.cassandra.db.Slice; +import org.apache.cassandra.db.Slices; +import org.apache.cassandra.db.TruncateRequest; +import org.apache.cassandra.db.filter.ClusteringIndexFilter; +import org.apache.cassandra.db.filter.ClusteringIndexSliceFilter; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.filter.DataLimits; +import org.apache.cassandra.db.filter.IndexHints; +import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.marshal.ByteBufferAccessor; +import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.rows.BTreeRow; +import org.apache.cassandra.db.rows.Cell; +import org.apache.cassandra.db.rows.ColumnData; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.LocalPartitioner; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestFailure; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.RequestCallback; +import org.apache.cassandra.net.Verb; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.membership.NodeId; +import org.apache.cassandra.utils.btree.BTree; +import org.apache.cassandra.utils.concurrent.AsyncPromise; +import org.apache.cassandra.utils.concurrent.Promise; +import org.apache.cassandra.utils.concurrent.SyncPromise; + +import static org.apache.cassandra.db.ClusteringBound.BOTTOM; +import static org.apache.cassandra.db.ClusteringBound.TOP; +import static org.apache.cassandra.db.ClusteringBound.boundKind; +import static org.apache.cassandra.db.ReadCommand.PotentialTxnConflicts.ALLOW; +import static org.apache.cassandra.db.virtual.VirtualTable.Sorted.SORTED; + +public class RemoteToLocalVirtualTable extends AbstractLazyVirtualTable +{ + private static final int MAX_CONCURRENCY = 8; + final TableMetadata local; + final boolean allowFilteringImplicitly; + final boolean allowFilteringLocalPartitionKeysImplicitly; + + public RemoteToLocalVirtualTable(String keyspace, VirtualTable virtualTable) + { + super(wrap(keyspace, virtualTable.name(), virtualTable.metadata()), virtualTable instanceof AbstractLazyVirtualTable ? ((AbstractLazyVirtualTable) virtualTable).onTimeout() : OnTimeout.FAIL, virtualTable.sorted(), SORTED); + this.local = virtualTable.metadata(); + this.allowFilteringImplicitly = virtualTable.allowFilteringImplicitly(); + this.allowFilteringLocalPartitionKeysImplicitly = virtualTable.allowFilteringPrimaryKeysImplicitly(); + } + + @Override + public boolean allowFilteringImplicitly() + { + return allowFilteringImplicitly; + } + + @Override + public boolean allowFilteringPrimaryKeysImplicitly() + { + return true; + } + + private static TableMetadata wrap(String keyspace, String name, TableMetadata local) + { + if (local.partitionKeyColumns().size() != 1 && !(local.partitionKeyType instanceof CompositeType)) + throw new IllegalArgumentException("Underlying table must have a single partition key, else use CompositeType for its partitioner"); + TableMetadata.Builder builder = TableMetadata.builder(keyspace, name); + builder.partitioner(new LocalPartitioner(Int32Type.instance)); + builder.addPartitionKeyColumn("node_id", Int32Type.instance); + for (ColumnMetadata cm : local.partitionKeyColumns()) + builder.addClusteringColumn(cm.name, cm.type, cm.getMask(), cm.getColumnConstraints()); + for (ColumnMetadata cm : local.clusteringColumns()) + builder.addClusteringColumn(cm.name, cm.type, cm.getMask(), cm.getColumnConstraints()); + // we don't add static columns as they can't be modelled correctly with the insertion of a prefix partition column + for (ColumnMetadata cm : local.regularColumns()) + { + if (!cm.isComplex()) + builder.addRegularColumn(cm.name, cm.type, cm.getMask(), cm.getColumnConstraints()); + } + builder.kind(TableMetadata.Kind.VIRTUAL); + return builder.build(); + } + + @Override + protected void collect(PartitionsCollector collector) + { + ClusterMetadata cm = ClusterMetadata.current(); + NavigableSet matchingIds = cm.directory.states.keySet(); + DataRange dataRange = collector.dataRange(); + AbstractBounds bounds = dataRange.keyRange(); + { + NodeId start = null; + if (!bounds.left.isMinimum()) + { + if (!(bounds.left instanceof DecoratedKey)) + throw new InvalidRequestException(metadata + " does not support filtering by token or incomplete partition keys"); + start = new NodeId(Int32Type.instance.compose(((DecoratedKey) bounds.left).getKey())); + } + NodeId end = null; + if (!bounds.right.isMaximum()) + { + if (!(bounds.right instanceof DecoratedKey)) + throw new InvalidRequestException(metadata + " does not support filtering by token or incomplete partition keys"); + end = new NodeId(Int32Type.instance.compose(((DecoratedKey) bounds.right).getKey())); + } + if (start != null && end != null) matchingIds = matchingIds.subSet(start, bounds.isStartInclusive(), end, bounds.isEndInclusive()); + else if (start != null) matchingIds = matchingIds.tailSet(start, bounds.isStartInclusive()); + else if (end != null) matchingIds = matchingIds.headSet(end, bounds.isEndInclusive()); + } + if (dataRange.isReversed()) + matchingIds = matchingIds.descendingSet(); + + RowFilter rowFilter = rebind(local, collector.rowFilter()); + ColumnFilter columnFilter = collector.columnFilter().rebind(local); + // TODO (expected): count this down as we progress where possible (or have AbstractLazyVirtualTable do it for us) + DataLimits limits = collector.limits(); + + Function pksToCks = partitionKeyToClusterings(metadata, local); + ArrayDeque pending = new ArrayDeque<>(); + matchingIds.forEach(id -> { + InetAddressAndPort endpoint = cm.directory.endpoint(id); + DecoratedKey remoteKey = metadata.partitioner.decorateKey(Int32Type.instance.decompose(id.id())); + ClusteringIndexFilter filter = dataRange.clusteringIndexFilter(remoteKey); + Slices slices = filter.getSlices(metadata); + + int i = 0, advance = 1, end = slices.size(); + if (dataRange.isReversed()) + { + i = slices.size() - 1; + end = -1; + advance = -1; + } + + PartitionCollector partition = collector.partition(id.id()); + while (i != end) + { + List request = rebind(local, slices.get(i), dataRange.isReversed(), rowFilter, columnFilter); + for (Request send : request) + { + ReadCommand readCommand; + if (send.dataRange.startKey().equals(send.dataRange.stopKey()) && !send.dataRange.startKey().isMinimum()) + readCommand = SinglePartitionReadCommand.create(local, collector.nowInSeconds(), send.columnFilter, send.rowFilter, limits, (DecoratedKey) send.dataRange.startKey(), send.dataRange.clusteringIndexFilter(remoteKey), ALLOW); + else + readCommand = PartitionRangeReadCommand.create(local, collector.nowInSeconds(), send.columnFilter, send.rowFilter, limits, send.dataRange); + + RequestAndResponse rr = new RequestAndResponse(partition, readCommand); + send(rr, endpoint); + pending.addLast(rr); + + boolean selectsOneRow = selectsOneRow(local, send.dataRange, remoteKey); + while (pending.size() >= (selectsOneRow ? 1 : MAX_CONCURRENCY)) + collect(collector, pending.pollFirst(), pksToCks); + } + i += advance; + } + }); + while (!pending.isEmpty()) + collect(collector, pending.pollFirst(), pksToCks); + } + + private static class RequestAndResponse extends SyncPromise + { + final PartitionCollector partition; + final ReadCommand readCommand; + private RequestAndResponse(PartitionCollector partition, ReadCommand readCommand) + { + this.partition = partition; + this.readCommand = readCommand; + } + } + + private static class Request + { + final DataRange dataRange; + final RowFilter rowFilter; + final ColumnFilter columnFilter; + + private Request(DataRange dataRange, RowFilter rowFilter, ColumnFilter columnFilter) + { + this.dataRange = dataRange; + this.rowFilter = rowFilter; + this.columnFilter = columnFilter; + } + } + + private void send(RequestAndResponse rr, InetAddressAndPort endpoint) + { + send(Verb.READ_REQ, rr.readCommand, rr, endpoint); + } + + private Promise send(Verb verb, Object payload, InetAddressAndPort endpoint) + { + Promise promise = new AsyncPromise<>(); + send(verb, payload, promise, endpoint); + return promise; + } + + private void send(Verb verb, Object payload, Promise promise, InetAddressAndPort endpoint) + { + // we have to send inline some of the MessagingService logic to circumvent the requirement to use AbstractWriteResponseHandler + Message message = Message.out(verb, payload); + RequestCallback callback = new RequestCallback() + { + @Override public void onResponse(Message msg) { promise.trySuccess(msg.payload); } + @Override public boolean invokeOnFailure() { return true; } + @Override public void onFailure(InetAddressAndPort from, RequestFailure failure) + { + if (failure.failure == null) promise.tryFailure(new RuntimeException(failure.reason.toString())); + else promise.tryFailure(failure.failure); + } + }; + + MessagingService.instance().sendWithCallback(message, endpoint, callback); + } + + private void collect(PartitionsCollector collector, RequestAndResponse rr, Function pksToCks) + { + if (!rr.awaitUntilThrowUncheckedOnInterrupt(collector.deadlineNanos())) + throw new InternalTimeoutException(); + + rr.rethrowIfFailed(); + int pkCount = local.partitionKeyColumns().size(); + ReadResponse response = Invariants.nonNull(rr.getNow()); + try (UnfilteredPartitionIterator partitions = response.makeIterator(rr.readCommand)) + { + while (partitions.hasNext()) + { + try (UnfilteredRowIterator iter = partitions.next()) + { + ByteBuffer[] clusterings = pksToCks.apply(iter.partitionKey()); + while (iter.hasNext()) + { + Unfiltered next = iter.next(); + if (!next.isRow()) + throw new UnsupportedOperationException("Range tombstones not supported"); + + Row row = (Row)next; + { + Clustering clustering = row.clustering(); + for (int j = 0 ; j < clustering.size(); ++j) + clusterings[pkCount + j] = clustering.bufferAt(j); + } + rr.partition.collect(rows -> { + rows.add((Object[])clusterings) + .lazyCollect(columns -> { + row.forEach(cd -> { + Invariants.require(cd instanceof Cell); + columns.add(cd.column().name.toString(), ((Cell) cd).buffer()); + }); + }); + }); + } + } + } + } + } + + private static boolean selectsOneRow(TableMetadata metadata, DataRange dataRange, DecoratedKey key) + { + if (dataRange.startKey().isMinimum() || !dataRange.startKey().equals(dataRange.stopKey())) + return false; + + if (metadata.clusteringColumns().isEmpty()) + return true; + + Slices slices = dataRange.clusteringIndexFilter(key).getSlices(metadata); + if (slices.size() != 1) + return false; + + Slice slice = slices.get(0); + return slice.start().equals(slice.end()); + } + + private static Function partitionKeyToClusterings(TableMetadata distributed, TableMetadata local) + { + ByteBuffer[] cks = new ByteBuffer[distributed.clusteringColumns().size()]; + if (local.partitionKeyColumns().size() == 1) + { + return pk -> { + cks[0] = pk.getKey(); + return cks.clone(); + }; + } + + CompositeType type = (CompositeType) local.partitionKeyType; + int pkCount = type.types.size(); + return (pk) -> { + System.arraycopy(type.split(pk.getKey()), 0, cks, 0, pkCount); + return cks.clone(); + }; + } + + private static RowFilter rebind(TableMetadata local, RowFilter rowFilter) + { + if (rowFilter.isEmpty()) + return rowFilter; + + RowFilter result = RowFilter.create(false, IndexHints.NONE); + for (RowFilter.Expression in : rowFilter.getExpressions()) + { + RowFilter.Expression out = in.rebind(local); + if (out != null) + result.add(out); + } + return result; + } + + private List rebind(TableMetadata local, Slice slice, boolean reversed, RowFilter rowFilter, ColumnFilter columnFilter) + { + ClusteringBound start = slice.start(); + ClusteringBound end = slice.end(); + int pkCount = local.partitionKeyColumns().size(); + // TODO (expected): we can filter by partition key by inserting a new row filter, but need to impose ALLOW FILTERING restrictions + if (((start.size() > 0 && start.size() < pkCount) || (end.size() > 0 && end.size() < pkCount))) + { + if (!allowFilteringLocalPartitionKeysImplicitly) + throw new InvalidRequestException("Must specify full partition key bounds for the underlying table"); + + List pks = local.partitionKeyColumns(); + ByteBuffer[] starts = start.getBufferArray(); + ByteBuffer[] ends = end.getBufferArray(); + + int minCount = Math.min(start.size(), end.size()); + int maxCount = Math.max(start.size(), end.size()); + int commonPrefixLength = 0; + while (commonPrefixLength < minCount && equalPart(start, end, commonPrefixLength)) + ++commonPrefixLength; + + RowFilter commonRowFilter = rowFilter; + if (commonPrefixLength > 0) + { + commonRowFilter = copy(commonRowFilter); + for (int i = 0 ; i < commonPrefixLength ; ++i) + commonRowFilter.add(pks.get(i), Operator.EQ, starts[i]); + } + + Operator lastStartOp = start.isInclusive() ? Operator.GTE : Operator.GT; + Operator lastEndOp = end.isInclusive() ? Operator.LTE : Operator.LT; + if (commonPrefixLength == Math.max(minCount, maxCount - 1)) + { + // can simply add our remaining filters and continue on our way + addExpressions(commonRowFilter, pks, commonPrefixLength, starts, Operator.GTE, lastStartOp); + addExpressions(commonRowFilter, pks, commonPrefixLength, ends, Operator.LTE, lastEndOp); + return List.of(new Request(DataRange.allData(local.partitioner), commonRowFilter, columnFilter)); + } + + throw new InvalidRequestException("This table currently does not support the complex partial partition key filters implied for the underlying table"); + } + + ByteBuffer[] startBuffers = start.getBufferArray(); + PartitionPosition startBound; + if (start.size() == 0) startBound = local.partitioner.getMinimumToken().minKeyBound(); + else if (pkCount == 1) startBound = local.partitioner.decorateKey(startBuffers[0]); + else startBound = local.partitioner.decorateKey(CompositeType.build(ByteBufferAccessor.instance, Arrays.copyOf(startBuffers, pkCount))); + + ByteBuffer[] endBuffers = end.getBufferArray(); + PartitionPosition endBound; + if (end.size() == 0) endBound = local.partitioner.getMinimumToken().maxKeyBound(); + else if (pkCount == 1) endBound = local.partitioner.decorateKey(endBuffers[0]); + else endBound = local.partitioner.decorateKey(CompositeType.build(ByteBufferAccessor.instance, Arrays.copyOf(endBuffers, pkCount))); + + AbstractBounds bounds = AbstractBounds.bounds(startBound, start.isEmpty() || start.size() > pkCount || start.isInclusive(), + endBound, end.isEmpty() || end.size() > pkCount || end.isInclusive()); + boolean hasSlices = start.size() > pkCount || end.size() > pkCount; + if (!hasSlices) + return List.of(new Request(new DataRange(bounds, new ClusteringIndexSliceFilter(Slices.ALL, reversed)), rowFilter, columnFilter)); + + ClusteringBound startSlice = ClusteringBound.BOTTOM; + if (start.size() > pkCount) + startSlice = BufferClusteringBound.create(boundKind(true, start.isInclusive()), Arrays.copyOfRange(startBuffers, pkCount, startBuffers.length)); + + ClusteringBound endSlice = ClusteringBound.TOP; + if (end.size() > pkCount) + endSlice = BufferClusteringBound.create(boundKind(false, end.isInclusive()), Arrays.copyOfRange(startBuffers, pkCount, startBuffers.length)); + + if (startBound.equals(endBound)) + return List.of(new Request(new DataRange(bounds, filter(local, startSlice, endSlice, reversed)), rowFilter, columnFilter)); + + List result = new ArrayList<>(3); + if (startSlice != BOTTOM) + { + AbstractBounds startBoundOnly = AbstractBounds.bounds(startBound, true, startBound, true); + result.add(new Request(new DataRange(startBoundOnly, filter(local, startSlice, TOP, reversed)), rowFilter, columnFilter)); + } + result.add(new Request(new DataRange(AbstractBounds.bounds(bounds.left, bounds.inclusiveLeft() && startSlice == BOTTOM, + bounds.right, bounds.inclusiveRight() && endSlice == TOP), + new ClusteringIndexSliceFilter(Slices.ALL, reversed)), rowFilter, columnFilter) + ); + if (endSlice != TOP) + { + AbstractBounds endBoundOnly = AbstractBounds.bounds(endBound, true, endBound, true); + result.add(new Request(new DataRange(endBoundOnly, filter(local, BOTTOM, endSlice, reversed)), rowFilter, columnFilter)); + } + if (reversed) + Collections.reverse(result); + return result; + } + + @SuppressWarnings({ "rawtypes", "unchecked" }) + private static boolean equalPart(ClusteringBound start, ClusteringBound end, int i) + { + return 0 == start.accessor().compare(start.get(i), end.get(i), end.accessor()); + } + + private static RowFilter copy(RowFilter copy) + { + RowFilter newRowFilter = RowFilter.create(false, IndexHints.NONE); + for (RowFilter.Expression expression : copy) + newRowFilter.add(expression); + return newRowFilter; + } + + private static void addExpressions(RowFilter rowFilter, List cms, int start, ByteBuffer[] values, Operator op, Operator lastOp) + { + for (int i = start ; i < values.length ; ++i) + rowFilter.add(cms.get(i), i + 1 == values.length ? lastOp : op, values[i]); + } + + private static ClusteringIndexSliceFilter filter(TableMetadata metadata, ClusteringBound start, ClusteringBound end, boolean reversed) + { + return new ClusteringIndexSliceFilter(Slices.with(metadata.comparator, Slice.make(start, end)), reversed); + } + + @Override + public void apply(PartitionUpdate update) + { + int nodeId = Int32Type.instance.compose(update.partitionKey().getKey()); + InetAddressAndPort endpoint = ClusterMetadata.current().directory.endpoint(new NodeId(nodeId)); + if (endpoint == null) + throw new InvalidRequestException("Unknown node " + nodeId); + + DeletionInfo deletionInfo = update.deletionInfo(); + if (!deletionInfo.getPartitionDeletion().isLive()) + { + truncate(endpoint).syncThrowUncheckedOnInterrupt(); + return; + } + + int pkCount = local.partitionKeyColumns().size(); + ByteBuffer[] pkBuffer, ckBuffer; + { + int ckCount = local.clusteringColumns().size(); + pkBuffer = pkCount == 1 ? null : new ByteBuffer[pkCount]; + ckBuffer = new ByteBuffer[ckCount]; + } + + PartitionUpdate.Builder builder = null; + ArrayDeque> results = new ArrayDeque<>(); + + if (deletionInfo.hasRanges()) + { + Iterator iterator = deletionInfo.rangeIterator(false); + while (iterator.hasNext()) + { + RangeTombstone rt = iterator.next(); + ClusteringBound start = rt.deletedSlice().start(); + ClusteringBound end = rt.deletedSlice().end(); + if (start.size() < pkCount || end.size() < pkCount) + throw new InvalidRequestException("Range deletions must specify a complete partition key in the underlying table " + metadata); + + for (int i = 0 ; i < pkCount ; ++i) + { + if (0 != start.accessor().compare(start.get(i), end.get(i), end.accessor())) + throw new InvalidRequestException("Range deletions must specify a single partition key in the underlying table " + metadata); + } + + DecoratedKey key = remoteClusteringToLocalPartitionKey(local, start, pkCount, pkBuffer); + builder = maybeRolloverAndWait(key, builder, results, endpoint); + if (start.size() == pkCount && end.size() == pkCount) + { + builder.addPartitionDeletion(rt.deletionTime()); + } + else + { + start = ClusteringBound.create(start.kind(), Clustering.make(remoteClusteringToLocalClustering(start.clustering(), pkCount, ckBuffer))); + end = ClusteringBound.create(end.kind(), Clustering.make(remoteClusteringToLocalClustering(end.clustering(), pkCount, ckBuffer))); + builder.add(new RangeTombstone(Slice.make(start, end), rt.deletionTime())); + } + } + } + + if (!update.staticRow().isEmpty()) + throw new InvalidRequestException("Static rows are not supported for remote table " + metadata); + + try (BTree.FastBuilder columns = BTree.fastBuilder()) + { + for (Row row : update) + { + Clustering clustering = row.clustering(); + DecoratedKey key = remoteClusteringToLocalPartitionKey(local, clustering, pkCount, pkBuffer); + builder = maybeRolloverAndWait(key, builder, results, endpoint); + Clustering newClustering = Clustering.make(remoteClusteringToLocalClustering(clustering, pkCount, ckBuffer)); + columns.reset(); + for (ColumnData cd : row) + columns.add(rebind(local, cd)); + builder.add(BTreeRow.create(newClustering, row.primaryKeyLivenessInfo(), row.deletion(), columns.build())); + } + } + + if (builder != null) + results.add(send(Verb.VIRTUAL_MUTATION_REQ, new VirtualMutation(builder.build()), endpoint)); + + while (!results.isEmpty()) + results.pollFirst().syncThrowUncheckedOnInterrupt(); + } + + private PartitionUpdate.Builder maybeRolloverAndWait(DecoratedKey key, PartitionUpdate.Builder builder, ArrayDeque> waiting, InetAddressAndPort endpoint) + { + if (builder == null || !builder.partitionKey().equals(key)) + { + if (builder != null) + waiting.add(send(Verb.VIRTUAL_MUTATION_REQ, new VirtualMutation(builder.build()), endpoint)); + builder = new PartitionUpdate.Builder(local, key, local.regularAndStaticColumns(), 8); + while (waiting.size() >= MAX_CONCURRENCY) + waiting.pollFirst().syncThrowUncheckedOnInterrupt(); + } + return builder; + } + + private Promise truncate(InetAddressAndPort endpoint) + { + return send(Verb.TRUNCATE_REQ, new TruncateRequest(local.keyspace, local.name), endpoint); + } + + private static ColumnData rebind(TableMetadata local, ColumnData cd) + { + ColumnMetadata column = local.getColumn(cd.column().name); + + Invariants.require(column != null, cd.column() + " not found in " + local); + Invariants.require(!column.isComplex(), "Complex column " + column + " not supported; should have been removed from metadata"); + + return ((Cell) cd).withUpdatedColumn(column); + } + + private static DecoratedKey remoteClusteringToLocalPartitionKey(TableMetadata local, ClusteringPrefix clustering, int pkCount, ByteBuffer[] pkBuffer) + { + ByteBuffer bytes; + if (pkCount == 1) bytes = clustering.bufferAt(0); + else + { + for (int i = 0 ; i < pkBuffer.length ; ++i) + pkBuffer[i] = clustering.bufferAt(i); + bytes = CompositeType.build(ByteBufferAccessor.instance, pkBuffer); + } + return local.partitioner.decorateKey(bytes); + } + + private static ByteBuffer[] remoteClusteringToLocalClustering(ClusteringPrefix clustering, int pkCount, ByteBuffer[] ckBuffer) + { + for (int i = pkCount ; i < clustering.size(); ++i) + ckBuffer[i - pkCount] = clustering.bufferAt(i); + + return Arrays.copyOf(ckBuffer, clustering.size() - pkCount); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/VirtualKeyspaceRegistry.java b/src/java/org/apache/cassandra/db/virtual/VirtualKeyspaceRegistry.java index 23814cdf59a0..653536c4320d 100644 --- a/src/java/org/apache/cassandra/db/virtual/VirtualKeyspaceRegistry.java +++ b/src/java/org/apache/cassandra/db/virtual/VirtualKeyspaceRegistry.java @@ -24,8 +24,11 @@ import com.google.common.collect.Iterables; import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.ClusterMetadataService; public final class VirtualKeyspaceRegistry { diff --git a/src/java/org/apache/cassandra/db/virtual/VirtualMutation.java b/src/java/org/apache/cassandra/db/virtual/VirtualMutation.java index ee98da29c115..ddfe5a843ec4 100644 --- a/src/java/org/apache/cassandra/db/virtual/VirtualMutation.java +++ b/src/java/org/apache/cassandra/db/virtual/VirtualMutation.java @@ -17,7 +17,9 @@ */ package org.apache.cassandra.db.virtual; +import java.io.IOException; import java.util.Collection; +import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.function.Predicate; import java.util.function.Supplier; @@ -26,14 +28,27 @@ import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableMap; +import accord.utils.Invariants; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.IMutation; import org.apache.cassandra.db.Mutation; import org.apache.cassandra.db.ReadCommand.PotentialTxnConflicts; +import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.rows.DeserializationHelper; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.net.IVerbHandler; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.NoPayload; import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.serializers.CollectionSerializer; import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.utils.CollectionSerializers; /** * A specialised IMutation implementation for virtual keyspaces. @@ -42,6 +57,35 @@ */ public final class VirtualMutation implements IMutation { + public static final IVersionedSerializer serializer = new IVersionedSerializer() + { + @Override + public void serialize(VirtualMutation t, DataOutputPlus out, int version) throws IOException + { + Invariants.require(t.modifications.size() == 1); + PartitionUpdate.serializer.serialize(t.modifications.values().iterator().next(), out, version); + } + + @Override + public VirtualMutation deserialize(DataInputPlus in, int version) throws IOException + { + PartitionUpdate update = PartitionUpdate.serializer.deserialize(in, version, DeserializationHelper.Flag.FROM_REMOTE); + return new VirtualMutation(update); + } + + @Override + public long serializedSize(VirtualMutation t, int version) + { + Invariants.require(t.modifications.size() == 1); + return PartitionUpdate.serializer.serializedSize(t.modifications.values().iterator().next(), version); + } + }; + + public static final IVerbHandler handler = message -> { + message.payload.apply(); + MessagingService.instance().respond(NoPayload.noPayload, message); + }; + private final String keyspaceName; private final DecoratedKey partitionKey; private final ImmutableMap modifications; diff --git a/src/java/org/apache/cassandra/db/virtual/VirtualTable.java b/src/java/org/apache/cassandra/db/virtual/VirtualTable.java index 770cb139830f..0423b12a4d05 100644 --- a/src/java/org/apache/cassandra/db/virtual/VirtualTable.java +++ b/src/java/org/apache/cassandra/db/virtual/VirtualTable.java @@ -21,6 +21,7 @@ import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.filter.ClusteringIndexFilter; 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.PartitionUpdate; import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; @@ -31,6 +32,8 @@ */ public interface VirtualTable { + enum Sorted { UNSORTED, ASC, DESC, SORTED } + /** * Returns the view name. * @@ -57,23 +60,25 @@ default String name() /** * Selects the rows from a single partition. * - * @param partitionKey the partition key + * @param partitionKey the partition key * @param clusteringIndexFilter the clustering columns to selected - * @param columnFilter the selected columns - * @param rowFilter filter on which rows a given query should include or exclude + * @param columnFilter the selected columns + * @param rowFilter filter on which rows a given query should include or exclude + * @param limits result limits to apply * @return the rows corresponding to the requested data. */ - UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter); + UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits); /** * Selects the rows from a range of partitions. * - * @param dataRange the range of data to retrieve + * @param dataRange the range of data to retrieve * @param columnFilter the selected columns - * @param rowFilter filter on which rows a given query should include or exclude + * @param rowFilter filter on which rows a given query should include or exclude + * @param limits * @return the rows corresponding to the requested data. */ - UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter); + UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits); /** * Truncates data from the underlying source, if supported. @@ -90,4 +95,11 @@ default boolean allowFilteringImplicitly() { return true; } + + default boolean allowFilteringPrimaryKeysImplicitly() + { + return allowFilteringImplicitly(); + } + + default Sorted sorted() { return Sorted.UNSORTED; } } diff --git a/src/java/org/apache/cassandra/exceptions/ExceptionSerializer.java b/src/java/org/apache/cassandra/exceptions/ExceptionSerializer.java index de379739a383..834abca49dd0 100644 --- a/src/java/org/apache/cassandra/exceptions/ExceptionSerializer.java +++ b/src/java/org/apache/cassandra/exceptions/ExceptionSerializer.java @@ -46,7 +46,7 @@ public class ExceptionSerializer { public static class RemoteException extends RuntimeException { - private final String originalClass; + public final String originalClass; public RemoteException(String originalClass, String originalMessage, StackTraceElement[] stackTrace) { diff --git a/src/java/org/apache/cassandra/exceptions/RequestFailure.java b/src/java/org/apache/cassandra/exceptions/RequestFailure.java index b9bba7fc7061..03d088a239b6 100644 --- a/src/java/org/apache/cassandra/exceptions/RequestFailure.java +++ b/src/java/org/apache/cassandra/exceptions/RequestFailure.java @@ -31,6 +31,7 @@ import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.cassandra.exceptions.ExceptionSerializer.nullableRemoteExceptionSerializer; +import static org.apache.cassandra.exceptions.RequestFailureReason.UNKNOWN; /** * Allow inclusion of a serialized exception in failure response messages @@ -125,7 +126,7 @@ public static RequestFailure forException(Throwable t) if (t instanceof CoordinatorBehindException) return COORDINATOR_BEHIND; - return UNKNOWN; + return new RequestFailure(t); } public static RequestFailure forReason(RequestFailureReason reason) diff --git a/src/java/org/apache/cassandra/journal/InMemoryIndex.java b/src/java/org/apache/cassandra/journal/InMemoryIndex.java index 49fe4d136714..974767590a37 100644 --- a/src/java/org/apache/cassandra/journal/InMemoryIndex.java +++ b/src/java/org/apache/cassandra/journal/InMemoryIndex.java @@ -18,6 +18,7 @@ package org.apache.cassandra.journal; import java.io.IOException; +import java.util.Iterator; import java.util.NavigableMap; import java.util.TreeMap; import java.util.concurrent.ConcurrentSkipListMap; @@ -126,6 +127,16 @@ long[] lookUpAll(K id) return lookUp(id); } + public Iterator keyIterator(@Nullable K min, @Nullable K max) + { + NavigableMap m; + if (min == null && max == null) m = index; + else if (min == null) m = index.headMap(max, true); + else if (max == null) m = index.tailMap(min, true); + else m = index.subMap(min, true, max, true); + return m.keySet().iterator(); + } + public void persist(Descriptor descriptor) { File tmpFile = descriptor.tmpFileFor(Component.INDEX); diff --git a/src/java/org/apache/cassandra/journal/Journal.java b/src/java/org/apache/cassandra/journal/Journal.java index d0793d39a7b8..2a63ab7cc4b0 100644 --- a/src/java/org/apache/cassandra/journal/Journal.java +++ b/src/java/org/apache/cassandra/journal/Journal.java @@ -934,11 +934,11 @@ public interface Writer } /** - * Static segment iterator iterates all keys in _static_ segments in order. + * segment iterator iterates all keys in order. */ - public StaticSegmentKeyIterator staticSegmentKeyIterator(K min, K max) + public SegmentKeyIterator segmentKeyIterator(K min, K max, Predicate> include) { - return new StaticSegmentKeyIterator(min, max); + return new SegmentKeyIterator(min, max, include); } /** @@ -1000,53 +1000,36 @@ public String toString() } } - public class StaticSegmentKeyIterator implements CloseableIterator> + public class SegmentKeyIterator implements CloseableIterator> { private final ReferencedSegments segments; private final MergeIterator> iterator; - public StaticSegmentKeyIterator(K min, K max) + public SegmentKeyIterator(K min, K max, Predicate> include) { - this.segments = selectAndReference(s -> s.isStatic() - && s.asStatic().index().entryCount() > 0 + this.segments = selectAndReference(s -> include.test(s) && !s.isEmpty() && (min == null || keySupport.compare(s.index().lastId(), min) >= 0) && (max == null || keySupport.compare(s.index().firstId(), max) <= 0)); List> iterators = new ArrayList<>(segments.count()); for (Segment segment : segments.allSorted(true)) { - final StaticSegment staticSegment = (StaticSegment) segment; - final OnDiskIndex.IndexReader iter = staticSegment.index().reader(); - if (min != null) iter.seek(min); - if (max != null) iter.seekEnd(max); - if (!iter.hasNext()) - continue; - - iterators.add(new AbstractIterator<>() + if (segment.isStatic()) { - final Head head = new Head(staticSegment.descriptor.timestamp); - - @Override - protected Head computeNext() - { - if (!iter.hasNext()) - return endOfData(); - - K next = iter.next(); - while (next.equals(head.key)) - { - if (!iter.hasNext()) - return endOfData(); - - next = iter.next(); - } - - Invariants.require(!next.equals(head.key), - "%s == %s", next, head.key); - head.key = next; - return head; - } - }); + final StaticSegment staticSegment = (StaticSegment) segment; + final OnDiskIndex.IndexReader iter = staticSegment.index().reader(); + if (min != null) iter.seek(min); + if (max != null) iter.seekEnd(max); + if (iter.hasNext()) + iterators.add(keyIterator(segment.descriptor.timestamp, iter)); + } + else + { + final ActiveSegment activeSegment = (ActiveSegment) segment; + final Iterator iter = activeSegment.index().keyIterator(min, max); + if (iter.hasNext()) + iterators.add(keyIterator(segment.descriptor.timestamp, iter)); + } } this.iterator = MergeIterator.get(iterators, @@ -1077,6 +1060,34 @@ protected void onKeyChange() }); } + private Iterator keyIterator(long segment, Iterator iter) + { + final Head head = new Head(segment); + return new AbstractIterator<>() + { + @Override + protected Head computeNext() + { + if (!iter.hasNext()) + return endOfData(); + + K next = iter.next(); + while (next.equals(head.key)) + { + if (!iter.hasNext()) + return endOfData(); + + next = iter.next(); + } + + Invariants.require(!next.equals(head.key), + "%s == %s", next, head.key); + head.key = next; + return head; + } + }; + } + @Override public void close() { diff --git a/src/java/org/apache/cassandra/journal/Segment.java b/src/java/org/apache/cassandra/journal/Segment.java index 3854f0ee27b5..1fda2f57c982 100644 --- a/src/java/org/apache/cassandra/journal/Segment.java +++ b/src/java/org/apache/cassandra/journal/Segment.java @@ -66,7 +66,8 @@ public final void tidy() abstract boolean isActive(); abstract boolean isFlushed(long position); - boolean isStatic() { return !isActive(); } + public boolean isStatic() { return !isActive(); } + abstract boolean isEmpty(); abstract ActiveSegment asActive(); abstract StaticSegment asStatic(); diff --git a/src/java/org/apache/cassandra/journal/StaticSegment.java b/src/java/org/apache/cassandra/journal/StaticSegment.java index 35c987c8a480..bc425dda7e2f 100644 --- a/src/java/org/apache/cassandra/journal/StaticSegment.java +++ b/src/java/org/apache/cassandra/journal/StaticSegment.java @@ -246,6 +246,12 @@ public int entryCount() return index.entryCount(); } + @Override + boolean isEmpty() + { + return entryCount() == 0; + } + @Override boolean isActive() { diff --git a/src/java/org/apache/cassandra/locator/RemoteStrategy.java b/src/java/org/apache/cassandra/locator/RemoteStrategy.java new file mode 100644 index 000000000000..515462ee980b --- /dev/null +++ b/src/java/org/apache/cassandra/locator/RemoteStrategy.java @@ -0,0 +1,29 @@ +/* + * 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.locator; + +import java.util.Map; + +public class RemoteStrategy extends LocalStrategy +{ + public RemoteStrategy(String keyspaceName, Map configOptions) + { + super(keyspaceName, configOptions); + } +} diff --git a/src/java/org/apache/cassandra/metrics/AccordSystemMetrics.java b/src/java/org/apache/cassandra/metrics/AccordSystemMetrics.java index a47fff2302b3..c9f813022537 100644 --- a/src/java/org/apache/cassandra/metrics/AccordSystemMetrics.java +++ b/src/java/org/apache/cassandra/metrics/AccordSystemMetrics.java @@ -175,7 +175,7 @@ private synchronized void refreshHistograms() int nowSeconds = (int) (Clock.Global.currentTimeMillis() / 1000); SnapshotBuilder builder = new SnapshotBuilder(); - service.node().commandStores().forEachCommandStore(commandStore -> { + service.node().commandStores().forAllUnsafe(commandStore -> { DefaultProgressLog.ImmutableView view = ((DefaultProgressLog)commandStore.unsafeProgressLog()).immutableView(); builder.progressLogActive += view.activeCount(); builder.progressLogSize.increment(view.size()); diff --git a/src/java/org/apache/cassandra/net/Verb.java b/src/java/org/apache/cassandra/net/Verb.java index d24c9e64adff..4a0ebb2c2216 100644 --- a/src/java/org/apache/cassandra/net/Verb.java +++ b/src/java/org/apache/cassandra/net/Verb.java @@ -44,6 +44,7 @@ import org.apache.cassandra.db.TruncateRequest; import org.apache.cassandra.db.TruncateResponse; import org.apache.cassandra.db.TruncateVerbHandler; +import org.apache.cassandra.db.virtual.VirtualMutation; import org.apache.cassandra.exceptions.RequestFailure; import org.apache.cassandra.gms.GossipDigestAck; import org.apache.cassandra.gms.GossipDigestAck2; @@ -195,6 +196,8 @@ public enum Verb { MUTATION_RSP (60, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, RESPONSE_HANDLER ), MUTATION_REQ (0, P3, writeTimeout, MUTATION, () -> Mutation.serializer, () -> MutationVerbHandler.instance, MUTATION_RSP ), + VIRTUAL_MUTATION_RSP (200, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, RESPONSE_HANDLER ), + VIRTUAL_MUTATION_REQ (201, P3, writeTimeout, MUTATION, () -> VirtualMutation.serializer, () -> VirtualMutation.handler, VIRTUAL_MUTATION_RSP), HINT_RSP (61, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, RESPONSE_HANDLER ), HINT_REQ (1, P4, writeTimeout, MUTATION, () -> HintMessage.serializer, () -> HintVerbHandler.instance, HINT_RSP ), READ_REPAIR_RSP (62, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, RESPONSE_HANDLER ), diff --git a/src/java/org/apache/cassandra/schema/SchemaConstants.java b/src/java/org/apache/cassandra/schema/SchemaConstants.java index 6e3cb7a72eb7..ada413537d15 100644 --- a/src/java/org/apache/cassandra/schema/SchemaConstants.java +++ b/src/java/org/apache/cassandra/schema/SchemaConstants.java @@ -57,6 +57,7 @@ public final class SchemaConstants public static final String VIRTUAL_VIEWS = "system_views"; public static final String VIRTUAL_METRICS = "system_metrics"; public static final String VIRTUAL_ACCORD_DEBUG = "system_accord_debug"; + public static final String VIRTUAL_ACCORD_DEBUG_REMOTE = "system_accord_debug_remote"; public static final String DUMMY_KEYSPACE_OR_TABLE_NAME = "--dummy--"; @@ -66,7 +67,7 @@ public final class SchemaConstants /* virtual table system keyspace names */ public static final Set VIRTUAL_SYSTEM_KEYSPACE_NAMES = - ImmutableSet.of(VIRTUAL_SCHEMA, VIRTUAL_VIEWS, VIRTUAL_METRICS); + ImmutableSet.of(VIRTUAL_SCHEMA, VIRTUAL_VIEWS, VIRTUAL_METRICS, VIRTUAL_ACCORD_DEBUG, VIRTUAL_ACCORD_DEBUG_REMOTE); /* replicate system keyspace names (the ones with a "true" replication strategy) */ public static final Set REPLICATED_SYSTEM_KEYSPACE_NAMES = diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java index c297ab191297..27e50157a3b5 100644 --- a/src/java/org/apache/cassandra/service/ActiveRepairService.java +++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java @@ -582,8 +582,7 @@ else if (range.intersects(toRepair)) { throw new IllegalArgumentException(String.format("Requested range %s intersects a local range (%s) " + "but is not fully contained in one; this would lead to " + - "imprecise repair. keyspace: %s", toRepair.toString(), - range.toString(), keyspaceName)); + "imprecise repair. keyspace: %s", toRepair, range, keyspaceName)); } } if (rangeSuperSet == null || !replicaSets.containsKey(rangeSuperSet)) diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index d8acdbc26666..e9aa4c96c013 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -59,6 +59,7 @@ import org.apache.cassandra.db.SystemKeyspaceMigrator41; import org.apache.cassandra.db.commitlog.CommitLog; import org.apache.cassandra.db.virtual.AccordDebugKeyspace; +import org.apache.cassandra.db.virtual.AccordDebugRemoteKeyspace; import org.apache.cassandra.db.virtual.LogMessagesTable; import org.apache.cassandra.db.virtual.SlowQueriesTable; import org.apache.cassandra.db.virtual.SystemViewsKeyspace; @@ -556,7 +557,10 @@ public void setupVirtualKeyspaces() VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(VIRTUAL_METRICS, createMetricsKeyspaceTables())); if (DatabaseDescriptor.getAccord().enable_virtual_debug_only_keyspace) + { VirtualKeyspaceRegistry.instance.register(AccordDebugKeyspace.instance); + VirtualKeyspaceRegistry.instance.register(AccordDebugRemoteKeyspace.instance); + } // Flush log messages to system_views.system_logs virtual table as there were messages already logged // before that virtual table was instantiated. diff --git a/src/java/org/apache/cassandra/service/Rebuild.java b/src/java/org/apache/cassandra/service/Rebuild.java index c7d40f08bf5b..56ad1f47ba88 100644 --- a/src/java/org/apache/cassandra/service/Rebuild.java +++ b/src/java/org/apache/cassandra/service/Rebuild.java @@ -33,6 +33,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import accord.api.ConfigurationService.EpochReady; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.dht.Range; @@ -160,7 +161,7 @@ else if (tokens == null) StreamResultFuture streamResult = streamer.fetchAsync(); - Future accordReady = AccordService.instance().epochReadyFor(metadata); + Future accordReady = AccordService.instance().epochReadyFor(metadata, EpochReady::reads); Future ready = FutureCombiner.allOf(streamResult, accordReady); // wait for result diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 8d861618a58f..84a61f6b2646 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -71,9 +71,6 @@ import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.common.util.concurrent.Uninterruptibles; - -import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.repair.autorepair.AutoRepair; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -134,6 +131,7 @@ import org.apache.cassandra.io.sstable.IScrubber; import org.apache.cassandra.io.sstable.IVerifier; import org.apache.cassandra.io.sstable.SSTableLoader; +import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.format.Version; import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileUtils; @@ -158,7 +156,9 @@ import org.apache.cassandra.metrics.StorageMetrics; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.RepairCoordinator; +import org.apache.cassandra.repair.RepairParallelism; import org.apache.cassandra.repair.SharedContext; +import org.apache.cassandra.repair.autorepair.AutoRepair; import org.apache.cassandra.repair.messages.RepairOption; import org.apache.cassandra.schema.CompactionParams.TombstoneOption; import org.apache.cassandra.schema.KeyspaceMetadata; @@ -183,6 +183,7 @@ import org.apache.cassandra.service.paxos.cleanup.PaxosCleanupLocalCoordinator; import org.apache.cassandra.service.paxos.cleanup.PaxosRepairState; import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.streaming.StreamManager; import org.apache.cassandra.streaming.StreamResultFuture; import org.apache.cassandra.streaming.StreamState; @@ -206,9 +207,9 @@ import org.apache.cassandra.tcm.sequences.BootstrapAndReplace; import org.apache.cassandra.tcm.sequences.InProgressSequences; import org.apache.cassandra.tcm.sequences.SingleNodeSequences; +import org.apache.cassandra.tcm.transformations.AlterTopology; import org.apache.cassandra.tcm.transformations.Assassinate; import org.apache.cassandra.tcm.transformations.CancelInProgressSequence; -import org.apache.cassandra.tcm.transformations.AlterTopology; import org.apache.cassandra.tcm.transformations.Register; import org.apache.cassandra.tcm.transformations.Startup; import org.apache.cassandra.tcm.transformations.Unregister; @@ -3145,6 +3146,29 @@ private FutureTask createRepairTask(final int cmd, final String keyspace return new FutureTask<>(task); } + public RepairCoordinator repairAccordKeyspace(String keyspace, Collection> ranges) + { + int cmd = nextRepairCommand.incrementAndGet(); + RepairOption options = new RepairOption(RepairParallelism.PARALLEL, // parallelism + false, // primaryRange + false, // incremental + false, // trace + 5, // jobThreads + ranges, // ranges + true, // pullRepair + true, // forceRepair + PreviewKind.NONE, // previewKind + false, // optimiseStreams + true, // ignoreUnreplicatedKeyspaces + true, // repairData + false, // repairPaxos + true, // dontPurgeTombstones + false // repairAccord + ); + + return new RepairCoordinator(this, cmd, options, keyspace); + } + private void tryRepairPaxosForTopologyChange(String reason) { try diff --git a/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java b/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java index cbf1fe49dc64..68fcaa8e4aec 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java +++ b/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java @@ -37,9 +37,9 @@ import org.apache.cassandra.cache.CacheSize; import org.apache.cassandra.config.AccordSpec.QueueShardModel; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.schema.TableId; import org.apache.cassandra.service.accord.AccordExecutor.AccordExecutorFactory; -import org.apache.cassandra.service.accord.api.TokenKey; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.membership.NodeState; import static org.apache.cassandra.config.AccordSpec.QueueShardModel.THREAD_PER_SHARD; import static org.apache.cassandra.config.DatabaseDescriptor.getAccordQueueShardCount; @@ -112,15 +112,6 @@ static Factory factory() }; } - @Override - protected boolean shouldBootstrap(Node node, Topology previous, Topology updated, Range range) - { - if (!super.shouldBootstrap(node, previous, updated, range)) - return false; - // we see new ranges when a new keyspace is added, so avoid bootstrap in these cases - return contains(previous, ((TokenKey) range.start()).table()); - } - @Override public SequentialAsyncExecutor someSequentialExecutor() { @@ -128,17 +119,6 @@ public SequentialAsyncExecutor someSequentialExecutor() return executors[idx].newSequentialExecutor(); } - private static boolean contains(Topology previous, TableId searchTable) - { - for (Range range : previous.ranges()) - { - TableId table = ((TokenKey) range.start()).table(); - if (table.equals(searchTable)) - return true; - } - return false; - } - public synchronized void setCapacity(long bytes) { cacheSize = bytes; diff --git a/src/java/org/apache/cassandra/service/accord/AccordConfigurationService.java b/src/java/org/apache/cassandra/service/accord/AccordConfigurationService.java index 5dc0d19d0969..80e3296f5695 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordConfigurationService.java +++ b/src/java/org/apache/cassandra/service/accord/AccordConfigurationService.java @@ -113,7 +113,7 @@ AsyncResult acknowledged() @Nullable AsyncResult reads() { - return reads; + return ready == null ? null : ready.reads; } AsyncResult.Settable localSyncNotified() @@ -449,7 +449,7 @@ public void reportTopology(Topology topology, boolean isLoad, boolean startSync) } @Override - protected void localSyncComplete(Topology topology, boolean startSync) + protected void onReadyToCoordinate(Topology topology, boolean startSync) { long epoch = topology.epoch(); EpochState epochState = getOrCreateEpochState(epoch); diff --git a/src/java/org/apache/cassandra/service/accord/AccordDataStore.java b/src/java/org/apache/cassandra/service/accord/AccordDataStore.java index 0234184a383b..7934aebf769f 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordDataStore.java +++ b/src/java/org/apache/cassandra/service/accord/AccordDataStore.java @@ -30,6 +30,7 @@ import accord.primitives.Range; import accord.primitives.Ranges; import accord.primitives.SyncPoint; +import accord.utils.UnhandledEnum; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.memtable.Memtable; import org.apache.cassandra.schema.Schema; @@ -40,14 +41,6 @@ public class AccordDataStore implements DataStore private static final Logger logger = LoggerFactory.getLogger(AccordDataStore.class); enum FlushListenerKey { KEY } - @Override - public FetchResult fetch(Node node, SafeCommandStore safeStore, Ranges ranges, SyncPoint syncPoint, FetchRanges callback) - { - AccordFetchCoordinator coordinator = new AccordFetchCoordinator(node, ranges, syncPoint, callback, safeStore.commandStore()); - coordinator.start(); - return coordinator.result(); - } - /** * Ensures data for the intersecting ranges is flushed to sstable before calling back with reportOnSuccess. * This is used to gate journal cleanup, since we skip the CommitLog for applying to the data table. @@ -95,4 +88,23 @@ public void ensureDurable(CommandStore commandStore, Ranges ranges, RedundantBef prev = cfs; } } + + @Override + public FetchResult fetch(Node node, SafeCommandStore safeStore, Ranges ranges, SyncPoint syncPoint, FetchRanges callback, FetchKind kind) + { + switch (kind) + { + default: throw new UnhandledEnum(kind); + case Image: + { + AccordFetchCoordinator coordinator = new AccordFetchCoordinator(node, ranges, syncPoint, callback, safeStore.commandStore()); + coordinator.start(); + return coordinator.result(); + } + case Sync: + { + throw new UnsupportedOperationException(); + } + } + } } diff --git a/src/java/org/apache/cassandra/service/accord/AccordExecutor.java b/src/java/org/apache/cassandra/service/accord/AccordExecutor.java index 637b158177fd..2fa8675563c4 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordExecutor.java +++ b/src/java/org/apache/cassandra/service/accord/AccordExecutor.java @@ -1655,7 +1655,8 @@ public DebuggableTask debuggable() public static class TaskInfo implements Comparable { - public enum Status { WAITING_TO_LOAD, SCANNING_RANGES, LOADING, WAITING_TO_RUN, RUNNING } + // sorted in name order for reporting to virtual tables + public enum Status { LOADING, RUNNING, SCANNING_RANGES, WAITING_TO_LOAD, WAITING_TO_RUN } final Status status; final int commandStoreId; @@ -1706,7 +1707,7 @@ public int position() public int compareTo(TaskInfo that) { int c = this.status.compareTo(that.status); - if (c == 0) c = this.position() - that.position(); + if (c == 0) c = Integer.compare(this.position(), that.position()); return c; } } diff --git a/src/java/org/apache/cassandra/service/accord/AccordJournal.java b/src/java/org/apache/cassandra/service/accord/AccordJournal.java index 045acea8dfab..54f952212472 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordJournal.java +++ b/src/java/org/apache/cassandra/service/accord/AccordJournal.java @@ -376,7 +376,8 @@ public List replayTopologies() try (CloseableIterator iter = new CloseableIterator<>() { final CloseableIterator> iter = journalTable.keyIterator(topologyUpdateKey(0L), - topologyUpdateKey(Timestamp.MAX_EPOCH)); + topologyUpdateKey(Timestamp.MAX_EPOCH), + true); TopologyImage prev = null; @Override @@ -571,9 +572,14 @@ public void purge(CommandStores commandStores, EpochSupplier minEpoch) journalTable.forceCompaction(); } - public void forEach(Consumer consumer) + public void forEach(Consumer consumer, boolean includeActive) { - try (CloseableIterator> iter = journalTable.keyIterator(null, null)) + forEach(consumer, null, null, includeActive); + } + + public void forEach(Consumer consumer, @Nullable JournalKey min, @Nullable JournalKey max, boolean includeActive) + { + try (CloseableIterator> iter = journalTable.keyIterator(min, max, includeActive)) { while (iter.hasNext()) { @@ -585,7 +591,7 @@ public void forEach(Consumer consumer) @SuppressWarnings("unchecked") @Override - public void replay(CommandStores commandStores) + public boolean replay(CommandStores commandStores) { // TODO (expected): make the parallelisms configurable // Replay is performed in parallel, where at most X commands can be in flight, accross at most Y commands stores. @@ -610,7 +616,7 @@ public ReplayStream(CommandStore commandStore) this.commandStore = commandStore; this.replayer = commandStore.replayer(); // Keys in the index are sorted by command store id, so index iteration will be sequential - this.iter = journalTable.keyIterator(new JournalKey(TxnId.NONE, COMMAND_DIFF, commandStore.id()), new JournalKey(TxnId.MAX.withoutNonIdentityFlags(), COMMAND_DIFF, commandStore.id())); + this.iter = journalTable.keyIterator(new JournalKey(TxnId.NONE, COMMAND_DIFF, commandStore.id()), new JournalKey(TxnId.MAX.withoutNonIdentityFlags(), COMMAND_DIFF, commandStore.id()), false); } boolean replay() @@ -716,6 +722,7 @@ public void close() ++cur; } + return true; } catch (Throwable t) { diff --git a/src/java/org/apache/cassandra/service/accord/AccordJournalTable.java b/src/java/org/apache/cassandra/service/accord/AccordJournalTable.java index 984ab7ed6210..17aff49f86a0 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordJournalTable.java +++ b/src/java/org/apache/cassandra/service/accord/AccordJournalTable.java @@ -75,6 +75,7 @@ import org.apache.cassandra.journal.Journal; import org.apache.cassandra.journal.KeySupport; import org.apache.cassandra.journal.RecordConsumer; +import org.apache.cassandra.journal.Segment; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.service.RetryStrategy; import org.apache.cassandra.service.accord.AccordKeyspace.JournalColumns; @@ -457,9 +458,9 @@ private TableKeyIterator readAllFromTable(K key) } @SuppressWarnings("resource") // Auto-closeable iterator will release related resources - public CloseableIterator> keyIterator(@Nullable K min, @Nullable K max) + public CloseableIterator> keyIterator(@Nullable K min, @Nullable K max, boolean includeActive) { - return new JournalAndTableKeyIterator(min, max); + return new JournalAndTableKeyIterator(min, max, includeActive); } private class TableIterator extends AbstractIterator implements CloseableIterator @@ -515,12 +516,12 @@ public void close() private class JournalAndTableKeyIterator extends AbstractIterator> implements CloseableIterator> { final TableIterator tableIterator; - final Journal.StaticSegmentKeyIterator journalIterator; + final Journal.SegmentKeyIterator journalIterator; - private JournalAndTableKeyIterator(K min, K max) + private JournalAndTableKeyIterator(K min, K max, boolean includeActive) { this.tableIterator = new TableIterator(min, max); - this.journalIterator = journal.staticSegmentKeyIterator(min, max); + this.journalIterator = journal.segmentKeyIterator(min, max, includeActive ? ignore -> true : Segment::isStatic); } K prevFromTable = null; diff --git a/src/java/org/apache/cassandra/service/accord/AccordService.java b/src/java/org/apache/cassandra/service/accord/AccordService.java index 058b9171ac16..6742f2484369 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordService.java +++ b/src/java/org/apache/cassandra/service/accord/AccordService.java @@ -21,7 +21,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -39,6 +38,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.primitives.Ints; +import accord.api.ConfigurationService.EpochReady; +import accord.primitives.Txn; import org.apache.cassandra.metrics.AccordReplicaMetrics; import org.apache.cassandra.service.accord.api.AccordViolationHandler; import org.apache.cassandra.utils.Clock; @@ -57,17 +58,11 @@ import accord.impl.RequestCallbacks; import accord.impl.SizeOfIntersectionSorter; import accord.impl.progresslog.DefaultProgressLogs; -import accord.local.Command; -import accord.local.CommandStore; -import accord.local.CommandStores; import accord.local.Node; import accord.local.Node.Id; -import accord.local.PreLoadContext; -import accord.local.SafeCommand; import accord.local.ShardDistributor.EvenSplit; import accord.local.UniqueTimeService.AtomicUniqueTimeWithStaleReservation; import accord.local.cfk.CommandsForKey; -import accord.local.cfk.SafeCommandsForKey; import accord.local.durability.DurabilityService; import accord.local.durability.ShardDurability; import accord.messages.Reply; @@ -75,13 +70,9 @@ import accord.primitives.FullRoute; import accord.primitives.Keys; import accord.primitives.Ranges; -import accord.primitives.RoutingKeys; -import accord.primitives.SaveStatus; import accord.primitives.Seekable; import accord.primitives.Seekables; -import accord.primitives.Status; import accord.primitives.Timestamp; -import accord.primitives.Txn; import accord.primitives.TxnId; import accord.topology.Shard; import accord.topology.Topology; @@ -89,7 +80,6 @@ import accord.utils.DefaultRandom; import accord.utils.Invariants; import accord.utils.async.AsyncChain; -import accord.utils.async.AsyncChains; import accord.utils.async.AsyncResult; import accord.utils.async.AsyncResults; import org.apache.cassandra.concurrent.Shutdownable; @@ -114,7 +104,6 @@ import org.apache.cassandra.service.accord.api.AccordTimeService; import org.apache.cassandra.service.accord.api.AccordTopologySorter; import org.apache.cassandra.service.accord.api.CompositeTopologySorter; -import org.apache.cassandra.service.accord.api.TokenKey; import org.apache.cassandra.service.accord.api.TokenKey.KeyspaceSplitter; import org.apache.cassandra.service.accord.interop.AccordInteropAdapter.AccordInteropFactory; import org.apache.cassandra.service.accord.serializers.TableMetadatas; @@ -139,8 +128,6 @@ import static accord.api.Journal.TopologyUpdate; import static accord.api.ProtocolModifiers.Toggles.FastExec.MAY_BYPASS_SAFESTORE; -import static accord.local.LoadKeys.SYNC; -import static accord.local.LoadKeysFor.READ_WRITE; import static accord.local.durability.DurabilityService.SyncLocal.Self; import static accord.local.durability.DurabilityService.SyncRemote.All; import static accord.messages.SimpleReply.Ok; @@ -327,7 +314,7 @@ public synchronized static AccordService startup(NodeId tcmId) } @VisibleForTesting - public static void replayJournal(AccordService as) + public static boolean replayJournal(AccordService as) { logger.info("Starting journal replay."); long before = Clock.Global.nanoTime(); @@ -337,12 +324,12 @@ public static void replayJournal(AccordService as) if (as.journalConfiguration().replayMode() == RESET) AccordKeyspace.truncateCommandsForKey(); - as.node.commandStores().forEachCommandStore(cs -> cs.unsafeProgressLog().stop()); + as.node.commandStores().forAllUnsafe(cs -> cs.unsafeProgressLog().stop()); as.journal().replay(as.node().commandStores()); logger.info("Waiting for command stores to quiesce."); ((AccordCommandStores)as.node.commandStores()).waitForQuiescense(); as.journal.unsafeSetStarted(); - as.node.commandStores().forEachCommandStore(cs -> cs.unsafeProgressLog().start()); + as.node.commandStores().forAllUnsafe(cs -> cs.unsafeProgressLog().start()); } finally { @@ -351,14 +338,7 @@ public static void replayJournal(AccordService as) long after = Clock.Global.nanoTime(); logger.info("Finished journal replay. {}ms elapsed", NANOSECONDS.toMillis(after - before)); - } - - public static void shutdownServiceAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException - { - IAccordService i = instance; - if (i == null) - return; - i.shutdownAndWait(timeout, unit); + return true; } @Override @@ -565,7 +545,7 @@ public AsyncChain sync(Timestamp minBound, Keys keys, DurabilityService.Sy if (keys.size() != 1) return syncInternal(minBound, keys, syncLocal, syncRemote); - return KeyBarriers.find(node, minBound, keys.get(0).toUnseekable(), syncLocal, syncRemote) + return KeyBarriers.find(node, minBound, keys.get(0).toUnseekable(), syncLocal, syncRemote).chain() .flatMap(found -> KeyBarriers.await(node, node.someSequentialExecutor(), found, syncLocal, syncRemote)) .flatMap(success -> { if (success) @@ -799,8 +779,8 @@ class Ready extends AsyncResults.CountingResult implements Runnable } Ready ready = new Ready(); AccordCommandStores commandStores = (AccordCommandStores) node.commandStores(); - getBlocking(commandStores.forEach((PreLoadContext.Empty)() -> "Flush Caches", safeStore -> { - AccordCommandStore commandStore = (AccordCommandStore)safeStore.commandStore(); + commandStores.forAllUnsafe(unsafeStore -> { + AccordCommandStore commandStore = (AccordCommandStore)unsafeStore; try (AccordCommandStore.ExclusiveCaches caches = commandStore.lockCaches()) { caches.commandsForKeys().forEach(entry -> { @@ -811,7 +791,7 @@ class Ready extends AsyncResults.CountingResult implements Runnable } }); } - })); + }); ready.decrement(); AsyncPromise result = new AsyncPromise<>(); ready.invoke((success, fail) -> { @@ -881,139 +861,6 @@ public Id nodeId() return node.id(); } - @Override - public List debugTxnBlockedGraph(TxnId txnId) - { - return getBlocking(loadDebug(txnId)); - } - - public AsyncChain> loadDebug(TxnId original) - { - CommandStores commandStores = node.commandStores(); - if (commandStores.count() == 0) - return AsyncChains.success(Collections.emptyList()); - int[] ids = commandStores.ids(); - List> chains = new ArrayList<>(ids.length); - for (int id : ids) - chains.add(loadDebug(original, commandStores.forId(id)).chain()); - return AsyncChains.allOf(chains); - } - - private AsyncResult loadDebug(TxnId txnId, CommandStore store) - { - CommandStoreTxnBlockedGraph.Builder state = new CommandStoreTxnBlockedGraph.Builder(store.id()); - populateAsync(state, store, txnId); - return state; - } - - private static void populate(CommandStoreTxnBlockedGraph.Builder state, AccordSafeCommandStore safeStore, TxnId blockedBy) - { - if (safeStore.ifLoadedAndInitialised(blockedBy) != null) populateSync(state, safeStore, blockedBy); - else populateAsync(state, safeStore.commandStore(), blockedBy); - } - - private static void populateAsync(CommandStoreTxnBlockedGraph.Builder state, CommandStore store, TxnId txnId) - { - state.asyncTxns.incrementAndGet(); - store.execute(PreLoadContext.contextFor(txnId, "Populate txn_blocked_by"), in -> { - populateSync(state, (AccordSafeCommandStore) in, txnId); - if (0 == state.asyncTxns.decrementAndGet() && 0 == state.asyncKeys.get()) - state.complete(); - }); - } - - @Nullable - private static void populateSync(CommandStoreTxnBlockedGraph.Builder state, AccordSafeCommandStore safeStore, TxnId txnId) - { - try - { - if (state.txns.containsKey(txnId)) - return; // could plausibly request same txn twice - - SafeCommand safeCommand = safeStore.unsafeGet(txnId); - Invariants.nonNull(safeCommand, "Txn %s is not in the cache", txnId); - if (safeCommand.current() == null || safeCommand.current().saveStatus() == SaveStatus.Uninitialised) - return; - - CommandStoreTxnBlockedGraph.TxnState cmdTxnState = populateSync(state, safeCommand.current()); - if (cmdTxnState.notBlocked()) - return; - - for (TxnId blockedBy : cmdTxnState.blockedBy) - { - if (!state.knows(blockedBy)) - populate(state, safeStore, blockedBy); - } - for (TokenKey blockedBy : cmdTxnState.blockedByKey) - { - if (!state.keys.containsKey(blockedBy)) - populate(state, safeStore, blockedBy, txnId, safeCommand.current().executeAt()); - } - } - catch (Throwable t) - { - state.tryFailure(t); - } - } - - private static void populate(CommandStoreTxnBlockedGraph.Builder state, AccordSafeCommandStore safeStore, TokenKey blockedBy, TxnId txnId, Timestamp executeAt) - { - if (safeStore.ifLoadedAndInitialised(txnId) != null && safeStore.ifLoadedAndInitialised(blockedBy) != null) populateSync(state, safeStore, blockedBy, txnId, executeAt); - else populateAsync(state, safeStore.commandStore(), blockedBy, txnId, executeAt); - } - - private static void populateAsync(CommandStoreTxnBlockedGraph.Builder state, CommandStore commandStore, TokenKey blockedBy, TxnId txnId, Timestamp executeAt) - { - state.asyncKeys.incrementAndGet(); - commandStore.execute(PreLoadContext.contextFor(txnId, RoutingKeys.of(blockedBy.toUnseekable()), SYNC, READ_WRITE, "Populate txn_blocked_by"), in -> { - populateSync(state, (AccordSafeCommandStore) in, blockedBy, txnId, executeAt); - if (0 == state.asyncKeys.decrementAndGet() && 0 == state.asyncTxns.get()) - state.complete(); - }); - } - - private static void populateSync(CommandStoreTxnBlockedGraph.Builder state, AccordSafeCommandStore safeStore, TokenKey pk, TxnId txnId, Timestamp executeAt) - { - try - { - SafeCommandsForKey commandsForKey = safeStore.ifLoadedAndInitialised(pk); - TxnId blocking = commandsForKey.current().blockedOnTxnId(txnId, executeAt); - if (blocking instanceof CommandsForKey.TxnInfo) - blocking = ((CommandsForKey.TxnInfo) blocking).plainTxnId(); - state.keys.put(pk, blocking); - if (state.txns.containsKey(blocking)) - return; - populate(state, safeStore, blocking); - } - catch (Throwable t) - { - state.tryFailure(t); - } - } - - private static CommandStoreTxnBlockedGraph.TxnState populateSync(CommandStoreTxnBlockedGraph.Builder state, Command cmd) - { - CommandStoreTxnBlockedGraph.Builder.TxnBuilder cmdTxnState = state.txn(cmd.txnId(), cmd.executeAt(), cmd.saveStatus()); - if (!cmd.hasBeen(Status.Applied) && cmd.hasBeen(Status.Stable)) - { - // check blocking state - Command.WaitingOn waitingOn = cmd.asCommitted().waitingOn(); - waitingOn.waitingOn.reverseForEach(null, null, null, null, (i1, i2, i3, i4, i) -> { - if (i < waitingOn.txnIdCount()) - { - // blocked on txn - cmdTxnState.blockedBy.add(waitingOn.txnId(i)); - } - else - { - // blocked on key - cmdTxnState.blockedByKey.add((TokenKey) waitingOn.keys.get(i - waitingOn.txnIdCount())); - } - }); - } - return cmdTxnState.build(); - } - @Override public long minEpoch() { @@ -1037,18 +884,18 @@ public AccordJournal journal() } @Override - public Future epochReady(Epoch epoch) + public Future epochReady(Epoch epoch, Function> get) { - return toFuture(configService.epochReady(epoch.getEpoch())); + return toFuture(configService.epochReady(epoch.getEpoch(), get)); } @Override - public Future epochReadyFor(ClusterMetadata metadata) + public Future epochReadyFor(ClusterMetadata metadata, Function> get) { if (!metadata.schema.hasAccordKeyspaces()) return EPOCH_READY; - return epochReady(metadata.epoch); + return epochReady(metadata.epoch, get); } @Override @@ -1116,7 +963,7 @@ public AccordConfigurationService configService() public AccordCompactionInfos getCompactionInfo() { AccordCompactionInfos compactionInfos = new AccordCompactionInfos(node.durableBefore(), node.topology().minEpoch()); - node.commandStores().forEachCommandStore(commandStore -> { + node.commandStores().forAllUnsafe(commandStore -> { compactionInfos.put(commandStore.id(), ((AccordCommandStore)commandStore).getCompactionInfo()); }); return compactionInfos; diff --git a/src/java/org/apache/cassandra/service/accord/AccordTopology.java b/src/java/org/apache/cassandra/service/accord/AccordTopology.java index ca817e647d57..8c60ecae5e3a 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordTopology.java +++ b/src/java/org/apache/cassandra/service/accord/AccordTopology.java @@ -35,6 +35,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Sets; +import accord.api.ConfigurationService.EpochReady; import accord.local.Node; import accord.local.Node.Id; import accord.primitives.Ranges; @@ -399,7 +400,7 @@ public static void awaitTopologyReadiness(Keyspaces.KeyspacesDiff keyspacesDiff, { ClusterMetadataService.instance().fetchLogFromCMS(epoch); IAccordService service = AccordService.instance(); - service.epochReady(epoch).get(service.agent().expireEpochWait(MILLISECONDS), MILLISECONDS); + service.epochReady(epoch, EpochReady::reads).get(service.agent().expireEpochWait(MILLISECONDS), MILLISECONDS); } catch (InterruptedException e) { diff --git a/src/java/org/apache/cassandra/service/accord/CommandStoreTxnBlockedGraph.java b/src/java/org/apache/cassandra/service/accord/CommandStoreTxnBlockedGraph.java deleted file mode 100644 index 7d553dda7c59..000000000000 --- a/src/java/org/apache/cassandra/service/accord/CommandStoreTxnBlockedGraph.java +++ /dev/null @@ -1,135 +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.accord; - -import java.util.ArrayList; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; - -import accord.primitives.SaveStatus; -import accord.primitives.Timestamp; -import accord.primitives.TxnId; -import accord.utils.async.AsyncResults; -import org.apache.cassandra.service.accord.api.TokenKey; - -public class CommandStoreTxnBlockedGraph -{ - public final int commandStoreId; - public final Map txns; - public final Map keys; - - public CommandStoreTxnBlockedGraph(Builder builder) - { - commandStoreId = builder.storeId; - txns = ImmutableMap.copyOf(builder.txns); - keys = ImmutableMap.copyOf(builder.keys); - } - - public static class TxnState - { - public final TxnId txnId; - public final Timestamp executeAt; - public final SaveStatus saveStatus; - public final List blockedBy; - public final Set blockedByKey; - - public TxnState(Builder.TxnBuilder builder) - { - txnId = builder.txnId; - executeAt = builder.executeAt; - saveStatus = builder.saveStatus; - blockedBy = ImmutableList.copyOf(builder.blockedBy); - blockedByKey = ImmutableSet.copyOf(builder.blockedByKey); - } - - public boolean isBlocked() - { - return !notBlocked(); - } - - public boolean notBlocked() - { - return blockedBy.isEmpty() && blockedByKey.isEmpty(); - } - } - - public static class Builder extends AsyncResults.SettableResult - { - final AtomicInteger asyncTxns = new AtomicInteger(), asyncKeys = new AtomicInteger(); - final int storeId; - final Map txns = new LinkedHashMap<>(); - final Map keys = new LinkedHashMap<>(); - - public Builder(int storeId) - { - this.storeId = storeId; - } - - boolean knows(TxnId id) - { - return txns.containsKey(id); - } - - public void complete() - { - trySuccess(build()); - } - - public CommandStoreTxnBlockedGraph build() - { - return new CommandStoreTxnBlockedGraph(this); - } - - public TxnBuilder txn(TxnId txnId, Timestamp executeAt, SaveStatus saveStatus) - { - return new TxnBuilder(txnId, executeAt, saveStatus); - } - - public class TxnBuilder - { - final TxnId txnId; - final Timestamp executeAt; - final SaveStatus saveStatus; - List blockedBy = new ArrayList<>(); - Set blockedByKey = new LinkedHashSet<>(); - - public TxnBuilder(TxnId txnId, Timestamp executeAt, SaveStatus saveStatus) - { - this.txnId = txnId; - this.executeAt = executeAt; - this.saveStatus = saveStatus; - } - - public TxnState build() - { - TxnState state = new TxnState(this); - txns.put(txnId, state); - return state; - } - } - } -} diff --git a/src/java/org/apache/cassandra/service/accord/DebugBlockedTxns.java b/src/java/org/apache/cassandra/service/accord/DebugBlockedTxns.java new file mode 100644 index 000000000000..a4ca0af5e1dc --- /dev/null +++ b/src/java/org/apache/cassandra/service/accord/DebugBlockedTxns.java @@ -0,0 +1,249 @@ +/* + * 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.accord; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.TimeoutException; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +import javax.annotation.Nullable; + +import accord.api.RoutingKey; +import accord.local.Command; +import accord.local.CommandStore; +import accord.local.CommandStores; +import accord.local.PreLoadContext; +import accord.local.SafeCommandStore; +import accord.local.cfk.CommandsForKey; +import accord.local.cfk.SafeCommandsForKey; +import accord.primitives.RoutingKeys; +import accord.primitives.SaveStatus; +import accord.primitives.Status; +import accord.primitives.Timestamp; +import accord.primitives.TxnId; +import accord.utils.async.AsyncChain; +import accord.utils.async.AsyncChains; +import org.apache.cassandra.service.accord.api.TokenKey; +import org.apache.cassandra.utils.concurrent.Future; + +import static accord.local.LoadKeys.SYNC; +import static accord.local.LoadKeysFor.READ_WRITE; +import static java.util.Collections.emptyList; + +public class DebugBlockedTxns +{ + public static class Txn implements Comparable + { + public final int commandStoreId; + public final int depth; + public final TxnId txnId; + public final Timestamp executeAt; + public final SaveStatus saveStatus; + public final RoutingKey blockedViaKey; + public final List blockedBy; + public final List blockedByKey; + + public Txn(int commandStoreId, int depth, TxnId txnId, Timestamp executeAt, SaveStatus saveStatus, RoutingKey blockedViaKey, List blockedBy, List blockedByKey) + { + this.commandStoreId = commandStoreId; + this.depth = depth; + this.txnId = txnId; + this.executeAt = executeAt; + this.saveStatus = saveStatus; + this.blockedViaKey = blockedViaKey; + this.blockedBy = blockedBy; + this.blockedByKey = blockedByKey; + } + + public boolean isBlocked() + { + return !notBlocked(); + } + + public boolean notBlocked() + { + return blockedBy.isEmpty() && blockedByKey.isEmpty(); + } + + @Override + public int compareTo(Txn that) + { + int c = Integer.compare(this.commandStoreId, that.commandStoreId); + if (c == 0) c = Integer.compare(this.depth, that.depth); + if (c == 0) c = this.txnId.compareTo(that.txnId); + return c; + } + } + + final IAccordService service; + final Consumer visit; + final TxnId root; + final int maxDepth; + final Set visited = Collections.newSetFromMap(new ConcurrentHashMap<>()); + final ConcurrentLinkedQueue> queuedKeys = new ConcurrentLinkedQueue<>(); + final ConcurrentLinkedQueue> queuedTxn = new ConcurrentLinkedQueue<>(); + + public DebugBlockedTxns(IAccordService service, TxnId root, int maxDepth, Consumer visit) + { + this.service = service; + this.visit = visit; + this.root = root; + this.maxDepth = maxDepth; + } + + public static void visit(IAccordService accord, TxnId txnId, int maxDepth, long deadlineNanos, Consumer visit) throws TimeoutException + { + new DebugBlockedTxns(accord, txnId, maxDepth, visit).visit(deadlineNanos); + } + + private void visit(long deadlineNanos) throws TimeoutException + { + CommandStores commandStores = service.node().commandStores(); + if (commandStores.count() == 0) + return; + + int[] ids = commandStores.ids(); + List> chains = new ArrayList<>(ids.length); + for (int id : ids) + chains.add(visitRootTxnAsync(commandStores.forId(id), root)); + + List tmp = new ArrayList<>(); + Future> next = AccordService.toFuture(AsyncChains.allOf(chains)); + while (next != null) + { + if (!next.awaitUntilThrowUncheckedOnInterrupt(deadlineNanos)) + throw new TimeoutException(); + + next.rethrowIfFailed(); + List process = next.getNow().stream() + .filter(Objects::nonNull) + .sorted(Comparator.naturalOrder()) + .collect(Collectors.toList()); + + for (Txn txn : process) + visit.accept(txn); + + Future> awaitKeys = drainToFuture(queuedKeys, (List>)(List)tmp); + if (awaitKeys != null && !awaitKeys.awaitUntilThrowUncheckedOnInterrupt(deadlineNanos)) + throw new TimeoutException(); + + next = drainToFuture(queuedTxn, (List>)(List)tmp); + } + } + + private Future> drainToFuture(Queue> drain, List> tmp) + { + AsyncChain next; + while (null != (next = drain.poll())) + tmp.add(next); + if (tmp.isEmpty()) + return null; + Future> result = AccordService.toFuture(AsyncChains.allOf(List.copyOf(tmp))); + tmp.clear(); + return result; + } + + private AsyncChain visitRootTxnAsync(CommandStore commandStore, TxnId txnId) + { + return commandStore.chain(PreLoadContext.contextFor(txnId, "Populate txn_blocked_by"), safeStore -> { + Command command = safeStore.unsafeGetNoCleanup(txnId).current(); + if (command == null || command.saveStatus() == SaveStatus.Uninitialised) + return null; + return visitTxnSync(safeStore, command, command.executeAt(), null, 0); + }); + } + + private AsyncChain visitTxnAsync(CommandStore commandStore, TxnId txnId, Timestamp rootExecuteAt, @Nullable TokenKey byKey, int depth, boolean recurse) + { + return commandStore.chain(PreLoadContext.contextFor(txnId, "Populate txn_blocked_by"), safeStore -> { + Command command = safeStore.unsafeGetNoCleanup(txnId).current(); + if (command == null || command.saveStatus() == SaveStatus.Uninitialised) + return null; + return visitTxnSync(safeStore, command, rootExecuteAt, byKey, depth); + }); + } + + private Txn visitTxnSync(SafeCommandStore safeStore, Command command, Timestamp rootExecuteAt, @Nullable TokenKey byKey, int depth) + { + List waitingOnTxnId = new ArrayList<>(); + List waitingOnKey = new ArrayList<>(); + if (!command.hasBeen(Status.Applied) && command.hasBeen(Status.Stable)) + { + // check blocking state + Command.WaitingOn waitingOn = command.asCommitted().waitingOn(); + waitingOn.waitingOn.reverseForEach(null, null, null, null, (i1, i2, i3, i4, i) -> { + if (i < waitingOn.txnIdCount()) waitingOnTxnId.add(waitingOn.txnId(i)); + else waitingOnKey.add((TokenKey) waitingOn.keys.get(i - waitingOn.txnIdCount())); + }); + } + + CommandStore commandStore = safeStore.commandStore(); + if (depth < maxDepth) + { + for (TxnId waitingOn : waitingOnTxnId) + { + if (visited.add(waitingOn)) + queuedTxn.add(visitTxnAsync(commandStore, waitingOn, rootExecuteAt, null, depth + 1, true)); + } + for (TokenKey key : waitingOnKey) + { + if (visited.add(key)) + queuedKeys.add(visitKeysAsync(commandStore, key, rootExecuteAt, depth + 1)); + } + } + + return new Txn(commandStore.id(), depth, command.txnId(), command.executeAt(), command.saveStatus(), byKey, waitingOnTxnId, waitingOnKey); + } + + + private AsyncChain visitKeysAsync(CommandStore commandStore, TokenKey key, Timestamp rootExecuteAt, int depth) + { + return commandStore.chain(PreLoadContext.contextFor(RoutingKeys.of(key.toUnseekable()), SYNC, READ_WRITE, "Populate txn_blocked_by"), safeStore -> { + visitKeysSync(safeStore, key, rootExecuteAt, depth); + }); + } + + private void visitKeysSync(SafeCommandStore safeStore, TokenKey key, Timestamp rootExecuteAt, int depth) + { + SafeCommandsForKey commandsForKey = safeStore.ifLoadedAndInitialised(key); + TxnId blocking = commandsForKey.current().blockedOnTxnId(root, rootExecuteAt); + CommandStore commandStore = safeStore.commandStore(); + if (blocking == null) + { + queuedTxn.add(AsyncChains.success(new Txn(commandStore.id(), depth, null, null, null, key, emptyList(), emptyList()))); + } + else + { + // TODO (required): this type check should not be needed; release accord version that fixes it at origin + if (blocking instanceof CommandsForKey.TxnInfo) + blocking = ((CommandsForKey.TxnInfo) blocking).plainTxnId(); + boolean recurse = visited.add(blocking); + queuedTxn.add(visitTxnAsync(commandStore, blocking, rootExecuteAt, key, depth, recurse)); + } + } +} diff --git a/src/java/org/apache/cassandra/service/accord/IAccordService.java b/src/java/org/apache/cassandra/service/accord/IAccordService.java index 34453d22c30b..5915ad281daa 100644 --- a/src/java/org/apache/cassandra/service/accord/IAccordService.java +++ b/src/java/org/apache/cassandra/service/accord/IAccordService.java @@ -19,15 +19,16 @@ package org.apache.cassandra.service.accord; import java.util.Collection; -import java.util.Collections; import java.util.EnumSet; import java.util.List; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.function.BiConsumer; +import java.util.function.Function; import javax.annotation.Nonnull; import javax.annotation.Nullable; +import accord.api.ConfigurationService.EpochReady; import accord.utils.async.AsyncResult; import org.apache.cassandra.tcm.ClusterMetadata; import org.slf4j.Logger; @@ -127,9 +128,8 @@ interface IAccordResult * Return a future that will complete once the accord has completed it's local bootstrap process * for any ranges gained in the given epoch */ - Future epochReady(Epoch epoch); - - Future epochReadyFor(ClusterMetadata epoch); + Future epochReady(Epoch epoch, Function> f); + Future epochReadyFor(ClusterMetadata epoch, Function> f); void receive(Message message); @@ -177,8 +177,6 @@ public AccordCompactionInfos(DurableBefore durableBefore, long minEpoch, AccordC Id nodeId(); - List debugTxnBlockedGraph(TxnId txnId); - long minEpoch(); void awaitDone(TableId id, long epoch); @@ -308,13 +306,13 @@ public AccordScheduler scheduler() } @Override - public Future epochReady(Epoch epoch) + public Future epochReady(Epoch epoch, Function> get) { return BOOTSTRAP_SUCCESS; } @Override - public Future epochReadyFor(ClusterMetadata epoch) + public Future epochReadyFor(ClusterMetadata epoch, Function> get) { return BOOTSTRAP_SUCCESS; } @@ -340,12 +338,6 @@ public Id nodeId() throw new UnsupportedOperationException(); } - @Override - public List debugTxnBlockedGraph(TxnId txnId) - { - return Collections.emptyList(); - } - @Override public long minEpoch() { @@ -515,15 +507,15 @@ public AccordScheduler scheduler() } @Override - public Future epochReady(Epoch epoch) + public Future epochReady(Epoch epoch, Function> get) { - return delegate.epochReady(epoch); + return delegate.epochReady(epoch, get); } @Override - public Future epochReadyFor(ClusterMetadata epoch) + public Future epochReadyFor(ClusterMetadata epoch, Function> get) { - return delegate.epochReadyFor(epoch); + return delegate.epochReadyFor(epoch, get); } @Override @@ -550,12 +542,6 @@ public Id nodeId() return delegate.nodeId(); } - @Override - public List debugTxnBlockedGraph(TxnId txnId) - { - return delegate.debugTxnBlockedGraph(txnId); - } - @Override public long minEpoch() { diff --git a/src/java/org/apache/cassandra/service/accord/api/AccordAgent.java b/src/java/org/apache/cassandra/service/accord/api/AccordAgent.java index be7153fc2c0c..e09a565c4130 100644 --- a/src/java/org/apache/cassandra/service/accord/api/AccordAgent.java +++ b/src/java/org/apache/cassandra/service/accord/api/AccordAgent.java @@ -30,6 +30,7 @@ import accord.api.Agent; import accord.api.CoordinatorEventListener; +import accord.api.OwnershipEventListener; import accord.api.ReplicaEventListener; import accord.api.ProgressLog.BlockedUntil; import accord.api.RoutingKey; @@ -93,7 +94,7 @@ import static org.apache.cassandra.utils.Clock.Global.nanoTime; // TODO (expected): merge with AccordService -public class AccordAgent implements Agent +public class AccordAgent implements Agent, OwnershipEventListener { private static final Logger logger = LoggerFactory.getLogger(AccordAgent.class); private static final NoSpamLogger noSpamLogger = NoSpamLogger.getLogger(logger, 1L, MINUTES); @@ -125,6 +126,12 @@ public AccordTracing tracing() return tracing.trace(txnId, eventType); } + @Override + public OwnershipEventListener ownershipEvents() + { + return this; + } + public void setNodeId(Node.Id id) { self = id; diff --git a/src/java/org/apache/cassandra/service/accord/api/AccordViolationHandler.java b/src/java/org/apache/cassandra/service/accord/api/AccordViolationHandler.java index 67ec0111a8fe..c40278932942 100644 --- a/src/java/org/apache/cassandra/service/accord/api/AccordViolationHandler.java +++ b/src/java/org/apache/cassandra/service/accord/api/AccordViolationHandler.java @@ -18,6 +18,8 @@ package org.apache.cassandra.service.accord.api; +import javax.annotation.Nullable; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,8 +31,6 @@ import accord.primitives.Timestamp; import accord.primitives.TxnId; -import static accord.utils.Invariants.illegalState; - public class AccordViolationHandler implements ViolationHandler { private static final Logger logger = LoggerFactory.getLogger(AccordViolationHandler.class); @@ -40,13 +40,11 @@ public static void setup() ViolationHandlerHolder.set(AccordViolationHandler::new); } - @Override - public void onTimestampViolation(SafeCommandStore safeStore, Command command, Participants otherParticipants, Route otherRoute, Timestamp otherExecuteAt) + public void onTimestampViolation(@Nullable SafeCommandStore safeStore, Command command, Participants otherParticipants, @Nullable Route otherRoute, Timestamp otherExecuteAt) { - throw illegalState(ViolationHandler.timestampViolationMessage(safeStore, command, otherParticipants, otherRoute, otherExecuteAt)); + logger.error(ViolationHandler.timestampViolationMessage(safeStore, command, otherParticipants, otherRoute, otherExecuteAt)); } - @Override public void onDependencyViolation(Participants participants, TxnId notWitnessed, Timestamp notWitnessedExecuteAt, TxnId by, Timestamp byExecuteAt) { logger.error(ViolationHandler.dependencyViolationMessage(participants, notWitnessed, notWitnessedExecuteAt, by, byExecuteAt)); diff --git a/src/java/org/apache/cassandra/service/accord/interop/AccordInteropStableThenRead.java b/src/java/org/apache/cassandra/service/accord/interop/AccordInteropStableThenRead.java index 5dcbb4552fc7..468f5ff41373 100644 --- a/src/java/org/apache/cassandra/service/accord/interop/AccordInteropStableThenRead.java +++ b/src/java/org/apache/cassandra/service/accord/interop/AccordInteropStableThenRead.java @@ -144,13 +144,13 @@ public AccordInteropStableThenRead(TxnId txnId, Participants scope, Commit.Ki } @Override - public CommitOrReadNack apply(SafeCommandStore safeStore) + public CommitOrReadNack applyInternal(SafeCommandStore safeStore) { Route route = this.route == null ? (Route)scope : this.route; StoreParticipants participants = StoreParticipants.execute(safeStore, route, txnId, minEpoch(), executeAtEpoch); SafeCommand safeCommand = safeStore.get(txnId, participants); Commands.commit(safeStore, safeCommand, participants, kind.saveStatus, Ballot.ZERO, txnId, route, partialTxn, executeAt, partialDeps, kind); - return super.apply(safeStore, safeCommand, participants); + return super.applyInternal(safeStore, safeCommand, participants); } @Override diff --git a/src/java/org/apache/cassandra/service/accord/serializers/AcceptSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/AcceptSerializers.java index cd967961424d..53391cd322c5 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/AcceptSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/AcceptSerializers.java @@ -88,7 +88,7 @@ public void serialize(Accept.NotAccept invalidate, DataOutputPlus out) throws IO CommandSerializers.status.serialize(invalidate.status, out); CommandSerializers.ballot.serialize(invalidate.ballot, out); CommandSerializers.txnId.serialize(invalidate.txnId, out); - KeySerializers.participants.serialize(invalidate.participants, out); + KeySerializers.participants.serialize(invalidate.scope, out); } @Override @@ -106,7 +106,7 @@ public long serializedSize(Accept.NotAccept invalidate) return CommandSerializers.status.serializedSize(invalidate.status) + CommandSerializers.ballot.serializedSize(invalidate.ballot) + CommandSerializers.txnId.serializedSize(invalidate.txnId) - + KeySerializers.participants.serializedSize(invalidate.participants); + + KeySerializers.participants.serializedSize(invalidate.scope); } }; diff --git a/src/java/org/apache/cassandra/service/accord/serializers/AwaitSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/AwaitSerializers.java index af67c05b0482..0e017184fb53 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/AwaitSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/AwaitSerializers.java @@ -122,7 +122,7 @@ public long serializedSize(A await) public void serialize(AsyncAwaitComplete ok, DataOutputPlus out) throws IOException { CommandSerializers.txnId.serialize(ok.txnId, out); - KeySerializers.route.serialize(ok.route, out); + KeySerializers.route.serialize(ok.scope, out); out.writeByte(ok.newStatus.ordinal()); out.writeUnsignedVInt32(ok.callbackId); } @@ -141,7 +141,7 @@ public AsyncAwaitComplete deserialize(DataInputPlus in) throws IOException public long serializedSize(AsyncAwaitComplete ok) { return CommandSerializers.txnId.serializedSize(ok.txnId) - + KeySerializers.route.serializedSize(ok.route) + + KeySerializers.route.serializedSize(ok.scope) + TypeSizes.BYTE_SIZE + VIntCoding.computeVIntSize(ok.callbackId); } diff --git a/src/java/org/apache/cassandra/service/accord/serializers/BeginInvalidationSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/BeginInvalidationSerializers.java index 25a40a6a418c..231c93287836 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/BeginInvalidationSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/BeginInvalidationSerializers.java @@ -39,7 +39,7 @@ public class BeginInvalidationSerializers public void serialize(BeginInvalidation begin, DataOutputPlus out) throws IOException { CommandSerializers.txnId.serialize(begin.txnId, out); - KeySerializers.participants.serialize(begin.participants, out); + KeySerializers.participants.serialize(begin.scope, out); CommandSerializers.ballot.serialize(begin.ballot, out); } @@ -55,7 +55,7 @@ public BeginInvalidation deserialize(DataInputPlus in) throws IOException public long serializedSize(BeginInvalidation begin) { return CommandSerializers.txnId.serializedSize(begin.txnId) - + KeySerializers.participants.serializedSize(begin.participants) + + KeySerializers.participants.serializedSize(begin.scope) + CommandSerializers.ballot.serializedSize(begin.ballot); } }; diff --git a/src/java/org/apache/cassandra/service/accord/serializers/CheckStatusSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/CheckStatusSerializers.java index d1f81512b975..9747d9f72f4c 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/CheckStatusSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/CheckStatusSerializers.java @@ -128,8 +128,8 @@ public long serializedSize(KnownMap knownMap) public void serialize(CheckStatus check, DataOutputPlus out) throws IOException { CommandSerializers.txnId.serialize(check.txnId, out); - KeySerializers.participants.serialize(check.query, out); - out.writeUnsignedVInt(check.sourceEpoch); + KeySerializers.participants.serialize(check.scope, out); + out.writeUnsignedVInt(check.waitForEpoch); out.writeByte(check.includeInfo.ordinal()); CommandSerializers.ballot.serialize(check.bumpBallot, out); } @@ -149,8 +149,8 @@ public CheckStatus deserialize(DataInputPlus in) throws IOException public long serializedSize(CheckStatus check) { return CommandSerializers.txnId.serializedSize(check.txnId) - + KeySerializers.participants.serializedSize(check.query) - + TypeSizes.sizeofUnsignedVInt(check.sourceEpoch) + + KeySerializers.participants.serializedSize(check.scope) + + TypeSizes.sizeofUnsignedVInt(check.waitForEpoch) + TypeSizes.BYTE_SIZE + CommandSerializers.ballot.serializedSize(check.bumpBallot); } diff --git a/src/java/org/apache/cassandra/service/accord/serializers/CommandStoreSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/CommandStoreSerializers.java index a1cb244b13c6..a30ea5927056 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/CommandStoreSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/CommandStoreSerializers.java @@ -155,11 +155,18 @@ public void serialize(RedundantBefore.Bounds b, DataOutputPlus out) throws IOExc } for (int i = 0 ; i < b.bounds.length ; ++i) { - out.writeShort(b.status(i * 2)); - out.writeShort(b.status(i * 2 + 1)); + out.writeShort(cast(b.status(i * 2))); + out.writeShort(cast(b.status(i * 2 + 1))); } } + private short cast(long v) + { + if ((v & ~0xFFFF) != 0) + throw new IllegalStateException("Cannot serialize RedundantStatus larger than 0xFFFF. Requires serialization version bump."); + return (short)v; + } + @Override public RedundantBefore.Bounds deserialize(DataInputPlus in) throws IOException { @@ -174,7 +181,7 @@ public RedundantBefore.Bounds deserialize(DataInputPlus in) throws IOException TxnId[] bounds = new TxnId[count]; for (int i = 0 ; i < bounds.length ; ++i) bounds[i] = CommandSerializers.txnId.deserialize(in); - short[] statuses = new short[count * 2]; + int[] statuses = new int[count * 2]; for (int i = 0 ; i < statuses.length ; ++i) statuses[i] = in.readShort(); diff --git a/src/java/org/apache/cassandra/service/accord/serializers/CommitSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/CommitSerializers.java index 31d879ce7c87..2d2592eee820 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/CommitSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/CommitSerializers.java @@ -52,9 +52,9 @@ public void serializeBody(Commit msg, DataOutputPlus out, Version version) throw kind.serialize(msg.kind, out); CommandSerializers.ballot.serialize(msg.ballot, out); ExecuteAtSerializer.serialize(msg.txnId, msg.executeAt, out); - CommandSerializers.nullablePartialTxn.serialize(msg.partialTxn, out, version); + CommandSerializers.nullablePartialTxn.serialize(msg.partialTxn(), out, version); if (msg.kind.withDeps == Commit.WithDeps.HasDeps) - DepsSerializers.partialDeps.serialize(msg.partialDeps, out); + DepsSerializers.partialDeps.serialize(msg.partialDeps(), out); serializeNullable(msg.route, out, KeySerializers.fullRoute); } @@ -78,10 +78,10 @@ public long serializedBodySize(Commit msg, Version version) long size = kind.serializedSize(msg.kind) + CommandSerializers.ballot.serializedSize(msg.ballot) + ExecuteAtSerializer.serializedSize(msg.txnId, msg.executeAt) - + CommandSerializers.nullablePartialTxn.serializedSize(msg.partialTxn, version); + + CommandSerializers.nullablePartialTxn.serializedSize(msg.partialTxn(), version); if (msg.kind.withDeps == Commit.WithDeps.HasDeps) - size += DepsSerializers.partialDeps.serializedSize(msg.partialDeps); + size += DepsSerializers.partialDeps.serializedSize(msg.partialDeps()); size += serializedNullableSize(msg.route, KeySerializers.fullRoute); return size; diff --git a/src/java/org/apache/cassandra/service/accord/serializers/RecoverySerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/RecoverySerializers.java index 3eb2fa498454..b3f81bec281c 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/RecoverySerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/RecoverySerializers.java @@ -56,7 +56,7 @@ public class RecoverySerializers static final int HAS_EXECUTE_AT_EPOCH = 0x2; static final int IS_FAST_PATH_DECIDED = 0x4; static final int SIZE_OF_FLAGS = VIntCoding.computeUnsignedVIntSize(HAS_ROUTE | HAS_EXECUTE_AT_EPOCH | IS_FAST_PATH_DECIDED); - public static final IVersionedSerializer request = new WithUnsyncedSerializer() + public static final IVersionedSerializer request = new WithUnsyncedSerializer<>() { @Override public void serializeBody(BeginRecovery recover, DataOutputPlus out, Version version) throws IOException diff --git a/src/java/org/apache/cassandra/service/accord/serializers/TxnRequestSerializer.java b/src/java/org/apache/cassandra/service/accord/serializers/TxnRequestSerializer.java index fe2cbe26136b..37e5efdb8f13 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/TxnRequestSerializer.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/TxnRequestSerializer.java @@ -20,14 +20,14 @@ import java.io.IOException; -import accord.messages.TxnRequest; +import accord.messages.RouteRequest; import accord.primitives.Route; import accord.primitives.TxnId; import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -public abstract class TxnRequestSerializer> implements IVersionedSerializer +public abstract class TxnRequestSerializer> implements IVersionedSerializer { void serializeHeader(T msg, DataOutputPlus out, Version version) throws IOException { @@ -72,7 +72,7 @@ public final long serializedSize(T msg, Version version) return serializedHeaderSize(msg, version) + serializedBodySize(msg, version); } - public static abstract class WithUnsyncedSerializer> extends TxnRequestSerializer + public static abstract class WithUnsyncedSerializer> extends TxnRequestSerializer { @Override void serializeHeader(T msg, DataOutputPlus out, Version version) throws IOException diff --git a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java index 341eff845bcc..a40a42dded9c 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java +++ b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java @@ -28,6 +28,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import accord.api.ConfigurationService.EpochReady; import com.googlecode.concurrenttrees.common.Iterables; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.db.ColumnFamilyStore; @@ -362,7 +363,7 @@ public static boolean bootstrap(final Collection tokens, StorageService.instance.repairPaxosForTopologyChange("bootstrap"); Future bootstrapStream = StorageService.instance.startBootstrap(metadata, beingReplaced, movements, strictMovements); - Future accordReady = AccordService.instance().epochReadyFor(metadata); + Future accordReady = AccordService.instance().epochReadyFor(metadata, EpochReady::reads); Future ready = FutureCombiner.allOf(bootstrapStream, accordReady); try diff --git a/src/java/org/apache/cassandra/tcm/sequences/DropAccordTable.java b/src/java/org/apache/cassandra/tcm/sequences/DropAccordTable.java index 68fc2471ac59..8a121fd2c2a2 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/DropAccordTable.java +++ b/src/java/org/apache/cassandra/tcm/sequences/DropAccordTable.java @@ -26,6 +26,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import accord.api.ConfigurationService.EpochReady; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.schema.Keyspaces; @@ -168,7 +169,7 @@ private SequenceState awaitSafeFromAccord() throws ExecutionException, Interrupt return error(new IllegalStateException(String.format("Table %s is in an invalid state to be dropped", table))); long startNanos = nanoTime(); - AccordService.instance().epochReady(metadata.epoch).get(); + AccordService.instance().epochReady(metadata.epoch, EpochReady::reads).get(); long epochEndNanos = nanoTime(); // As of this writing this logic is based off ExclusiveSyncPoints which is a bit heavy weight for what is needed, this could cause timeouts for clusters that have a lot of data. diff --git a/src/java/org/apache/cassandra/tcm/sequences/Move.java b/src/java/org/apache/cassandra/tcm/sequences/Move.java index b54b796749fa..25058e20f2ae 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/Move.java +++ b/src/java/org/apache/cassandra/tcm/sequences/Move.java @@ -31,6 +31,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import accord.api.ConfigurationService.EpochReady; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.SystemKeyspace; import org.apache.cassandra.db.TypeSizes; @@ -258,7 +259,7 @@ else if (destination.isSelf()) StreamResultFuture streamResult = streamPlan.execute(); - Future accordReady = AccordService.instance().epochReadyFor(metadata); + Future accordReady = AccordService.instance().epochReadyFor(metadata, EpochReady::reads); Future ready = FutureCombiner.allOf(streamResult, accordReady); ready.get(); StorageService.instance.repairPaxosForTopologyChange("move"); diff --git a/src/java/org/apache/cassandra/tools/StandaloneJournalUtil.java b/src/java/org/apache/cassandra/tools/StandaloneJournalUtil.java index 93132c5467de..a14770472c51 100644 --- a/src/java/org/apache/cassandra/tools/StandaloneJournalUtil.java +++ b/src/java/org/apache/cassandra/tools/StandaloneJournalUtil.java @@ -274,7 +274,7 @@ public void run() Map cache = new HashMap<>(); journal.start(null); - journal.forEach(key -> processKey(cache, journal, key, txnId, sinceTimestamp, untilTimestamp, skipAllErrors, skipExceptionTypes)); + journal.forEach(key -> processKey(cache, journal, key, txnId, sinceTimestamp, untilTimestamp, skipAllErrors, skipExceptionTypes), false); } private void processKey(Map redundantBeforeCache, AccordJournal journal, JournalKey key, Timestamp txnId, Timestamp minTimestamp, Timestamp maxTimestamp, boolean skipAllErrors, Set skipExceptionTypes) diff --git a/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java b/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java index 51a551a19abd..a2d0429c3f66 100644 --- a/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java +++ b/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java @@ -49,6 +49,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.Futures; +import accord.api.ConfigurationService.EpochReady; import org.agrona.collections.IntArrayList; import org.apache.cassandra.tcm.compatibility.TokenRingUtils; import org.apache.cassandra.utils.FBUtilities; @@ -1698,7 +1699,7 @@ public static void awaitAccordEpochReady(Cluster cluster, long epoch) i.runOnInstance(() -> { try { - AccordService.instance().epochReady(Epoch.create(epoch)).get(); + AccordService.instance().epochReady(Epoch.create(epoch), EpochReady::reads).get(); } catch (InterruptedException | ExecutionException e) { diff --git a/test/distributed/org/apache/cassandra/distributed/test/TestBaseImpl.java b/test/distributed/org/apache/cassandra/distributed/test/TestBaseImpl.java index 7d4320aa01a0..6d91a592379a 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/TestBaseImpl.java +++ b/test/distributed/org/apache/cassandra/distributed/test/TestBaseImpl.java @@ -44,7 +44,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import accord.messages.AbstractRequest; +import accord.messages.NoWaitRequest; import net.openhft.chronicle.core.util.SerializablePredicate; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.cql3.CQLTester; @@ -109,8 +109,8 @@ protected static class MessageCountingSink implements IMessageSink // This isn't perfect at excluding messages so make sure it excludes the ones you care about in your test public static final SerializablePredicate> EXCLUDE_SYNC_POINT_MESSAGES = message -> { - if (message.payload instanceof AbstractRequest) - return !((AbstractRequest)message.payload).txnId.isSyncPoint(); + if (message.payload instanceof NoWaitRequest) + return !((NoWaitRequest)message.payload).txnId.isSyncPoint(); return true; }; @@ -251,7 +251,7 @@ public static String batch(boolean logged, String... queries) return sb.toString(); } - protected void bootstrapAndJoinNode(Cluster cluster) + protected IInvokableInstance bootstrapAndJoinNode(Cluster cluster) { IInstanceConfig config = cluster.newInstanceConfig(); config.set("auto_bootstrap", true); @@ -261,6 +261,7 @@ protected void bootstrapAndJoinNode(Cluster cluster) () -> newInstance.startup(cluster)); newInstance.nodetoolResult("join").asserts().success(); newInstance.nodetoolResult("cms", "describe").asserts().success(); // just make sure we're joined, remove later + return newInstance; } @SuppressWarnings("unchecked") diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java index d30c3d887a39..49632ed1535d 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java @@ -24,13 +24,16 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; +import java.util.function.Function; import org.junit.Assert; import org.junit.Test; -import accord.local.PreLoadContext; +import accord.api.ConfigurationService.EpochReady; +import accord.primitives.RoutingKeys; import accord.primitives.Timestamp; import accord.topology.TopologyManager; +import accord.utils.async.AsyncResult; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.cql3.UntypedResultSet; @@ -41,14 +44,15 @@ import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.IInstanceConfig; import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableFunction; import org.apache.cassandra.distributed.api.TokenSupplier; import org.apache.cassandra.distributed.shared.NetworkTopology; import org.apache.cassandra.distributed.test.TestBaseImpl; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.accord.AccordCommandStore; import org.apache.cassandra.service.accord.AccordConfigurationService; -import org.apache.cassandra.service.accord.AccordConfigurationService.EpochSnapshot; import org.apache.cassandra.service.accord.AccordSafeCommandStore; import org.apache.cassandra.service.accord.AccordService; import org.apache.cassandra.service.accord.api.PartitionKey; @@ -66,6 +70,8 @@ import static org.apache.cassandra.distributed.api.Feature.GOSSIP; import static org.apache.cassandra.distributed.api.Feature.NETWORK; import static org.apache.cassandra.service.accord.AccordService.getBlocking; +import static org.apache.cassandra.service.accord.AccordConfigurationService.EpochSnapshot.ResultStatus.SUCCESS; +import static org.apache.cassandra.service.accord.AccordConfigurationService.SyncStatus.COMPLETED; public class AccordBootstrapTest extends TestBaseImpl { @@ -81,7 +87,7 @@ private static PartitionKey pk(int key, String keyspace, String table) return new PartitionKey(tid, dk(key)); } - protected void bootstrapAndJoinNode(Cluster cluster) + protected IInvokableInstance bootstrapAndJoinNode(Cluster cluster) { IInstanceConfig config = cluster.newInstanceConfig(); config.set("auto_bootstrap", true); @@ -94,6 +100,7 @@ protected void bootstrapAndJoinNode(Cluster cluster) // () -> withProperty("cassandra.join_ring", false, () -> newInstance.startup(cluster))); // newInstance.nodetoolResult("join").asserts().success(); newInstance.nodetoolResult("cms", "describe").asserts().success(); // just make sure we're joined, remove later + return newInstance; } private static AccordService service() @@ -101,11 +108,11 @@ private static AccordService service() return (AccordService) AccordService.instance(); } - private static void awaitEpoch(long epoch) + private static void awaitEpoch(long epoch, Function> await) { try { - boolean completed = service().epochReady(Epoch.create(epoch)).await(60, TimeUnit.SECONDS); + boolean completed = service().epochReady(Epoch.create(epoch), await).await(60, TimeUnit.SECONDS); Assertions.assertThat(completed) .describedAs("Epoch %s did not become ready within timeout on %s -> %s", epoch, FBUtilities.getBroadcastAddressAndPort(), @@ -168,6 +175,14 @@ public synchronized void forSession(Consumer consumer) @Test public void bootstrapTest() throws Throwable + { + bootstrapTest(Function.identity(), cluster -> { + bootstrapAndJoinNode(cluster); + awaitMaxEpochReadyToRead(cluster); + }); + } + + public void bootstrapTest(Function setup, Consumer bootstrapAndJoinNode) throws Throwable { int originalNodeCount = 2; int expandedNodeCount = originalNodeCount + 1; @@ -188,49 +203,10 @@ public void bootstrapTest() throws Throwable cluster.schemaChange("CREATE KEYSPACE ks WITH REPLICATION={'class':'SimpleStrategy', 'replication_factor':2}"); cluster.schemaChange("CREATE TABLE ks.tbl (k int, c int, v int, primary key(k, c)) WITH transactional_mode='full'"); - long initialMax = maxEpoch(cluster); - + awaitMaxEpochReadyToRead(cluster); for (IInvokableInstance node : cluster) - { - node.runOnInstance(() -> { - Assert.assertEquals(initialMax, ClusterMetadata.current().epoch.getEpoch()); - awaitEpoch(initialMax); - AccordConfigurationService configService = service().configService(); - long minEpoch = configService.minEpoch(); - - Assert.assertEquals(initialMax, configService.maxEpoch()); - - for (long epoch = minEpoch; epoch < initialMax; epoch++) - { - awaitEpoch(epoch); - Assert.assertEquals(EpochSnapshot.completed(epoch), configService.getEpochSnapshot(epoch)); - } - - awaitLocalSyncNotification(initialMax); - Assert.assertEquals(EpochSnapshot.completed(initialMax), configService.getEpochSnapshot(initialMax)); - }); - } - - for (IInvokableInstance node : cluster) - { node.runOnInstance(StreamListener::register); - } - - long schemaChangeMax = maxEpoch(cluster); - for (IInvokableInstance node : cluster) - { - node.runOnInstance(() -> { - ClusterMetadataService.instance().fetchLogFromCMS(Epoch.create(schemaChangeMax)); - awaitEpoch(schemaChangeMax); - AccordConfigurationService configService = service().configService(); - - for (long epoch = initialMax + 1; epoch <= schemaChangeMax; epoch++) - { - awaitLocalSyncNotification(epoch); - Assert.assertEquals(EpochSnapshot.completed(epoch), configService.getEpochSnapshot(epoch)); - } - }); - } + awaitMaxEpochReadyToRead(cluster); for (int key = 0; key < 100; key++) { @@ -251,21 +227,7 @@ public void bootstrapTest() throws Throwable }); } - bootstrapAndJoinNode(cluster); - long bootstrapMax = maxEpoch(cluster); - for (IInvokableInstance node : cluster) - { - node.runOnInstance(() -> { - ClusterMetadataService.instance().fetchLogFromCMS(Epoch.create(bootstrapMax)); - Assert.assertEquals(bootstrapMax, ClusterMetadata.current().epoch.getEpoch()); - AccordService service = (AccordService) AccordService.instance(); - awaitEpoch(bootstrapMax); - AccordConfigurationService configService = service.configService(); - - awaitLocalSyncNotification(bootstrapMax); - Assert.assertEquals(EpochSnapshot.completed(bootstrapMax), configService.getEpochSnapshot(bootstrapMax)); - }); - } + bootstrapAndJoinNode.accept(cluster); InetAddress node3Addr = cluster.get(3).broadcastAddress().getAddress(); for (IInvokableInstance node : cluster.get(1, 2)) @@ -278,15 +240,11 @@ public void bootstrapTest() throws Throwable Assert.assertTrue(session.getNumKeyspaceTransfers() > 0); }); - getBlocking(service().node().commandStores().forEach((PreLoadContext.Empty)()->"Test", safeStore -> { - AccordSafeCommandStore ss = (AccordSafeCommandStore) safeStore; - Assert.assertEquals(Timestamp.NONE, getOnlyElement(ss.bootstrapBeganAt().keySet())); - Assert.assertEquals(Timestamp.NONE, getOnlyElement(ss.safeToReadAt().keySet())); -// -// Assert.assertTrue(commandStore.maxBootstrapEpoch() > 0); -// Assert.assertTrue(commandStore.bootstrapBeganAt().isEmpty()); -// Assert.assertTrue(commandStore.safeToRead().isEmpty()); - })); + service().node().commandStores().forAllUnsafe(unsafeStore -> { + AccordCommandStore ss = (AccordCommandStore) unsafeStore; + Assert.assertEquals(Timestamp.NONE, getOnlyElement(ss.unsafeGetBootstrapBeganAt().keySet())); + Assert.assertEquals(Timestamp.NONE, getOnlyElement(ss.unsafeGetSafeToRead().keySet())); + }); }); } @@ -321,7 +279,7 @@ public void bootstrapTest() throws Throwable Assert.assertEquals(key, row.getInt("c")); Assert.assertEquals(key, row.getInt("v")); - getBlocking(service().node().commandStores().forEach((PreLoadContext.Empty)()->"Test", safeStore -> { + getBlocking(service().node().commandStores().forEach("Test", RoutingKeys.of(partitionKey.toUnseekable()), Long.MIN_VALUE, Long.MAX_VALUE, safeStore -> { if (safeStore.ranges().currentRanges().contains(partitionKey)) { AccordSafeCommandStore ss = (AccordSafeCommandStore) safeStore; @@ -375,44 +333,7 @@ public void moveTest() throws Throwable tokens[i] = cluster.get(i+1).callOnInstance(() -> Long.valueOf(getOnlyElement(StorageService.instance.getTokens()))); } - for (IInvokableInstance node : cluster) - { - - node.runOnInstance(() -> { - Assert.assertEquals(initialMax, ClusterMetadata.current().epoch.getEpoch()); - awaitEpoch(initialMax); - AccordConfigurationService configService = service().configService(); - long minEpoch = configService.minEpoch(); - - Assert.assertEquals(initialMax, configService.maxEpoch()); - - for (long epoch = minEpoch; epoch < initialMax; epoch++) - { - awaitEpoch(epoch); - Assert.assertEquals(EpochSnapshot.completed(epoch), configService.getEpochSnapshot(epoch)); - } - - awaitLocalSyncNotification(initialMax); - Assert.assertEquals(EpochSnapshot.completed(initialMax), configService.getEpochSnapshot(initialMax)); - }); - } - - long schemaChangeMax = maxEpoch(cluster); - for (IInvokableInstance node : cluster) - { - node.runOnInstance(() -> { - Assert.assertEquals(schemaChangeMax, ClusterMetadata.current().epoch.getEpoch()); - AccordService service = (AccordService) AccordService.instance(); - awaitEpoch(schemaChangeMax); - AccordConfigurationService configService = service.configService(); - - for (long epoch = initialMax + 1; epoch <= schemaChangeMax; epoch++) - { - awaitLocalSyncNotification(epoch); - Assert.assertEquals(EpochSnapshot.completed(epoch), configService.getEpochSnapshot(epoch)); - } - }); - } + awaitMaxEpochReadyToRead(cluster); for (int key = 0; key < 100; key++) { @@ -431,20 +352,7 @@ public void moveTest() throws Throwable cluster.get(1).runOnInstance(() -> StorageService.instance.move(Long.toString(token))); - long moveMax = maxEpoch(cluster); - for (IInvokableInstance node : cluster) - { - node.runOnInstance(() -> { - ClusterMetadataService.instance().fetchLogFromCMS(Epoch.create(moveMax)); - Assert.assertEquals(moveMax, ClusterMetadata.current().epoch.getEpoch()); - AccordService service = (AccordService) AccordService.instance(); - awaitEpoch(moveMax); - AccordConfigurationService configService = service.configService(); - - awaitLocalSyncNotification(moveMax); - Assert.assertEquals(EpochSnapshot.completed(moveMax), configService.getEpochSnapshot(moveMax)); - }); - } + long moveMax = awaitMaxEpochReadyToRead(cluster); for (IInvokableInstance node : cluster) { @@ -464,9 +372,7 @@ public void moveTest() throws Throwable PartitionKey partitionKey = new PartitionKey(tableId, dk); - getBlocking(service().node().commandStores().forEach((PreLoadContext.Empty)()->"Test", - partitionKey.toUnseekable(), moveMax, moveMax, - safeStore -> { + getBlocking(service().node().commandStores().forEach("Test", RoutingKeys.of(partitionKey.toUnseekable()), moveMax, moveMax, safeStore -> { if (!safeStore.ranges().allAt(preMove).contains(partitionKey)) { AccordSafeCommandStore ss = (AccordSafeCommandStore) safeStore; @@ -493,4 +399,41 @@ public void moveTest() throws Throwable } } } + + private static long awaitMaxEpochReadyToRead(Cluster cluster) + { + return awaitMaxEpoch(cluster, EpochReady::reads, true); + } + + private static long awaitMaxEpochMetadataReady(Cluster cluster) + { + return awaitMaxEpoch(cluster, EpochReady::metadata, false); + } + + private static long awaitMaxEpoch(Cluster cluster, SerializableFunction> await, boolean expectReadyToRead) + { + long maxEpoch = maxEpoch(cluster); + for (IInvokableInstance node : cluster) + { + node.acceptOnInstance(aw -> { + ClusterMetadataService.instance().fetchLogFromCMS(Epoch.create(maxEpoch)); + Assert.assertEquals(maxEpoch, ClusterMetadata.current().epoch.getEpoch()); + AccordService service = (AccordService) AccordService.instance(); + awaitEpoch(maxEpoch, aw); + AccordConfigurationService configService = service.configService(); + + awaitLocalSyncNotification(maxEpoch); + for (long epoch = configService.minEpoch(); epoch <= maxEpoch; epoch++) + { + Assert.assertEquals(COMPLETED, configService.getEpochSnapshot(maxEpoch).syncStatus); + Assert.assertEquals(SUCCESS, configService.getEpochSnapshot(maxEpoch).acknowledged); + Assert.assertEquals(SUCCESS, configService.getEpochSnapshot(maxEpoch).received); + if (expectReadyToRead) + Assert.assertEquals(SUCCESS, configService.getEpochSnapshot(maxEpoch).reads); + } + }, node.transfer(await)); + } + return maxEpoch; + } + } diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIncrementalRepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIncrementalRepairTest.java index a5a38edadf74..bbb8a7f4cb4d 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIncrementalRepairTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIncrementalRepairTest.java @@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicReference; import javax.annotation.Nullable; import accord.local.Node; -import accord.local.PreLoadContext; import accord.local.SafeCommand; import accord.local.StoreParticipants; import accord.local.cfk.CommandsForKey; @@ -32,6 +31,7 @@ import accord.local.durability.DurabilityService; import accord.primitives.Keys; import accord.primitives.Ranges; +import accord.primitives.RoutingKeys; import accord.primitives.Status; import accord.primitives.Timestamp; import accord.primitives.TxnId; @@ -68,8 +68,6 @@ import org.slf4j.LoggerFactory; -import static accord.local.LoadKeys.SYNC; -import static accord.local.LoadKeysFor.READ_WRITE; import static java.lang.String.format; import static org.apache.cassandra.distributed.test.accord.AccordTestBase.executeWithRetry; import static org.apache.cassandra.service.accord.AccordService.getBlocking; @@ -158,7 +156,7 @@ private static void withCluster(ModelChecker.ThrowingConsumer run) thro { cluster.filters().reset(); for (IInvokableInstance instance : cluster) - instance.runOnInstance(() -> AccordService.instance().node().commandStores().forEachCommandStore(cs -> cs.unsafeProgressLog().start())); + instance.runOnInstance(() -> AccordService.instance().node().commandStores().forAllUnsafe(cs -> cs.unsafeProgressLog().start())); } } } @@ -207,7 +205,7 @@ private static TxnId awaitLocalApplyOnKey(TokenKey key) { Node node = accordService().node(); AtomicReference waitFor = new AtomicReference<>(null); - getBlocking(node.commandStores().ifLocal(PreLoadContext.contextFor(key, SYNC, READ_WRITE, "Test"), key.toUnseekable(), 0, Long.MAX_VALUE, safeStore -> { + getBlocking(node.commandStores().forEach("Test", RoutingKeys.of(key), Long.MIN_VALUE, Long.MAX_VALUE, safeStore -> { AccordSafeCommandStore store = (AccordSafeCommandStore) safeStore; SafeCommandsForKey safeCfk = store.ifLoadedAndInitialised(key); if (safeCfk == null) @@ -229,7 +227,7 @@ private static TxnId awaitLocalApplyOnKey(TokenKey key) long now = Clock.Global.currentTimeMillis(); if (now - start > TimeUnit.MINUTES.toMillis(1)) throw new AssertionError("Timeout"); - AccordService.getBlocking(node.commandStores().ifLocal(PreLoadContext.contextFor(txnId, "Test"), key.toUnseekable(), 0, Long.MAX_VALUE, safeStore -> { + getBlocking(node.commandStores().forEach("Test", RoutingKeys.of(key), Long.MIN_VALUE, Long.MAX_VALUE, safeStore -> { SafeCommand command = safeStore.get(txnId, StoreParticipants.empty(txnId)); Assert.assertNotNull(command.current()); if (command.current().status().hasBeen(Status.Applied)) @@ -291,7 +289,7 @@ public void txnRepairTest(Cluster cluster) throws Throwable // heal partition and wait for node 1 to see node 3 again for (IInvokableInstance instance : cluster) instance.runOnInstance(() -> { - AccordService.instance().node().commandStores().forEachCommandStore(cs -> cs.unsafeProgressLog().stop()); + AccordService.instance().node().commandStores().forAllUnsafe(cs -> cs.unsafeProgressLog().stop()); Assert.assertFalse(barrierRecordingService().executedBarriers); }); cluster.filters().reset(); diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIntegrationTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIntegrationTest.java index c2b8f1e48618..a7354e17b7cf 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIntegrationTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIntegrationTest.java @@ -125,6 +125,6 @@ public void testLostCommitReadTriggersFallbackRead() throws Exception private void pauseSimpleProgressLog() { for (IInvokableInstance instance : SHARED_CLUSTER) - instance.runOnInstance(() -> AccordService.instance().node().commandStores().forEachCommandStore(cs -> cs.unsafeProgressLog().stop())); + instance.runOnInstance(() -> AccordService.instance().node().commandStores().forAllUnsafe(cs -> cs.unsafeProgressLog().stop())); } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java index b4c53b6159b9..cb91e9a160e1 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java @@ -166,7 +166,7 @@ public void tearDown() throws Exception { SHARED_CLUSTER.filters().reset(); for (IInvokableInstance instance : SHARED_CLUSTER) - instance.runOnInstance(() -> AccordService.instance().node().commandStores().forEachCommandStore(cs -> cs.unsafeProgressLog().start())); + instance.runOnInstance(() -> AccordService.instance().node().commandStores().forAllUnsafe(cs -> cs.unsafeProgressLog().start())); truncateSystemTables(); diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordVirtualTableTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordVirtualTableTest.java new file mode 100644 index 000000000000..f1e9d17ed4b3 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordVirtualTableTest.java @@ -0,0 +1,109 @@ +/* + * 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.accord; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.test.log.FuzzTestBase; +import org.apache.cassandra.service.accord.AccordService; +import org.junit.Test; + + +import static org.apache.cassandra.distributed.api.Feature.GOSSIP; +import static org.apache.cassandra.distributed.api.Feature.NETWORK; + +public class AccordVirtualTableTest extends FuzzTestBase +{ + @Test + public void testCoordinationsTableDistributed() throws Exception + { + try (Cluster cluster = init(builder() + .withNodes(3) + .withConfig(config -> config.with(NETWORK, GOSSIP)) + .start())) + { + cluster.schemaChange(withKeyspace("CREATE TABLE %s.test_table (pk int PRIMARY KEY, v int) " + + "WITH transactional_mode = 'full'")); + + // Wait for accord to be fully init + // Perform concurrent transactional operations to generate coordination activity + for (int i = 0; i < 10; i++) { + final int key = i; + cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.test_table (pk, v) VALUES (?, ?)"), + ConsistencyLevel.QUORUM, key, key * 10); + } + + List nodeIds = new ArrayList<>(); + cluster.forEach(i -> { + nodeIds.add(i.callsOnInstance(() -> AccordService.instance().node().id().id).call()); + }); + + for (int callOn = 1; callOn <= 3; callOn++) + { + for (Integer remote : nodeIds) + { + IInvokableInstance node = cluster.get(callOn); + Object[][] remoteRes = cluster.get(remote).executeInternal("SELECT * FROM system_accord_debug_remote.redundant_before where node_id= " + remote); + Object[][] localRes = node.executeInternal("SELECT * FROM system_accord_debug.redundant_before"); + localMatchesRemote(remote, callOn, remoteRes, localRes); + } + } + } + } + + public static void localMatchesRemote(int remote, int callOn, Object[][] remoteRes, Object[][] localRes) + { + if (remoteRes.length == localRes.length) + { + for (int i = 0; i < remoteRes.length; i++) + { + Object[] remoteRow = remoteRes[i]; + Object[] localRow = localRes[i]; + for (int j = 0; j < localRow.length; j++) + { + if (!Objects.equals(remoteRow[i + 1], localRow[i])) + { + throw new AssertionError(String.format("Mismatch while executing query for %d on %d.Row %d column %d. %s != %s\n" + + "Remote: %s\n" + + "Local: %s", + remote, callOn, + i, j, + Objects.toString(remoteRow[i + 1]), Objects.toString(localRow[i]), + toString(remoteRes), + toString(localRes))); + } + } + } + } + } + public static String toString(Object[][] objects) + { + StringBuilder sb = new StringBuilder(); + for (Object[] o : objects) + { + sb.append(Arrays.toString(o)); + } + return sb.toString(); + } +} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/fuzz/topology/AccordBounceTest.java b/test/distributed/org/apache/cassandra/fuzz/topology/AccordBounceTest.java index 1e311bf8a541..b6aeb6231d56 100644 --- a/test/distributed/org/apache/cassandra/fuzz/topology/AccordBounceTest.java +++ b/test/distributed/org/apache/cassandra/fuzz/topology/AccordBounceTest.java @@ -150,9 +150,9 @@ public String get() // Command Stores should not be lost on bounce Map> before = cluster.get(1).callOnInstance(() -> { Map> m = new HashMap<>(); - AccordService.instance().node().commandStores().forEach((store, ranges) -> { + AccordService.instance().node().commandStores().forAllUnsafe((store) -> { Set set = new HashSet<>(); - for (Range range : ranges.all()) + for (Range range : store.unsafeGetRangesForEpoch().all()) set.add(range.toString()); m.put(store.id(), set); }); @@ -169,9 +169,9 @@ public String get() Map> after = cluster.get(1).callOnInstance(() -> { Map> m = new HashMap<>(); - AccordService.instance().node().commandStores().forEach((store, ranges) -> { + AccordService.instance().node().commandStores().forAllUnsafe(store -> { Set set = new HashSet<>(); - for (Range range : ranges.all()) + for (Range range : store.unsafeGetRangesForEpoch().all()) set.add(range.toString()); m.put(store.id(), set); }); diff --git a/test/distributed/org/apache/cassandra/fuzz/topology/JournalGCTest.java b/test/distributed/org/apache/cassandra/fuzz/topology/JournalGCTest.java index 5ba2a7c5d308..e4c497657d2b 100644 --- a/test/distributed/org/apache/cassandra/fuzz/topology/JournalGCTest.java +++ b/test/distributed/org/apache/cassandra/fuzz/topology/JournalGCTest.java @@ -113,7 +113,7 @@ public void journalGCTest() throws Throwable ((AccordService) AccordService.instance()).journal().forEach((v) -> { if (v.type == JournalKey.Type.COMMAND_DIFF && (a.get() == null || v.id.compareTo(a.get()) > 0)) a.set(v.id); - }); + }, false); return a.get() == null ? "" : a.get().toString(); }); @@ -123,7 +123,7 @@ public void journalGCTest() throws Throwable ((AccordService) AccordService.instance()).journal().forEach((v) -> { if (v.type == JournalKey.Type.COMMAND_DIFF && v.id.compareTo(maxId) <= 0) a.incrementAndGet(); - }); + }, false); return a.get(); }, maximumId); diff --git a/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java b/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java index 25b7b952c888..e074ed950e3e 100644 --- a/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java +++ b/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java @@ -332,7 +332,7 @@ public void purge(CommandStores commandStores, EpochSupplier minEpoch) private TreeMap read(CommandStores commandStores) { TreeMap result = new TreeMap<>(JournalKey.SUPPORT::compare); - try (CloseableIterator> iter = journalTable.keyIterator(null, null)) + try (CloseableIterator> iter = journalTable.keyIterator(null, null, false)) { JournalKey prev = null; while (iter.hasNext()) @@ -354,11 +354,11 @@ private TreeMap read(CommandStores commandStores) } @Override - public void replay(CommandStores commandStores) + public boolean replay(CommandStores commandStores) { // Make sure to replay _only_ static segments this.closeCurrentSegmentForTestingIfNonEmpty(); - super.replay(commandStores); + return super.replay(commandStores); } @Override @@ -388,7 +388,7 @@ public PersistentField.Persister durableBeforePers public static IAccordService.AccordCompactionInfos getCompactionInfo(Node node, TableId tableId) { IAccordService.AccordCompactionInfos compactionInfos = new IAccordService.AccordCompactionInfos(node.durableBefore(), node.topology().minEpoch()); - node.commandStores().forEachCommandStore(commandStore -> { + node.commandStores().forAllUnsafe(commandStore -> { RedundantBefore redundantBefore = commandStore.unsafeGetRedundantBefore(); if (redundantBefore == null) redundantBefore = RedundantBefore.EMPTY; diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 23ff085c778f..413b1553fca3 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -3185,7 +3185,7 @@ public String describe() private static String formatValue(ByteBuffer bb, AbstractType type) { - if (bb == null) + if (bb == null || (!bb.hasRemaining() && type.isEmptyValueMeaningless())) return "null"; if (type instanceof CollectionType) diff --git a/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java b/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java index 7f7e44fd9f64..c6ab0b04322b 100644 --- a/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java +++ b/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java @@ -19,6 +19,8 @@ package org.apache.cassandra.db.virtual; import java.util.Collections; +import java.util.ArrayList; +import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -34,8 +36,8 @@ import org.slf4j.LoggerFactory; import accord.api.ProtocolModifiers; -import accord.local.PreLoadContext; -import accord.messages.TxnRequest; +import accord.messages.NoWaitRequest; +import accord.api.RoutingKey; import accord.primitives.Ranges; import accord.primitives.Routable; import accord.primitives.SaveStatus; @@ -48,7 +50,10 @@ import org.apache.cassandra.config.YamlConfigurationLoader; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.cql3.UntypedResultSet; +import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.dht.Murmur3Partitioner.LongToken; +import org.apache.cassandra.exceptions.ExceptionSerializer; +import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; @@ -57,11 +62,14 @@ import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.service.CassandraDaemon; +import org.apache.cassandra.service.accord.AccordCommandStore; import org.apache.cassandra.service.accord.AccordService; +import org.apache.cassandra.service.accord.IAccordService; import org.apache.cassandra.service.accord.TokenRange; import org.apache.cassandra.service.accord.api.TokenKey; import org.apache.cassandra.tcm.ClusterMetadata; -import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.transport.Dispatcher; +import org.apache.cassandra.utils.Clock; import org.apache.cassandra.utils.concurrent.Condition; import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; @@ -82,48 +90,108 @@ public class AccordDebugKeyspaceTest extends CQLTester private static final String QUERY_TXN_BLOCKED_BY = String.format("SELECT * FROM %s.%s WHERE txn_id=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_BLOCKED_BY); + private static final String QUERY_TXN_BLOCKED_BY_REMOTE = + String.format("SELECT * FROM %s.%s WHERE node_id = ? AND txn_id=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_BLOCKED_BY); + private static final String QUERY_COMMANDS_FOR_KEY = String.format("SELECT txn_id, status FROM %s.%s WHERE key=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.COMMANDS_FOR_KEY); + private static final String QUERY_COMMANDS_FOR_KEY_REMOTE = + String.format("SELECT txn_id, status FROM %s.%s WHERE node_id = ? AND key=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.COMMANDS_FOR_KEY); + private static final String QUERY_TXN = String.format("SELECT txn_id, save_status FROM %s.%s WHERE txn_id=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN); + private static final String QUERY_TXN_REMOTE = + String.format("SELECT txn_id, save_status FROM %s.%s WHERE node_id = ? AND txn_id=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN); + + private static final String QUERY_TXNS = + String.format("SELECT save_status FROM %s.%s WHERE command_store_id = ? LIMIT 5", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN); + + private static final String QUERY_TXNS_REMOTE = + String.format("SELECT save_status FROM %s.%s WHERE node_id = ? AND command_store_id = ? LIMIT 5", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN); + + private static final String QUERY_TXNS_SEARCH = + String.format("SELECT save_status FROM %s.%s WHERE command_store_id = ? AND txn_id > ? LIMIT 5", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN); + + private static final String QUERY_TXNS_SEARCH_REMOTE = + String.format("SELECT save_status FROM %s.%s WHERE node_id = ? AND command_store_id = ? AND txn_id > ? LIMIT 5", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN); + private static final String QUERY_JOURNAL = String.format("SELECT txn_id, save_status FROM %s.%s WHERE txn_id=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.JOURNAL); + private static final String ERASE_JOURNAL_REMOTE = + String.format("DELETE FROM %s.%s WHERE node_id = ? AND command_store_id = ? AND txn_id=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.JOURNAL); + + private static final String QUERY_JOURNAL_REMOTE = + String.format("SELECT txn_id, save_status FROM %s.%s WHERE node_id = ? AND txn_id=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.JOURNAL); + private static final String SET_TRACE = String.format("UPDATE %s.%s SET permits = ? WHERE txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_TRACE); + private static final String SET_TRACE_REMOTE = + String.format("UPDATE %s.%s SET permits = ? WHERE node_id = ? AND txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_TRACE); + private static final String QUERY_TRACE = String.format("SELECT * FROM %s.%s WHERE txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_TRACE); + private static final String QUERY_TRACE_REMOTE = + String.format("SELECT * FROM %s.%s WHERE node_id = ? AND txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_TRACE); + private static final String UNSET_TRACE1 = String.format("DELETE FROM %s.%s WHERE txn_id = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_TRACE); + private static final String UNSET_TRACE1_REMOTE = + String.format("DELETE FROM %s.%s WHERE node_id = ? AND txn_id = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_TRACE); + private static final String UNSET_TRACE2 = String.format("DELETE FROM %s.%s WHERE txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_TRACE); + private static final String UNSET_TRACE2_REMOTE = + String.format("DELETE FROM %s.%s WHERE node_id = ? AND txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_TRACE); + private static final String QUERY_TRACES = String.format("SELECT * FROM %s.%s WHERE txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_TRACES); + private static final String QUERY_TRACES_REMOTE = + String.format("SELECT * FROM %s.%s WHERE node_id = ? AND txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_TRACES); + private static final String ERASE_TRACES1 = String.format("DELETE FROM %s.%s WHERE txn_id = ? AND event_type = ? AND id_micros < ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_TRACES); + private static final String ERASE_TRACES1_REMOTE = + String.format("DELETE FROM %s.%s WHERE node_id = ? AND txn_id = ? AND event_type = ? AND id_micros < ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_TRACES); + private static final String ERASE_TRACES2 = String.format("DELETE FROM %s.%s WHERE txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_TRACES); + private static final String ERASE_TRACES2_REMOTE = + String.format("DELETE FROM %s.%s WHERE node_id = ? AND txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_TRACES); + private static final String ERASE_TRACES3 = String.format("DELETE FROM %s.%s WHERE txn_id = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_TRACES); + private static final String ERASE_TRACES3_REMOTE = + String.format("DELETE FROM %s.%s WHERE node_id = ? AND txn_id = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_TRACES); + private static final String QUERY_REDUNDANT_BEFORE = String.format("SELECT * FROM %s.%s", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.REDUNDANT_BEFORE); + private static final String QUERY_REDUNDANT_BEFORE_REMOTE = + String.format("SELECT * FROM %s.%s WHERE node_id = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.REDUNDANT_BEFORE); + private static final String QUERY_REDUNDANT_BEFORE_FILTER_QUORUM_APPLIED_GEQ = String.format("SELECT * FROM %s.%s WHERE quorum_applied >= ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.REDUNDANT_BEFORE); + private static final String QUERY_REDUNDANT_BEFORE_FILTER_QUORUM_APPLIED_GEQ_REMOTE = + String.format("SELECT * FROM %s.%s WHERE node_id = ? AND quorum_applied >= ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.REDUNDANT_BEFORE); + private static final String QUERY_REDUNDANT_BEFORE_FILTER_SHARD_APPLIED_GEQ = String.format("SELECT * FROM %s.%s WHERE shard_applied >= ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.REDUNDANT_BEFORE); + private static final String QUERY_REDUNDANT_BEFORE_FILTER_SHARD_APPLIED_GEQ_REMOTE = + String.format("SELECT * FROM %s.%s WHERE node_id = ? AND shard_applied >= ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.REDUNDANT_BEFORE); + @BeforeClass public static void setUpClass() { @@ -158,43 +226,103 @@ public void unknownIsEmpty() public void tracing() { // simple test to confirm basic tracing functionality works, doesn't validate specific behaviours only requesting/querying/erasing + String tableName = createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH transactional_mode = 'full'"); + AccordService accord = accord(); + DatabaseDescriptor.getAccord().fetch_txn = "1s"; + int nodeId = accord.nodeId().id; + AccordMsgFilter filter = new AccordMsgFilter(); MessagingService.instance().outboundSink.add(filter); try { - String tableName = createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH transactional_mode = 'full'"); - AccordService accord = accord(); - DatabaseDescriptor.getAccord().fetch_txn = "1s"; TxnId id = accord.node().nextTxnIdWithDefaultFlags(Txn.Kind.Write, Routable.Domain.Key); Txn txn = createTxn(wrapInTxn(String.format("INSERT INTO %s.%s(k, c, v) VALUES (?, ?, ?)", KEYSPACE, tableName)), 0, 0, 0); + filter.appliesTo(id); execute(SET_TRACE, 1, id.toString(), "WAIT_PROGRESS"); assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS"), row(id.toString(), "WAIT_PROGRESS", 1)); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"), row(nodeId, id.toString(), "WAIT_PROGRESS", 1)); execute(SET_TRACE, 0, id.toString(), "WAIT_PROGRESS"); assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS")); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS")); execute(SET_TRACE, 1, id.toString(), "WAIT_PROGRESS"); assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS"), row(id.toString(), "WAIT_PROGRESS", 1)); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"), row(nodeId, id.toString(), "WAIT_PROGRESS", 1)); execute(UNSET_TRACE1, id.toString()); assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS")); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS")); execute(SET_TRACE, 1, id.toString(), "WAIT_PROGRESS"); assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS"), row(id.toString(), "WAIT_PROGRESS", 1)); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"), row(nodeId, id.toString(), "WAIT_PROGRESS", 1)); execute(UNSET_TRACE2, id.toString(), "WAIT_PROGRESS"); assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS")); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS")); execute(SET_TRACE, 1, id.toString(), "WAIT_PROGRESS"); assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS"), row(id.toString(), "WAIT_PROGRESS", 1)); - accord.node().coordinate(id, txn).beginAsResult(); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"), row(nodeId, id.toString(), "WAIT_PROGRESS", 1)); filter.appliesTo(id); + accord.node().coordinate(id, txn).beginAsResult(); filter.preAccept.awaitThrowUncheckedOnInterrupt(); - filter.apply.awaitThrowUncheckedOnInterrupt(); spinUntilSuccess(() -> Assertions.assertThat(execute(QUERY_TRACES, id.toString(), "WAIT_PROGRESS").size()).isGreaterThan(0)); + spinUntilSuccess(() -> Assertions.assertThat(execute(QUERY_TRACES_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS").size()).isGreaterThan(0)); execute(ERASE_TRACES1, id.toString(), "FETCH", Long.MAX_VALUE); execute(ERASE_TRACES2, id.toString(), "FETCH"); execute(ERASE_TRACES1, id.toString(), "WAIT_PROGRESS", Long.MAX_VALUE); Assertions.assertThat(execute(QUERY_TRACES, id.toString(), "WAIT_PROGRESS").size()).isEqualTo(0); + Assertions.assertThat(execute(QUERY_TRACES_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS").size()).isEqualTo(0); // just check other variants don't fail execute(ERASE_TRACES2, id.toString(), "WAIT_PROGRESS"); execute(ERASE_TRACES3, id.toString()); + + } + finally + { + MessagingService.instance().outboundSink.remove(filter); + } + + filter = new AccordMsgFilter(); + MessagingService.instance().outboundSink.add(filter); + try + { + TxnId id = accord.node().nextTxnIdWithDefaultFlags(Txn.Kind.Write, Routable.Domain.Key); + Txn txn = createTxn(wrapInTxn(String.format("INSERT INTO %s.%s(k, c, v) VALUES (?, ?, ?)", KEYSPACE, tableName)), 1, 1, 1); + filter.appliesTo(id); + + execute(SET_TRACE_REMOTE, 1, nodeId, id.toString(), "WAIT_PROGRESS"); + assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS"), row(id.toString(), "WAIT_PROGRESS", 1)); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"), row(nodeId, id.toString(), "WAIT_PROGRESS", 1)); + execute(SET_TRACE_REMOTE, 0, nodeId, id.toString(), "WAIT_PROGRESS"); + assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS")); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS")); + execute(SET_TRACE_REMOTE, 1, nodeId, id.toString(), "WAIT_PROGRESS"); + assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS"), row(id.toString(), "WAIT_PROGRESS", 1)); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"), row(nodeId, id.toString(), "WAIT_PROGRESS", 1)); + execute(UNSET_TRACE1_REMOTE, nodeId, id.toString()); + assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS")); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS")); + execute(SET_TRACE_REMOTE, 1, nodeId, id.toString(), "WAIT_PROGRESS"); + assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS"), row(id.toString(), "WAIT_PROGRESS", 1)); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"), row(nodeId, id.toString(), "WAIT_PROGRESS", 1)); + execute(UNSET_TRACE2_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"); + assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS")); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS")); + execute(SET_TRACE_REMOTE, 1, nodeId, id.toString(), "WAIT_PROGRESS"); + assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS"), row(id.toString(), "WAIT_PROGRESS", 1)); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"), row(nodeId, id.toString(), "WAIT_PROGRESS", 1)); + accord.node().coordinate(id, txn); + filter.preAccept.awaitThrowUncheckedOnInterrupt(); + filter.apply.awaitThrowUncheckedOnInterrupt(); + spinUntilSuccess(() -> Assertions.assertThat(execute(QUERY_TRACES, id.toString(), "WAIT_PROGRESS").size()).isGreaterThan(0)); + spinUntilSuccess(() -> Assertions.assertThat(execute(QUERY_TRACES_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS").size()).isGreaterThan(0)); + execute(ERASE_TRACES1_REMOTE, nodeId, id.toString(), "FETCH", Long.MAX_VALUE); + execute(ERASE_TRACES2_REMOTE, nodeId, id.toString(), "FETCH"); + execute(ERASE_TRACES1_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS", Long.MAX_VALUE); + Assertions.assertThat(execute(QUERY_TRACES, id.toString(), "WAIT_PROGRESS").size()).isEqualTo(0); + Assertions.assertThat(execute(QUERY_TRACES_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS").size()).isEqualTo(0); + // just check other variants don't fail + execute(ERASE_TRACES2_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"); + execute(ERASE_TRACES3_REMOTE, nodeId, id.toString()); } finally { @@ -207,12 +335,13 @@ public void redundantBefore() throws ExecutionException, InterruptedException { String tableName = createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH transactional_mode = 'full'"); var accord = accord(); + int nodeId = accord.nodeId().id; TableId tableId = Schema.instance.getTableMetadata(KEYSPACE, tableName).id; TxnId syncId1 = new TxnId(100, 200, Txn.Kind.ExclusiveSyncPoint, Routable.Domain.Range, accord.nodeId()); TxnId syncId2 = new TxnId(101, 300, Txn.Kind.ExclusiveSyncPoint, Routable.Domain.Range, accord.nodeId()); Ranges ranges1 = Ranges.of(TokenRange.create(new TokenKey(tableId, new LongToken(1)), new TokenKey(tableId, new LongToken(100)))); Ranges ranges2 = Ranges.of(TokenRange.create(new TokenKey(tableId, new LongToken(100)), new TokenKey(tableId, new LongToken(200)))); - getBlocking(accord.node().commandStores().forEach((PreLoadContext.Empty)() -> "Test", safeStore -> { + getBlocking(accord.node().commandStores().forAll("Test", safeStore -> { safeStore.commandStore().markShardDurable(safeStore, syncId1, ranges1, HasOutcome.Universal); safeStore.commandStore().markShardDurable(safeStore, syncId2, ranges2, HasOutcome.Quorum); })); @@ -222,6 +351,27 @@ public void redundantBefore() throws ExecutionException, InterruptedException Assertions.assertThat(execute(QUERY_REDUNDANT_BEFORE_FILTER_QUORUM_APPLIED_GEQ, syncId2.toString()).size()).isEqualTo(1); Assertions.assertThat(execute(QUERY_REDUNDANT_BEFORE_FILTER_SHARD_APPLIED_GEQ, syncId1.toString()).size()).isEqualTo(1); Assertions.assertThat(execute(QUERY_REDUNDANT_BEFORE_FILTER_SHARD_APPLIED_GEQ, syncId2.toString()).size()).isEqualTo(0); + Assertions.assertThat(execute(QUERY_REDUNDANT_BEFORE_REMOTE, nodeId).size()).isGreaterThan(0); + Assertions.assertThat(execute(QUERY_REDUNDANT_BEFORE_FILTER_QUORUM_APPLIED_GEQ_REMOTE, nodeId, syncId1.toString()).size()).isEqualTo(2); + Assertions.assertThat(execute(QUERY_REDUNDANT_BEFORE_FILTER_QUORUM_APPLIED_GEQ_REMOTE, nodeId, syncId2.toString()).size()).isEqualTo(1); + Assertions.assertThat(execute(QUERY_REDUNDANT_BEFORE_FILTER_SHARD_APPLIED_GEQ_REMOTE, nodeId, syncId1.toString()).size()).isEqualTo(1); + Assertions.assertThat(execute(QUERY_REDUNDANT_BEFORE_FILTER_SHARD_APPLIED_GEQ_REMOTE, nodeId, syncId2.toString()).size()).isEqualTo(0); + } + + @Test + public void reportInvalidRequestForUnsupportedRemoteToLocal() + { + AccordService accord = accord(); + int nodeId = accord.nodeId().id; + TxnId id = accord.node().nextTxnIdWithDefaultFlags(Txn.Kind.Write, Routable.Domain.Key); + try + { + execute(ERASE_JOURNAL_REMOTE, nodeId, 1, id.toString()); + } + catch (ExceptionSerializer.RemoteException t) + { + Assertions.assertThat(t.originalClass).isEqualTo(InvalidRequestException.class.getName()); + } } @Test @@ -229,16 +379,67 @@ public void completedTxn() throws ExecutionException, InterruptedException { String tableName = createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH transactional_mode = 'full'"); AccordService accord = accord(); + int nodeId = accord.nodeId().id; TxnId id = accord.node().nextTxnIdWithDefaultFlags(Txn.Kind.Write, Routable.Domain.Key); Txn txn = createTxn(wrapInTxn(String.format("INSERT INTO %s.%s(k, c, v) VALUES (?, ?, ?)", KEYSPACE, tableName)), 0, 0, 0); String keyStr = txn.keys().get(0).toUnseekable().toString(); getBlocking(accord.node().coordinate(id, txn)); spinUntilSuccess(() -> assertRows(execute(QUERY_TXN_BLOCKED_BY, id.toString()), - row(id.toString(), KEYSPACE, tableName, anyInt(), 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, "Self", any(), null, anyOf(SaveStatus.ReadyToExecute.name(), SaveStatus.Applying.name(), SaveStatus.Applied.name())))); - spinUntilSuccess(() -> assertRows(execute(QUERY_TXN, id.toString()), row(id.toString(), "Applied"))); + row(id.toString(), anyInt(), 0, "", "", any(), anyOf(SaveStatus.ReadyToExecute.name(), SaveStatus.Applying.name(), SaveStatus.Applied.name())))); + assertRows(execute(QUERY_TXN, id.toString()), row(id.toString(), "Applied")); + assertRows(execute(QUERY_TXN_REMOTE, nodeId, id.toString()), row(id.toString(), "Applied")); assertRows(execute(QUERY_JOURNAL, id.toString()), row(id.toString(), "PreAccepted"), row(id.toString(), "Applying"), row(id.toString(), "Applied"), row(id.toString(), null)); + assertRows(execute(QUERY_JOURNAL_REMOTE, nodeId, id.toString()), row(id.toString(), "PreAccepted"), row(id.toString(), "Applying"), row(id.toString(), "Applied"), row(id.toString(), null)); assertRows(execute(QUERY_COMMANDS_FOR_KEY, keyStr), row(id.toString(), "APPLIED_DURABLE")); + assertRows(execute(QUERY_COMMANDS_FOR_KEY_REMOTE, nodeId, keyStr), row(id.toString(), "APPLIED_DURABLE")); + } + + @Test + public void manyTxns() + { + String tableName = createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH transactional_mode = 'full'"); + AccordService accord = accord(); + int nodeId = accord.nodeId().id; + List await = new ArrayList<>(); + Txn txn = createTxn(wrapInTxn(String.format("INSERT INTO %s.%s(k, c, v) VALUES (?, ?, ?)", KEYSPACE, tableName)), 0, 0, 0); + for (int i = 0 ; i < 100; ++i) + await.add(accord.coordinateAsync(0, 0, txn, ConsistencyLevel.QUORUM, new Dispatcher.RequestTime(Clock.Global.nanoTime()))); + + AccordCommandStore commandStore = (AccordCommandStore) accord.node().commandStores().unsafeForKey((RoutingKey) txn.keys().get(0).toUnseekable()); + await.forEach(IAccordService.IAccordResult::awaitAndGet); + + assertRows(execute(QUERY_TXNS, commandStore.id()), + row("Applied"), + row("Applied"), + row("Applied"), + row("Applied"), + row("Applied") + ); + + assertRows(execute(QUERY_TXNS_SEARCH, commandStore.id(), TxnId.NONE.toString()), + row("Applied"), + row("Applied"), + row("Applied"), + row("Applied"), + row("Applied") + ); + + assertRows(execute(QUERY_TXNS_REMOTE, nodeId, commandStore.id()), + row("Applied"), + row("Applied"), + row("Applied"), + row("Applied"), + row("Applied") + ); + + assertRows(execute(QUERY_TXNS_SEARCH_REMOTE, nodeId, commandStore.id(), TxnId.NONE.toString()), + row("Applied"), + row("Applied"), + row("Applied"), + row("Applied"), + row("Applied") + ); } @Test @@ -251,6 +452,7 @@ public void inflight() throws ExecutionException, InterruptedException { String tableName = createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH transactional_mode = 'full'"); AccordService accord = accord(); + int nodeId = accord.nodeId().id; TxnId id = accord.node().nextTxnIdWithDefaultFlags(Txn.Kind.Write, Routable.Domain.Key); String insertTxn = String.format("BEGIN TRANSACTION\n" + " LET r = (SELECT * FROM %s.%s WHERE k = ? AND c = ?);\n" + @@ -264,11 +466,14 @@ public void inflight() throws ExecutionException, InterruptedException filter.preAccept.awaitThrowUncheckedOnInterrupt(); assertRows(execute(QUERY_TXN_BLOCKED_BY, id.toString()), - row(id.toString(), KEYSPACE, tableName, anyInt(), 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, "Self", any(), null, anyOf(SaveStatus.PreAccepted.name(), SaveStatus.ReadyToExecute.name()))); - + row(id.toString(), anyInt(), 0, "", "", any(), anyOf(SaveStatus.PreAccepted.name(), SaveStatus.ReadyToExecute.name()))); + assertRows(execute(QUERY_TXN_BLOCKED_BY_REMOTE, nodeId, id.toString()), + row(nodeId, id.toString(), anyInt(), 0, "", "", any(), anyOf(SaveStatus.PreAccepted.name(), SaveStatus.ReadyToExecute.name()))); filter.apply.awaitThrowUncheckedOnInterrupt(); assertRows(execute(QUERY_TXN_BLOCKED_BY, id.toString()), - row(id.toString(), KEYSPACE, tableName, anyInt(), 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, "Self", any(), null, SaveStatus.ReadyToExecute.name())); + row(id.toString(), anyInt(), 0, "", "", any(), SaveStatus.ReadyToExecute.name())); + assertRows(execute(QUERY_TXN_BLOCKED_BY_REMOTE, nodeId, id.toString()), + row(nodeId, id.toString(), anyInt(), 0, "", "", any(), SaveStatus.ReadyToExecute.name())); } finally { @@ -288,6 +493,7 @@ public void blocked() throws ExecutionException, InterruptedException { String tableName = createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH transactional_mode = 'full'"); AccordService accord = accord(); + int nodeId = accord.nodeId().id; TxnId first = accord.node().nextTxnIdWithDefaultFlags(Txn.Kind.Write, Routable.Domain.Key); String insertTxn = String.format("BEGIN TRANSACTION\n" + " LET r = (SELECT * FROM %s.%s WHERE k = ? AND c = ?);\n" + @@ -300,12 +506,17 @@ public void blocked() throws ExecutionException, InterruptedException accord.node().coordinate(first, createTxn(insertTxn, 0, 0, 0, 0, 0)).beginAsResult(); filter.preAccept.awaitThrowUncheckedOnInterrupt(); - spinUntilSuccess(() ->assertRows(execute(QUERY_TXN_BLOCKED_BY, first.toString()), - row(first.toString(), KEYSPACE, tableName, anyInt(), 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, "Self", any(), null, anyOf(SaveStatus.PreAccepted.name(), SaveStatus.ReadyToExecute.name())))); - + assertRows(execute(QUERY_TXN_BLOCKED_BY, first.toString()), + row(first.toString(), anyInt(), 0, "", any(), any(), anyOf(SaveStatus.PreAccepted.name(), SaveStatus.ReadyToExecute.name()))); + assertRows(execute(QUERY_TXN_BLOCKED_BY_REMOTE, nodeId, first.toString()), + row(first.toString(), anyInt(), 0, "", any(), any(), anyOf(SaveStatus.PreAccepted.name(), SaveStatus.ReadyToExecute.name()))); filter.apply.awaitThrowUncheckedOnInterrupt(); - spinUntilSuccess(() -> assertRows(execute(QUERY_TXN_BLOCKED_BY, first.toString()), - row(first.toString(), KEYSPACE, tableName, anyInt(), 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, "Self", anyNonNull(), null, SaveStatus.ReadyToExecute.name()))); + assertRows(execute(QUERY_TXN_BLOCKED_BY, first.toString()), + row(first.toString(), anyInt(), 0, "", any(), anyNonNull(), SaveStatus.ReadyToExecute.name())); + assertRows(execute(QUERY_TXN_BLOCKED_BY_REMOTE, nodeId, first.toString()), + row(nodeId, first.toString(), anyInt(), 0, "", any(), anyNonNull(), SaveStatus.ReadyToExecute.name())); + + filter.reset(); TxnId second = accord.node().nextTxnIdWithDefaultFlags(Txn.Kind.Write, Routable.Domain.Key); filter.reset(); @@ -320,12 +531,15 @@ public void blocked() throws ExecutionException, InterruptedException return rs.size() == 2; }); assertRows(execute(QUERY_TXN_BLOCKED_BY, second.toString()), - row(second.toString(), KEYSPACE, tableName, anyInt(), 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, "Self", anyNonNull(), null, SaveStatus.Stable.name()), - row(second.toString(), KEYSPACE, tableName, anyInt(), 1, first.toString(), "Key", anyNonNull(), anyNonNull(), SaveStatus.ReadyToExecute.name())); - + row(second.toString(), anyInt(), 0, "", "", anyNonNull(), SaveStatus.Stable.name()), + row(second.toString(), anyInt(), 1, any(), first.toString(), anyNonNull(), SaveStatus.ReadyToExecute.name())); assertRows(execute(QUERY_TXN_BLOCKED_BY + " AND depth < 1", second.toString()), - row(second.toString(), KEYSPACE, tableName, anyInt(), 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, "Self", anyNonNull(), null, SaveStatus.Stable.name())); - + row(second.toString(), anyInt(), 0, any(), "", anyNonNull(), SaveStatus.Stable.name())); + assertRows(execute(QUERY_TXN_BLOCKED_BY_REMOTE, nodeId, second.toString()), + row(nodeId, second.toString(), anyInt(), 0, "", "", anyNonNull(), SaveStatus.Stable.name()), + row(nodeId, second.toString(), anyInt(), 1, any(), first.toString(), anyNonNull(), SaveStatus.ReadyToExecute.name())); + assertRows(execute(QUERY_TXN_BLOCKED_BY_REMOTE + " AND depth < 1", nodeId, second.toString()), + row(nodeId, second.toString(), anyInt(), 0, any(), "", anyNonNull(), SaveStatus.Stable.name())); } finally { @@ -436,9 +650,9 @@ public boolean test(Message msg, InetAddressAndPort to) if (!msg.verb().name().startsWith("ACCORD_")) return true; TxnId txnId = null; - if (msg.payload instanceof TxnRequest) + if (msg.payload instanceof NoWaitRequest) { - txnId = ((TxnRequest) msg.payload).txnId; + txnId = ((NoWaitRequest) msg.payload).txnId; if (applyTo != null && !applyTo.contains(txnId)) return true; } @@ -464,4 +678,6 @@ public boolean test(Message msg, InetAddressAndPort to) return !dropVerbs.contains(msg.verb()); } } + + } \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/index/accord/RouteIndexTest.java b/test/unit/org/apache/cassandra/index/accord/RouteIndexTest.java index d0c2d87f1eba..b467174c42be 100644 --- a/test/unit/org/apache/cassandra/index/accord/RouteIndexTest.java +++ b/test/unit/org/apache/cassandra/index/accord/RouteIndexTest.java @@ -30,6 +30,8 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import javax.annotation.Nullable; + +import accord.api.ConfigurationService.EpochReady; import accord.api.Journal; import accord.api.RoutingKey; import accord.local.CommandStores; @@ -238,7 +240,7 @@ private InsertTxn(int storeId, TxnId txnId, SaveStatus saveStatus, Route rout this.storeId = storeId; this.txnId = txnId; this.saveStatus = saveStatus; - this.participants = StoreParticipants.all(route); + this.participants = StoreParticipants.all(route, saveStatus); } @Override @@ -503,7 +505,7 @@ public State(RandomSource rs) storeRangesForEpochs.put(i, new RangesForEpoch(1, Ranges.of(TokenRange.fullRange(tableId, getPartitioner())))); accordService = startAccord(); - accordService.epochReady(ClusterMetadata.current().epoch).awaitUninterruptibly(); + accordService.epochReady(ClusterMetadata.current().epoch, EpochReady::reads).awaitUninterruptibly(); minDecidedIdNull = rs.nextFloat(); txnWriteFrequency = rs.pickInt(1, // every txn is a Write diff --git a/test/unit/org/apache/cassandra/service/accord/AccordCommandTest.java b/test/unit/org/apache/cassandra/service/accord/AccordCommandTest.java index b714b833f1a8..ac322403d15d 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordCommandTest.java +++ b/test/unit/org/apache/cassandra/service/accord/AccordCommandTest.java @@ -176,7 +176,7 @@ public void basicCycleTest() throws Throwable Command before = safeStore.ifInitialised(txnId).current(); Assert.assertEquals(commit.executeAt, before.executeAt()); Assert.assertTrue(before.hasBeen(Status.Committed)); - Assert.assertEquals(commit.partialDeps, before.partialDeps()); + Assert.assertEquals(commit.partialDeps(), before.partialDeps()); CommandsForKey cfk = safeStore.get(key(1).toUnseekable()).current(); Assert.assertTrue(cfk.indexOf(txnId) >= 0); diff --git a/test/unit/org/apache/cassandra/service/accord/AccordSyncPropagatorTest.java b/test/unit/org/apache/cassandra/service/accord/AccordSyncPropagatorTest.java index adaff55578d4..081aaeec473e 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordSyncPropagatorTest.java +++ b/test/unit/org/apache/cassandra/service/accord/AccordSyncPropagatorTest.java @@ -41,7 +41,7 @@ import org.junit.Test; import accord.api.Agent; -import accord.impl.AbstractConfigurationService; +import accord.impl.AbstractTestConfigurationService; import accord.impl.TestAgent; import accord.impl.basic.Pending; import accord.impl.basic.PendingQueue; @@ -410,7 +410,7 @@ public void unregisterFailureDetectionEventListener(IFailureDetectionEventListen } } - private class ConfigService extends AbstractConfigurationService.Minimal implements AccordSyncPropagator.Listener + private class ConfigService extends AbstractTestConfigurationService implements AccordSyncPropagator.Listener { private final Map> syncCompletes = new HashMap<>(); private final Map> endpointAcks = new HashMap<>(); @@ -436,7 +436,7 @@ public void fetchTopologyForEpoch(long epoch) } @Override - protected void localSyncComplete(Topology topology, boolean startSync) + protected void onReadyToCoordinate(Topology topology, boolean startSync) { Set notify = topology.nodes().stream().filter(i -> !localId.equals(i)).collect(Collectors.toSet()); instances.get(localId).propagator.reportSyncComplete(topology.epoch(), notify, localId); diff --git a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java index b9602dec07fc..d73763ba5aac 100644 --- a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java +++ b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java @@ -59,7 +59,7 @@ import accord.messages.BeginRecovery; import accord.messages.PreAccept; import accord.messages.Reply; -import accord.messages.TxnRequest; +import accord.messages.RouteRequest; import accord.primitives.AbstractUnseekableKeys; import accord.primitives.Ballot; import accord.primitives.EpochSupplier; @@ -421,9 +421,9 @@ public void checkFailures() throw error; } - public T process(TxnRequest request) throws ExecutionException, InterruptedException + public T process(RouteRequest request) throws ExecutionException, InterruptedException { - return process(request, request::apply); + return process(request, request); } public T process(PreLoadContext loadCtx, Function function) throws ExecutionException, InterruptedException @@ -433,9 +433,9 @@ public T process(PreLoadContext loadCtx, Function AsyncResult processAsync(TxnRequest request) + public AsyncResult processAsync(RouteRequest request) { - return processAsync(request, request::apply); + return processAsync(request, request); } public AsyncResult processAsync(PreLoadContext loadCtx, Function function) diff --git a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordTaskTest.java b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordTaskTest.java index d54058b72703..f18065adc96f 100644 --- a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordTaskTest.java +++ b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordTaskTest.java @@ -120,9 +120,9 @@ private static void test(RandomSource rs, int numSamples, TableMetadata tbl, Gen FullRoute route = txnWithRoute.right; PreAccept preAccept = new PreAccept(nodeId, instance.topologies, txnId, txn, null, false, route) { @Override - public PreAcceptReply apply(SafeCommandStore safeStore) + public PreAcceptReply applyInternal(SafeCommandStore safeStore) { - PreAcceptReply result = super.apply(safeStore); + PreAcceptReply result = super.applyInternal(safeStore); if (action == Action.FAILURE) throw new SimulatedFault("PreAccept failed for keys " + keys()); return result; diff --git a/test/unit/org/apache/cassandra/service/accord/serializers/CommandsForKeySerializerTest.java b/test/unit/org/apache/cassandra/service/accord/serializers/CommandsForKeySerializerTest.java index 884638da98dc..edca7e1c6b0c 100644 --- a/test/unit/org/apache/cassandra/service/accord/serializers/CommandsForKeySerializerTest.java +++ b/test/unit/org/apache/cassandra/service/accord/serializers/CommandsForKeySerializerTest.java @@ -28,6 +28,7 @@ import java.util.Random; import java.util.Set; import java.util.TreeSet; +import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; import java.util.function.BooleanSupplier; import java.util.function.Consumer; @@ -49,6 +50,7 @@ import accord.api.DataStore; import accord.api.Journal; import accord.api.Key; +import accord.api.OwnershipEventListener; import accord.api.ProgressLog; import accord.api.RoutingKey; import accord.api.Timeouts; @@ -657,8 +659,8 @@ protected TestCommandStore() @Override public Agent agent() { return this; } @Override public void execute(Runnable run) {} @Override public void shutdown() { } - @Override public void onFailedBootstrap(int attempts, String phase, Ranges ranges, Runnable retry, Throwable failure) { throw new UnsupportedOperationException(); } - @Override public void onStale(Timestamp staleSince, Ranges ranges) { throw new UnsupportedOperationException(); } + @Override public AsyncChain chain(Callable call) { throw new UnsupportedOperationException(); } + @Override public OwnershipEventListener ownershipEvents() { return null; } @Override public void onUncaughtException(Throwable t) { throw new UnsupportedOperationException(); } @Override public void onCaughtException(Throwable t, String context) { throw new UnsupportedOperationException(); } @Override public boolean rejectPreAccept(TimeService time, TxnId txnId) { throw new UnsupportedOperationException(); } diff --git a/test/unit/org/apache/cassandra/utils/AccordGenerators.java b/test/unit/org/apache/cassandra/utils/AccordGenerators.java index bd994fc27a10..2c8f292b6814 100644 --- a/test/unit/org/apache/cassandra/utils/AccordGenerators.java +++ b/test/unit/org/apache/cassandra/utils/AccordGenerators.java @@ -90,7 +90,7 @@ import static accord.local.CommandStores.RangesForEpoch; import static accord.local.RedundantStatus.Property.GC_BEFORE; -import static accord.local.RedundantStatus.Property.PRE_BOOTSTRAP; +import static accord.local.RedundantStatus.Property.UNREADY; import static accord.local.RedundantStatus.SomeStatus.LOCALLY_APPLIED_ONLY; import static accord.local.RedundantStatus.SomeStatus.LOCALLY_WITNESSED_ONLY; import static accord.local.RedundantStatus.SomeStatus.SHARD_APPLIED_ONLY; @@ -276,7 +276,7 @@ private ICommand attributes(SaveStatus saveStatus) if (saveStatus.known.deps().hasPreAcceptedOrProposedOrDecidedDeps()) builder.partialDeps(partialDeps); - builder.setParticipants(StoreParticipants.all(route)); + builder.setParticipants(StoreParticipants.all(route, saveStatus)); builder.durability(NotDurable); if (saveStatus.compareTo(SaveStatus.PreAccepted) >= 0) builder.executeAt(executeAt); @@ -601,9 +601,9 @@ public static Gen redundantBeforeEntry(Gen emptyGen, Gen if (rs.nextBoolean()) bounds.add(Bounds.create(range, txnIdGen.next(rs).addFlag(SHARD_BOUND), oneSlow(GC_BEFORE), null )); if (rs.nextBoolean()) - bounds.add(Bounds.create(range, txnIdGen.next(rs), oneSlow(PRE_BOOTSTRAP), null )); + bounds.add(Bounds.create(range, txnIdGen.next(rs), oneSlow(UNREADY), null )); if (rs.nextBoolean()) - bounds.add(new Bounds(range, Long.MIN_VALUE, Long.MAX_VALUE, new TxnId[0], new short[0], txnIdGen.next(rs))); + bounds.add(new Bounds(range, Long.MIN_VALUE, Long.MAX_VALUE, new TxnId[0], new int[0], txnIdGen.next(rs))); Collections.shuffle(bounds); long endEpoch = emptyGen.next(rs) ? Long.MAX_VALUE : rs.nextLong(0, Long.MAX_VALUE); @@ -618,7 +618,7 @@ public static Gen redundantBeforeEntry(Gen emptyGen, Gen } long startEpoch = rs.nextLong(Math.min(minEpoch, endEpoch)); - Bounds epochBounds = new Bounds(range, startEpoch, endEpoch, new TxnId[0], new short[0], null); + Bounds epochBounds = new Bounds(range, startEpoch, endEpoch, new TxnId[0], new int[0], null); if (result == null) return epochBounds; return Bounds.reduce(result, epochBounds);