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/01/03 19:50:23 UTC

[GitHub] [pinot] npawar commented on a change in pull request #7969: extend SegmentDirectoryLoader interface and refactor BaseTableDataManager

npawar commented on a change in pull request #7969:
URL: https://github.com/apache/pinot/pull/7969#discussion_r777649669



##########
File path: pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
##########
@@ -293,6 +292,26 @@ private void reloadSegment(String tableNameWithType, SegmentMetadata segmentMeta
     }
   }
 
+  @VisibleForTesting
+  boolean reloadMutableSegment(String tableNameWithType, String segmentName,
+      SegmentDataManager segmentDataManager, @Nullable Schema schema) {
+    IndexSegment segment = segmentDataManager.getSegment();
+    if (segment instanceof ImmutableSegment) {
+      LOGGER.info("Reloading OFFLINE segment: {} in table: {} not using local tier backend", segmentName,

Review comment:
       this comment is confusing. The segment is not in OFFLINE state right? this is simply a completed segment of a table that could be either realtime or offline, and is on tier backend?
   How about something like "Reloading ImmutableSegment: {} of table: {} on remote tier" ?

##########
File path: pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/loader/SegmentDirectoryLoader.java
##########
@@ -18,19 +18,48 @@
  */
 package org.apache.pinot.segment.spi.loader;
 
+import java.io.File;
 import java.net.URI;
 import org.apache.pinot.segment.spi.store.SegmentDirectory;
 
 
 /**
- * Interface for creating and loading the {@link SegmentDirectory} instance using provided config
+ * Interface for creating and loading the {@link SegmentDirectory} instance using provided config.
+ *
+ * The segment may be kept in local or remote tier backend. When the segment needs reprocessing,
+ * like to add or remote indices, the SegmentDirectoryLoader downloads the segment from tier backend

Review comment:
       typo s/remote/remove

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
##########
@@ -333,92 +315,131 @@ public void reloadSegment(String segmentName, IndexLoadingConfig indexLoadingCon
 
   @Override
   public void addOrReplaceSegment(String segmentName, IndexLoadingConfig indexLoadingConfig,
-      SegmentZKMetadata zkMetadata, @Nullable SegmentMetadata localMetadata)
+      SegmentZKMetadata zkMetadata, @Nullable SegmentMetadata segmentMetadata)
       throws Exception {
-    if (!isNewSegment(zkMetadata, localMetadata)) {
-      LOGGER.info("Segment: {} of table: {} has crc: {} same as before, already loaded, do nothing", segmentName,
-          _tableNameWithType, localMetadata.getCrc());
+    // Non-null segment metadata means the segment has already been loaded.
+    if (segmentMetadata != null) {
+      if (hasSameCRC(zkMetadata, segmentMetadata)) {
+        // Simply returns if the CRC hasn't changed. The table config may have changed
+        // since segment is loaded, but that is handled by reloadSegment() method.
+        LOGGER.info("Segment: {} of table: {} has crc: {} same as before, already loaded, do nothing", segmentName,
+            _tableNameWithType, segmentMetadata.getCrc());
+      } else {
+        // Download the raw segment, reprocess and load it if the CRC has changed.
+        LOGGER.info("Segment: {} of table: {} already loaded but its crc: {} differs from new crc: {}", segmentName,
+            _tableNameWithType, segmentMetadata.getCrc(), zkMetadata.getCrc());
+        downloadRawSegmentAndProcess(segmentName, indexLoadingConfig, zkMetadata,
+            ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType));
+      }
       return;
     }
 
-    // Try to recover if no local metadata is provided.
-    if (localMetadata == null) {
-      LOGGER.info("Segment: {} of table: {} is not loaded, checking disk", segmentName, _tableNameWithType);
-      localMetadata = recoverSegmentQuietly(segmentName);
-      if (!isNewSegment(zkMetadata, localMetadata)) {
-        LOGGER.info("Segment: {} of table {} has crc: {} same as before, loading", segmentName, _tableNameWithType,
-            localMetadata.getCrc());
-        if (loadSegmentQuietly(segmentName, indexLoadingConfig)) {
-          return;
-        }
-        // Set local metadata to null to indicate that the local segment fails to load,
-        // although it exists and has same crc with the remote one.
-        localMetadata = null;
-      }
+    // For local tier backend, try to recover the segment from potential
+    // reload failure. Continue upon any failure.
+    File indexDir = getSegmentDataDir(segmentName);
+    recoverReloadFailureQuietly(_tableNameWithType, segmentName, indexDir);
+
+    // Creates the SegmentDirectory object to access the segment metadata that
+    // may be from local tier backend or remote tier backend.
+    SegmentDirectoryLoaderContext segmentLoaderContext =
+        new SegmentDirectoryLoaderContext(indexLoadingConfig.getTableConfig(), indexLoadingConfig.getInstanceId(),
+            segmentName, indexLoadingConfig.getSegmentDirectoryConfigs());
+    SegmentDirectoryLoader segmentLoader =
+        SegmentDirectoryLoaderRegistry.getSegmentDirectoryLoader(indexLoadingConfig.getSegmentDirectoryLoader());
+    SegmentDirectory segmentDirectory = null;
+    try {
+      segmentDirectory = segmentLoader.load(indexDir.toURI(), segmentLoaderContext);
+    } catch (Exception e) {
+      LOGGER.warn("Failed to create SegmentDirectory for segment: {} of table: {} due to error: {}", segmentName,
+          _tableNameWithType, e.getMessage());
     }
 
-    Preconditions.checkState(allowDownload(segmentName, zkMetadata), "Segment: %s of table: %s does not allow download",
-        segmentName, _tableNameWithType);
-
-    // Download segment and replace the local one, either due to failure to recover local segment,
-    // or the segment data is updated and has new CRC now.
-    if (localMetadata == null) {
-      LOGGER.info("Download segment: {} of table: {} as no good one exists locally", segmentName, _tableNameWithType);
-    } else {
-      LOGGER.info("Download segment: {} of table: {} as local crc: {} mismatches remote crc: {}.", segmentName,
-          _tableNameWithType, localMetadata.getCrc(), zkMetadata.getCrc());
+    // Download the raw segment, reprocess and load it if it is never loaded or its CRC has changed.
+    if (segmentDirectory == null || !hasSameCRC(zkMetadata, segmentDirectory.getSegmentMetadata())) {
+      LOGGER.info("Segment: {} of table: {} not exist or its crc: {} differs from new crc: {}", segmentName,
+          _tableNameWithType, (segmentDirectory == null) ? "none" : segmentDirectory.getSegmentMetadata().getCrc(),
+          zkMetadata.getCrc());
+      closeSegmentDirectoryQuietly(segmentDirectory);
+      downloadRawSegmentAndProcess(segmentName, indexLoadingConfig, zkMetadata,
+          ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType));
+      return;
     }
-    File indexDir = downloadSegment(segmentName, zkMetadata);
-    addSegment(indexDir, indexLoadingConfig);
-    LOGGER.info("Downloaded and loaded segment: {} of table: {} with crc: {}", segmentName, _tableNameWithType,
-        zkMetadata.getCrc());
-  }
 
-  protected boolean allowDownload(String segmentName, SegmentZKMetadata zkMetadata) {
-    return true;
+    try {
+      Schema schema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+      if (!ImmutableSegmentLoader.needPreprocess(segmentDirectory, indexLoadingConfig, schema)) {
+        // The loaded segment is still consistent with current table config or schema.
+        LOGGER.info("Segment: {} of table: {} is consistent with table config and schema", segmentName,
+            _tableNameWithType);
+        addSegment(ImmutableSegmentLoader.load(segmentDirectory, indexLoadingConfig, schema));
+        return;
+      }
+      // If any discrepancy is found, get the segment from tier backend, reprocess and load it.
+      // Please note that the segment is from tier backed, not deep store, for incremental processing.
+      LOGGER.info("Segment: {} of table: {} needs reprocess with table config and schema", segmentName,

Review comment:
       nit: "Segment {} of table {} needs reprocess to reflect latest table config and schema" ?

##########
File path: pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/metadata/SegmentMetadataImpl.java
##########
@@ -80,6 +80,9 @@
 
   private SegmentVersion _segmentVersion;
   private List<StarTreeV2Metadata> _starTreeV2MetadataList;
+  // Caching properties around can be costly when the number of segments is high according to the

Review comment:
       good catch!

##########
File path: pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
##########
@@ -293,6 +292,26 @@ private void reloadSegment(String tableNameWithType, SegmentMetadata segmentMeta
     }
   }
 
+  @VisibleForTesting
+  boolean reloadMutableSegment(String tableNameWithType, String segmentName,
+      SegmentDataManager segmentDataManager, @Nullable Schema schema) {
+    IndexSegment segment = segmentDataManager.getSegment();
+    if (segment instanceof ImmutableSegment) {
+      LOGGER.info("Reloading OFFLINE segment: {} in table: {} not using local tier backend", segmentName,

Review comment:
       Can we pull out the check for ImmutableSegment out of this method? Maybe another helper method 
   ```
   boolean reloadSegmentWithNullIndexDir(params..) {
   IndexSegment segment = segmentDataManager.getSegment();
       if (segment instanceof ImmutableSegment) {
         LOGGER.info("Reloading OFFLINE segment: {} in table: {} not using local tier backend", segmentName,
   tableNameWithType);
         return false;
       }
   return reloadMutableSegment(args..);
   ```
   

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentLoader.java
##########
@@ -95,35 +96,52 @@ public static ImmutableSegment load(File indexDir, IndexLoadingConfig indexLoadi
    * modify the segment like to convert segment format, add or remove indices.
    */
   public static ImmutableSegment load(File indexDir, IndexLoadingConfig indexLoadingConfig, @Nullable Schema schema,

Review comment:
       where is this load method being used now? All calls I saw in BaseDataTableManager were for the new load method

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
##########
@@ -275,53 +278,32 @@ public void addSegmentError(String segmentName, SegmentErrorInfo segmentErrorInf
 
   @Override
   public void reloadSegment(String segmentName, IndexLoadingConfig indexLoadingConfig, SegmentZKMetadata zkMetadata,
-      SegmentMetadata localMetadata, @Nullable Schema schema, boolean forceDownload)
+      SegmentMetadata segmentMetadata, @Nullable Schema schema, boolean forceDownload)
       throws Exception {
-    File indexDir = localMetadata.getIndexDir();
-    Preconditions.checkState(indexDir.isDirectory(), "Index directory: %s is not a directory", indexDir);
-
-    File parentFile = indexDir.getParentFile();
-    File segmentBackupDir =
-        new File(parentFile, indexDir.getName() + CommonConstants.Segment.SEGMENT_BACKUP_DIR_SUFFIX);
-
+    File indexDir = getSegmentDataDir(segmentName);
+    // Create backup dir to make segment reloading atomic for local tier backend.
+    LoaderUtils.createBackup(indexDir);
     try {
-      // First rename index directory to segment backup directory so that original segment have all file descriptors
-      // point to the segment backup directory to ensure original segment serves queries properly
-
-      // Rename index directory to segment backup directory (atomic)
-      Preconditions.checkState(indexDir.renameTo(segmentBackupDir),
-          "Failed to rename index directory: %s to segment backup directory: %s", indexDir, segmentBackupDir);
-
-      // Download from remote or copy from local backup directory into index directory,
-      // and then continue to load the segment from index directory.
-      boolean shouldDownload = forceDownload || !hasSameCRC(zkMetadata, localMetadata);
-      if (shouldDownload && allowDownload(segmentName, zkMetadata)) {
-        if (forceDownload) {
-          LOGGER.info("Segment: {} of table: {} is forced to download", segmentName, _tableNameWithType);
-        } else {
-          LOGGER.info("Download segment:{} of table: {} as local crc: {} mismatches remote crc: {}", segmentName,
-              _tableNameWithType, localMetadata.getCrc(), zkMetadata.getCrc());
-        }
-        indexDir = downloadSegment(segmentName, zkMetadata);
+      boolean shouldDownloadRawSegment = forceDownload || !hasSameCRC(zkMetadata, segmentMetadata);
+      if (shouldDownloadRawSegment && allowDownloadRawSegment(segmentName, zkMetadata)) {
+        downloadRawSegmentAndProcess(segmentName, indexLoadingConfig, zkMetadata, schema);
       } else {
-        LOGGER.info("Reload the local copy of segment: {} of table: {}", segmentName, _tableNameWithType);
-        FileUtils.copyDirectory(segmentBackupDir, indexDir);
+        SegmentDirectoryLoaderContext segmentLoaderContext =
+            new SegmentDirectoryLoaderContext(indexLoadingConfig.getTableConfig(), indexLoadingConfig.getInstanceId(),

Review comment:
       may i suggest adding one more small method, similar to `downloadRawSegmentAndProcess`, as `downloadTierSegmentAndProcess` to fold lines in this else block. It can also be used i the "if not consistent" block of addOrReplace.

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
##########
@@ -333,92 +315,131 @@ public void reloadSegment(String segmentName, IndexLoadingConfig indexLoadingCon
 
   @Override
   public void addOrReplaceSegment(String segmentName, IndexLoadingConfig indexLoadingConfig,
-      SegmentZKMetadata zkMetadata, @Nullable SegmentMetadata localMetadata)
+      SegmentZKMetadata zkMetadata, @Nullable SegmentMetadata segmentMetadata)
       throws Exception {
-    if (!isNewSegment(zkMetadata, localMetadata)) {
-      LOGGER.info("Segment: {} of table: {} has crc: {} same as before, already loaded, do nothing", segmentName,
-          _tableNameWithType, localMetadata.getCrc());
+    // Non-null segment metadata means the segment has already been loaded.
+    if (segmentMetadata != null) {
+      if (hasSameCRC(zkMetadata, segmentMetadata)) {
+        // Simply returns if the CRC hasn't changed. The table config may have changed
+        // since segment is loaded, but that is handled by reloadSegment() method.
+        LOGGER.info("Segment: {} of table: {} has crc: {} same as before, already loaded, do nothing", segmentName,
+            _tableNameWithType, segmentMetadata.getCrc());
+      } else {
+        // Download the raw segment, reprocess and load it if the CRC has changed.
+        LOGGER.info("Segment: {} of table: {} already loaded but its crc: {} differs from new crc: {}", segmentName,
+            _tableNameWithType, segmentMetadata.getCrc(), zkMetadata.getCrc());
+        downloadRawSegmentAndProcess(segmentName, indexLoadingConfig, zkMetadata,
+            ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType));
+      }
       return;
     }
 
-    // Try to recover if no local metadata is provided.
-    if (localMetadata == null) {
-      LOGGER.info("Segment: {} of table: {} is not loaded, checking disk", segmentName, _tableNameWithType);
-      localMetadata = recoverSegmentQuietly(segmentName);
-      if (!isNewSegment(zkMetadata, localMetadata)) {
-        LOGGER.info("Segment: {} of table {} has crc: {} same as before, loading", segmentName, _tableNameWithType,
-            localMetadata.getCrc());
-        if (loadSegmentQuietly(segmentName, indexLoadingConfig)) {
-          return;
-        }
-        // Set local metadata to null to indicate that the local segment fails to load,
-        // although it exists and has same crc with the remote one.
-        localMetadata = null;
-      }
+    // For local tier backend, try to recover the segment from potential
+    // reload failure. Continue upon any failure.
+    File indexDir = getSegmentDataDir(segmentName);
+    recoverReloadFailureQuietly(_tableNameWithType, segmentName, indexDir);
+
+    // Creates the SegmentDirectory object to access the segment metadata that
+    // may be from local tier backend or remote tier backend.
+    SegmentDirectoryLoaderContext segmentLoaderContext =
+        new SegmentDirectoryLoaderContext(indexLoadingConfig.getTableConfig(), indexLoadingConfig.getInstanceId(),
+            segmentName, indexLoadingConfig.getSegmentDirectoryConfigs());
+    SegmentDirectoryLoader segmentLoader =
+        SegmentDirectoryLoaderRegistry.getSegmentDirectoryLoader(indexLoadingConfig.getSegmentDirectoryLoader());
+    SegmentDirectory segmentDirectory = null;
+    try {
+      segmentDirectory = segmentLoader.load(indexDir.toURI(), segmentLoaderContext);
+    } catch (Exception e) {
+      LOGGER.warn("Failed to create SegmentDirectory for segment: {} of table: {} due to error: {}", segmentName,
+          _tableNameWithType, e.getMessage());
     }
 
-    Preconditions.checkState(allowDownload(segmentName, zkMetadata), "Segment: %s of table: %s does not allow download",
-        segmentName, _tableNameWithType);
-
-    // Download segment and replace the local one, either due to failure to recover local segment,
-    // or the segment data is updated and has new CRC now.
-    if (localMetadata == null) {
-      LOGGER.info("Download segment: {} of table: {} as no good one exists locally", segmentName, _tableNameWithType);
-    } else {
-      LOGGER.info("Download segment: {} of table: {} as local crc: {} mismatches remote crc: {}.", segmentName,
-          _tableNameWithType, localMetadata.getCrc(), zkMetadata.getCrc());
+    // Download the raw segment, reprocess and load it if it is never loaded or its CRC has changed.
+    if (segmentDirectory == null || !hasSameCRC(zkMetadata, segmentDirectory.getSegmentMetadata())) {
+      LOGGER.info("Segment: {} of table: {} not exist or its crc: {} differs from new crc: {}", segmentName,
+          _tableNameWithType, (segmentDirectory == null) ? "none" : segmentDirectory.getSegmentMetadata().getCrc(),
+          zkMetadata.getCrc());
+      closeSegmentDirectoryQuietly(segmentDirectory);
+      downloadRawSegmentAndProcess(segmentName, indexLoadingConfig, zkMetadata,
+          ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType));
+      return;
     }
-    File indexDir = downloadSegment(segmentName, zkMetadata);
-    addSegment(indexDir, indexLoadingConfig);
-    LOGGER.info("Downloaded and loaded segment: {} of table: {} with crc: {}", segmentName, _tableNameWithType,
-        zkMetadata.getCrc());
-  }
 
-  protected boolean allowDownload(String segmentName, SegmentZKMetadata zkMetadata) {
-    return true;
+    try {
+      Schema schema = ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType);
+      if (!ImmutableSegmentLoader.needPreprocess(segmentDirectory, indexLoadingConfig, schema)) {
+        // The loaded segment is still consistent with current table config or schema.
+        LOGGER.info("Segment: {} of table: {} is consistent with table config and schema", segmentName,
+            _tableNameWithType);
+        addSegment(ImmutableSegmentLoader.load(segmentDirectory, indexLoadingConfig, schema));

Review comment:
       (optional) and also another sub method, `loadSegment` to fold in this line. This way this addOrReplace driver method will be easier to read

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
##########
@@ -333,92 +315,131 @@ public void reloadSegment(String segmentName, IndexLoadingConfig indexLoadingCon
 
   @Override
   public void addOrReplaceSegment(String segmentName, IndexLoadingConfig indexLoadingConfig,
-      SegmentZKMetadata zkMetadata, @Nullable SegmentMetadata localMetadata)
+      SegmentZKMetadata zkMetadata, @Nullable SegmentMetadata segmentMetadata)
       throws Exception {
-    if (!isNewSegment(zkMetadata, localMetadata)) {
-      LOGGER.info("Segment: {} of table: {} has crc: {} same as before, already loaded, do nothing", segmentName,
-          _tableNameWithType, localMetadata.getCrc());
+    // Non-null segment metadata means the segment has already been loaded.
+    if (segmentMetadata != null) {
+      if (hasSameCRC(zkMetadata, segmentMetadata)) {
+        // Simply returns if the CRC hasn't changed. The table config may have changed
+        // since segment is loaded, but that is handled by reloadSegment() method.
+        LOGGER.info("Segment: {} of table: {} has crc: {} same as before, already loaded, do nothing", segmentName,
+            _tableNameWithType, segmentMetadata.getCrc());
+      } else {
+        // Download the raw segment, reprocess and load it if the CRC has changed.
+        LOGGER.info("Segment: {} of table: {} already loaded but its crc: {} differs from new crc: {}", segmentName,
+            _tableNameWithType, segmentMetadata.getCrc(), zkMetadata.getCrc());
+        downloadRawSegmentAndProcess(segmentName, indexLoadingConfig, zkMetadata,
+            ZKMetadataProvider.getTableSchema(_propertyStore, _tableNameWithType));
+      }
       return;
     }
 
-    // Try to recover if no local metadata is provided.
-    if (localMetadata == null) {
-      LOGGER.info("Segment: {} of table: {} is not loaded, checking disk", segmentName, _tableNameWithType);
-      localMetadata = recoverSegmentQuietly(segmentName);
-      if (!isNewSegment(zkMetadata, localMetadata)) {
-        LOGGER.info("Segment: {} of table {} has crc: {} same as before, loading", segmentName, _tableNameWithType,
-            localMetadata.getCrc());
-        if (loadSegmentQuietly(segmentName, indexLoadingConfig)) {
-          return;
-        }
-        // Set local metadata to null to indicate that the local segment fails to load,
-        // although it exists and has same crc with the remote one.
-        localMetadata = null;
-      }
+    // For local tier backend, try to recover the segment from potential
+    // reload failure. Continue upon any failure.
+    File indexDir = getSegmentDataDir(segmentName);
+    recoverReloadFailureQuietly(_tableNameWithType, segmentName, indexDir);
+
+    // Creates the SegmentDirectory object to access the segment metadata that
+    // may be from local tier backend or remote tier backend.
+    SegmentDirectoryLoaderContext segmentLoaderContext =
+        new SegmentDirectoryLoaderContext(indexLoadingConfig.getTableConfig(), indexLoadingConfig.getInstanceId(),
+            segmentName, indexLoadingConfig.getSegmentDirectoryConfigs());
+    SegmentDirectoryLoader segmentLoader =
+        SegmentDirectoryLoaderRegistry.getSegmentDirectoryLoader(indexLoadingConfig.getSegmentDirectoryLoader());
+    SegmentDirectory segmentDirectory = null;
+    try {
+      segmentDirectory = segmentLoader.load(indexDir.toURI(), segmentLoaderContext);
+    } catch (Exception e) {
+      LOGGER.warn("Failed to create SegmentDirectory for segment: {} of table: {} due to error: {}", segmentName,
+          _tableNameWithType, e.getMessage());
     }
 
-    Preconditions.checkState(allowDownload(segmentName, zkMetadata), "Segment: %s of table: %s does not allow download",
-        segmentName, _tableNameWithType);
-
-    // Download segment and replace the local one, either due to failure to recover local segment,
-    // or the segment data is updated and has new CRC now.
-    if (localMetadata == null) {
-      LOGGER.info("Download segment: {} of table: {} as no good one exists locally", segmentName, _tableNameWithType);
-    } else {
-      LOGGER.info("Download segment: {} of table: {} as local crc: {} mismatches remote crc: {}.", segmentName,
-          _tableNameWithType, localMetadata.getCrc(), zkMetadata.getCrc());
+    // Download the raw segment, reprocess and load it if it is never loaded or its CRC has changed.
+    if (segmentDirectory == null || !hasSameCRC(zkMetadata, segmentDirectory.getSegmentMetadata())) {
+      LOGGER.info("Segment: {} of table: {} not exist or its crc: {} differs from new crc: {}", segmentName,

Review comment:
       can we split these conditions and their respective log statements so it's easier to read and follow in the logs?




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