adelapena commented on code in PR #2673:
URL: https://github.com/apache/cassandra/pull/2673#discussion_r1327569820


##########
src/java/org/apache/cassandra/index/sai/disk/SSTableRowIdPostingList.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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;
+
+import java.io.IOException;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.index.sai.postings.PostingList;
+
+public class SSTableRowIdPostingList implements PostingList

Review Comment:
   This entire class seems unused



##########
src/java/org/apache/cassandra/index/sai/disk/v1/MemtableIndexWriter.java:
##########
@@ -104,22 +105,20 @@ public void complete(Stopwatch stopwatch) throws 
IOException
                 return;
             }
 
-            final Iterator<Pair<ByteComparable, LongArrayList>> iterator = 
rowMapping.merge(memtable);
-
-            try (MemtableTermsIterator terms = new 
MemtableTermsIterator(memtable.getMinTerm(), memtable.getMaxTerm(), iterator))
+            if (indexContext.isVector())
             {
-                long cellCount = flush(rowMapping.minKey, rowMapping.maxKey, 
indexContext.getValidator(), terms, rowMapping.maxSSTableRowId);
-
-                
indexDescriptor.createComponentOnDisk(IndexComponent.COLUMN_COMPLETION_MARKER, 
indexContext);
-
-                indexContext.getIndexMetrics().memtableIndexFlushCount.inc();
-
-                long elapsed = stopwatch.elapsed(TimeUnit.MILLISECONDS);
+                flushVectorIndex(rowMapping.minKey, rowMapping.maxKey, start, 
stopwatch);

Review Comment:
   We probably don't need the min/max keys as arguments of `flushVectorIndex`, 
since it has access to `rowMapping`.



##########
src/java/org/apache/cassandra/index/sai/disk/v1/PerColumnIndexFiles.java:
##########
@@ -71,6 +77,11 @@ public FileHandle balancedTree()
         return getFile(IndexComponent.BALANCED_TREE);
     }
 
+    public FileHandle vectors()
+    {
+        return getFile(IndexComponent.VECTOR);

Review Comment:
   Maybe the index component should be named `VECTORS`, in the plural, the same 
as the method?



##########
src/java/org/apache/cassandra/index/sai/disk/v1/vector/CheckpointingIterator.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.vector;
+
+import com.google.common.collect.Iterables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.index.sai.QueryContext;
+import org.apache.cassandra.index.sai.disk.SSTableIndex;
+import org.apache.cassandra.index.sai.iterators.KeyRangeIterator;
+import org.apache.cassandra.index.sai.utils.PrimaryKey;
+import org.apache.cassandra.io.util.FileUtils;
+
+public class CheckpointingIterator extends KeyRangeIterator
+{
+    private static final Logger logger = 
LoggerFactory.getLogger(CheckpointingIterator.class);
+
+    private final QueryContext context;
+    private final KeyRangeIterator union;
+    private final Iterable<SSTableIndex> referencedIndexes;
+
+    public CheckpointingIterator(KeyRangeIterator wrapped, 
Iterable<SSTableIndex> referencedIndexes, Iterable<SSTableIndex> 
referencedAnnIndexesInHybridSearch, QueryContext queryContext)
+    {
+        super(wrapped.getMinimum(), wrapped.getMaximum(), wrapped.getCount());
+
+        this.union = wrapped;
+        if (referencedAnnIndexesInHybridSearch != null)
+            this.referencedIndexes = Iterables.concat(referencedIndexes, 
referencedAnnIndexesInHybridSearch);
+        else
+            this.referencedIndexes = referencedIndexes;
+        this.context = queryContext;
+    }
+
+    protected PrimaryKey computeNext()
+    {
+        try
+        {
+            return union.hasNext() ? union.next() : endOfData();
+        }
+        finally
+        {
+            context.checkpoint();
+        }
+    }
+
+    protected void performSkipTo(PrimaryKey nextKey)
+    {
+        try
+        {
+            union.skipTo(nextKey);
+        }
+        finally
+        {
+            context.checkpoint();
+        }
+    }
+
+    public void close()

Review Comment:
   Nit: add `@Override`



##########
src/java/org/apache/cassandra/index/sai/disk/v1/vector/CheckpointingIterator.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.vector;
+
+import com.google.common.collect.Iterables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.index.sai.QueryContext;
+import org.apache.cassandra.index.sai.disk.SSTableIndex;
+import org.apache.cassandra.index.sai.iterators.KeyRangeIterator;
+import org.apache.cassandra.index.sai.utils.PrimaryKey;
+import org.apache.cassandra.io.util.FileUtils;
+
+public class CheckpointingIterator extends KeyRangeIterator

Review Comment:
   Can we add some class JavaDoc?



##########
src/java/org/apache/cassandra/index/sai/disk/v1/SSTableIndexWriter.java:
##########
@@ -82,7 +84,7 @@ public SSTableIndexWriter(IndexDescriptor indexDescriptor, 
IndexContext indexCon
         this.analyzer = indexContext.getAnalyzerFactory().create();
         this.limiter = limiter;
         this.isIndexValid = isIndexValid;
-        this.maxTermSize = indexContext.isFrozen() ? MAX_FROZEN_TERM_SIZE : 
MAX_STRING_TERM_SIZE;
+        this.maxTermSize = indexContext.isVector() ? MAX_VECTOR_TERM_SIZE : 
indexContext.isFrozen() ? MAX_FROZEN_TERM_SIZE : MAX_STRING_TERM_SIZE;

Review Comment:
   Using parenthesis could ease the reading of the nested ternary operator.



##########
src/java/org/apache/cassandra/index/sai/disk/v1/vector/CheckpointingIterator.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.vector;
+
+import com.google.common.collect.Iterables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.index.sai.QueryContext;
+import org.apache.cassandra.index.sai.disk.SSTableIndex;
+import org.apache.cassandra.index.sai.iterators.KeyRangeIterator;
+import org.apache.cassandra.index.sai.utils.PrimaryKey;
+import org.apache.cassandra.io.util.FileUtils;
+
+public class CheckpointingIterator extends KeyRangeIterator
+{
+    private static final Logger logger = 
LoggerFactory.getLogger(CheckpointingIterator.class);
+
+    private final QueryContext context;
+    private final KeyRangeIterator union;
+    private final Iterable<SSTableIndex> referencedIndexes;
+
+    public CheckpointingIterator(KeyRangeIterator wrapped, 
Iterable<SSTableIndex> referencedIndexes, Iterable<SSTableIndex> 
referencedAnnIndexesInHybridSearch, QueryContext queryContext)

Review Comment:
   Can we break the long line?
   ```suggestion
       public CheckpointingIterator(KeyRangeIterator wrapped, 
                                    Iterable<SSTableIndex> referencedIndexes, 
                                    Iterable<SSTableIndex> 
referencedAnnIndexesInHybridSearch, 
                                    QueryContext queryContext)
   ```



##########
src/java/org/apache/cassandra/index/sai/disk/v1/vector/hnsw/BitsUtil.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.vector.hnsw;
+
+import org.apache.lucene.util.Bits;
+import org.cliffc.high_scale_lib.NonBlockingHashMapLong;
+
+import java.util.Set;
+
+public class BitsUtil
+{
+    public static Bits bitsIgnoringDeleted(Bits toAccept, Set<Integer> 
deletedOrdinals)
+    {
+        return deletedOrdinals.isEmpty()
+               ? toAccept
+               : toAccept == null ? new NoDeletedBits(deletedOrdinals) : new 
NoDeletedIntersectingBits(toAccept, deletedOrdinals);
+    }
+
+    public static <T> Bits bitsIgnoringDeleted(Bits toAccept, 
NonBlockingHashMapLong<VectorPostings<T>> postings)
+    {
+        return toAccept == null ? new NoDeletedPostings(postings) : new 
NoDeletedIntersectingPostings(toAccept, postings);

Review Comment:
   ```suggestion
           return toAccept == null ? new NoDeletedPostings<>(postings) : new 
NoDeletedIntersectingPostings<>(toAccept, postings);
   ```



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