maedhroz commented on code in PR #2540:
URL: https://github.com/apache/cassandra/pull/2540#discussion_r1289348898


##########
src/java/org/apache/cassandra/index/sai/disk/v1/WidePrimaryKeyMap.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.index.sai.disk.v1;
+
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.index.sai.disk.PrimaryKeyMap;
+import org.apache.cassandra.index.sai.disk.format.IndexComponent;
+import org.apache.cassandra.index.sai.disk.format.IndexDescriptor;
+import org.apache.cassandra.index.sai.disk.v1.bitpack.NumericValuesMeta;
+import org.apache.cassandra.index.sai.disk.v1.sortedterms.SortedTermsMeta;
+import org.apache.cassandra.index.sai.disk.v1.sortedterms.SortedTermsReader;
+import org.apache.cassandra.index.sai.utils.PrimaryKey;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import javax.annotation.concurrent.ThreadSafe;
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * An extension of the {@link SkinnyPrimaryKeyMap} for wide tables (those with 
clustering columns).
+ * <p>
+ * This used the following additional on-disk structures to the {@link 
SkinnyPrimaryKeyMap}
+ * <ul>
+ *     <li>A sorted terms structure for rowId to {@link Clustering} and {@link 
Clustering} to rowId lookups using
+ *     {@link SortedTermsReader}. Uses the {@link 
IndexComponent#CLUSTERING_KEY_BLOCKS} and
+ *     {@link IndexComponent#CLUSTERING_KEY_BLOCK_OFFSETS} components</li>
+ * </ul>
+ * While the {@link Factory} is threadsafe, individual instances of the {@link 
WidePrimaryKeyMap}
+ * are not.
+ */
+@NotThreadSafe
+public class WidePrimaryKeyMap extends SkinnyPrimaryKeyMap
+{
+    @ThreadSafe
+    public static class Factory extends SkinnyPrimaryKeyMap.Factory
+    {
+        private final ClusteringComparator clusteringComparator;
+        private final SortedTermsReader clusteringKeyReader;
+
+        private FileHandle clusteringKeyBlockOffsetsFile;
+        private FileHandle clustingingKeyBlocksFile;
+
+        public Factory(IndexDescriptor indexDescriptor, SSTableReader sstable)
+        {
+            super(indexDescriptor, sstable);
+
+            try
+            {
+                this.clusteringComparator = 
indexDescriptor.clusteringComparator;
+                this.clusteringKeyBlockOffsetsFile = 
indexDescriptor.createPerSSTableFileHandle(IndexComponent.CLUSTERING_KEY_BLOCK_OFFSETS);
+                this.clustingingKeyBlocksFile = 
indexDescriptor.createPerSSTableFileHandle(IndexComponent.CLUSTERING_KEY_BLOCKS);
+                NumericValuesMeta clusteringKeyBlockOffsetsMeta = new 
NumericValuesMeta(metadataSource.get(indexDescriptor.componentName(IndexComponent.CLUSTERING_KEY_BLOCK_OFFSETS)));
+                SortedTermsMeta clusteringKeyMeta = new 
SortedTermsMeta(metadataSource.get(indexDescriptor.componentName(IndexComponent.CLUSTERING_KEY_BLOCKS)));
+                this.clusteringKeyReader = new 
SortedTermsReader(clustingingKeyBlocksFile, clusteringKeyBlockOffsetsFile, 
clusteringKeyMeta, clusteringKeyBlockOffsetsMeta);
+            }
+            catch (Throwable t)
+            {
+                throw Throwables.unchecked(Throwables.close(t, 
clustingingKeyBlocksFile, clusteringKeyBlockOffsetsFile));
+            }
+        }
+
+        @Override
+        @SuppressWarnings({ "resource", "RedundantSuppression" })
+        public PrimaryKeyMap newPerSSTablePrimaryKeyMap() throws IOException
+        {
+            LongArray rowIdToToken = new 
LongArray.DeferredLongArray(tokenReaderFactory::open);
+            LongArray partitionIdToToken = new 
LongArray.DeferredLongArray(partitionReaderFactory::open);
+
+            return new WidePrimaryKeyMap(rowIdToToken,
+                                         partitionIdToToken,
+                                         partitionKeyReader.openCursor(),
+                                         clusteringKeyReader.openCursor(),
+                                         partitioner,
+                                         primaryKeyFactory,
+                                         clusteringComparator);
+        }
+
+        @Override
+        public void close()
+        {
+            super.close();
+            FileUtils.closeQuietly(Arrays.asList(clustingingKeyBlocksFile, 
clusteringKeyBlockOffsetsFile));
+        }
+    }
+
+    private final ClusteringComparator clusteringComparator;
+    private final SortedTermsReader.Cursor clusteringKeyCursor;
+
+    private WidePrimaryKeyMap(LongArray tokenArray,
+                              LongArray partitionArray,
+                              SortedTermsReader.Cursor partitionKeyCursor,
+                              SortedTermsReader.Cursor clusteringKeyCursor,
+                              IPartitioner partitioner,
+                              PrimaryKey.Factory primaryKeyFactory,
+                              ClusteringComparator clusteringComparator)
+    {
+        super(tokenArray, partitionArray, partitionKeyCursor, partitioner, 
primaryKeyFactory);
+
+        this.clusteringComparator = clusteringComparator;
+        this.clusteringKeyCursor = clusteringKeyCursor;
+    }
+
+    @Override
+    public long rowIdFromPrimaryKey(PrimaryKey primaryKey)
+    {
+        long rowId = tokenArray.indexOf(primaryKey.token().getLongValue());
+
+        // If the key only has a token (initial range skip in the query), the 
token is out of range,
+        // or we have skipped a token, return the rowId from the token array.

Review Comment:
   > or we have skipped a token
   
   This means we skipped past the token of the provided primary key to a new 
partition whose token is greater?



-- 
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]

Reply via email to