You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "klsince (via GitHub)" <gi...@apache.org> on 2023/04/28 20:33:43 UTC

[GitHub] [pinot] klsince commented on a diff in pull request #10687: Implement mutable index using index SPI

klsince commented on code in PR #10687:
URL: https://github.com/apache/pinot/pull/10687#discussion_r1180780395


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/fst/FstIndexType.java:
##########
@@ -77,7 +78,32 @@ public Map<String, FstIndexConfig> fromIndexLoadingConfig(IndexLoadingConfig ind
     Set<String> fstIndexColumns = indexLoadingConfig.getFSTIndexColumns();
     for (String column : indexLoadingConfig.getAllKnownColumns()) {
       if (fstIndexColumns.contains(column)) {
-        FstIndexConfig conf = new FstIndexConfig(indexLoadingConfig.getFSTIndexType());
+        FSTType fstType = indexLoadingConfig.getFSTIndexType();
+
+        TableConfig tableConfig = indexLoadingConfig.getTableConfig();

Review Comment:
   add a comment or wrap those changes to a helper method, as to get `fstType` from index props. 



##########
pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java:
##########
@@ -0,0 +1,48 @@
+/**
+ * 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.pinot.segment.spi.index.mutable;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import org.apache.pinot.segment.spi.index.IndexReader;
+
+
+public interface MutableIndex extends IndexReader {
+
+  /**
+   * Adds the given single value cell to the index.
+   *
+   * Rows will be added in no particular order.

Review Comment:
   add `@param` for the docId, and same for next method. 
   
   nit: for the comment, "... no particular order, so the docId is required by this method"



##########
pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableIndex.java:
##########
@@ -0,0 +1,48 @@
+/**
+ * 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.pinot.segment.spi.index.mutable;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import org.apache.pinot.segment.spi.index.IndexReader;
+
+
+public interface MutableIndex extends IndexReader {

Review Comment:
   does it have to extend IndexReader? As this is the counter part of `IndexCreator` but for creating index on mutable segments, maybe we call this `MutableIndexCreator` and extends `Closable`



##########
pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/AbstractIndexType.java:
##########
@@ -123,4 +123,16 @@ public boolean equals(Object o) {
   public int hashCode() {
     return Objects.hash(_id);
   }
+
+  /**
+   * Helper method that builds allocation context that includes segment name, column name, and index type.
+   *
+   * @param segmentName Name of segment.
+   * @param columnName Name of column.
+   * @param indexType Index type.
+   * @return Allocation context built from segment name, column name and index type.
+   */
+  public static String buildAllocationContext(String segmentName, String columnName, String indexType) {
+    return segmentName + ":" + columnName + indexType;

Review Comment:
   add a util class like IndexUtils for those?
   I see there is a ColumnIndexUtils in spi pkg already, perhaps reuse or rename it.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java:
##########
@@ -637,23 +595,23 @@ private void addNewRow(int docId, GenericRow row) {
         // Update numValues info
         indexContainer._valuesInfo.updateSVNumValues();
 
-        MutableForwardIndex forwardIndex = indexContainer._forwardIndex;
+        MutableIndex forwardIndex = indexContainer._mutableIndexes.get(StandardIndexes.forward());
         FieldSpec fieldSpec = indexContainer._fieldSpec;
 
         DataType dataType = fieldSpec.getDataType();
         value = indexContainer._valueAggregator.getInitialAggregatedValue(value);
         switch (dataType.getStoredType()) {

Review Comment:
   it looks like forwardIndex.add() checks data types internally, so can remove this switch-case block here, saving cost of box/unbox as well.



##########
pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/provider/MutableIndexContext.java:
##########
@@ -18,36 +18,97 @@
  */
 package org.apache.pinot.segment.spi.index.mutable.provider;
 
+import java.io.File;
 import java.util.Objects;
+import javax.annotation.Nullable;
 import org.apache.pinot.segment.spi.memory.PinotDataBufferMemoryManager;
-import org.apache.pinot.spi.config.table.JsonIndexConfig;
 import org.apache.pinot.spi.data.FieldSpec;
 
 
-public interface MutableIndexContext {
-  PinotDataBufferMemoryManager getMemoryManager();
+public class MutableIndexContext {

Review Comment:
   looks like IndexCreationContext is still an interface, but it keeps the inner Common classes to hold the context fields. should it be a class too to be consistent with here.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/text/TextIndexType.java:
##########
@@ -178,4 +182,24 @@ public TextIndexReader createIndexReader(SegmentDirectory.Reader segmentReader,
       return new LuceneTextIndexReader(metadata.getColumnName(), segmentDir, metadata.getTotalDocs(), indexConfig);
     }
   }
+
+  @Nullable
+  @Override
+  public MutableIndex createMutableIndex(MutableIndexContext context, TextIndexConfig config) {
+    if (config.isDisabled()) {
+      return null;
+    }
+    if (!context.getFieldSpec().isSingleValueField()) {
+      return null;
+    }
+    if (config.getFstType() == FSTType.NATIVE) {
+      return new NativeMutableTextIndex(context.getFieldSpec().getName());
+    } else {

Review Comment:
   as return early in the if-branch, can move things out of else branch and save some indents.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java:
##########
@@ -52,11 +57,18 @@
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.Schema;
 
+import static org.apache.pinot.spi.data.FieldSpec.DataType.INT;

Review Comment:
   nit: inline this import



-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org