Jackie-Jiang commented on code in PR #8589:
URL: https://github.com/apache/pinot/pull/8589#discussion_r884010829


##########
pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PurgeMinionClusterIntegrationTest.java:
##########
@@ -0,0 +1,350 @@
+/**
+ * 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.integration.tests;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.io.FileUtils;
+import org.apache.helix.task.TaskState;
+import org.apache.pinot.common.lineage.SegmentLineageAccessHelper;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.common.minion.MinionTaskMetadataUtils;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import 
org.apache.pinot.controller.helix.core.minion.PinotHelixTaskResourceManager;
+import org.apache.pinot.controller.helix.core.minion.PinotTaskManager;
+import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.minion.MinionContext;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableTaskConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.apache.pinot.util.TestUtils;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.*;
+
+
+/**
+ * Integration test for minion task of type "PurgeTask"
+ */
+public class PurgeMinionClusterIntegrationTest extends 
BaseClusterIntegrationTest {
+  private static final String PURGE_FIRST_RUN_TABLE = "myTable1";
+  private static final String PURGE_DELTA_PASSED_TABLE = "myTable2";
+  private static final String PURGE_DELTA_NOT_PASSED_TABLE = "myTable3";
+
+  protected PinotHelixTaskResourceManager _helixTaskResourceManager;
+  protected PinotTaskManager _taskManager;
+  protected PinotHelixResourceManager _pinotHelixResourceManager;
+  protected String _tableName;
+
+  protected final File _segmentDir1 = new File(_tempDir, "segmentDir1");
+  protected final File _segmentDir2 = new File(_tempDir, "segmentDir2");
+  protected final File _segmentDir3 = new File(_tempDir, "segmentDir3");
+
+  protected final File _tarDir1 = new File(_tempDir, "tarDir1");
+  protected final File _tarDir2 = new File(_tempDir, "tarDir2");
+  protected final File _tarDir3 = new File(_tempDir, "tarDir3");
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir1, _tarDir1,
+        _segmentDir2, _tarDir2, _segmentDir3, _tarDir3);
+
+    // Start the Pinot cluster
+    startZk();
+    startController();
+    startBrokers(1);
+    startServers(1);
+
+    // Create and upload the schema and table config
+    Schema schema = createSchema();
+    addSchema(schema);
+    setTableName(PURGE_DELTA_NOT_PASSED_TABLE);
+    TableConfig purgeDeltaNotPassedTableConfig = createOfflineTableConfig();
+    purgeDeltaNotPassedTableConfig.setTaskConfig(getPurgeTaskConfig());
+    setTableName(PURGE_FIRST_RUN_TABLE);
+    TableConfig purgeTableConfig = createOfflineTableConfig();
+    purgeTableConfig.setTaskConfig(getPurgeTaskConfig());
+
+    setTableName(PURGE_DELTA_PASSED_TABLE);
+    TableConfig purgeDeltaPassedTableConfig = createOfflineTableConfig();
+    purgeDeltaPassedTableConfig.setTaskConfig(getPurgeTaskConfig());
+
+    addTableConfig(purgeTableConfig);
+    addTableConfig(purgeDeltaPassedTableConfig);
+    addTableConfig(purgeDeltaNotPassedTableConfig);
+
+    // Unpack the Avro files
+    List<File> avroFiles = unpackAvroData(_tempDir);
+
+    // Create and upload segments
+    ClusterIntegrationTestUtils
+        .buildSegmentsFromAvro(avroFiles, purgeTableConfig, schema, 0, 
_segmentDir1, _tarDir1);
+    ClusterIntegrationTestUtils
+        .buildSegmentsFromAvro(avroFiles, purgeDeltaPassedTableConfig,
+            schema, 0, _segmentDir2, _tarDir2);
+    ClusterIntegrationTestUtils
+        .buildSegmentsFromAvro(avroFiles, purgeDeltaNotPassedTableConfig,
+            schema, 0, _segmentDir3, _tarDir3);
+
+    uploadSegments(PURGE_FIRST_RUN_TABLE, _tarDir1);
+    uploadSegments(PURGE_DELTA_PASSED_TABLE, _tarDir2);
+    uploadSegments(PURGE_DELTA_NOT_PASSED_TABLE, _tarDir3);
+
+    // Initialize the query generator
+    setUpQueryGenerator(avroFiles);
+    startMinion();
+    setRecordPurger();
+    _helixTaskResourceManager = 
_controllerStarter.getHelixTaskResourceManager();
+    _taskManager = _controllerStarter.getTaskManager();
+    _pinotHelixResourceManager = _controllerStarter.getHelixResourceManager();
+
+    //set up metadata on segment to check how code handle passed and not 
passed delay
+    String tablenameOfflineNotPassed = 
TableNameBuilder.OFFLINE.tableNameWithType(PURGE_DELTA_NOT_PASSED_TABLE);
+    String tablenameOfflinePassed = 
TableNameBuilder.OFFLINE.tableNameWithType(PURGE_DELTA_PASSED_TABLE);
+
+    //set up passed delay
+    List<SegmentZKMetadata> segmentsZKMetadataDeltaPassed = 
_pinotHelixResourceManager
+        .getSegmentsZKMetadata(tablenameOfflinePassed);
+
+    Map<String, String> customSegmentMetadataPassed = new HashMap<>();
+    customSegmentMetadataPassed.put(MinionConstants.PurgeTask.TASK_TYPE
+        + MinionConstants.TASK_TIME_SUFFIX, 
String.valueOf(System.currentTimeMillis() - 88400000));
+
+    for (SegmentZKMetadata segmentZKMetadata : segmentsZKMetadataDeltaPassed) {
+      segmentZKMetadata.setCustomMap(customSegmentMetadataPassed);
+      _pinotHelixResourceManager.updateZkMetadata(tablenameOfflinePassed, 
segmentZKMetadata);
+    }
+    //set up not passed delay
+    List<SegmentZKMetadata> segmentsZKMetadataDeltaNotPassed = 
_pinotHelixResourceManager
+        .getSegmentsZKMetadata(tablenameOfflineNotPassed);
+    Map<String, String> customSegmentMetadataNotPassed = new HashMap<>();
+    customSegmentMetadataNotPassed.put(MinionConstants.PurgeTask.TASK_TYPE
+        + MinionConstants.TASK_TIME_SUFFIX, 
String.valueOf(System.currentTimeMillis() - 4000));
+    for (SegmentZKMetadata segmentZKMetadata : 
segmentsZKMetadataDeltaNotPassed) {
+      segmentZKMetadata.setCustomMap(customSegmentMetadataNotPassed);
+      _pinotHelixResourceManager.updateZkMetadata(tablenameOfflineNotPassed, 
segmentZKMetadata);
+    }
+  }
+
+
+  private void setRecordPurger() {
+    MinionContext minionContext = MinionContext.getInstance();
+    minionContext.setRecordPurgerFactory(rawTableName -> {
+      List<String> tableNames = Arrays.asList(PURGE_FIRST_RUN_TABLE,
+          PURGE_DELTA_PASSED_TABLE, PURGE_DELTA_NOT_PASSED_TABLE);
+      if (tableNames.contains(rawTableName)) {
+        return row -> row.getValue("Quarter").equals(1);
+      } else {
+        return null;
+      }
+    });
+  }
+
+  @Override
+  public String getTableName() {
+    return _tableName;
+  }
+
+  public void setTableName(String tableName) {
+    _tableName = tableName;
+  }
+
+  private TableTaskConfig getPurgeTaskConfig() {
+    Map<String, String> tableTaskConfigs = new HashMap<>();
+    
tableTaskConfigs.put(MinionConstants.PurgeTask.LAST_PURGE_TIME_THREESOLD_PERIOD,
 "1d");
+    return new 
TableTaskConfig(Collections.singletonMap(MinionConstants.PurgeTask.TASK_TYPE, 
tableTaskConfigs));
+  }
+
+  /**
+   * Test purge with no metadata on the segments (checking null safe 
implementation)
+   */
+  @Test
+  public void testFirstRunPurge()
+      throws Exception {
+    // Expected purge task generation :
+    // 1. No previous purge run so all segment should be processed and purge 
metadata sould be added to the segments
+    //

Review Comment:
   Is this incomplete?



##########
pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PurgeMinionClusterIntegrationTest.java:
##########
@@ -0,0 +1,350 @@
+/**
+ * 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.integration.tests;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.io.FileUtils;
+import org.apache.helix.task.TaskState;
+import org.apache.pinot.common.lineage.SegmentLineageAccessHelper;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.common.minion.MinionTaskMetadataUtils;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import 
org.apache.pinot.controller.helix.core.minion.PinotHelixTaskResourceManager;
+import org.apache.pinot.controller.helix.core.minion.PinotTaskManager;
+import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.minion.MinionContext;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableTaskConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.apache.pinot.util.TestUtils;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.*;
+
+
+/**
+ * Integration test for minion task of type "PurgeTask"
+ */
+public class PurgeMinionClusterIntegrationTest extends 
BaseClusterIntegrationTest {
+  private static final String PURGE_FIRST_RUN_TABLE = "myTable1";
+  private static final String PURGE_DELTA_PASSED_TABLE = "myTable2";
+  private static final String PURGE_DELTA_NOT_PASSED_TABLE = "myTable3";
+
+  protected PinotHelixTaskResourceManager _helixTaskResourceManager;
+  protected PinotTaskManager _taskManager;
+  protected PinotHelixResourceManager _pinotHelixResourceManager;
+  protected String _tableName;
+
+  protected final File _segmentDir1 = new File(_tempDir, "segmentDir1");
+  protected final File _segmentDir2 = new File(_tempDir, "segmentDir2");
+  protected final File _segmentDir3 = new File(_tempDir, "segmentDir3");
+
+  protected final File _tarDir1 = new File(_tempDir, "tarDir1");
+  protected final File _tarDir2 = new File(_tempDir, "tarDir2");
+  protected final File _tarDir3 = new File(_tempDir, "tarDir3");
+
+  @BeforeClass
+  public void setUp()
+      throws Exception {
+    TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir1, _tarDir1,
+        _segmentDir2, _tarDir2, _segmentDir3, _tarDir3);
+
+    // Start the Pinot cluster
+    startZk();
+    startController();
+    startBrokers(1);
+    startServers(1);
+
+    // Create and upload the schema and table config
+    Schema schema = createSchema();
+    addSchema(schema);
+    setTableName(PURGE_DELTA_NOT_PASSED_TABLE);
+    TableConfig purgeDeltaNotPassedTableConfig = createOfflineTableConfig();
+    purgeDeltaNotPassedTableConfig.setTaskConfig(getPurgeTaskConfig());
+    setTableName(PURGE_FIRST_RUN_TABLE);
+    TableConfig purgeTableConfig = createOfflineTableConfig();
+    purgeTableConfig.setTaskConfig(getPurgeTaskConfig());
+
+    setTableName(PURGE_DELTA_PASSED_TABLE);
+    TableConfig purgeDeltaPassedTableConfig = createOfflineTableConfig();
+    purgeDeltaPassedTableConfig.setTaskConfig(getPurgeTaskConfig());
+
+    addTableConfig(purgeTableConfig);
+    addTableConfig(purgeDeltaPassedTableConfig);
+    addTableConfig(purgeDeltaNotPassedTableConfig);
+
+    // Unpack the Avro files
+    List<File> avroFiles = unpackAvroData(_tempDir);
+
+    // Create and upload segments
+    ClusterIntegrationTestUtils
+        .buildSegmentsFromAvro(avroFiles, purgeTableConfig, schema, 0, 
_segmentDir1, _tarDir1);
+    ClusterIntegrationTestUtils
+        .buildSegmentsFromAvro(avroFiles, purgeDeltaPassedTableConfig,
+            schema, 0, _segmentDir2, _tarDir2);
+    ClusterIntegrationTestUtils
+        .buildSegmentsFromAvro(avroFiles, purgeDeltaNotPassedTableConfig,
+            schema, 0, _segmentDir3, _tarDir3);
+
+    uploadSegments(PURGE_FIRST_RUN_TABLE, _tarDir1);
+    uploadSegments(PURGE_DELTA_PASSED_TABLE, _tarDir2);
+    uploadSegments(PURGE_DELTA_NOT_PASSED_TABLE, _tarDir3);
+
+    // Initialize the query generator
+    setUpQueryGenerator(avroFiles);
+    startMinion();
+    setRecordPurger();
+    _helixTaskResourceManager = 
_controllerStarter.getHelixTaskResourceManager();
+    _taskManager = _controllerStarter.getTaskManager();
+    _pinotHelixResourceManager = _controllerStarter.getHelixResourceManager();
+
+    //set up metadata on segment to check how code handle passed and not 
passed delay
+    String tablenameOfflineNotPassed = 
TableNameBuilder.OFFLINE.tableNameWithType(PURGE_DELTA_NOT_PASSED_TABLE);
+    String tablenameOfflinePassed = 
TableNameBuilder.OFFLINE.tableNameWithType(PURGE_DELTA_PASSED_TABLE);
+
+    //set up passed delay
+    List<SegmentZKMetadata> segmentsZKMetadataDeltaPassed = 
_pinotHelixResourceManager
+        .getSegmentsZKMetadata(tablenameOfflinePassed);
+
+    Map<String, String> customSegmentMetadataPassed = new HashMap<>();
+    customSegmentMetadataPassed.put(MinionConstants.PurgeTask.TASK_TYPE
+        + MinionConstants.TASK_TIME_SUFFIX, 
String.valueOf(System.currentTimeMillis() - 88400000));
+
+    for (SegmentZKMetadata segmentZKMetadata : segmentsZKMetadataDeltaPassed) {
+      segmentZKMetadata.setCustomMap(customSegmentMetadataPassed);
+      _pinotHelixResourceManager.updateZkMetadata(tablenameOfflinePassed, 
segmentZKMetadata);
+    }
+    //set up not passed delay
+    List<SegmentZKMetadata> segmentsZKMetadataDeltaNotPassed = 
_pinotHelixResourceManager
+        .getSegmentsZKMetadata(tablenameOfflineNotPassed);
+    Map<String, String> customSegmentMetadataNotPassed = new HashMap<>();
+    customSegmentMetadataNotPassed.put(MinionConstants.PurgeTask.TASK_TYPE
+        + MinionConstants.TASK_TIME_SUFFIX, 
String.valueOf(System.currentTimeMillis() - 4000));
+    for (SegmentZKMetadata segmentZKMetadata : 
segmentsZKMetadataDeltaNotPassed) {
+      segmentZKMetadata.setCustomMap(customSegmentMetadataNotPassed);
+      _pinotHelixResourceManager.updateZkMetadata(tablenameOfflineNotPassed, 
segmentZKMetadata);
+    }
+  }
+

Review Comment:
   (minor, code style) Avoid multiple empty lines



##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/purge/PurgeTaskGenerator.java:
##########
@@ -0,0 +1,160 @@
+/**
+ * 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.plugin.minion.tasks.purge;
+
+import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import org.apache.pinot.common.data.Segment;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.controller.api.exception.UnknownTaskTypeException;
+import 
org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator;
+import 
org.apache.pinot.controller.helix.core.minion.generator.TaskGeneratorUtils;
+import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.minion.PinotTaskConfig;
+import org.apache.pinot.spi.annotations.minion.TaskGenerator;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableTaskConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.utils.TimeUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@TaskGenerator
+public class PurgeTaskGenerator extends BaseTaskGenerator {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PurgeTaskGenerator.class);
+  private static final String DEFAULT_DELTA_PERIOD = "1d";
+
+  @Override
+  public String getTaskType() {
+    return MinionConstants.PurgeTask.TASK_TYPE;
+  }
+
+  @Override
+  public List<PinotTaskConfig> generateTasks(List<TableConfig> tableConfigs) {
+    LOGGER.info("Start generating PurgeTask");
+    String taskType = MinionConstants.PurgeTask.TASK_TYPE;
+    List<PinotTaskConfig> pinotTaskConfigs = new ArrayList<>();
+
+    for (TableConfig tableConfig : tableConfigs) {
+
+      String tableName = tableConfig.getTableName();
+      if (tableConfig.getTableType() == TableType.REALTIME) {
+        LOGGER.info("Task type : {}, cannot be run on table of type  {}", 
taskType, TableType.REALTIME);
+        continue;
+      }
+
+      Map<String, String> taskConfigs;
+      try {
+        TableTaskConfig tableTaskConfig = tableConfig.getTaskConfig();
+        Preconditions.checkNotNull(tableTaskConfig);
+        taskConfigs = 
tableTaskConfig.getConfigsForTaskType(MinionConstants.PurgeTask.TASK_TYPE);
+        Preconditions.checkNotNull(taskConfigs, "Task config shouldn't be null 
for Table: {}", tableName);
+      } catch (Exception e) {
+        continue;
+      }
+      String deltaTimePeriod =
+          
taskConfigs.getOrDefault(MinionConstants.PurgeTask.DELTA_TIME_PERIOD_KEY, 
DEFAULT_DELTA_PERIOD);
+      long purgeDeltaMs = TimeUtils.convertPeriodToMillis(deltaTimePeriod);
+
+      LOGGER.info("Start generating task configs for table: {} for task: {}", 
tableName, taskType);
+      // Get max number of tasks for this table
+      int tableMaxNumTasks;
+      String tableMaxNumTasksConfig = 
taskConfigs.get(MinionConstants.TABLE_MAX_NUM_TASKS_KEY);
+      if (tableMaxNumTasksConfig != null) {
+        try {
+          tableMaxNumTasks = Integer.parseInt(tableMaxNumTasksConfig);
+        } catch (Exception e) {
+          tableMaxNumTasks = Integer.MAX_VALUE;
+          LOGGER.warn("MaxNumTasks have been wrongly set for table : {}, and 
task {}", tableName, taskType);
+        }
+      } else {
+        tableMaxNumTasks = Integer.MAX_VALUE;
+      }
+      List<SegmentZKMetadata> offlineSegmentsZKMetadata = 
_clusterInfoAccessor.getSegmentsZKMetadata(tableName);
+
+      Predicate<SegmentZKMetadata> alreadyPurged = segment -> 
segment.getCustomMap() != null;
+      Predicate<SegmentZKMetadata> notPurged = segment -> 
segment.getCustomMap() == null;
+
+      List<SegmentZKMetadata> purgedSegmentsZKMetadata =
+          
offlineSegmentsZKMetadata.stream().filter(alreadyPurged).collect(Collectors.toList());
+
+      List<SegmentZKMetadata> notpurgedSegmentsZKMetadata =
+          
offlineSegmentsZKMetadata.stream().filter(notPurged).collect(Collectors.toList());
+
+      Collections.sort(purgedSegmentsZKMetadata, Comparator.comparing(
+          segmentZKMetadata -> segmentZKMetadata.getCustomMap()
+              .get(MinionConstants.PurgeTask.TASK_TYPE + 
MinionConstants.TASK_TIME_SUFFIX),
+          Comparator.nullsFirst(Comparator.naturalOrder())));
+      //add already purged segment at the end
+      notpurgedSegmentsZKMetadata.addAll(purgedSegmentsZKMetadata);

Review Comment:
   We can use `org.apache.commons.lang3.tuple.Pair` which does not require key 
to be `Serializable`



-- 
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-unsubscr...@pinot.apache.org

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


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

Reply via email to