From e0ea87253a6911542295569741fd9f36bd3c67de Mon Sep 17 00:00:00 2001 From: Jonathan Ellis Date: Tue, 19 Nov 2024 15:39:18 -0600 Subject: [PATCH] CNDB-11725 add SYNTHETIC ColumnMetadata.Kind to represent the score column --- .../cql3/selection/ColumnFilterFactory.java | 16 ++- .../cassandra/cql3/selection/Selection.java | 23 ++++- .../cql3/statements/SelectStatement.java | 18 ++-- src/java/org/apache/cassandra/db/Columns.java | 99 ++++++++++++++++--- .../org/apache/cassandra/db/ReadCommand.java | 20 +++- .../cassandra/db/RegularAndStaticColumns.java | 2 +- .../cassandra/db/filter/ColumnFilter.java | 25 +++-- .../plan/StorageAttachedIndexSearcher.java | 11 ++- .../cassandra/schema/ColumnMetadata.java | 35 +++++-- .../cassandra/schema/TableMetadata.java | 6 +- .../index/sai/StorageAttachedIndexTest.java | 8 -- 11 files changed, 196 insertions(+), 67 deletions(-) diff --git a/src/java/org/apache/cassandra/cql3/selection/ColumnFilterFactory.java b/src/java/org/apache/cassandra/cql3/selection/ColumnFilterFactory.java index 63fa0520101e..00225cca4108 100644 --- a/src/java/org/apache/cassandra/cql3/selection/ColumnFilterFactory.java +++ b/src/java/org/apache/cassandra/cql3/selection/ColumnFilterFactory.java @@ -38,9 +38,21 @@ abstract class ColumnFilterFactory */ abstract ColumnFilter newInstance(List selectors); - public static ColumnFilterFactory wildcard(TableMetadata table) + public static ColumnFilterFactory wildcard(TableMetadata table, Set orderingColumns) { - return new PrecomputedColumnFilter(ColumnFilter.all(table)); + ColumnFilter cf; + if (orderingColumns.isEmpty()) + { + cf = ColumnFilter.all(table); + } + else + { + ColumnFilter.Builder builder = ColumnFilter.selectionBuilder(); + builder.addAll(table.regularAndStaticColumns()); + builder.addAll(orderingColumns); + cf = builder.build(); + } + return new PrecomputedColumnFilter(cf); } public static ColumnFilterFactory fromColumns(TableMetadata table, diff --git a/src/java/org/apache/cassandra/cql3/selection/Selection.java b/src/java/org/apache/cassandra/cql3/selection/Selection.java index 02aae61dd5ff..12d8aa014e19 100644 --- a/src/java/org/apache/cassandra/cql3/selection/Selection.java +++ b/src/java/org/apache/cassandra/cql3/selection/Selection.java @@ -43,10 +43,24 @@ public abstract class Selection private static final Predicate STATIC_COLUMN_FILTER = (column) -> column.isStatic(); private final TableMetadata table; + + // Full list of columns needed for processing the query, including selected columns, ordering columns, + // and columns needed for restrictions. Wildcard columns are fully materialized here. + // + // This also includes synthetic columns, because unlike all the other not-physical-columns selectables, they are + // computed on the replica instead of the coordinator and so, like physical columns, they need to be sent back + // as part of the result. private final List columns; + + // maps ColumnSpecifications (columns, function calls, aliases) to the columns backing them private final SelectionColumnMapping columnMapping; + + // metadata matching the ColumnSpcifications protected final ResultSet.ResultMetadata metadata; + + // creates a ColumnFilter that breaks columns into `queried` and `fetched` protected final ColumnFilterFactory columnFilterFactory; + protected final boolean isJson; // Columns used to order the result set for JSON queries with post ordering. @@ -126,10 +140,15 @@ public ResultSet.ResultMetadata getResultMetadata() } public static Selection wildcard(TableMetadata table, boolean isJson, boolean returnStaticContentOnPartitionWithNoRows) + { + return wildcard(table, Collections.emptySet(), isJson, returnStaticContentOnPartitionWithNoRows); + } + + public static Selection wildcard(TableMetadata table, Set orderingColumns, boolean isJson, boolean returnStaticContentOnPartitionWithNoRows) { List all = new ArrayList<>(table.columns().size()); Iterators.addAll(all, table.allColumnsInSelectOrder()); - return new SimpleSelection(table, all, Collections.emptySet(), true, isJson, returnStaticContentOnPartitionWithNoRows); + return new SimpleSelection(table, all, orderingColumns, true, isJson, returnStaticContentOnPartitionWithNoRows); } public static Selection wildcardWithGroupBy(TableMetadata table, @@ -400,7 +419,7 @@ public SimpleSelection(TableMetadata table, selectedColumns, orderingColumns, SelectionColumnMapping.simpleMapping(selectedColumns), - isWildcard ? ColumnFilterFactory.wildcard(table) + isWildcard ? ColumnFilterFactory.wildcard(table, orderingColumns) : ColumnFilterFactory.fromColumns(table, selectedColumns, orderingColumns, Collections.emptySet(), returnStaticContentOnPartitionWithNoRows), isWildcard, isJson); diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java index 3ab85e5adf6d..eb1b5f9fa0ba 100644 --- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java @@ -1080,12 +1080,16 @@ void processPartition(RowIterator partition, QueryOptions options, ResultSetBuil case CLUSTERING: result.add(row.clustering().bufferAt(def.position())); break; + case SYNTHETIC: + // treat as REGULAR case REGULAR: result.add(row.getColumnData(def), nowInSec); break; case STATIC: result.add(staticRow.getColumnData(def), nowInSec); break; + default: + throw new AssertionError(); } } } @@ -1159,7 +1163,7 @@ public SelectStatement prepare(boolean forView, UnaryOperator keyspaceMa // Besides actual restrictions (where clauses), prepareRestrictions will include pseudo-restrictions // on indexed columns to allow pushing ORDER BY into the index; see StatementRestrictions::addOrderingRestrictions. - // Therefore, we don't want to convert the Ordering column into a +score column until after that. + // Therefore, we don't want to convert an ANN Ordering column into a +score column until after that. List orderings = getOrderings(table); StatementRestrictions restrictions = prepareRestrictions( table, bindVariables, orderings, containsOnlyStaticColumns, forView); @@ -1235,14 +1239,8 @@ private Map getScoreOrdering(List orderings) return null; // Create synthetic score column - // Use the original column's table metadata but create new identifier and type ColumnMetadata sourceColumn = expr.getColumn(); - var cm = new ColumnMetadata(sourceColumn.ksName, - sourceColumn.cfName, - new ColumnIdentifier("+score", true), - FloatType.instance, - ColumnMetadata.NO_POSITION, - ColumnMetadata.Kind.REGULAR); + var cm = ColumnMetadata.syntheticColumn(sourceColumn.ksName, sourceColumn.cfName, ColumnMetadata.SYNTHETIC_SCORE_ID, FloatType.instance); return Map.of(cm, orderings.get(0)); } @@ -1264,7 +1262,7 @@ private Selection prepareSelection(TableMetadata table, if (selectables.isEmpty()) // wildcard query { return hasGroupBy ? Selection.wildcardWithGroupBy(table, boundNames, parameters.isJson, restrictions.returnStaticContentOnPartitionWithNoRows()) - : Selection.wildcard(table, parameters.isJson, restrictions.returnStaticContentOnPartitionWithNoRows()); + : Selection.wildcard(table, resultSetOrderingColumns, parameters.isJson, restrictions.returnStaticContentOnPartitionWithNoRows()); } return Selection.fromSelectors(table, @@ -1502,7 +1500,7 @@ private boolean isReversed(TableMetadata table, Map or { ColumnMetadata def = entry.getKey(); Ordering ordering = entry.getValue(); - // We defined ANN OF to be ASC ordering, as in, "order by near-ness". But since score goves from + // We defined ANN OF to be ASC ordering, as in, "order by near-ness". But since score goes from // 0 (worst) to 1 (closest), we need to reverse the ordering for the comparator when we're sorting // by synthetic +score column. boolean cqlReversed = ordering.direction == Ordering.Direction.DESC; diff --git a/src/java/org/apache/cassandra/db/Columns.java b/src/java/org/apache/cassandra/db/Columns.java index 7ce9bd68cf15..cef03393a1db 100644 --- a/src/java/org/apache/cassandra/db/Columns.java +++ b/src/java/org/apache/cassandra/db/Columns.java @@ -28,6 +28,7 @@ import net.nicoulaj.compilecommand.annotations.DontInline; import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.SetType; import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.db.rows.ColumnData; @@ -36,6 +37,7 @@ import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.serializers.AbstractTypeSerializer; import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.ObjectSizes; import org.apache.cassandra.utils.SearchIterator; @@ -334,6 +336,11 @@ public Iterator simpleColumns() return BTree.iterator(columns, 0, complexIdx - 1, BTree.Dir.ASC); } + public Iterator simpleColumnsDesc() + { + return BTree.iterator(columns, 0, complexIdx - 1, BTree.Dir.DESC); + } + /** * Iterator over the complex columns of this object. * @@ -459,42 +466,112 @@ public String toString() public static class Serializer { + AbstractTypeSerializer typeSerializer = new AbstractTypeSerializer(); + public void serialize(Columns columns, DataOutputPlus out) throws IOException { - out.writeUnsignedVInt(columns.size()); + int regularCount = 0; + int syntheticCount = 0; + + // Count regular and synthetic columns + for (ColumnMetadata column : columns) + { + if (column.isSynthetic()) + syntheticCount++; + else + regularCount++; + } + + // Jam the two counts into a single value to avoid massive backwards compatibility issues + long packedCount = getPackedCount(syntheticCount, regularCount); + out.writeUnsignedVInt(packedCount); + + // First pass - write regular columns for (ColumnMetadata column : columns) - ByteBufferUtil.writeWithVIntLength(column.name.bytes, out); + { + if (!column.isSynthetic()) + ByteBufferUtil.writeWithVIntLength(column.name.bytes, out); + } + + // Second pass - write synthetic columns with their full metadata + for (ColumnMetadata column : columns) + { + if (column.isSynthetic()) + { + ByteBufferUtil.writeWithVIntLength(column.name.bytes, out); + typeSerializer.serialize(column.type, out); + } + } + } + + private static long getPackedCount(int syntheticCount, int regularCount) + { + // Left shift of 20 gives us over 1M regular columns, and up to 4 synthetic columns + // before overflowing to a 4th byte. + return ((long) syntheticCount << 20) | regularCount; } public long serializedSize(Columns columns) { - long size = TypeSizes.sizeofUnsignedVInt(columns.size()); + int regularCount = 0; + int syntheticCount = 0; + long size = 0; + + // Count and calculate sizes for (ColumnMetadata column : columns) - size += ByteBufferUtil.serializedSizeWithVIntLength(column.name.bytes); - return size; + { + if (column.isSynthetic()) + { + syntheticCount++; + size += ByteBufferUtil.serializedSizeWithVIntLength(column.name.bytes); + size += typeSerializer.serializedSize(column.type); + } + else + { + regularCount++; + size += ByteBufferUtil.serializedSizeWithVIntLength(column.name.bytes); + } + } + + return TypeSizes.sizeofUnsignedVInt(getPackedCount(syntheticCount, regularCount)) + + size; } public Columns deserialize(DataInputPlus in, TableMetadata metadata) throws IOException { - int length = (int)in.readUnsignedVInt(); try (BTree.FastBuilder builder = BTree.fastBuilder()) { - for (int i = 0; i < length; i++) + long packedCount = in.readUnsignedVInt() ; + int regularCount = (int) (packedCount & 0xFFFFF); + int syntheticCount = (int) (packedCount >> 20); + + // First pass - regular columns + for (int i = 0; i < regularCount; i++) { ByteBuffer name = ByteBufferUtil.readWithVIntLength(in); ColumnMetadata column = metadata.getColumn(name); if (column == null) { - // If we don't find the definition, it could be we have data for a dropped column, and we shouldn't - // fail deserialization because of that. So we grab a "fake" ColumnMetadata that ensure proper - // deserialization. The column will be ignore later on anyway. + // If we don't find the definition, it could be we have data for a dropped column column = metadata.getDroppedColumn(name); - if (column == null) throw new RuntimeException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization"); } builder.add(column); } + + // Second pass - synthetic columns + for (int i = 0; i < syntheticCount; i++) + { + ByteBuffer name = ByteBufferUtil.readWithVIntLength(in); + AbstractType type = typeSerializer.deserialize(in); + + if (!name.equals(ColumnMetadata.SYNTHETIC_SCORE_ID.bytes)) + throw new IllegalStateException("Unknown synthetic column " + UTF8Type.instance.getString(name)); + + ColumnMetadata column = ColumnMetadata.syntheticColumn(metadata.keyspace, metadata.name, ColumnMetadata.SYNTHETIC_SCORE_ID, type); + builder.add(column); + } return new Columns(builder.build()); } } diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java index d872147c0d0f..a289987a1adc 100644 --- a/src/java/org/apache/cassandra/db/ReadCommand.java +++ b/src/java/org/apache/cassandra/db/ReadCommand.java @@ -72,6 +72,7 @@ import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessageFlag; import org.apache.cassandra.net.Verb; +import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.IndexMetadata; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.SchemaConstants; @@ -413,9 +414,9 @@ public UnfilteredPartitionIterator executeLocally(ReadExecutionController execut } Context context = Context.from(this); - UnfilteredPartitionIterator iterator = (null == searcher) ? Transformation.apply(queryStorage(cfs, executionController), new TrackingRowIterator(context)) - : Transformation.apply(searchStorage(searcher, executionController), new TrackingRowIterator(context)); - + var storageTarget = (null == searcher) ? queryStorage(cfs, executionController) + : searchStorage(searcher, executionController); + UnfilteredPartitionIterator iterator = Transformation.apply(storageTarget, new TrackingRowIterator(context)); iterator = RTBoundValidator.validate(iterator, Stage.MERGED, false); try @@ -1047,6 +1048,19 @@ public ReadCommand deserialize(DataInputPlus in, int version) throws IOException TableMetadata metadata = schema.getExistingTableMetadata(TableId.deserialize(in)); int nowInSec = in.readInt(); ColumnFilter columnFilter = ColumnFilter.serializer.deserialize(in, version, metadata); + + // add synthetic columns to the tablemetadata so we can serialize them in our response + var tmb = metadata.unbuild(); + for (var it = columnFilter.fetchedColumns().regulars.simpleColumnsDesc(); it.hasNext(); ) + { + var c = it.next(); + // synthetic columns sort last, so when we hit the first non-synthetic, we're done + if (!c.isSynthetic()) + break; + tmb.addColumn(ColumnMetadata.syntheticColumn(c.ksName, c.cfName, c.name, c.type)); + } + metadata = tmb.build(); + RowFilter rowFilter = RowFilter.serializer.deserialize(in, version, metadata); DataLimits limits = DataLimits.serializer.deserialize(in, version, metadata.comparator); Index.QueryPlan indexQueryPlan = null; diff --git a/src/java/org/apache/cassandra/db/RegularAndStaticColumns.java b/src/java/org/apache/cassandra/db/RegularAndStaticColumns.java index 05c5251c34e8..55533eda0e97 100644 --- a/src/java/org/apache/cassandra/db/RegularAndStaticColumns.java +++ b/src/java/org/apache/cassandra/db/RegularAndStaticColumns.java @@ -163,7 +163,7 @@ public Builder add(ColumnMetadata c) } else { - assert c.isRegular(); + assert c.isRegular() || c.isSynthetic(); if (regularColumns == null) regularColumns = BTree.builder(naturalOrder()); regularColumns.add(c); diff --git a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java index d9a1b9d4e51a..2d4b240270a8 100644 --- a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java +++ b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java @@ -103,7 +103,8 @@ boolean fetchesAllColumns(boolean isStatic) @Override RegularAndStaticColumns getFetchedColumns(TableMetadata metadata, RegularAndStaticColumns queried) { - return metadata.regularAndStaticColumns(); + var merged = queried.regulars.mergeTo(metadata.regularColumns()); + return new RegularAndStaticColumns(metadata.staticColumns(), merged); } }, @@ -124,7 +125,8 @@ boolean fetchesAllColumns(boolean isStatic) @Override RegularAndStaticColumns getFetchedColumns(TableMetadata metadata, RegularAndStaticColumns queried) { - return new RegularAndStaticColumns(queried.statics, metadata.regularColumns()); + var merged = queried.regulars.mergeTo(metadata.regularColumns()); + return new RegularAndStaticColumns(queried.statics, merged); } }, @@ -295,14 +297,16 @@ public static ColumnFilter selection(TableMetadata metadata, } /** - * The columns that needs to be fetched internally for this filter. + * The columns that needs to be fetched internally. See FetchingStrategy for why this is + * always a superset of the queried columns. * * @return the columns to fetch for this filter. */ public abstract RegularAndStaticColumns fetchedColumns(); /** - * The columns actually queried by the user. + * The columns needed to process the query, including selected columns, ordering columns, + * restriction (predicate) columns, and synthetic columns. *

* Note that this is in general not all the columns that are fetched internally (see {@link #fetchedColumns}). */ @@ -619,9 +623,7 @@ private SortedSetMultimap buildSubSelectio */ public static class WildCardColumnFilter extends ColumnFilter { - /** - * The queried and fetched columns. - */ + // for wildcards, there is no distinction between fetched and queried because queried is already "everything" private final RegularAndStaticColumns fetchedAndQueried; /** @@ -739,14 +741,9 @@ public static class SelectionColumnFilter extends ColumnFilter { public final FetchingStrategy fetchingStrategy; - /** - * The selected columns - */ + // Materializes the columns required to implement queriedColumns() and fetchedColumns(), + // see the comments to superclass's methods private final RegularAndStaticColumns queried; - - /** - * The columns that need to be fetched to be able - */ private final RegularAndStaticColumns fetched; private final SortedSetMultimap subSelections; // can be null diff --git a/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java b/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java index 774f06745110..7dd620be9ebe 100644 --- a/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java +++ b/src/java/org/apache/cassandra/index/sai/plan/StorageAttachedIndexSearcher.java @@ -585,7 +585,8 @@ public static class PrimaryKeyIterator extends AbstractUnfilteredRowIterator private final Unfiltered row; public final PrimaryKeyWithSortKey primaryKeyWithSortKey; - public PrimaryKeyIterator(PrimaryKeyWithSortKey key, UnfilteredRowIterator partition, Row staticRow, Unfiltered content) { + public PrimaryKeyIterator(PrimaryKeyWithSortKey key, UnfilteredRowIterator partition, Row staticRow, Unfiltered content) + { super(partition.metadata(), partition.partitionKey(), partition.partitionLevelDeletion(), @@ -602,15 +603,14 @@ public PrimaryKeyIterator(PrimaryKeyWithSortKey key, UnfilteredRowIterator parti return; } - var tm = metadata(); - var scoreColumn = ColumnMetadata.regularColumn(tm.keyspace, tm.name, "+score", FloatType.instance); - // clone the original Row Row originalRow = (Row) content; ArrayList columnData = new ArrayList<>(originalRow.columnCount() + 1); columnData.addAll(originalRow.columnData()); // inject +score as a new column + var tm = metadata(); + var scoreColumn = ColumnMetadata.syntheticColumn(tm.keyspace, tm.name, ColumnMetadata.SYNTHETIC_SCORE_ID, FloatType.instance); var pkWithScore = (PrimaryKeyWithScore) key; columnData.add(BufferCell.live(scoreColumn, FBUtilities.nowInSeconds(), @@ -626,7 +626,8 @@ public PrimaryKeyIterator(PrimaryKeyWithSortKey key, UnfilteredRowIterator parti } @Override - protected Unfiltered computeNext() { + protected Unfiltered computeNext() + { if (consumed) return endOfData(); consumed = true; diff --git a/src/java/org/apache/cassandra/schema/ColumnMetadata.java b/src/java/org/apache/cassandra/schema/ColumnMetadata.java index 38784e72acb5..2a59ee3ebf58 100644 --- a/src/java/org/apache/cassandra/schema/ColumnMetadata.java +++ b/src/java/org/apache/cassandra/schema/ColumnMetadata.java @@ -71,9 +71,9 @@ public enum ClusteringOrder /** * The type of CQL3 column this definition represents. - * There is 4 main type of CQL3 columns: those parts of the partition key, - * those parts of the clustering columns and amongst the others, regular and - * static ones. + * There are 5 types of columns: those parts of the partition key, + * those parts of the clustering columns and amongst the others, regular, + * static, and synthetic ones. * * IMPORTANT: this enum is serialized as toString() and deserialized by calling * Kind.valueOf(), so do not override toString() or rename existing values. @@ -84,15 +84,17 @@ public enum Kind PARTITION_KEY, CLUSTERING, REGULAR, - STATIC; + STATIC, + SYNTHETIC; public boolean isPrimaryKeyKind() { return this == PARTITION_KEY || this == CLUSTERING; } - } + public static final ColumnIdentifier SYNTHETIC_SCORE_ID = ColumnIdentifier.getInterned("+:!score", true); + /** * Whether this is a dropped column. */ @@ -121,10 +123,17 @@ public boolean isPrimaryKeyKind() */ private final long comparisonOrder; + /** + * Bit layout (from most to least significant): + * - Bits 61-63: Kind ordinal (3 bits, supporting up to 8 Kind values) + * - Bit 60: isComplex flag + * - Bits 48-59: position (12 bits, see assert) + * - Bits 0-47: name.prefixComparison (shifted right by 16) + */ private static long comparisonOrder(Kind kind, boolean isComplex, long position, ColumnIdentifier name) { assert position >= 0 && position < 1 << 12; - return (((long) kind.ordinal()) << 61) + return (((long) kind.ordinal()) << 61) | (isComplex ? 1L << 60 : 0) | (position << 48) | (name.prefixComparison >>> 16); @@ -170,6 +179,14 @@ public static ColumnMetadata staticColumn(String keyspace, String table, String return new ColumnMetadata(keyspace, table, ColumnIdentifier.getInterned(name, true), type, NO_POSITION, Kind.STATIC); } + /** + * Creates a new synthetic column metadata instance. + */ + public static ColumnMetadata syntheticColumn(String keyspace, String table, ColumnIdentifier id, AbstractType type) + { + return new ColumnMetadata(keyspace, table, id, type, NO_POSITION, Kind.SYNTHETIC); + } + /** * Rebuild the metadata for a dropped column from its recorded data. * @@ -225,6 +242,7 @@ public ColumnMetadata(String ksName, this.kind = kind; this.position = position; this.cellPathComparator = makeCellPathComparator(kind, type); + assert kind != Kind.SYNTHETIC || cellPathComparator == null; this.cellComparator = cellPathComparator == null ? ColumnData.comparator : new Comparator>() { @Override @@ -593,6 +611,11 @@ public boolean isCounterColumn() return type.isCounter(); } + public boolean isSynthetic() + { + return kind == Kind.SYNTHETIC; + } + public Selector.Factory newSelectorFactory(TableMetadata table, AbstractType expectedType, List defs, VariableSpecifications boundNames) throws InvalidRequestException { return SimpleSelector.newFactory(this, addAndGetIndex(this, defs)); diff --git a/src/java/org/apache/cassandra/schema/TableMetadata.java b/src/java/org/apache/cassandra/schema/TableMetadata.java index 6442326c9a2d..8885b85fdd43 100644 --- a/src/java/org/apache/cassandra/schema/TableMetadata.java +++ b/src/java/org/apache/cassandra/schema/TableMetadata.java @@ -167,9 +167,6 @@ protected TableMetadata(Builder builder) name = builder.name; id = builder.id; - // FIXME - builder.addColumn(new ColumnMetadata(keyspace, name, new ColumnIdentifier("+score", true), FloatType.instance, ColumnMetadata.NO_POSITION, ColumnMetadata.Kind.REGULAR)); - partitioner = builder.partitioner; kind = builder.kind; params = builder.params.build(); @@ -1125,8 +1122,7 @@ public Builder addStaticColumn(ColumnIdentifier name, AbstractType type) public Builder addColumn(ColumnMetadata column) { - if (columns.containsKey(column.name.bytes)) - return this; // FIXME + assert !columns.containsKey(column.name.bytes) : column.name + " is already present"; switch (column.kind) { diff --git a/test/unit/org/apache/cassandra/index/sai/StorageAttachedIndexTest.java b/test/unit/org/apache/cassandra/index/sai/StorageAttachedIndexTest.java index f475e466e28c..7538d65a4d88 100644 --- a/test/unit/org/apache/cassandra/index/sai/StorageAttachedIndexTest.java +++ b/test/unit/org/apache/cassandra/index/sai/StorageAttachedIndexTest.java @@ -135,14 +135,6 @@ private static ByteBuffer floatVectorToByteBuffer(CQLTester.Vector vector @Test public void testOrderResults() { - QueryOptions queryOptions = QueryOptions.create(ConsistencyLevel.ONE, - byteBufferList, - false, - PageSize.inRows(1), - null, - null, - ProtocolVersion.CURRENT, - KEYSPACE); List> rows = new ArrayList<>(); rows.add(byteBufferList);