blambov commented on code in PR #4402:
URL: https://github.com/apache/cassandra/pull/4402#discussion_r2426224236
##########
src/java/org/apache/cassandra/db/ClusteringComparator.java:
##########
@@ -156,6 +159,126 @@ public <V1, V2> int compare(ClusteringPrefix<V1> c1,
ClusteringPrefix<V2> c2)
return s1 < s2 ? c1.kind().comparedToClustering :
-c2.kind().comparedToClustering;
}
+ public static int compare(ClusteringDescriptor c1, ClusteringDescriptor c2)
+ {
+ final int c1Size = c1.clusteringColumnsBound();
+ final int c2Size = c2.clusteringColumnsBound();
+ final int minColumns = Math.min(c1Size, c2Size);
+
+ final int cmp = compare(c1.clusteringTypes(), c1.clusteringBuffer(),
c2.clusteringBuffer(), minColumns);
+ if (cmp != 0)
+ return cmp;
+
+ final ClusteringPrefix.Kind c1Kind = c1.clusteringKind();
+ final ClusteringPrefix.Kind c2Kind = c2.clusteringKind();
+ if (c1Size == c2Size)
+ {
+ return ClusteringPrefix.Kind.compare(c1Kind, c2Kind);
+ }
+
+ return c1Size < c2Size ? c1Kind.comparedToClustering :
-c2Kind.comparedToClustering;
+ }
+
+ public static int compare(AbstractType<?>[] types, ByteBuffer c1,
ByteBuffer c2) {
+ return compare(types, c1, c2, types.length);
+ }
+
+ private static int compare(AbstractType<?>[] types, ByteBuffer c1,
ByteBuffer c2, int size)
+ {
+ long clusteringBlock1 = 0;
+ long clusteringBlock2 = 0;
+ final int position1 = c1.position();
+ final int position2 = c2.position();
+ final int limit1 = c1.limit();
+ final int limit2 = c2.limit();
+ try
+ {
+ int ofst1 = position1;
+ int ofst2 = position2;
+ for (int clusteringIndex = 0; clusteringIndex < size;
clusteringIndex++)
+ {
+ if (clusteringIndex % 32 == 0)
+ {
+ clusteringBlock1 = VIntCoding.getUnsignedVInt(c1, ofst1,
limit1);
+ ofst1 +=
VIntCoding.computeUnsignedVIntSize(clusteringBlock1);
+ clusteringBlock2 = VIntCoding.getUnsignedVInt(c2, ofst2,
limit2);
+ ofst2 +=
VIntCoding.computeUnsignedVIntSize(clusteringBlock2);
+ }
+
+ AbstractType<?> type = types[clusteringIndex];
+
+ boolean v1Present = (clusteringBlock1 & 0x11) == 0;
Review Comment:
Shouldn't this be `0b11`?
##########
src/java/org/apache/cassandra/db/ClusteringComparator.java:
##########
@@ -156,6 +159,126 @@ public <V1, V2> int compare(ClusteringPrefix<V1> c1,
ClusteringPrefix<V2> c2)
return s1 < s2 ? c1.kind().comparedToClustering :
-c2.kind().comparedToClustering;
}
+ public static int compare(ClusteringDescriptor c1, ClusteringDescriptor c2)
+ {
+ final int c1Size = c1.clusteringColumnsBound();
+ final int c2Size = c2.clusteringColumnsBound();
+ final int minColumns = Math.min(c1Size, c2Size);
+
+ final int cmp = compare(c1.clusteringTypes(), c1.clusteringBuffer(),
c2.clusteringBuffer(), minColumns);
+ if (cmp != 0)
+ return cmp;
+
+ final ClusteringPrefix.Kind c1Kind = c1.clusteringKind();
+ final ClusteringPrefix.Kind c2Kind = c2.clusteringKind();
+ if (c1Size == c2Size)
+ {
+ return ClusteringPrefix.Kind.compare(c1Kind, c2Kind);
+ }
+
+ return c1Size < c2Size ? c1Kind.comparedToClustering :
-c2Kind.comparedToClustering;
+ }
+
+ public static int compare(AbstractType<?>[] types, ByteBuffer c1,
ByteBuffer c2) {
+ return compare(types, c1, c2, types.length);
+ }
+
+ private static int compare(AbstractType<?>[] types, ByteBuffer c1,
ByteBuffer c2, int size)
+ {
+ long clusteringBlock1 = 0;
+ long clusteringBlock2 = 0;
+ final int position1 = c1.position();
+ final int position2 = c2.position();
+ final int limit1 = c1.limit();
+ final int limit2 = c2.limit();
+ try
+ {
+ int ofst1 = position1;
+ int ofst2 = position2;
+ for (int clusteringIndex = 0; clusteringIndex < size;
clusteringIndex++)
+ {
+ if (clusteringIndex % 32 == 0)
+ {
+ clusteringBlock1 = VIntCoding.getUnsignedVInt(c1, ofst1,
limit1);
+ ofst1 +=
VIntCoding.computeUnsignedVIntSize(clusteringBlock1);
+ clusteringBlock2 = VIntCoding.getUnsignedVInt(c2, ofst2,
limit2);
+ ofst2 +=
VIntCoding.computeUnsignedVIntSize(clusteringBlock2);
+ }
+
+ AbstractType<?> type = types[clusteringIndex];
+
+ boolean v1Present = (clusteringBlock1 & 0x11) == 0;
+ boolean v2Present = (clusteringBlock2 & 0x11) == 0;
+
+ if (v1Present && v2Present)
+ {
+ boolean isByteOrderComparable = type.isByteOrderComparable;
+ int vlen1,vlen2;
+ if (type.isValueLengthFixed())
+ {
+ vlen1 = vlen2 = type.valueLengthIfFixed();
+ }
+ else
+ {
+ vlen1 = VIntCoding.getUnsignedVInt32(c1, ofst1,
limit1);
+ ofst1 += VIntCoding.computeUnsignedVIntSize(vlen1);
+ vlen2 = VIntCoding.getUnsignedVInt32(c2, ofst2,
limit2);
+ ofst2 += VIntCoding.computeUnsignedVIntSize(vlen2);
+ }
+ int v1Limit = ofst1 + vlen1;
+ if (v1Limit > limit1)
+ throw new IllegalArgumentException("Value limit
exceeds buffer limit.");
+ c1.position(ofst1).limit(v1Limit);
+ int v2Limit = ofst2 + vlen2;
+ if (v2Limit > limit2)
+ throw new IllegalArgumentException("Value limit
exceeds buffer limit.");
+ c2.position(ofst2).limit(v2Limit);
+ int cmp = isByteOrderComparable ?
+ ByteBufferUtil.compareUnsigned(c1, c2) :
+ type.compareCustom(c1,
ByteBufferAccessor.instance, c2, ByteBufferAccessor.instance);
+ if (cmp != 0)
+ return cmp;
+ c1.limit(limit1);
+ c2.limit(limit2);
+ ofst1 += vlen1;
+ ofst2 += vlen2;
+ }
+ // present > not present
+ else if (v1Present && !v2Present)
Review Comment:
Nit: The `else` case can be done as
```
// null (0b10) is smaller than empty (0b01) which is
smaller than valued (0b00);
// compare swapped arguments to reverse the order
int cmp = Long.compare(clusteringBlock2 & 0b11,
clusteringBlock1 & 0b11);
if (cmp != 0)
return cmp;
```
##########
src/java/org/apache/cassandra/db/ClusteringComparator.java:
##########
@@ -156,6 +159,126 @@ public <V1, V2> int compare(ClusteringPrefix<V1> c1,
ClusteringPrefix<V2> c2)
return s1 < s2 ? c1.kind().comparedToClustering :
-c2.kind().comparedToClustering;
}
+ public static int compare(ClusteringDescriptor c1, ClusteringDescriptor c2)
+ {
+ final int c1Size = c1.clusteringColumnsBound();
+ final int c2Size = c2.clusteringColumnsBound();
+ final int minColumns = Math.min(c1Size, c2Size);
+
+ final int cmp = compare(c1.clusteringTypes(), c1.clusteringBuffer(),
c2.clusteringBuffer(), minColumns);
+ if (cmp != 0)
+ return cmp;
+
+ final ClusteringPrefix.Kind c1Kind = c1.clusteringKind();
+ final ClusteringPrefix.Kind c2Kind = c2.clusteringKind();
+ if (c1Size == c2Size)
+ {
+ return ClusteringPrefix.Kind.compare(c1Kind, c2Kind);
+ }
+
+ return c1Size < c2Size ? c1Kind.comparedToClustering :
-c2Kind.comparedToClustering;
+ }
+
+ public static int compare(AbstractType<?>[] types, ByteBuffer c1,
ByteBuffer c2) {
+ return compare(types, c1, c2, types.length);
+ }
+
+ private static int compare(AbstractType<?>[] types, ByteBuffer c1,
ByteBuffer c2, int size)
+ {
+ long clusteringBlock1 = 0;
+ long clusteringBlock2 = 0;
+ final int position1 = c1.position();
+ final int position2 = c2.position();
+ final int limit1 = c1.limit();
+ final int limit2 = c2.limit();
+ try
+ {
+ int ofst1 = position1;
+ int ofst2 = position2;
+ for (int clusteringIndex = 0; clusteringIndex < size;
clusteringIndex++)
+ {
+ if (clusteringIndex % 32 == 0)
+ {
+ clusteringBlock1 = VIntCoding.getUnsignedVInt(c1, ofst1,
limit1);
Review Comment:
Is the manual offset tracking more efficient than advancing `c1` and `c2`
with `VIntCoding.readUnsignedVInt`?
##########
src/java/org/apache/cassandra/io/sstable/format/big/RowIndexEntry.java:
##########
@@ -465,7 +464,7 @@ private static int serializedSize(DeletionTime
deletionTime, long headerLength,
+ TypeSizes.sizeofUnsignedVInt(columnIndexCount);
}
- public void serialize(DataOutputPlus out, ByteBuffer indexInfo) throws
IOException
+ public void serialize(DataOutputPlus out, ByteBuffer unused) throws
IOException
Review Comment:
This method has overrides that use the buffer, the name should not be
changed.
##########
src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java:
##########
@@ -361,6 +405,13 @@ public Map<MetadataType, MetadataComponent>
finalizeMetadata(String partitioner,
Map<MetadataType, MetadataComponent> components = new
EnumMap<>(MetadataType.class);
components.put(MetadataType.VALIDATION, new
ValidationMetadata(partitioner, bloomFilterFPChance));
+ Slice coveredClustering;
+ if (minClusteringDescriptor.clusteringKind() !=
ClusteringPrefix.Kind.EXCL_START_BOUND) // min is end only if the descriptors
are unused
Review Comment:
The minimum can certainly be `EXCL_START_BOUND` when it is used, if a
partition starts with a range tombstone. The maximum, on the other hand, can't.
If you want to do this by a single operation (and also remove the
`minClusteringDescriptor.clusteringColumnsBound() == 0` check in
`updateClusteringValues`), you can change the uninitialized min kind to
`SSTABLE_UPPER_BOUND`, because that won't ever be given to
`updateClusteringValues`.
##########
src/java/org/apache/cassandra/io/sstable/format/SortedTableWriter.java:
##########
@@ -237,15 +237,17 @@ private void addRangeTomstoneMarker(RangeTombstoneMarker
marker) throws IOExcept
private AbstractRowIndexEntry endPartition(DecoratedKey key, DeletionTime
partitionLevelDeletion) throws IOException
{
- long finishResult = partitionWriter.finish();
+ // not inclusive of last byte
+ long partitionLength = partitionWriter.finish();
Review Comment:
This is not at all guaranteed to be the partition length, changing the name
here is very misleading.
##########
src/java/org/apache/cassandra/io/sstable/ElementDescriptor.java:
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.cassandra.db.Columns;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.rows.DeserializationHelper;
+import org.apache.cassandra.db.rows.UnfilteredSerializer;
+import org.apache.cassandra.io.util.RandomAccessReader;
+
+import static
org.apache.cassandra.io.sstable.SSTableCursorReader.readUnfilteredDeletionTime;
+
+public class ElementDescriptor extends ClusteringDescriptor
+{
+ private final ReusableLivenessInfo rowLivenessInfo = new
ReusableLivenessInfo();
+ private final DeletionTime deletionTime = DeletionTime.build(0, 0);
+ private final DeletionTime deletionTime2 = DeletionTime.build(0, 0);
+
+ private long position;
+ private int flags;
+ private int extendedFlags;
+
+ private long unfilteredSize;
+ private long unfilteredDataStart;
+// private long prevUnfilteredSize;
+ private long unfilteredCellStart;
+ Columns rowColumns;
+
+ void loadTombstone(RandomAccessReader dataReader,
+ SerializationHeader serializationHeader,
+ DeserializationHelper deserializationHelper,
+ AbstractType<?>[] clusteringColumnTypes,
+ int flags) throws IOException
+ {
+ this.flags = flags;
+ this.extendedFlags = 0;
+ rowColumns = null;
+ byte clusteringKind = dataReader.readByte();
+ if (clusteringKind == STATIC_CLUSTERING_TYPE || clusteringKind ==
ROW_CLUSTERING_TYPE) {
+ // STATIC_CLUSTERING or CLUSTERING -> no deletion info, should not
happen
+ throw new IllegalStateException();
+ }
+
+ int columnsBound = dataReader.readUnsignedShort();
+ loadClustering(dataReader, clusteringColumnTypes, clusteringKind,
columnsBound);
+ this.unfilteredSize = dataReader.readUnsignedVInt();
+ dataReader.readUnsignedVInt(); // Unused: prevUnfilteredSize
+ if (clusteringKind == EXCL_END_INCL_START_BOUNDARY_CLUSTERING_TYPE ||
clusteringKind == INCL_END_EXCL_START_BOUNDARY_CLUSTERING_TYPE)
+ {
+ // boundary
+ readUnfilteredDeletionTime(dataReader, serializationHeader,
deletionTime); // CLOSE
+ readUnfilteredDeletionTime(dataReader, serializationHeader,
deletionTime2); // OPEN
+ }
+ else
+ {
+ // bound
+ readUnfilteredDeletionTime(dataReader, serializationHeader,
deletionTime); // CLOSE|OPEN
+ }
+ }
+
+ void loadRow(RandomAccessReader dataReader,
+ SerializationHeader serializationHeader,
+ DeserializationHelper deserializationHelper,
+ AbstractType<?>[] clusteringTypes,
+ int flags) throws IOException {
+ // body = whatever is covered by size, so inclusive of the
prev_row_size inclusive of flags
+ position = dataReader.getPosition() - 1;
+ this.flags = flags;
+ this.extendedFlags = 0;
+
+ loadClustering(dataReader, clusteringTypes, ROW_CLUSTERING_TYPE,
clusteringTypes.length);
+
+ rowColumns = serializationHeader.columns(false);
+
+ loadCommonRowFields(dataReader, serializationHeader,
deserializationHelper, flags);
+ }
+
+ void loadStaticRow(RandomAccessReader dataReader,
+ SerializationHeader serializationHeader,
+ DeserializationHelper deserializationHelper,
+ int flags,
+ int extendedFlags) throws IOException {
+ // body = whatever is covered by size, so inclusive of the
prev_row_size inclusive of flags
+ position = dataReader.getPosition() - 2;
+ this.flags = flags;
+ this.extendedFlags = extendedFlags;
+ // no clustering
+ loadClustering(dataReader, null, STATIC_CLUSTERING_TYPE, 0);
+ rowColumns = serializationHeader.columns(true);
+
+ loadCommonRowFields(dataReader, serializationHeader,
deserializationHelper, flags);
+ }
+
+ private void loadCommonRowFields(RandomAccessReader dataReader,
SerializationHeader serializationHeader, DeserializationHelper
deserializationHelper, int flags) throws IOException
+ {
+ unfilteredSize = dataReader.readUnsignedVInt();
+ unfilteredDataStart = dataReader.getPosition();
+ // prevUnfilteredSize = ;
+ dataReader.readUnsignedVInt(); // unused
+
+ SSTableCursorReader.readLivenessInfo(dataReader, serializationHeader,
deserializationHelper, flags, rowLivenessInfo);
+ if (UnfilteredSerializer.hasDeletion(flags))
+ {
+ // struct delta_deletion_time {
+ // varint delta_marked_for_delete_at;
+ // varint delta_local_deletion_time;
+ //};
+ readUnfilteredDeletionTime(dataReader, serializationHeader,
deletionTime);
+ }
+ else
+ {
+ deletionTime.resetLive();
+ }
+ if (!UnfilteredSerializer.hasAllColumns(flags))
+ {
+ // TODO: re-implement GC free
Review Comment:
DataStax's branch has [an implementation of
it](https://github.com/datastax/cassandra/commit/9f999fb11fd04eb04cbd9728e11a389b180c14a1#diff-0da9619f64836c20d71cc73876639051b9f1dded554af97638e742105565f184).
##########
src/java/org/apache/cassandra/io/sstable/format/big/BigTableVerifier.java:
##########
@@ -99,6 +106,51 @@ protected void verifyIndex()
super.verifyIndex();
}
+ @Override
+ protected void deserializeIndex(SSTableReader sstable) throws IOException
+ {
+ try (BigTableKeyReader it = (BigTableKeyReader)sstable.keyReader(true))
+ {
+ if (it.isExhausted())
+ return;
+
+ ByteBuffer last = it.key();
+ if (!Objects.equals(last, sstable.getFirst().getKey()))
+ throw new CorruptSSTableException(new IOException("First
partition does not match index"), it.toString());
+ RowIndexEntry rowIndexEntry = it.rowIndexEntry();
+ long partitionBase = it.dataPosition();
+ int blockCount = rowIndexEntry.blockCount();
+ if (blockCount > 0) {
Review Comment:
Why not check this for every partition rather than just the first? Perhaps
as part of the `verifyPartition` method?
##########
src/java/org/apache/cassandra/db/ClusteringPrefix.java:
##########
@@ -425,18 +425,18 @@ public default String
clusteringString(List<AbstractType<?>> types)
public static class Serializer
{
- public void serialize(ClusteringPrefix<?> clustering, DataOutputPlus
out, int version, List<AbstractType<?>> types) throws IOException
+ public void serialize(ClusteringPrefix<?> clustering, DataOutputPlus
out, int unused, List<AbstractType<?>> types) throws IOException
{
// We shouldn't serialize static clusterings
assert clustering.kind() != Kind.STATIC_CLUSTERING;
if (clustering.kind() == Kind.CLUSTERING)
{
out.writeByte(clustering.kind().ordinal());
- Clustering.serializer.serialize((Clustering<?>)clustering,
out, version, types);
+ Clustering.serializer.serialize((Clustering<?>)clustering,
out, unused, types);
Review Comment:
It's not quite clear if this really is unused. Do you rely on them being
unused? If so, shouldn't we remove the versioning support for clusterings
altogether (preferably in a separate commit)?
##########
src/java/org/apache/cassandra/io/sstable/format/big/BigTableVerifier.java:
##########
@@ -99,6 +106,51 @@ protected void verifyIndex()
super.verifyIndex();
}
+ @Override
+ protected void deserializeIndex(SSTableReader sstable) throws IOException
+ {
+ try (BigTableKeyReader it = (BigTableKeyReader)sstable.keyReader(true))
+ {
+ if (it.isExhausted())
+ return;
+
+ ByteBuffer last = it.key();
+ if (!Objects.equals(last, sstable.getFirst().getKey()))
+ throw new CorruptSSTableException(new IOException("First
partition does not match index"), it.toString());
+ RowIndexEntry rowIndexEntry = it.rowIndexEntry();
+ long partitionBase = it.dataPosition();
+ int blockCount = rowIndexEntry.blockCount();
+ if (blockCount > 0) {
+ long expectedNextOffset = 0;
+ RowIndexEntry.IndexInfoRetriever indexInfoRetriever =
rowIndexEntry.openWithIndex(it.indexFile());
+ for (int blockIndex=0;blockIndex<blockCount;blockIndex++) {
+ IndexInfo indexInfo =
indexInfoRetriever.columnsIndex(blockIndex);
+ long dataFileOffset = partitionBase + indexInfo.offset;
+ if (expectedNextOffset != 0) {
+ if (expectedNextOffset != indexInfo.offset)
+ throw new CorruptSSTableException(new
IOException("Row entry indexInfo offset + width should match next block
offset:" + indexInfo.offset + " expected: " + expectedNextOffset),
it.toString());
+ }
+ expectedNextOffset = indexInfo.offset + indexInfo.width;
+ dataFile.seek(dataFileOffset);
+ Unfiltered unfiltered =
UnfilteredSerializer.serializer.deserialize(dataFile, sstable.header, new
DeserializationHelper(sstable.metadata(),
sstable.descriptor.version.correspondingMessagingVersion(),
DeserializationHelper.Flag.LOCAL), BTreeRow.sortedBuilder());
+ if (unfiltered.isRow()) {
+ if (!Objects.equals(indexInfo.firstName,
unfiltered.clustering()))
+ throw new CorruptSSTableException(new
IOException("Row entry clustering does not match data:{info=" + indexInfo + ",
row:" + unfiltered.clustering() + "}"), it.toString());
+ }
+ else {
+ throw new CorruptSSTableException(new IOException("Row
entry does not point to row:" + indexInfo), it.toString());
Review Comment:
I don't think anything prevents the row index from pointing to a range
tombstone marker.
##########
src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java:
##########
@@ -49,7 +49,7 @@ public DefaultCompactionWriter(ColumnFamilyStore cfs,
Directories directories, I
}
@Override
- protected boolean shouldSwitchWriterInCurrentLocation(DecoratedKey key)
+ protected boolean shouldSwitchWriterInCurrentLocation(DecoratedKey unused)
Review Comment:
Do we need to rename the parameter?
##########
src/java/org/apache/cassandra/db/ClusteringComparator.java:
##########
@@ -156,6 +159,126 @@ public <V1, V2> int compare(ClusteringPrefix<V1> c1,
ClusteringPrefix<V2> c2)
return s1 < s2 ? c1.kind().comparedToClustering :
-c2.kind().comparedToClustering;
}
+ public static int compare(ClusteringDescriptor c1, ClusteringDescriptor c2)
+ {
+ final int c1Size = c1.clusteringColumnsBound();
+ final int c2Size = c2.clusteringColumnsBound();
+ final int minColumns = Math.min(c1Size, c2Size);
+
+ final int cmp = compare(c1.clusteringTypes(), c1.clusteringBuffer(),
c2.clusteringBuffer(), minColumns);
+ if (cmp != 0)
+ return cmp;
+
+ final ClusteringPrefix.Kind c1Kind = c1.clusteringKind();
+ final ClusteringPrefix.Kind c2Kind = c2.clusteringKind();
+ if (c1Size == c2Size)
+ {
+ return ClusteringPrefix.Kind.compare(c1Kind, c2Kind);
+ }
+
+ return c1Size < c2Size ? c1Kind.comparedToClustering :
-c2Kind.comparedToClustering;
+ }
+
+ public static int compare(AbstractType<?>[] types, ByteBuffer c1,
ByteBuffer c2) {
+ return compare(types, c1, c2, types.length);
+ }
+
+ private static int compare(AbstractType<?>[] types, ByteBuffer c1,
ByteBuffer c2, int size)
+ {
+ long clusteringBlock1 = 0;
+ long clusteringBlock2 = 0;
+ final int position1 = c1.position();
+ final int position2 = c2.position();
+ final int limit1 = c1.limit();
+ final int limit2 = c2.limit();
+ try
+ {
+ int ofst1 = position1;
+ int ofst2 = position2;
+ for (int clusteringIndex = 0; clusteringIndex < size;
clusteringIndex++)
+ {
+ if (clusteringIndex % 32 == 0)
+ {
+ clusteringBlock1 = VIntCoding.getUnsignedVInt(c1, ofst1,
limit1);
+ ofst1 +=
VIntCoding.computeUnsignedVIntSize(clusteringBlock1);
+ clusteringBlock2 = VIntCoding.getUnsignedVInt(c2, ofst2,
limit2);
+ ofst2 +=
VIntCoding.computeUnsignedVIntSize(clusteringBlock2);
+ }
+
+ AbstractType<?> type = types[clusteringIndex];
+
+ boolean v1Present = (clusteringBlock1 & 0x11) == 0;
+ boolean v2Present = (clusteringBlock2 & 0x11) == 0;
+
+ if (v1Present && v2Present)
+ {
+ boolean isByteOrderComparable = type.isByteOrderComparable;
+ int vlen1,vlen2;
+ if (type.isValueLengthFixed())
+ {
+ vlen1 = vlen2 = type.valueLengthIfFixed();
+ }
+ else
+ {
+ vlen1 = VIntCoding.getUnsignedVInt32(c1, ofst1,
limit1);
+ ofst1 += VIntCoding.computeUnsignedVIntSize(vlen1);
+ vlen2 = VIntCoding.getUnsignedVInt32(c2, ofst2,
limit2);
+ ofst2 += VIntCoding.computeUnsignedVIntSize(vlen2);
+ }
+ int v1Limit = ofst1 + vlen1;
+ if (v1Limit > limit1)
+ throw new IllegalArgumentException("Value limit
exceeds buffer limit.");
+ c1.position(ofst1).limit(v1Limit);
+ int v2Limit = ofst2 + vlen2;
+ if (v2Limit > limit2)
+ throw new IllegalArgumentException("Value limit
exceeds buffer limit.");
+ c2.position(ofst2).limit(v2Limit);
+ int cmp = isByteOrderComparable ?
+ ByteBufferUtil.compareUnsigned(c1, c2) :
+ type.compareCustom(c1,
ByteBufferAccessor.instance, c2, ByteBufferAccessor.instance);
+ if (cmp != 0)
+ return cmp;
+ c1.limit(limit1);
+ c2.limit(limit2);
+ ofst1 += vlen1;
+ ofst2 += vlen2;
+ }
+ // present > not present
+ else if (v1Present && !v2Present)
+ {
+ return 1;
+ }
+ else if (!v1Present && v2Present)
+ {
+ return -1;
+ }
+ else
+ {
+ boolean v1Null = (clusteringBlock1 & 0x10) == 0;
Review Comment:
Likewise, this should be `0b10`.
##########
src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java:
##########
@@ -299,6 +320,29 @@ public MetadataCollector tokenSpaceCoverage(double
coverage)
return this;
}
+ public void updateClusteringValues(ClusteringDescriptor newClustering) {
+ if (newClustering == null ||
newClustering.clusteringKind().isBoundary())
Review Comment:
I think you need to copy the comment from
`updateClusteringValuesByBoundOrBoundary` to explain skipping boundaries.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]