You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2022/12/14 18:54:39 UTC

[GitHub] [pinot] vvivekiyer commented on a diff in pull request #9982: Reload the SegmentMetadata even for temporary forward index

vvivekiyer commented on code in PR #9982:
URL: https://github.com/apache/pinot/pull/9982#discussion_r1048846678


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/InvertedIndexAndDictionaryBasedForwardIndexCreator.java:
##########
@@ -218,20 +218,23 @@ public void regenerateForwardIndex()
 
     LoaderUtils.writeIndexToV3Format(_segmentWriter, _columnName, _forwardIndexFile, ColumnIndexType.FORWARD_INDEX);
 
-    if (!_isTemporaryForwardIndex) {
-      // Only update the metadata and cleanup other indexes if the forward index to be created is permanent. If the
-      // forward index is temporary, it is meant to be used only for construction of other indexes and will be deleted
-      // once all the IndexHandlers have completed.
-      try {
-        LOGGER.info("Created forward index from inverted index and dictionary. Updating metadata properties for "
-            + "segment: {}, column: {}, property list: {}", segmentName, _columnName, metadataProperties);
-        SegmentMetadataUtils.updateMetadataProperties(_segmentMetadata, metadataProperties);
-      } catch (Exception e) {
-        throw new IOException(
-            String.format("Failed to update metadata properties for segment: %s, column: %s", segmentName, _columnName),
-            e);
-      }
+    try {
+      // Update the metadata even for temporary forward index as other IndexHandlers may rely on the updated metadata
+      // to construct their indexes based on the forward index.
+      LOGGER.info("Created forward index from inverted index and dictionary. Updating metadata properties for "
+          + "segment: {}, column: {}, property list: {}, is temporary: {}", segmentName, _columnName,
+          metadataProperties, _isTemporaryForwardIndex);
+      SegmentMetadataUtils.updateMetadataProperties(_segmentMetadata, metadataProperties);
+    } catch (Exception e) {
+      throw new IOException(
+          String.format("Failed to update metadata properties for segment: %s, column: %s", segmentName, _columnName),
+          e);
+    }
 
+    if (!_isTemporaryForwardIndex) {

Review Comment:
   Question:
   Earlier, we weren't updating metadata properties for temporary forward index creations.
   
   Now that we are updating properties, shouldn't there also be a cleanup step (where we revert these updates) when the temp forward index is removed? Is that already guaranteed? If yes, can you please add a comment stating how the cleanup is handled for temporary forward index?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/BaseIndexHandler.java:
##########
@@ -93,6 +100,24 @@ protected void createForwardIndexIfNeeded(SegmentDirectory.Writer segmentWriter,
       _tmpForwardIndexColumns.add(columnName);
     }
 
+    // Update the segmentMetadata
+    File indexDir = _segmentMetadata.getIndexDir();

Review Comment:
   As per my understanding, whenever we update metadata properties file, we also want to update the in-memory segmentMetadata data-structure. Is that correct?
   
   Can we move the reloadSegmentMetadata() call closer to the place where the we actually update the file - or maybe wrap in-memory and file updates into a common function that can be called?
   
   This will also make sure that future callers don't miss updating in-memory DS when updating the file.



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/SegmentPreProcessor.java:
##########
@@ -107,17 +107,20 @@ public void process()
       List<IndexHandler> indexHandlers = new ArrayList<>();
       for (ColumnIndexType type : ColumnIndexType.values()) {
         IndexHandler handler =
-            IndexHandlerFactory.getIndexHandler(type, _segmentMetadata, _indexLoadingConfig, _schema);
+            IndexHandlerFactory.getIndexHandler(type, _segmentMetadata, _indexLoadingConfig, _schema,
+                _segmentDirectory);
         indexHandlers.add(handler);
+        // TODO: Find a way to ensure ForwardIndexHandler is always executed before other handlers instead of
+        // relying on enum ordering.
         handler.updateIndices(segmentWriter, indexCreatorProvider);
-        if (type == ColumnIndexType.FORWARD_INDEX) {
-          // TODO: Find a way to ensure ForwardIndexHandler is always executed before other handlers instead of
-          // relying on enum ordering.
-          // ForwardIndexHandler may modify the segment metadata while rewriting forward index to create a dictionary
-          // This new metadata is needed to construct other indexes like RangeIndex.
-          _segmentMetadata = new SegmentMetadataImpl(indexDir);
-          _segmentDirectory.reloadMetadata();
-        }
+        // ForwardIndexHandler may modify the segment metadata while rewriting forward index to create / remove a
+        // dictionary. Other IndexHandler classes may modify the segment metadata while creating a temporary forward
+        // index to generate their respective indexes from if the forward index was disabled. This new metadata is
+        // needed to construct other indexes like RangeIndex. This also needs to be done outside of the IndexHandler
+        // code since modifying the `_segmentMetadata` within the IndexHandler doesn't modify this object directly but
+        // creates a new one for use within the IndexHandler.
+        _segmentMetadata = new SegmentMetadataImpl(indexDir);

Review Comment:
   Based on my comment above -
   
   if we wrap file and in-memory update into a common function and use it in all places where we want to update metadata properties, the call here is not needed. Is my understanding correct? 



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