You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "Jackie-Jiang (via GitHub)" <gi...@apache.org> on 2023/07/19 07:03:23 UTC

[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #10463: Segment compaction for upsert real-time tables

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


##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java:
##########
@@ -0,0 +1,292 @@
+/**
+ * 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.upsertcompaction;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.BiMap;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.pinot.common.exception.InvalidConfigException;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator;
+import org.apache.pinot.controller.util.CompletionServiceHelper;
+import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.common.MinionConstants.UpsertCompactionTask;
+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.TableType;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.TimeUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@TaskGenerator
+public class UpsertCompactionTaskGenerator extends BaseTaskGenerator {
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactionTaskGenerator.class);
+  private static final String DEFAULT_BUFFER_PERIOD = "7d";
+  private static final double DEFAULT_INVALID_RECORDS_THRESHOLD_PERCENT = 0.0;
+  private static final long DEFAULT_MIN_RECORD_COUNT = 0;
+  public static class SegmentSelectionResult {
+    private List<SegmentZKMetadata> _segmentsForCompaction;
+    private List<String> _segmentsForDeletion;
+    SegmentSelectionResult(List<SegmentZKMetadata> segmentsForCompaction, List<String> segmentsForDeletion) {
+      _segmentsForCompaction = segmentsForCompaction;
+      _segmentsForDeletion = segmentsForDeletion;
+    }
+    public List<SegmentZKMetadata> getSegmentsForCompaction() {
+      return _segmentsForCompaction;
+    }
+    public List<String> getSegmentsForDeletion() {
+      return _segmentsForDeletion;
+    }
+  }

Review Comment:
   (format) Can you add some empty lines for readability?



##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java:
##########
@@ -0,0 +1,292 @@
+/**
+ * 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.upsertcompaction;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.BiMap;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.pinot.common.exception.InvalidConfigException;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator;
+import org.apache.pinot.controller.util.CompletionServiceHelper;
+import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.common.MinionConstants.UpsertCompactionTask;
+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.TableType;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.TimeUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@TaskGenerator
+public class UpsertCompactionTaskGenerator extends BaseTaskGenerator {
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactionTaskGenerator.class);
+  private static final String DEFAULT_BUFFER_PERIOD = "7d";
+  private static final double DEFAULT_INVALID_RECORDS_THRESHOLD_PERCENT = 0.0;
+  private static final long DEFAULT_MIN_RECORD_COUNT = 0;
+  public static class SegmentSelectionResult {
+    private List<SegmentZKMetadata> _segmentsForCompaction;
+    private List<String> _segmentsForDeletion;
+    SegmentSelectionResult(List<SegmentZKMetadata> segmentsForCompaction, List<String> segmentsForDeletion) {
+      _segmentsForCompaction = segmentsForCompaction;
+      _segmentsForDeletion = segmentsForDeletion;
+    }
+    public List<SegmentZKMetadata> getSegmentsForCompaction() {
+      return _segmentsForCompaction;
+    }
+    public List<String> getSegmentsForDeletion() {
+      return _segmentsForDeletion;
+    }
+  }
+
+  @Override
+  public String getTaskType() {
+    return MinionConstants.UpsertCompactionTask.TASK_TYPE;
+  }
+
+  @Override
+  public List<PinotTaskConfig> generateTasks(List<TableConfig> tableConfigs) {
+    String taskType = MinionConstants.UpsertCompactionTask.TASK_TYPE;
+    List<PinotTaskConfig> pinotTaskConfigs = new ArrayList<>();
+    for (TableConfig tableConfig: tableConfigs) {
+      if (!validate(tableConfig)) {
+        continue;
+      }
+
+      String tableNameWithType = tableConfig.getTableName();
+      LOGGER.info("Start generating task configs for table: {} for task: {}",
+          tableNameWithType, taskType);
+
+      Map<String, String> taskConfigs = tableConfig.getTaskConfig().getConfigsForTaskType(taskType);
+      Map<String, String> compactionConfigs = getCompactionConfigs(taskConfigs);
+      List<SegmentZKMetadata> completedSegments = getCompletedSegments(tableNameWithType, compactionConfigs);
+
+      if (completedSegments.isEmpty()) {
+        LOGGER.info("No completed segments were eligible for compaction for table: {}", tableNameWithType);
+        continue;
+      }
+
+      // get server to segment mappings
+      Map<String, List<String>> serverToSegments = _clusterInfoAccessor.getServerToSegmentsMap(tableNameWithType);
+      PinotHelixResourceManager pinotHelixResourceManager = _clusterInfoAccessor.getPinotHelixResourceManager();
+      BiMap<String, String> serverToEndpoints;
+      try {
+        serverToEndpoints = pinotHelixResourceManager.getDataInstanceAdminEndpoints(serverToSegments.keySet());
+      } catch (InvalidConfigException e) {
+        throw new RuntimeException(e);
+      }
+
+      Map<String, SegmentZKMetadata> completedSegmentsMap = completedSegments.stream()
+          .collect(Collectors.toMap(SegmentZKMetadata::getSegmentName, Function.identity()));
+
+      List<String> validDocIdUrls;
+      try {
+        validDocIdUrls = getValidDocIdMetadataUrls(
+            serverToSegments, serverToEndpoints, tableNameWithType, completedSegmentsMap);
+      } catch (URISyntaxException e) {
+        throw new RuntimeException(e);
+      }
+
+      // request the urls from the servers
+      CompletionServiceHelper completionServiceHelper = new CompletionServiceHelper(
+            Executors.newCachedThreadPool(), new MultiThreadedHttpConnectionManager(), serverToEndpoints.inverse());
+      CompletionServiceHelper.CompletionServiceResponse serviceResponse =
+            completionServiceHelper.doMultiGetRequest(validDocIdUrls, tableNameWithType, false, 3000);
+
+      SegmentSelectionResult segmentSelectionResult =
+          processValidDocIdMetadata(compactionConfigs, completedSegmentsMap, serviceResponse._httpResponses.entrySet());
+
+      if (!segmentSelectionResult.getSegmentsForDeletion().isEmpty()) {
+          pinotHelixResourceManager.deleteSegments(
+              tableNameWithType, segmentSelectionResult.getSegmentsForDeletion(), "0d");
+      }
+
+      int numTasks = 0;
+      int maxTasks = getMaxTasks(taskType, tableNameWithType, taskConfigs);
+      for (SegmentZKMetadata segment : segmentSelectionResult.getSegmentsForCompaction()) {
+        if (numTasks == maxTasks) {
+          break;
+        }
+        Map<String, String> configs = new HashMap<>();
+        configs.put(MinionConstants.TABLE_NAME_KEY, tableNameWithType);
+        configs.put(MinionConstants.SEGMENT_NAME_KEY, segment.getSegmentName());
+        configs.put(MinionConstants.DOWNLOAD_URL_KEY, segment.getDownloadUrl());
+        configs.put(MinionConstants.UPLOAD_URL_KEY, _clusterInfoAccessor.getVipUrl() + "/segments");
+        configs.put(MinionConstants.ORIGINAL_SEGMENT_CRC_KEY, String.valueOf(segment.getCrc()));
+        pinotTaskConfigs.add(new PinotTaskConfig(UpsertCompactionTask.TASK_TYPE, configs));
+        numTasks++;
+      }
+      LOGGER.info("Finished generating {} tasks configs for table: {} " + "for task: {}",
+          numTasks, tableNameWithType, taskType);
+    }
+    return pinotTaskConfigs;
+  }
+
+  @VisibleForTesting
+  public static SegmentSelectionResult processValidDocIdMetadata(Map<String, String> compactionConfigs,
+      Map<String, SegmentZKMetadata> completedSegmentsMap, Set<Map.Entry<String, String>> responseSet) {
+    double invalidRecordsThresholdPercent =
+        Double.parseDouble(compactionConfigs.getOrDefault(UpsertCompactionTask.INVALID_RECORDS_THRESHOLD_PERCENT,
+            String.valueOf(DEFAULT_INVALID_RECORDS_THRESHOLD_PERCENT)));
+    long minRecordCount =
+        Long.parseLong(compactionConfigs.getOrDefault(UpsertCompactionTask.MIN_RECORD_COUNT,
+            String.valueOf(DEFAULT_MIN_RECORD_COUNT)));
+    List<SegmentZKMetadata> segmentsForCompaction = new ArrayList<>();
+    List<String> segmentsForDeletion = new ArrayList<>();
+    for (Map.Entry<String, String> streamResponse : responseSet) {
+      JsonNode allValidDocIdMetadata;
+      try {
+        allValidDocIdMetadata = JsonUtils.stringToJsonNode(streamResponse.getValue());
+      } catch (IOException e) {
+        LOGGER.error("Unable to parse validDocIdMetadata response for: {}", streamResponse.getKey());
+        continue;
+      }
+      Iterator<JsonNode> iterator = allValidDocIdMetadata.elements();
+      while (iterator.hasNext()) {
+        JsonNode validDocIdMetadata = iterator.next();
+        long invalidRecordCount = validDocIdMetadata.get("totalInvalidDocs").asLong();
+        String segmentName = validDocIdMetadata.get("segmentName").asText();
+        SegmentZKMetadata segment = completedSegmentsMap.get(segmentName);
+        double invalidRecordPercent = ((double) invalidRecordCount / segment.getTotalDocs()) * 100;
+        if (invalidRecordCount == segment.getTotalDocs()) {
+          segmentsForDeletion.add(segment.getSegmentName());
+        } else if (invalidRecordPercent > invalidRecordsThresholdPercent && invalidRecordCount > minRecordCount) {
+          segmentsForCompaction.add(segment);
+        }
+      }
+    }
+    return new SegmentSelectionResult(segmentsForCompaction, segmentsForDeletion);
+  }
+
+  @VisibleForTesting
+  public static List<String> getValidDocIdMetadataUrls(Map<String, List<String>> serverToSegments,
+      BiMap<String, String> serverToEndpoints, String tableNameWithType,
+      Map<String, SegmentZKMetadata> completedSegments) throws URISyntaxException {
+    List<String> urls = new ArrayList<>();
+    for (Map.Entry<String, List<String>> entry : serverToSegments.entrySet()) {

Review Comment:
   Since we are looping over the server to segments list, if the table has multiple replicas, are we adding the same segment multiple times?



##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java:
##########
@@ -0,0 +1,292 @@
+/**
+ * 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.upsertcompaction;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.BiMap;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.pinot.common.exception.InvalidConfigException;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator;
+import org.apache.pinot.controller.util.CompletionServiceHelper;
+import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.common.MinionConstants.UpsertCompactionTask;
+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.TableType;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.TimeUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@TaskGenerator
+public class UpsertCompactionTaskGenerator extends BaseTaskGenerator {
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactionTaskGenerator.class);
+  private static final String DEFAULT_BUFFER_PERIOD = "7d";
+  private static final double DEFAULT_INVALID_RECORDS_THRESHOLD_PERCENT = 0.0;
+  private static final long DEFAULT_MIN_RECORD_COUNT = 0;
+  public static class SegmentSelectionResult {
+    private List<SegmentZKMetadata> _segmentsForCompaction;
+    private List<String> _segmentsForDeletion;
+    SegmentSelectionResult(List<SegmentZKMetadata> segmentsForCompaction, List<String> segmentsForDeletion) {
+      _segmentsForCompaction = segmentsForCompaction;
+      _segmentsForDeletion = segmentsForDeletion;
+    }
+    public List<SegmentZKMetadata> getSegmentsForCompaction() {
+      return _segmentsForCompaction;
+    }
+    public List<String> getSegmentsForDeletion() {
+      return _segmentsForDeletion;
+    }
+  }
+
+  @Override
+  public String getTaskType() {
+    return MinionConstants.UpsertCompactionTask.TASK_TYPE;
+  }
+
+  @Override
+  public List<PinotTaskConfig> generateTasks(List<TableConfig> tableConfigs) {
+    String taskType = MinionConstants.UpsertCompactionTask.TASK_TYPE;
+    List<PinotTaskConfig> pinotTaskConfigs = new ArrayList<>();
+    for (TableConfig tableConfig: tableConfigs) {
+      if (!validate(tableConfig)) {
+        continue;
+      }
+
+      String tableNameWithType = tableConfig.getTableName();
+      LOGGER.info("Start generating task configs for table: {} for task: {}",
+          tableNameWithType, taskType);
+
+      Map<String, String> taskConfigs = tableConfig.getTaskConfig().getConfigsForTaskType(taskType);
+      Map<String, String> compactionConfigs = getCompactionConfigs(taskConfigs);
+      List<SegmentZKMetadata> completedSegments = getCompletedSegments(tableNameWithType, compactionConfigs);
+
+      if (completedSegments.isEmpty()) {
+        LOGGER.info("No completed segments were eligible for compaction for table: {}", tableNameWithType);
+        continue;
+      }
+
+      // get server to segment mappings
+      Map<String, List<String>> serverToSegments = _clusterInfoAccessor.getServerToSegmentsMap(tableNameWithType);
+      PinotHelixResourceManager pinotHelixResourceManager = _clusterInfoAccessor.getPinotHelixResourceManager();
+      BiMap<String, String> serverToEndpoints;
+      try {
+        serverToEndpoints = pinotHelixResourceManager.getDataInstanceAdminEndpoints(serverToSegments.keySet());
+      } catch (InvalidConfigException e) {
+        throw new RuntimeException(e);
+      }
+
+      Map<String, SegmentZKMetadata> completedSegmentsMap = completedSegments.stream()
+          .collect(Collectors.toMap(SegmentZKMetadata::getSegmentName, Function.identity()));
+
+      List<String> validDocIdUrls;
+      try {
+        validDocIdUrls = getValidDocIdMetadataUrls(
+            serverToSegments, serverToEndpoints, tableNameWithType, completedSegmentsMap);
+      } catch (URISyntaxException e) {
+        throw new RuntimeException(e);
+      }
+
+      // request the urls from the servers
+      CompletionServiceHelper completionServiceHelper = new CompletionServiceHelper(
+            Executors.newCachedThreadPool(), new MultiThreadedHttpConnectionManager(), serverToEndpoints.inverse());
+      CompletionServiceHelper.CompletionServiceResponse serviceResponse =
+            completionServiceHelper.doMultiGetRequest(validDocIdUrls, tableNameWithType, false, 3000);
+
+      SegmentSelectionResult segmentSelectionResult =
+          processValidDocIdMetadata(compactionConfigs, completedSegmentsMap, serviceResponse._httpResponses.entrySet());
+
+      if (!segmentSelectionResult.getSegmentsForDeletion().isEmpty()) {
+          pinotHelixResourceManager.deleteSegments(
+              tableNameWithType, segmentSelectionResult.getSegmentsForDeletion(), "0d");
+      }
+
+      int numTasks = 0;
+      int maxTasks = getMaxTasks(taskType, tableNameWithType, taskConfigs);
+      for (SegmentZKMetadata segment : segmentSelectionResult.getSegmentsForCompaction()) {
+        if (numTasks == maxTasks) {
+          break;
+        }
+        Map<String, String> configs = new HashMap<>();
+        configs.put(MinionConstants.TABLE_NAME_KEY, tableNameWithType);
+        configs.put(MinionConstants.SEGMENT_NAME_KEY, segment.getSegmentName());
+        configs.put(MinionConstants.DOWNLOAD_URL_KEY, segment.getDownloadUrl());
+        configs.put(MinionConstants.UPLOAD_URL_KEY, _clusterInfoAccessor.getVipUrl() + "/segments");
+        configs.put(MinionConstants.ORIGINAL_SEGMENT_CRC_KEY, String.valueOf(segment.getCrc()));
+        pinotTaskConfigs.add(new PinotTaskConfig(UpsertCompactionTask.TASK_TYPE, configs));
+        numTasks++;
+      }
+      LOGGER.info("Finished generating {} tasks configs for table: {} " + "for task: {}",
+          numTasks, tableNameWithType, taskType);
+    }
+    return pinotTaskConfigs;
+  }
+
+  @VisibleForTesting
+  public static SegmentSelectionResult processValidDocIdMetadata(Map<String, String> compactionConfigs,
+      Map<String, SegmentZKMetadata> completedSegmentsMap, Set<Map.Entry<String, String>> responseSet) {
+    double invalidRecordsThresholdPercent =
+        Double.parseDouble(compactionConfigs.getOrDefault(UpsertCompactionTask.INVALID_RECORDS_THRESHOLD_PERCENT,
+            String.valueOf(DEFAULT_INVALID_RECORDS_THRESHOLD_PERCENT)));
+    long minRecordCount =
+        Long.parseLong(compactionConfigs.getOrDefault(UpsertCompactionTask.MIN_RECORD_COUNT,
+            String.valueOf(DEFAULT_MIN_RECORD_COUNT)));
+    List<SegmentZKMetadata> segmentsForCompaction = new ArrayList<>();
+    List<String> segmentsForDeletion = new ArrayList<>();
+    for (Map.Entry<String, String> streamResponse : responseSet) {
+      JsonNode allValidDocIdMetadata;
+      try {
+        allValidDocIdMetadata = JsonUtils.stringToJsonNode(streamResponse.getValue());
+      } catch (IOException e) {
+        LOGGER.error("Unable to parse validDocIdMetadata response for: {}", streamResponse.getKey());
+        continue;
+      }
+      Iterator<JsonNode> iterator = allValidDocIdMetadata.elements();
+      while (iterator.hasNext()) {
+        JsonNode validDocIdMetadata = iterator.next();
+        long invalidRecordCount = validDocIdMetadata.get("totalInvalidDocs").asLong();
+        String segmentName = validDocIdMetadata.get("segmentName").asText();
+        SegmentZKMetadata segment = completedSegmentsMap.get(segmentName);
+        double invalidRecordPercent = ((double) invalidRecordCount / segment.getTotalDocs()) * 100;
+        if (invalidRecordCount == segment.getTotalDocs()) {
+          segmentsForDeletion.add(segment.getSegmentName());
+        } else if (invalidRecordPercent > invalidRecordsThresholdPercent && invalidRecordCount > minRecordCount) {
+          segmentsForCompaction.add(segment);
+        }
+      }
+    }
+    return new SegmentSelectionResult(segmentsForCompaction, segmentsForDeletion);
+  }
+
+  @VisibleForTesting
+  public static List<String> getValidDocIdMetadataUrls(Map<String, List<String>> serverToSegments,
+      BiMap<String, String> serverToEndpoints, String tableNameWithType,
+      Map<String, SegmentZKMetadata> completedSegments) throws URISyntaxException {
+    List<String> urls = new ArrayList<>();
+    for (Map.Entry<String, List<String>> entry : serverToSegments.entrySet()) {
+      String server = entry.getKey();
+      List<String> segmentNames = entry.getValue();
+      URIBuilder uriBuilder = new URIBuilder(serverToEndpoints.get(server)).setPath(
+          String.format("/tables/%s/validDocIdMetadata", tableNameWithType));
+      int completedSegmentCountPerServer = 0;
+      for (String segmentName : segmentNames) {
+        if (completedSegments.containsKey(segmentName)) {
+          completedSegmentCountPerServer++;
+          uriBuilder.addParameter("segmentNames", segmentName);
+        }
+      }
+      if (completedSegmentCountPerServer > 0) {
+        // only add to the list if the server has completed segments
+        urls.add(uriBuilder.toString());
+      }
+    }
+    return urls;
+  }
+
+  private List<SegmentZKMetadata> getCompletedSegments(String tableNameWithType,
+      Map<String, String> compactionConfigs) {
+    List<SegmentZKMetadata> completedSegments = new ArrayList<>();
+    String bufferPeriod = compactionConfigs.getOrDefault(
+        UpsertCompactionTask.BUFFER_TIME_PERIOD_KEY, DEFAULT_BUFFER_PERIOD);
+    long bufferMs = TimeUtils.convertPeriodToMillis(bufferPeriod);
+    List<SegmentZKMetadata> allSegments = _clusterInfoAccessor.getSegmentsZKMetadata(tableNameWithType);
+    for (SegmentZKMetadata segment : allSegments) {
+      CommonConstants.Segment.Realtime.Status status = segment.getStatus();
+      // initial segments selection based on status and age
+      if (status.isCompleted()) {
+        boolean endedWithinBufferPeriod = segment.getEndTimeMs() <= (System.currentTimeMillis() - bufferMs);
+        boolean endsInTheFuture = segment.getEndTimeMs() > System.currentTimeMillis();
+        if (endedWithinBufferPeriod || endsInTheFuture) {
+          completedSegments.add(segment);
+        }
+      }
+    }
+    return completedSegments;
+  }
+
+  @VisibleForTesting
+  public static int getMaxTasks(String taskType, String tableNameWithType, Map<String, String> taskConfigs) {
+    int maxTasks = Integer.MAX_VALUE;
+    String tableMaxNumTasksConfig = taskConfigs.get(MinionConstants.TABLE_MAX_NUM_TASKS_KEY);
+    if (tableMaxNumTasksConfig != null) {
+      try {
+        maxTasks = Integer.parseInt(tableMaxNumTasksConfig);
+      } catch (Exception e) {
+        LOGGER.warn("MaxNumTasks have been wrongly set for table : {}, and task {}", tableNameWithType, taskType);
+      }
+    }
+    return maxTasks;
+  }
+
+  private static final String[] VALID_CONFIG_KEYS = {
+      UpsertCompactionTask.BUFFER_TIME_PERIOD_KEY,
+      UpsertCompactionTask.INVALID_RECORDS_THRESHOLD_PERCENT,
+      UpsertCompactionTask.MIN_RECORD_COUNT
+  };
+
+  private Map<String, String> getCompactionConfigs(Map<String, String> taskConfig) {
+    Map<String, String> compactionConfigs = new HashMap<>();
+
+    for (Map.Entry<String, String> entry : taskConfig.entrySet()) {
+      String key = entry.getKey();
+      for (String configKey : VALID_CONFIG_KEYS) {
+        if (key.endsWith(configKey)) {
+          compactionConfigs.put(configKey, entry.getValue());
+        }
+      }
+    }
+
+    return compactionConfigs;
+  }
+
+  @VisibleForTesting
+  static boolean validate(TableConfig tableConfig) {
+    String taskType = MinionConstants.UpsertCompactionTask.TASK_TYPE;
+    String tableNameWithType = tableConfig.getTableName();
+    if (tableConfig.getTableType() == TableType.OFFLINE) {
+      String message = "Skip generation task: {} for table: {}, offline table is not supported";
+      LOGGER.warn(message, taskType, tableNameWithType);

Review Comment:
   (minor) Inline the message for readability



##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java:
##########
@@ -0,0 +1,292 @@
+/**
+ * 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.upsertcompaction;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.BiMap;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.pinot.common.exception.InvalidConfigException;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator;
+import org.apache.pinot.controller.util.CompletionServiceHelper;
+import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.common.MinionConstants.UpsertCompactionTask;
+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.TableType;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.TimeUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@TaskGenerator
+public class UpsertCompactionTaskGenerator extends BaseTaskGenerator {
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactionTaskGenerator.class);
+  private static final String DEFAULT_BUFFER_PERIOD = "7d";
+  private static final double DEFAULT_INVALID_RECORDS_THRESHOLD_PERCENT = 0.0;
+  private static final long DEFAULT_MIN_RECORD_COUNT = 0;
+  public static class SegmentSelectionResult {
+    private List<SegmentZKMetadata> _segmentsForCompaction;
+    private List<String> _segmentsForDeletion;
+    SegmentSelectionResult(List<SegmentZKMetadata> segmentsForCompaction, List<String> segmentsForDeletion) {
+      _segmentsForCompaction = segmentsForCompaction;
+      _segmentsForDeletion = segmentsForDeletion;
+    }
+    public List<SegmentZKMetadata> getSegmentsForCompaction() {
+      return _segmentsForCompaction;
+    }
+    public List<String> getSegmentsForDeletion() {
+      return _segmentsForDeletion;
+    }
+  }
+
+  @Override
+  public String getTaskType() {
+    return MinionConstants.UpsertCompactionTask.TASK_TYPE;
+  }
+
+  @Override
+  public List<PinotTaskConfig> generateTasks(List<TableConfig> tableConfigs) {
+    String taskType = MinionConstants.UpsertCompactionTask.TASK_TYPE;
+    List<PinotTaskConfig> pinotTaskConfigs = new ArrayList<>();
+    for (TableConfig tableConfig: tableConfigs) {
+      if (!validate(tableConfig)) {
+        continue;
+      }
+
+      String tableNameWithType = tableConfig.getTableName();
+      LOGGER.info("Start generating task configs for table: {} for task: {}",
+          tableNameWithType, taskType);
+
+      Map<String, String> taskConfigs = tableConfig.getTaskConfig().getConfigsForTaskType(taskType);
+      Map<String, String> compactionConfigs = getCompactionConfigs(taskConfigs);
+      List<SegmentZKMetadata> completedSegments = getCompletedSegments(tableNameWithType, compactionConfigs);
+
+      if (completedSegments.isEmpty()) {
+        LOGGER.info("No completed segments were eligible for compaction for table: {}", tableNameWithType);
+        continue;
+      }
+
+      // get server to segment mappings
+      Map<String, List<String>> serverToSegments = _clusterInfoAccessor.getServerToSegmentsMap(tableNameWithType);
+      PinotHelixResourceManager pinotHelixResourceManager = _clusterInfoAccessor.getPinotHelixResourceManager();
+      BiMap<String, String> serverToEndpoints;
+      try {
+        serverToEndpoints = pinotHelixResourceManager.getDataInstanceAdminEndpoints(serverToSegments.keySet());
+      } catch (InvalidConfigException e) {
+        throw new RuntimeException(e);
+      }
+
+      Map<String, SegmentZKMetadata> completedSegmentsMap = completedSegments.stream()
+          .collect(Collectors.toMap(SegmentZKMetadata::getSegmentName, Function.identity()));
+
+      List<String> validDocIdUrls;
+      try {
+        validDocIdUrls = getValidDocIdMetadataUrls(
+            serverToSegments, serverToEndpoints, tableNameWithType, completedSegmentsMap);
+      } catch (URISyntaxException e) {
+        throw new RuntimeException(e);
+      }
+
+      // request the urls from the servers
+      CompletionServiceHelper completionServiceHelper = new CompletionServiceHelper(
+            Executors.newCachedThreadPool(), new MultiThreadedHttpConnectionManager(), serverToEndpoints.inverse());
+      CompletionServiceHelper.CompletionServiceResponse serviceResponse =
+            completionServiceHelper.doMultiGetRequest(validDocIdUrls, tableNameWithType, false, 3000);
+
+      SegmentSelectionResult segmentSelectionResult =
+          processValidDocIdMetadata(compactionConfigs, completedSegmentsMap, serviceResponse._httpResponses.entrySet());
+
+      if (!segmentSelectionResult.getSegmentsForDeletion().isEmpty()) {
+          pinotHelixResourceManager.deleteSegments(

Review Comment:
   (code format) The indentation is incorrect. Can you auto-reformat the changes



##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java:
##########
@@ -0,0 +1,292 @@
+/**
+ * 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.upsertcompaction;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.BiMap;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.pinot.common.exception.InvalidConfigException;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator;
+import org.apache.pinot.controller.util.CompletionServiceHelper;
+import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.common.MinionConstants.UpsertCompactionTask;
+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.TableType;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.TimeUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@TaskGenerator
+public class UpsertCompactionTaskGenerator extends BaseTaskGenerator {
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactionTaskGenerator.class);
+  private static final String DEFAULT_BUFFER_PERIOD = "7d";
+  private static final double DEFAULT_INVALID_RECORDS_THRESHOLD_PERCENT = 0.0;
+  private static final long DEFAULT_MIN_RECORD_COUNT = 0;
+  public static class SegmentSelectionResult {
+    private List<SegmentZKMetadata> _segmentsForCompaction;
+    private List<String> _segmentsForDeletion;
+    SegmentSelectionResult(List<SegmentZKMetadata> segmentsForCompaction, List<String> segmentsForDeletion) {
+      _segmentsForCompaction = segmentsForCompaction;
+      _segmentsForDeletion = segmentsForDeletion;
+    }
+    public List<SegmentZKMetadata> getSegmentsForCompaction() {
+      return _segmentsForCompaction;
+    }
+    public List<String> getSegmentsForDeletion() {
+      return _segmentsForDeletion;
+    }
+  }
+
+  @Override
+  public String getTaskType() {
+    return MinionConstants.UpsertCompactionTask.TASK_TYPE;
+  }
+
+  @Override
+  public List<PinotTaskConfig> generateTasks(List<TableConfig> tableConfigs) {
+    String taskType = MinionConstants.UpsertCompactionTask.TASK_TYPE;
+    List<PinotTaskConfig> pinotTaskConfigs = new ArrayList<>();
+    for (TableConfig tableConfig: tableConfigs) {
+      if (!validate(tableConfig)) {
+        continue;
+      }
+
+      String tableNameWithType = tableConfig.getTableName();
+      LOGGER.info("Start generating task configs for table: {} for task: {}",
+          tableNameWithType, taskType);
+
+      Map<String, String> taskConfigs = tableConfig.getTaskConfig().getConfigsForTaskType(taskType);
+      Map<String, String> compactionConfigs = getCompactionConfigs(taskConfigs);
+      List<SegmentZKMetadata> completedSegments = getCompletedSegments(tableNameWithType, compactionConfigs);
+
+      if (completedSegments.isEmpty()) {
+        LOGGER.info("No completed segments were eligible for compaction for table: {}", tableNameWithType);
+        continue;
+      }
+
+      // get server to segment mappings
+      Map<String, List<String>> serverToSegments = _clusterInfoAccessor.getServerToSegmentsMap(tableNameWithType);
+      PinotHelixResourceManager pinotHelixResourceManager = _clusterInfoAccessor.getPinotHelixResourceManager();
+      BiMap<String, String> serverToEndpoints;
+      try {
+        serverToEndpoints = pinotHelixResourceManager.getDataInstanceAdminEndpoints(serverToSegments.keySet());
+      } catch (InvalidConfigException e) {
+        throw new RuntimeException(e);
+      }
+
+      Map<String, SegmentZKMetadata> completedSegmentsMap = completedSegments.stream()
+          .collect(Collectors.toMap(SegmentZKMetadata::getSegmentName, Function.identity()));
+
+      List<String> validDocIdUrls;
+      try {
+        validDocIdUrls = getValidDocIdMetadataUrls(
+            serverToSegments, serverToEndpoints, tableNameWithType, completedSegmentsMap);
+      } catch (URISyntaxException e) {
+        throw new RuntimeException(e);
+      }
+
+      // request the urls from the servers
+      CompletionServiceHelper completionServiceHelper = new CompletionServiceHelper(
+            Executors.newCachedThreadPool(), new MultiThreadedHttpConnectionManager(), serverToEndpoints.inverse());
+      CompletionServiceHelper.CompletionServiceResponse serviceResponse =
+            completionServiceHelper.doMultiGetRequest(validDocIdUrls, tableNameWithType, false, 3000);
+
+      SegmentSelectionResult segmentSelectionResult =
+          processValidDocIdMetadata(compactionConfigs, completedSegmentsMap, serviceResponse._httpResponses.entrySet());
+
+      if (!segmentSelectionResult.getSegmentsForDeletion().isEmpty()) {
+          pinotHelixResourceManager.deleteSegments(
+              tableNameWithType, segmentSelectionResult.getSegmentsForDeletion(), "0d");
+      }
+
+      int numTasks = 0;
+      int maxTasks = getMaxTasks(taskType, tableNameWithType, taskConfigs);
+      for (SegmentZKMetadata segment : segmentSelectionResult.getSegmentsForCompaction()) {
+        if (numTasks == maxTasks) {
+          break;
+        }
+        Map<String, String> configs = new HashMap<>();
+        configs.put(MinionConstants.TABLE_NAME_KEY, tableNameWithType);
+        configs.put(MinionConstants.SEGMENT_NAME_KEY, segment.getSegmentName());
+        configs.put(MinionConstants.DOWNLOAD_URL_KEY, segment.getDownloadUrl());
+        configs.put(MinionConstants.UPLOAD_URL_KEY, _clusterInfoAccessor.getVipUrl() + "/segments");
+        configs.put(MinionConstants.ORIGINAL_SEGMENT_CRC_KEY, String.valueOf(segment.getCrc()));
+        pinotTaskConfigs.add(new PinotTaskConfig(UpsertCompactionTask.TASK_TYPE, configs));
+        numTasks++;
+      }
+      LOGGER.info("Finished generating {} tasks configs for table: {} " + "for task: {}",
+          numTasks, tableNameWithType, taskType);
+    }
+    return pinotTaskConfigs;
+  }
+
+  @VisibleForTesting
+  public static SegmentSelectionResult processValidDocIdMetadata(Map<String, String> compactionConfigs,
+      Map<String, SegmentZKMetadata> completedSegmentsMap, Set<Map.Entry<String, String>> responseSet) {
+    double invalidRecordsThresholdPercent =
+        Double.parseDouble(compactionConfigs.getOrDefault(UpsertCompactionTask.INVALID_RECORDS_THRESHOLD_PERCENT,
+            String.valueOf(DEFAULT_INVALID_RECORDS_THRESHOLD_PERCENT)));
+    long minRecordCount =
+        Long.parseLong(compactionConfigs.getOrDefault(UpsertCompactionTask.MIN_RECORD_COUNT,
+            String.valueOf(DEFAULT_MIN_RECORD_COUNT)));
+    List<SegmentZKMetadata> segmentsForCompaction = new ArrayList<>();
+    List<String> segmentsForDeletion = new ArrayList<>();
+    for (Map.Entry<String, String> streamResponse : responseSet) {
+      JsonNode allValidDocIdMetadata;
+      try {
+        allValidDocIdMetadata = JsonUtils.stringToJsonNode(streamResponse.getValue());
+      } catch (IOException e) {
+        LOGGER.error("Unable to parse validDocIdMetadata response for: {}", streamResponse.getKey());
+        continue;
+      }
+      Iterator<JsonNode> iterator = allValidDocIdMetadata.elements();
+      while (iterator.hasNext()) {
+        JsonNode validDocIdMetadata = iterator.next();
+        long invalidRecordCount = validDocIdMetadata.get("totalInvalidDocs").asLong();
+        String segmentName = validDocIdMetadata.get("segmentName").asText();
+        SegmentZKMetadata segment = completedSegmentsMap.get(segmentName);
+        double invalidRecordPercent = ((double) invalidRecordCount / segment.getTotalDocs()) * 100;
+        if (invalidRecordCount == segment.getTotalDocs()) {
+          segmentsForDeletion.add(segment.getSegmentName());
+        } else if (invalidRecordPercent > invalidRecordsThresholdPercent && invalidRecordCount > minRecordCount) {
+          segmentsForCompaction.add(segment);
+        }
+      }
+    }
+    return new SegmentSelectionResult(segmentsForCompaction, segmentsForDeletion);
+  }
+
+  @VisibleForTesting
+  public static List<String> getValidDocIdMetadataUrls(Map<String, List<String>> serverToSegments,
+      BiMap<String, String> serverToEndpoints, String tableNameWithType,
+      Map<String, SegmentZKMetadata> completedSegments) throws URISyntaxException {
+    List<String> urls = new ArrayList<>();
+    for (Map.Entry<String, List<String>> entry : serverToSegments.entrySet()) {
+      String server = entry.getKey();
+      List<String> segmentNames = entry.getValue();
+      URIBuilder uriBuilder = new URIBuilder(serverToEndpoints.get(server)).setPath(
+          String.format("/tables/%s/validDocIdMetadata", tableNameWithType));
+      int completedSegmentCountPerServer = 0;
+      for (String segmentName : segmentNames) {
+        if (completedSegments.containsKey(segmentName)) {
+          completedSegmentCountPerServer++;
+          uriBuilder.addParameter("segmentNames", segmentName);
+        }
+      }
+      if (completedSegmentCountPerServer > 0) {
+        // only add to the list if the server has completed segments
+        urls.add(uriBuilder.toString());
+      }
+    }
+    return urls;
+  }
+
+  private List<SegmentZKMetadata> getCompletedSegments(String tableNameWithType,
+      Map<String, String> compactionConfigs) {
+    List<SegmentZKMetadata> completedSegments = new ArrayList<>();
+    String bufferPeriod = compactionConfigs.getOrDefault(
+        UpsertCompactionTask.BUFFER_TIME_PERIOD_KEY, DEFAULT_BUFFER_PERIOD);
+    long bufferMs = TimeUtils.convertPeriodToMillis(bufferPeriod);
+    List<SegmentZKMetadata> allSegments = _clusterInfoAccessor.getSegmentsZKMetadata(tableNameWithType);
+    for (SegmentZKMetadata segment : allSegments) {
+      CommonConstants.Segment.Realtime.Status status = segment.getStatus();
+      // initial segments selection based on status and age
+      if (status.isCompleted()) {
+        boolean endedWithinBufferPeriod = segment.getEndTimeMs() <= (System.currentTimeMillis() - bufferMs);
+        boolean endsInTheFuture = segment.getEndTimeMs() > System.currentTimeMillis();
+        if (endedWithinBufferPeriod || endsInTheFuture) {
+          completedSegments.add(segment);
+        }
+      }
+    }
+    return completedSegments;
+  }
+
+  @VisibleForTesting
+  public static int getMaxTasks(String taskType, String tableNameWithType, Map<String, String> taskConfigs) {
+    int maxTasks = Integer.MAX_VALUE;
+    String tableMaxNumTasksConfig = taskConfigs.get(MinionConstants.TABLE_MAX_NUM_TASKS_KEY);
+    if (tableMaxNumTasksConfig != null) {
+      try {
+        maxTasks = Integer.parseInt(tableMaxNumTasksConfig);
+      } catch (Exception e) {
+        LOGGER.warn("MaxNumTasks have been wrongly set for table : {}, and task {}", tableNameWithType, taskType);
+      }
+    }
+    return maxTasks;
+  }
+
+  private static final String[] VALID_CONFIG_KEYS = {
+      UpsertCompactionTask.BUFFER_TIME_PERIOD_KEY,
+      UpsertCompactionTask.INVALID_RECORDS_THRESHOLD_PERCENT,
+      UpsertCompactionTask.MIN_RECORD_COUNT
+  };
+
+  private Map<String, String> getCompactionConfigs(Map<String, String> taskConfig) {
+    Map<String, String> compactionConfigs = new HashMap<>();
+
+    for (Map.Entry<String, String> entry : taskConfig.entrySet()) {
+      String key = entry.getKey();
+      for (String configKey : VALID_CONFIG_KEYS) {
+        if (key.endsWith(configKey)) {
+          compactionConfigs.put(configKey, entry.getValue());
+        }
+      }
+    }
+
+    return compactionConfigs;
+  }
+
+  @VisibleForTesting
+  static boolean validate(TableConfig tableConfig) {
+    String taskType = MinionConstants.UpsertCompactionTask.TASK_TYPE;
+    String tableNameWithType = tableConfig.getTableName();
+    if (tableConfig.getTableType() == TableType.OFFLINE) {
+      String message = "Skip generation task: {} for table: {}, offline table is not supported";
+      LOGGER.warn(message, taskType, tableNameWithType);
+      return false;
+    }
+    if (!tableConfig.isUpsertEnabled()) {
+      String message = "Skip generation task: {} for table: {}, table without upsert enabled is not supported";
+      LOGGER.warn(message, taskType, tableNameWithType);

Review Comment:
   (minor) Same here



##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java:
##########
@@ -0,0 +1,292 @@
+/**
+ * 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.upsertcompaction;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.BiMap;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.pinot.common.exception.InvalidConfigException;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator;
+import org.apache.pinot.controller.util.CompletionServiceHelper;
+import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.common.MinionConstants.UpsertCompactionTask;
+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.TableType;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.TimeUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@TaskGenerator
+public class UpsertCompactionTaskGenerator extends BaseTaskGenerator {
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactionTaskGenerator.class);
+  private static final String DEFAULT_BUFFER_PERIOD = "7d";
+  private static final double DEFAULT_INVALID_RECORDS_THRESHOLD_PERCENT = 0.0;
+  private static final long DEFAULT_MIN_RECORD_COUNT = 0;
+  public static class SegmentSelectionResult {
+    private List<SegmentZKMetadata> _segmentsForCompaction;
+    private List<String> _segmentsForDeletion;
+    SegmentSelectionResult(List<SegmentZKMetadata> segmentsForCompaction, List<String> segmentsForDeletion) {
+      _segmentsForCompaction = segmentsForCompaction;
+      _segmentsForDeletion = segmentsForDeletion;
+    }
+    public List<SegmentZKMetadata> getSegmentsForCompaction() {
+      return _segmentsForCompaction;
+    }
+    public List<String> getSegmentsForDeletion() {
+      return _segmentsForDeletion;
+    }
+  }
+
+  @Override
+  public String getTaskType() {
+    return MinionConstants.UpsertCompactionTask.TASK_TYPE;
+  }
+
+  @Override
+  public List<PinotTaskConfig> generateTasks(List<TableConfig> tableConfigs) {
+    String taskType = MinionConstants.UpsertCompactionTask.TASK_TYPE;
+    List<PinotTaskConfig> pinotTaskConfigs = new ArrayList<>();
+    for (TableConfig tableConfig: tableConfigs) {
+      if (!validate(tableConfig)) {
+        continue;
+      }
+
+      String tableNameWithType = tableConfig.getTableName();
+      LOGGER.info("Start generating task configs for table: {} for task: {}",
+          tableNameWithType, taskType);
+
+      Map<String, String> taskConfigs = tableConfig.getTaskConfig().getConfigsForTaskType(taskType);
+      Map<String, String> compactionConfigs = getCompactionConfigs(taskConfigs);
+      List<SegmentZKMetadata> completedSegments = getCompletedSegments(tableNameWithType, compactionConfigs);
+
+      if (completedSegments.isEmpty()) {
+        LOGGER.info("No completed segments were eligible for compaction for table: {}", tableNameWithType);
+        continue;
+      }
+
+      // get server to segment mappings
+      Map<String, List<String>> serverToSegments = _clusterInfoAccessor.getServerToSegmentsMap(tableNameWithType);
+      PinotHelixResourceManager pinotHelixResourceManager = _clusterInfoAccessor.getPinotHelixResourceManager();
+      BiMap<String, String> serverToEndpoints;
+      try {
+        serverToEndpoints = pinotHelixResourceManager.getDataInstanceAdminEndpoints(serverToSegments.keySet());
+      } catch (InvalidConfigException e) {
+        throw new RuntimeException(e);
+      }
+
+      Map<String, SegmentZKMetadata> completedSegmentsMap = completedSegments.stream()
+          .collect(Collectors.toMap(SegmentZKMetadata::getSegmentName, Function.identity()));
+
+      List<String> validDocIdUrls;
+      try {
+        validDocIdUrls = getValidDocIdMetadataUrls(
+            serverToSegments, serverToEndpoints, tableNameWithType, completedSegmentsMap);
+      } catch (URISyntaxException e) {
+        throw new RuntimeException(e);
+      }
+
+      // request the urls from the servers
+      CompletionServiceHelper completionServiceHelper = new CompletionServiceHelper(
+            Executors.newCachedThreadPool(), new MultiThreadedHttpConnectionManager(), serverToEndpoints.inverse());
+      CompletionServiceHelper.CompletionServiceResponse serviceResponse =
+            completionServiceHelper.doMultiGetRequest(validDocIdUrls, tableNameWithType, false, 3000);
+
+      SegmentSelectionResult segmentSelectionResult =
+          processValidDocIdMetadata(compactionConfigs, completedSegmentsMap, serviceResponse._httpResponses.entrySet());
+
+      if (!segmentSelectionResult.getSegmentsForDeletion().isEmpty()) {
+          pinotHelixResourceManager.deleteSegments(

Review Comment:
   Please add some logs here



##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java:
##########
@@ -0,0 +1,292 @@
+/**
+ * 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.upsertcompaction;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.BiMap;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.pinot.common.exception.InvalidConfigException;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator;
+import org.apache.pinot.controller.util.CompletionServiceHelper;
+import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.common.MinionConstants.UpsertCompactionTask;
+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.TableType;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.TimeUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@TaskGenerator
+public class UpsertCompactionTaskGenerator extends BaseTaskGenerator {
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactionTaskGenerator.class);
+  private static final String DEFAULT_BUFFER_PERIOD = "7d";
+  private static final double DEFAULT_INVALID_RECORDS_THRESHOLD_PERCENT = 0.0;
+  private static final long DEFAULT_MIN_RECORD_COUNT = 0;
+  public static class SegmentSelectionResult {
+    private List<SegmentZKMetadata> _segmentsForCompaction;
+    private List<String> _segmentsForDeletion;
+    SegmentSelectionResult(List<SegmentZKMetadata> segmentsForCompaction, List<String> segmentsForDeletion) {
+      _segmentsForCompaction = segmentsForCompaction;
+      _segmentsForDeletion = segmentsForDeletion;
+    }
+    public List<SegmentZKMetadata> getSegmentsForCompaction() {
+      return _segmentsForCompaction;
+    }
+    public List<String> getSegmentsForDeletion() {
+      return _segmentsForDeletion;
+    }
+  }
+
+  @Override
+  public String getTaskType() {
+    return MinionConstants.UpsertCompactionTask.TASK_TYPE;
+  }
+
+  @Override
+  public List<PinotTaskConfig> generateTasks(List<TableConfig> tableConfigs) {
+    String taskType = MinionConstants.UpsertCompactionTask.TASK_TYPE;
+    List<PinotTaskConfig> pinotTaskConfigs = new ArrayList<>();
+    for (TableConfig tableConfig: tableConfigs) {
+      if (!validate(tableConfig)) {
+        continue;
+      }
+
+      String tableNameWithType = tableConfig.getTableName();
+      LOGGER.info("Start generating task configs for table: {} for task: {}",
+          tableNameWithType, taskType);
+
+      Map<String, String> taskConfigs = tableConfig.getTaskConfig().getConfigsForTaskType(taskType);
+      Map<String, String> compactionConfigs = getCompactionConfigs(taskConfigs);
+      List<SegmentZKMetadata> completedSegments = getCompletedSegments(tableNameWithType, compactionConfigs);
+
+      if (completedSegments.isEmpty()) {
+        LOGGER.info("No completed segments were eligible for compaction for table: {}", tableNameWithType);
+        continue;
+      }
+
+      // get server to segment mappings
+      Map<String, List<String>> serverToSegments = _clusterInfoAccessor.getServerToSegmentsMap(tableNameWithType);
+      PinotHelixResourceManager pinotHelixResourceManager = _clusterInfoAccessor.getPinotHelixResourceManager();
+      BiMap<String, String> serverToEndpoints;
+      try {
+        serverToEndpoints = pinotHelixResourceManager.getDataInstanceAdminEndpoints(serverToSegments.keySet());
+      } catch (InvalidConfigException e) {
+        throw new RuntimeException(e);
+      }
+
+      Map<String, SegmentZKMetadata> completedSegmentsMap = completedSegments.stream()
+          .collect(Collectors.toMap(SegmentZKMetadata::getSegmentName, Function.identity()));
+
+      List<String> validDocIdUrls;
+      try {
+        validDocIdUrls = getValidDocIdMetadataUrls(
+            serverToSegments, serverToEndpoints, tableNameWithType, completedSegmentsMap);
+      } catch (URISyntaxException e) {
+        throw new RuntimeException(e);
+      }
+
+      // request the urls from the servers
+      CompletionServiceHelper completionServiceHelper = new CompletionServiceHelper(
+            Executors.newCachedThreadPool(), new MultiThreadedHttpConnectionManager(), serverToEndpoints.inverse());
+      CompletionServiceHelper.CompletionServiceResponse serviceResponse =
+            completionServiceHelper.doMultiGetRequest(validDocIdUrls, tableNameWithType, false, 3000);
+
+      SegmentSelectionResult segmentSelectionResult =
+          processValidDocIdMetadata(compactionConfigs, completedSegmentsMap, serviceResponse._httpResponses.entrySet());
+
+      if (!segmentSelectionResult.getSegmentsForDeletion().isEmpty()) {
+          pinotHelixResourceManager.deleteSegments(
+              tableNameWithType, segmentSelectionResult.getSegmentsForDeletion(), "0d");
+      }
+
+      int numTasks = 0;
+      int maxTasks = getMaxTasks(taskType, tableNameWithType, taskConfigs);
+      for (SegmentZKMetadata segment : segmentSelectionResult.getSegmentsForCompaction()) {
+        if (numTasks == maxTasks) {
+          break;
+        }
+        Map<String, String> configs = new HashMap<>();
+        configs.put(MinionConstants.TABLE_NAME_KEY, tableNameWithType);
+        configs.put(MinionConstants.SEGMENT_NAME_KEY, segment.getSegmentName());
+        configs.put(MinionConstants.DOWNLOAD_URL_KEY, segment.getDownloadUrl());
+        configs.put(MinionConstants.UPLOAD_URL_KEY, _clusterInfoAccessor.getVipUrl() + "/segments");
+        configs.put(MinionConstants.ORIGINAL_SEGMENT_CRC_KEY, String.valueOf(segment.getCrc()));
+        pinotTaskConfigs.add(new PinotTaskConfig(UpsertCompactionTask.TASK_TYPE, configs));
+        numTasks++;
+      }
+      LOGGER.info("Finished generating {} tasks configs for table: {} " + "for task: {}",
+          numTasks, tableNameWithType, taskType);
+    }
+    return pinotTaskConfigs;
+  }
+
+  @VisibleForTesting
+  public static SegmentSelectionResult processValidDocIdMetadata(Map<String, String> compactionConfigs,
+      Map<String, SegmentZKMetadata> completedSegmentsMap, Set<Map.Entry<String, String>> responseSet) {
+    double invalidRecordsThresholdPercent =
+        Double.parseDouble(compactionConfigs.getOrDefault(UpsertCompactionTask.INVALID_RECORDS_THRESHOLD_PERCENT,
+            String.valueOf(DEFAULT_INVALID_RECORDS_THRESHOLD_PERCENT)));
+    long minRecordCount =
+        Long.parseLong(compactionConfigs.getOrDefault(UpsertCompactionTask.MIN_RECORD_COUNT,
+            String.valueOf(DEFAULT_MIN_RECORD_COUNT)));
+    List<SegmentZKMetadata> segmentsForCompaction = new ArrayList<>();
+    List<String> segmentsForDeletion = new ArrayList<>();
+    for (Map.Entry<String, String> streamResponse : responseSet) {
+      JsonNode allValidDocIdMetadata;
+      try {
+        allValidDocIdMetadata = JsonUtils.stringToJsonNode(streamResponse.getValue());
+      } catch (IOException e) {
+        LOGGER.error("Unable to parse validDocIdMetadata response for: {}", streamResponse.getKey());
+        continue;
+      }
+      Iterator<JsonNode> iterator = allValidDocIdMetadata.elements();
+      while (iterator.hasNext()) {
+        JsonNode validDocIdMetadata = iterator.next();
+        long invalidRecordCount = validDocIdMetadata.get("totalInvalidDocs").asLong();
+        String segmentName = validDocIdMetadata.get("segmentName").asText();
+        SegmentZKMetadata segment = completedSegmentsMap.get(segmentName);
+        double invalidRecordPercent = ((double) invalidRecordCount / segment.getTotalDocs()) * 100;
+        if (invalidRecordCount == segment.getTotalDocs()) {
+          segmentsForDeletion.add(segment.getSegmentName());
+        } else if (invalidRecordPercent > invalidRecordsThresholdPercent && invalidRecordCount > minRecordCount) {
+          segmentsForCompaction.add(segment);
+        }
+      }
+    }
+    return new SegmentSelectionResult(segmentsForCompaction, segmentsForDeletion);
+  }
+
+  @VisibleForTesting
+  public static List<String> getValidDocIdMetadataUrls(Map<String, List<String>> serverToSegments,
+      BiMap<String, String> serverToEndpoints, String tableNameWithType,
+      Map<String, SegmentZKMetadata> completedSegments) throws URISyntaxException {
+    List<String> urls = new ArrayList<>();
+    for (Map.Entry<String, List<String>> entry : serverToSegments.entrySet()) {
+      String server = entry.getKey();
+      List<String> segmentNames = entry.getValue();
+      URIBuilder uriBuilder = new URIBuilder(serverToEndpoints.get(server)).setPath(
+          String.format("/tables/%s/validDocIdMetadata", tableNameWithType));
+      int completedSegmentCountPerServer = 0;
+      for (String segmentName : segmentNames) {
+        if (completedSegments.containsKey(segmentName)) {
+          completedSegmentCountPerServer++;
+          uriBuilder.addParameter("segmentNames", segmentName);
+        }
+      }
+      if (completedSegmentCountPerServer > 0) {
+        // only add to the list if the server has completed segments
+        urls.add(uriBuilder.toString());
+      }
+    }
+    return urls;
+  }
+
+  private List<SegmentZKMetadata> getCompletedSegments(String tableNameWithType,
+      Map<String, String> compactionConfigs) {
+    List<SegmentZKMetadata> completedSegments = new ArrayList<>();
+    String bufferPeriod = compactionConfigs.getOrDefault(
+        UpsertCompactionTask.BUFFER_TIME_PERIOD_KEY, DEFAULT_BUFFER_PERIOD);
+    long bufferMs = TimeUtils.convertPeriodToMillis(bufferPeriod);
+    List<SegmentZKMetadata> allSegments = _clusterInfoAccessor.getSegmentsZKMetadata(tableNameWithType);
+    for (SegmentZKMetadata segment : allSegments) {
+      CommonConstants.Segment.Realtime.Status status = segment.getStatus();
+      // initial segments selection based on status and age
+      if (status.isCompleted()) {
+        boolean endedWithinBufferPeriod = segment.getEndTimeMs() <= (System.currentTimeMillis() - bufferMs);
+        boolean endsInTheFuture = segment.getEndTimeMs() > System.currentTimeMillis();
+        if (endedWithinBufferPeriod || endsInTheFuture) {
+          completedSegments.add(segment);
+        }
+      }
+    }
+    return completedSegments;
+  }
+
+  @VisibleForTesting
+  public static int getMaxTasks(String taskType, String tableNameWithType, Map<String, String> taskConfigs) {
+    int maxTasks = Integer.MAX_VALUE;
+    String tableMaxNumTasksConfig = taskConfigs.get(MinionConstants.TABLE_MAX_NUM_TASKS_KEY);
+    if (tableMaxNumTasksConfig != null) {
+      try {
+        maxTasks = Integer.parseInt(tableMaxNumTasksConfig);
+      } catch (Exception e) {
+        LOGGER.warn("MaxNumTasks have been wrongly set for table : {}, and task {}", tableNameWithType, taskType);
+      }
+    }
+    return maxTasks;
+  }
+
+  private static final String[] VALID_CONFIG_KEYS = {
+      UpsertCompactionTask.BUFFER_TIME_PERIOD_KEY,
+      UpsertCompactionTask.INVALID_RECORDS_THRESHOLD_PERCENT,
+      UpsertCompactionTask.MIN_RECORD_COUNT
+  };
+
+  private Map<String, String> getCompactionConfigs(Map<String, String> taskConfig) {
+    Map<String, String> compactionConfigs = new HashMap<>();
+
+    for (Map.Entry<String, String> entry : taskConfig.entrySet()) {
+      String key = entry.getKey();
+      for (String configKey : VALID_CONFIG_KEYS) {
+        if (key.endsWith(configKey)) {

Review Comment:
   Why is this `endsWith()`?



##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java:
##########
@@ -0,0 +1,245 @@
+/**
+ * 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.upsertcompaction;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.BiMap;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.pinot.common.exception.InvalidConfigException;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator;
+import org.apache.pinot.controller.util.CompletionServiceHelper;
+import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.common.MinionConstants.UpsertCompactionTask;
+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.TableType;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.apache.pinot.spi.utils.TimeUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@TaskGenerator
+public class UpsertCompactionTaskGenerator extends BaseTaskGenerator {
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactionTaskGenerator.class);
+  private static final String DEFAULT_BUFFER_PERIOD = "7d";
+  private static final double DEFAULT_INVALID_RECORDS_THRESHOLD_PERCENT = 10.0;
+  @Override
+  public String getTaskType() {
+    return MinionConstants.UpsertCompactionTask.TASK_TYPE;
+  }
+
+  @Override
+  public List<PinotTaskConfig> generateTasks(List<TableConfig> tableConfigs) {
+    String taskType = MinionConstants.UpsertCompactionTask.TASK_TYPE;
+    List<PinotTaskConfig> pinotTaskConfigs = new ArrayList<>();
+    for (TableConfig tableConfig: tableConfigs) {
+      if (!validate(tableConfig)) {
+        continue;
+      }
+
+      String tableNameWithType = tableConfig.getTableName();
+      LOGGER.info("Start generating task configs for table: {} for task: {}",
+          tableNameWithType, taskType);
+
+      Map<String, String> taskConfigs = tableConfig.getTaskConfig().getConfigsForTaskType(taskType);
+      Map<String, String> compactionConfigs = getCompactionConfigs(taskConfigs);
+      List<SegmentZKMetadata> completedSegments = getCompletedSegments(tableNameWithType, compactionConfigs);
+
+      if (completedSegments.isEmpty()) {
+        LOGGER.info("No completed segments were available for compaction for table: {}", tableNameWithType);
+        continue;
+      }
+
+      // get server to segment mappings
+      Map<String, List<String>> serverToSegments = _clusterInfoAccessor.getServerToSegmentsMap(tableNameWithType);
+      Map<String, String> segmentToServer = getSegmentToServer(serverToSegments);
+      PinotHelixResourceManager pinotHelixResourceManager = _clusterInfoAccessor.getPinotHelixResourceManager();
+      BiMap<String, String> serverToEndpoints;
+      try {
+        serverToEndpoints = pinotHelixResourceManager.getDataInstanceAdminEndpoints(serverToSegments.keySet());
+      } catch (InvalidConfigException e) {
+        throw new RuntimeException(e);
+      }
+
+      Map<String, SegmentZKMetadata> urlToSegment;
+      try {
+        urlToSegment =
+            getUrlToSegmentMappings(tableNameWithType, completedSegments, segmentToServer, serverToEndpoints);
+      } catch (URISyntaxException e) {
+        throw new RuntimeException(e);
+      }
+
+      // request the urls from the servers
+      CompletionServiceHelper completionServiceHelper = new CompletionServiceHelper(
+            Executors.newCachedThreadPool(), new MultiThreadedHttpConnectionManager(), serverToEndpoints.inverse());
+      CompletionServiceHelper.CompletionServiceResponse serviceResponse =
+            completionServiceHelper.doMultiGetRequest(
+                new ArrayList<>(urlToSegment.keySet()), tableNameWithType, true, 3000);
+
+      // only compact segments that exceed the threshold
+      double invalidRecordsThresholdPercent =
+          Double.parseDouble(compactionConfigs.getOrDefault(UpsertCompactionTask.INVALID_RECORDS_THRESHOLD_PERCENT,
+              String.valueOf(DEFAULT_INVALID_RECORDS_THRESHOLD_PERCENT)));
+      List<SegmentZKMetadata> selectedSegments = new ArrayList<>();
+      for (Map.Entry<String, String> streamResponse : serviceResponse._httpResponses.entrySet()) {
+        double invalidRecordCount = Double.parseDouble(streamResponse.getValue());
+        SegmentZKMetadata segment = urlToSegment.get(streamResponse.getKey());
+        double invalidRecordPercent = (invalidRecordCount / segment.getTotalDocs()) * 100;
+        if (invalidRecordPercent > invalidRecordsThresholdPercent) {
+          selectedSegments.add(segment);
+        }
+      }
+
+      int numTasks = 0;
+      int maxTasks = getMaxTasks(taskType, tableNameWithType, taskConfigs);
+      for (SegmentZKMetadata selectedSegment : selectedSegments) {
+        if (numTasks == maxTasks) {
+          break;
+        }
+        Map<String, String> configs = new HashMap<>();
+        configs.put(MinionConstants.TABLE_NAME_KEY, tableNameWithType);
+        configs.put(MinionConstants.SEGMENT_NAME_KEY, selectedSegment.getSegmentName());
+        configs.put(MinionConstants.DOWNLOAD_URL_KEY, selectedSegment.getDownloadUrl());
+        configs.put(MinionConstants.UPLOAD_URL_KEY, _clusterInfoAccessor.getVipUrl() + "/segments");
+        configs.put(MinionConstants.ORIGINAL_SEGMENT_CRC_KEY, String.valueOf(selectedSegment.getCrc()));
+        pinotTaskConfigs.add(new PinotTaskConfig(UpsertCompactionTask.TASK_TYPE, configs));
+        numTasks++;
+      }
+      LOGGER.info("Finished generating {} tasks configs for table: {} " + "for task: {}",
+          numTasks, tableNameWithType, taskType);
+    }
+    return pinotTaskConfigs;
+  }
+
+  private List<SegmentZKMetadata> getCompletedSegments(String tableNameWithType,
+      Map<String, String> compactionConfigs) {
+    List<SegmentZKMetadata> completedSegments = new ArrayList<>();
+    String bufferPeriod = compactionConfigs.getOrDefault(
+        UpsertCompactionTask.BUFFER_TIME_PERIOD_KEY, DEFAULT_BUFFER_PERIOD);
+    long bufferMs = TimeUtils.convertPeriodToMillis(bufferPeriod);
+    List<SegmentZKMetadata> allSegments = _clusterInfoAccessor.getSegmentsZKMetadata(tableNameWithType);
+    for (SegmentZKMetadata segment : allSegments) {
+      CommonConstants.Segment.Realtime.Status status = segment.getStatus();
+      // initial segments selection based on status and age
+      if (status.isCompleted()) {
+        boolean endedWithinBufferPeriod = segment.getEndTimeMs() <= (System.currentTimeMillis() - bufferMs);
+        boolean endsInTheFuture = segment.getEndTimeMs() > System.currentTimeMillis();

Review Comment:
   I don't follow this. Why do we want to compact segments ended in the future? Segment ended in the future should just be invalid



##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java:
##########
@@ -0,0 +1,292 @@
+/**
+ * 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.upsertcompaction;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.BiMap;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.pinot.common.exception.InvalidConfigException;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator;
+import org.apache.pinot.controller.util.CompletionServiceHelper;
+import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.common.MinionConstants.UpsertCompactionTask;
+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.TableType;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.TimeUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@TaskGenerator
+public class UpsertCompactionTaskGenerator extends BaseTaskGenerator {
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactionTaskGenerator.class);
+  private static final String DEFAULT_BUFFER_PERIOD = "7d";
+  private static final double DEFAULT_INVALID_RECORDS_THRESHOLD_PERCENT = 0.0;
+  private static final long DEFAULT_MIN_RECORD_COUNT = 0;
+  public static class SegmentSelectionResult {
+    private List<SegmentZKMetadata> _segmentsForCompaction;
+    private List<String> _segmentsForDeletion;
+    SegmentSelectionResult(List<SegmentZKMetadata> segmentsForCompaction, List<String> segmentsForDeletion) {
+      _segmentsForCompaction = segmentsForCompaction;
+      _segmentsForDeletion = segmentsForDeletion;
+    }
+    public List<SegmentZKMetadata> getSegmentsForCompaction() {
+      return _segmentsForCompaction;
+    }
+    public List<String> getSegmentsForDeletion() {
+      return _segmentsForDeletion;
+    }
+  }
+
+  @Override
+  public String getTaskType() {
+    return MinionConstants.UpsertCompactionTask.TASK_TYPE;
+  }
+
+  @Override
+  public List<PinotTaskConfig> generateTasks(List<TableConfig> tableConfigs) {
+    String taskType = MinionConstants.UpsertCompactionTask.TASK_TYPE;
+    List<PinotTaskConfig> pinotTaskConfigs = new ArrayList<>();
+    for (TableConfig tableConfig: tableConfigs) {
+      if (!validate(tableConfig)) {
+        continue;
+      }
+
+      String tableNameWithType = tableConfig.getTableName();
+      LOGGER.info("Start generating task configs for table: {} for task: {}",
+          tableNameWithType, taskType);
+
+      Map<String, String> taskConfigs = tableConfig.getTaskConfig().getConfigsForTaskType(taskType);
+      Map<String, String> compactionConfigs = getCompactionConfigs(taskConfigs);
+      List<SegmentZKMetadata> completedSegments = getCompletedSegments(tableNameWithType, compactionConfigs);
+
+      if (completedSegments.isEmpty()) {
+        LOGGER.info("No completed segments were eligible for compaction for table: {}", tableNameWithType);
+        continue;
+      }
+
+      // get server to segment mappings
+      Map<String, List<String>> serverToSegments = _clusterInfoAccessor.getServerToSegmentsMap(tableNameWithType);
+      PinotHelixResourceManager pinotHelixResourceManager = _clusterInfoAccessor.getPinotHelixResourceManager();
+      BiMap<String, String> serverToEndpoints;
+      try {
+        serverToEndpoints = pinotHelixResourceManager.getDataInstanceAdminEndpoints(serverToSegments.keySet());
+      } catch (InvalidConfigException e) {
+        throw new RuntimeException(e);
+      }
+
+      Map<String, SegmentZKMetadata> completedSegmentsMap = completedSegments.stream()
+          .collect(Collectors.toMap(SegmentZKMetadata::getSegmentName, Function.identity()));
+
+      List<String> validDocIdUrls;
+      try {
+        validDocIdUrls = getValidDocIdMetadataUrls(
+            serverToSegments, serverToEndpoints, tableNameWithType, completedSegmentsMap);
+      } catch (URISyntaxException e) {
+        throw new RuntimeException(e);
+      }
+
+      // request the urls from the servers
+      CompletionServiceHelper completionServiceHelper = new CompletionServiceHelper(
+            Executors.newCachedThreadPool(), new MultiThreadedHttpConnectionManager(), serverToEndpoints.inverse());

Review Comment:
   We usually take a reusable thread pool, or at least we need to shut down the pool after use if creating a new one



##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java:
##########
@@ -0,0 +1,292 @@
+/**
+ * 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.upsertcompaction;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.BiMap;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.pinot.common.exception.InvalidConfigException;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator;
+import org.apache.pinot.controller.util.CompletionServiceHelper;
+import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.common.MinionConstants.UpsertCompactionTask;
+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.TableType;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.TimeUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@TaskGenerator
+public class UpsertCompactionTaskGenerator extends BaseTaskGenerator {
+  private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactionTaskGenerator.class);
+  private static final String DEFAULT_BUFFER_PERIOD = "7d";
+  private static final double DEFAULT_INVALID_RECORDS_THRESHOLD_PERCENT = 0.0;
+  private static final long DEFAULT_MIN_RECORD_COUNT = 0;

Review Comment:
   Should we pick a better default? Currently we will trigger a compaction even with only one invalid record



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