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/03/23 17:01:47 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_r1146471573


##########
pinot-core/src/main/java/org/apache/pinot/core/common/MinionConstants.java:
##########
@@ -133,4 +133,32 @@ public static class SegmentGenerationAndPushTask {
     public static final String CONFIG_NUMBER_CONCURRENT_TASKS_PER_INSTANCE =
         "SegmentGenerationAndPushTask.numConcurrentTasksPerInstance";
   }
+
+  public static class UpsertCompactionTask {
+    public static final String TASK_TYPE = "UpsertCompactionTask";
+
+    /**
+     * The time window size for the task.
+     * e.g. if set to "1d", then generated segments will include data for a 1 day window
+     */
+    public static final String BUCKET_TIME_PERIOD_KEY = "bucketTimePeriod";
+
+    /**
+     * The time period to wait before picking segments for this task
+     * e.g. if set to "2d", then a newly completed segment will not be compacted until 2 days since completion
+     */
+    public static final String BUFFER_TIME_PERIOD_KEY = "bufferTimePeriod";
+
+    /**
+     * The maximum amount of records allowed for a generated segment.
+     * e.g. if set to "5000000", then a generated segment will have at most 5000000 records
+     */
+    public static final String MAX_NUM_RECORDS_PER_SEGMENT = "maxNumRecordsPerSegment";

Review Comment:
   We don't need to define these keys as they are common for `MergeTask`. You may extend `MergeTask` (see `MergeRollupTask` as an example)



##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java:
##########
@@ -0,0 +1,205 @@
+/**
+ * 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.base.Preconditions;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.common.utils.SegmentUtils;
+import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignmentUtils;
+import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator;
+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.segment.spi.partition.metadata.ColumnPartitionMetadata;
+import org.apache.pinot.spi.annotations.minion.TaskGenerator;
+import org.apache.pinot.spi.config.table.ColumnPartitionConfig;
+import org.apache.pinot.spi.config.table.IndexingConfig;
+import org.apache.pinot.spi.config.table.SegmentPartitionConfig;
+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.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);
+  @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, taskType))
+      {
+        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);
+      String bufferPeriod = compactionConfigs.get(UpsertCompactionTask.BUFFER_TIME_PERIOD_KEY);
+      long bufferMs = TimeUtils.convertPeriodToMillis(bufferPeriod);
+      List<SegmentZKMetadata> completedSegments = new ArrayList<>();
+      List<SegmentZKMetadata> allSegments = _clusterInfoAccessor.getSegmentsZKMetadata(tableNameWithType);
+      for (SegmentZKMetadata segment : allSegments) {

Review Comment:
   We don't want to process all completed segments. We want to process the segments when:
   - The invalid docs are over the threshold
   - (optional) The segment is not processed recently



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/SegmentUtils.java:
##########
@@ -38,15 +38,18 @@ private SegmentUtils() {
   // path.
   @Nullable
   public static Integer getRealtimeSegmentPartitionId(String segmentName, String realtimeTableName,
-      HelixManager helixManager, @Nullable String partitionColumn) {
+      HelixManager helixManager, @Nullable String partitionColumn,

Review Comment:
   Suggest not changing this method, but add a new one to read the partition id from the `SegmentZKMetadata` only (we can extract the part of processing `SegmentZKMetadata` of this method into a separate method)



##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java:
##########
@@ -0,0 +1,205 @@
+/**
+ * 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.base.Preconditions;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.common.utils.SegmentUtils;
+import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignmentUtils;
+import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator;
+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.segment.spi.partition.metadata.ColumnPartitionMetadata;
+import org.apache.pinot.spi.annotations.minion.TaskGenerator;
+import org.apache.pinot.spi.config.table.ColumnPartitionConfig;
+import org.apache.pinot.spi.config.table.IndexingConfig;
+import org.apache.pinot.spi.config.table.SegmentPartitionConfig;
+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.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);
+  @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, taskType))
+      {
+        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);
+      String bufferPeriod = compactionConfigs.get(UpsertCompactionTask.BUFFER_TIME_PERIOD_KEY);
+      long bufferMs = TimeUtils.convertPeriodToMillis(bufferPeriod);
+      List<SegmentZKMetadata> completedSegments = new ArrayList<>();
+      List<SegmentZKMetadata> allSegments = _clusterInfoAccessor.getSegmentsZKMetadata(tableNameWithType);
+      for (SegmentZKMetadata segment : allSegments) {
+        CommonConstants.Segment.Realtime.Status status = segment.getStatus();
+        // only compact the completed segments that are older than the bufferTimePeriod
+        if (status.isCompleted() && segment.getEndTimeMs() <= (System.currentTimeMillis() - bufferMs)) {
+          completedSegments.add(segment);
+        }
+      }
+      if (completedSegments.isEmpty()) {
+        LOGGER.info("No segments were selected for compaction for table: {}", tableNameWithType);
+        continue;
+      }
+
+      // map each completedSegment to its partition
+      Map<Integer, List<String>> partitionToSegmentNames = new HashMap<>();
+      for (SegmentZKMetadata completedSegment : completedSegments) {
+        Integer partitionId =
+            SegmentUtils.getRealtimeSegmentPartitionId(completedSegment.getSegmentName(), tableNameWithType, null, null,
+                completedSegment);
+
+        Preconditions.checkState(partitionId != null,
+            "Unable to find partitionId for completedSegment");
+
+        partitionToSegmentNames.computeIfAbsent(partitionId, k -> new ArrayList<>())
+            .add(completedSegment.getSegmentName());
+      }
+      int numTaskConfigsForTable = 0;
+      for (Map.Entry<Integer, List<String>> entry : partitionToSegmentNames.entrySet()) {
+        List<String> completedSegmentNames = entry.getValue();
+        PinotTaskConfig pinotTaskConfig =
+            getPinotTaskConfig(tableNameWithType, compactionConfigs, completedSegmentNames);
+        pinotTaskConfigs.add(pinotTaskConfig);
+        numTaskConfigsForTable++;
+      }
+      LOGGER.info("Finished generating {} tasks configs for table: {} " + "for task: {}",
+          numTaskConfigsForTable, tableNameWithType, taskType);
+    }
+    return pinotTaskConfigs;
+  }
+
+  private static PinotTaskConfig getPinotTaskConfig(String tableNameWithType, Map<String, String> compactionConfigs,
+      List<String> completedSegmentNames) {
+    Map<String, String> configs = new HashMap<>();
+    configs.put(MinionConstants.TABLE_NAME_KEY, tableNameWithType);
+    configs.put(MinionConstants.SEGMENT_NAME_KEY,
+        StringUtils.join(completedSegmentNames, MinionConstants.SEGMENT_NAME_SEPARATOR));
+    configs.put(UpsertCompactionTask.BUCKET_TIME_PERIOD_KEY,
+        compactionConfigs.get(UpsertCompactionTask.BUCKET_TIME_PERIOD_KEY));
+    configs.put(UpsertCompactionTask.MAX_NUM_RECORDS_PER_SEGMENT,
+        compactionConfigs.get(UpsertCompactionTask.MAX_NUM_RECORDS_PER_SEGMENT));
+    configs.put(UpsertCompactionTask.INVALID_RECORDS_THRESHOLD,
+        compactionConfigs.get(UpsertCompactionTask.INVALID_RECORDS_THRESHOLD));
+    PinotTaskConfig pinotTaskConfig = new PinotTaskConfig(UpsertCompactionTask.TASK_TYPE, configs);
+    return pinotTaskConfig;
+  }
+
+  private static final String[] VALID_CONFIG_KEYS = {
+      UpsertCompactionTask.BUCKET_TIME_PERIOD_KEY,
+      UpsertCompactionTask.BUFFER_TIME_PERIOD_KEY,
+      UpsertCompactionTask.MAX_NUM_RECORDS_PER_SEGMENT,
+      UpsertCompactionTask.INVALID_RECORDS_THRESHOLD,
+  };
+
+  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) {
+    String tableNameWithType = tableConfig.getTableName();
+    if (tableConfig.getTableType() == TableType.OFFLINE) {
+      LOGGER.warn("Skip generation task: {} for table: {}, offline table is not supported", taskType, tableNameWithType);
+      return false;
+    }
+    if (!tableConfig.isUpsertEnabled()) {
+      LOGGER.warn("Skip generation task: {} for table: {}, table without upsert enabled is not supported", taskType, tableNameWithType);
+      return false;
+    }
+    TableTaskConfig tableTaskConfig = tableConfig.getTaskConfig();
+    if (tableTaskConfig == null) {
+      LOGGER.warn("Skip generation task: {} for table: {}, unable to find task config",
+          taskType, tableNameWithType);
+      return false;
+    }
+    Map<String, String> compactionConfigs = tableTaskConfig.getConfigsForTaskType(MinionConstants.UpsertCompactionTask.TASK_TYPE);

Review Comment:
   (minor) The config should always be their, or it won't trigger the generator



##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java:
##########
@@ -0,0 +1,205 @@
+/**
+ * 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.base.Preconditions;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.common.utils.SegmentUtils;
+import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignmentUtils;
+import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator;
+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.segment.spi.partition.metadata.ColumnPartitionMetadata;
+import org.apache.pinot.spi.annotations.minion.TaskGenerator;
+import org.apache.pinot.spi.config.table.ColumnPartitionConfig;
+import org.apache.pinot.spi.config.table.IndexingConfig;
+import org.apache.pinot.spi.config.table.SegmentPartitionConfig;
+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.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);
+  @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, taskType))
+      {
+        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);
+      String bufferPeriod = compactionConfigs.get(UpsertCompactionTask.BUFFER_TIME_PERIOD_KEY);
+      long bufferMs = TimeUtils.convertPeriodToMillis(bufferPeriod);
+      List<SegmentZKMetadata> completedSegments = new ArrayList<>();
+      List<SegmentZKMetadata> allSegments = _clusterInfoAccessor.getSegmentsZKMetadata(tableNameWithType);
+      for (SegmentZKMetadata segment : allSegments) {
+        CommonConstants.Segment.Realtime.Status status = segment.getStatus();
+        // only compact the completed segments that are older than the bufferTimePeriod
+        if (status.isCompleted() && segment.getEndTimeMs() <= (System.currentTimeMillis() - bufferMs)) {
+          completedSegments.add(segment);
+        }
+      }
+      if (completedSegments.isEmpty()) {
+        LOGGER.info("No segments were selected for compaction for table: {}", tableNameWithType);
+        continue;
+      }
+
+      // map each completedSegment to its partition
+      Map<Integer, List<String>> partitionToSegmentNames = new HashMap<>();
+      for (SegmentZKMetadata completedSegment : completedSegments) {
+        Integer partitionId =
+            SegmentUtils.getRealtimeSegmentPartitionId(completedSegment.getSegmentName(), tableNameWithType, null, null,
+                completedSegment);
+
+        Preconditions.checkState(partitionId != null,
+            "Unable to find partitionId for completedSegment");
+
+        partitionToSegmentNames.computeIfAbsent(partitionId, k -> new ArrayList<>())
+            .add(completedSegment.getSegmentName());
+      }
+      int numTaskConfigsForTable = 0;
+      for (Map.Entry<Integer, List<String>> entry : partitionToSegmentNames.entrySet()) {
+        List<String> completedSegmentNames = entry.getValue();
+        PinotTaskConfig pinotTaskConfig =
+            getPinotTaskConfig(tableNameWithType, compactionConfigs, completedSegmentNames);
+        pinotTaskConfigs.add(pinotTaskConfig);
+        numTaskConfigsForTable++;
+      }
+      LOGGER.info("Finished generating {} tasks configs for table: {} " + "for task: {}",
+          numTaskConfigsForTable, tableNameWithType, taskType);
+    }
+    return pinotTaskConfigs;
+  }
+
+  private static PinotTaskConfig getPinotTaskConfig(String tableNameWithType, Map<String, String> compactionConfigs,
+      List<String> completedSegmentNames) {
+    Map<String, String> configs = new HashMap<>();
+    configs.put(MinionConstants.TABLE_NAME_KEY, tableNameWithType);
+    configs.put(MinionConstants.SEGMENT_NAME_KEY,
+        StringUtils.join(completedSegmentNames, MinionConstants.SEGMENT_NAME_SEPARATOR));
+    configs.put(UpsertCompactionTask.BUCKET_TIME_PERIOD_KEY,
+        compactionConfigs.get(UpsertCompactionTask.BUCKET_TIME_PERIOD_KEY));
+    configs.put(UpsertCompactionTask.MAX_NUM_RECORDS_PER_SEGMENT,
+        compactionConfigs.get(UpsertCompactionTask.MAX_NUM_RECORDS_PER_SEGMENT));
+    configs.put(UpsertCompactionTask.INVALID_RECORDS_THRESHOLD,
+        compactionConfigs.get(UpsertCompactionTask.INVALID_RECORDS_THRESHOLD));
+    PinotTaskConfig pinotTaskConfig = new PinotTaskConfig(UpsertCompactionTask.TASK_TYPE, configs);
+    return pinotTaskConfig;
+  }
+
+  private static final String[] VALID_CONFIG_KEYS = {
+      UpsertCompactionTask.BUCKET_TIME_PERIOD_KEY,
+      UpsertCompactionTask.BUFFER_TIME_PERIOD_KEY,
+      UpsertCompactionTask.MAX_NUM_RECORDS_PER_SEGMENT,
+      UpsertCompactionTask.INVALID_RECORDS_THRESHOLD,
+  };
+
+  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) {
+    String tableNameWithType = tableConfig.getTableName();
+    if (tableConfig.getTableType() == TableType.OFFLINE) {
+      LOGGER.warn("Skip generation task: {} for table: {}, offline table is not supported", taskType, tableNameWithType);
+      return false;
+    }
+    if (!tableConfig.isUpsertEnabled()) {
+      LOGGER.warn("Skip generation task: {} for table: {}, table without upsert enabled is not supported", taskType, tableNameWithType);
+      return false;
+    }
+    TableTaskConfig tableTaskConfig = tableConfig.getTaskConfig();
+    if (tableTaskConfig == null) {
+      LOGGER.warn("Skip generation task: {} for table: {}, unable to find task config",
+          taskType, tableNameWithType);
+      return false;
+    }
+    Map<String, String> compactionConfigs = tableTaskConfig.getConfigsForTaskType(MinionConstants.UpsertCompactionTask.TASK_TYPE);
+    if (compactionConfigs == null) {
+      LOGGER.warn("Skip generation task: {} for table: {}, unable to find compaction task config",
+          taskType, tableNameWithType);
+      return false;
+    }
+    if (!compactionConfigs.containsKey(UpsertCompactionTask.BUCKET_TIME_PERIOD_KEY))

Review Comment:
   These configs are not mandatory. We can use the default when they are not configured



##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java:
##########
@@ -0,0 +1,205 @@
+/**
+ * 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.base.Preconditions;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.common.utils.SegmentUtils;
+import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignmentUtils;
+import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator;
+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.segment.spi.partition.metadata.ColumnPartitionMetadata;
+import org.apache.pinot.spi.annotations.minion.TaskGenerator;
+import org.apache.pinot.spi.config.table.ColumnPartitionConfig;
+import org.apache.pinot.spi.config.table.IndexingConfig;
+import org.apache.pinot.spi.config.table.SegmentPartitionConfig;
+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.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);
+  @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, taskType))
+      {
+        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);
+      String bufferPeriod = compactionConfigs.get(UpsertCompactionTask.BUFFER_TIME_PERIOD_KEY);
+      long bufferMs = TimeUtils.convertPeriodToMillis(bufferPeriod);
+      List<SegmentZKMetadata> completedSegments = new ArrayList<>();
+      List<SegmentZKMetadata> allSegments = _clusterInfoAccessor.getSegmentsZKMetadata(tableNameWithType);
+      for (SegmentZKMetadata segment : allSegments) {
+        CommonConstants.Segment.Realtime.Status status = segment.getStatus();
+        // only compact the completed segments that are older than the bufferTimePeriod
+        if (status.isCompleted() && segment.getEndTimeMs() <= (System.currentTimeMillis() - bufferMs)) {
+          completedSegments.add(segment);
+        }
+      }
+      if (completedSegments.isEmpty()) {
+        LOGGER.info("No segments were selected for compaction for table: {}", tableNameWithType);
+        continue;
+      }
+
+      // map each completedSegment to its partition
+      Map<Integer, List<String>> partitionToSegmentNames = new HashMap<>();
+      for (SegmentZKMetadata completedSegment : completedSegments) {
+        Integer partitionId =
+            SegmentUtils.getRealtimeSegmentPartitionId(completedSegment.getSegmentName(), tableNameWithType, null, null,
+                completedSegment);
+
+        Preconditions.checkState(partitionId != null,
+            "Unable to find partitionId for completedSegment");
+
+        partitionToSegmentNames.computeIfAbsent(partitionId, k -> new ArrayList<>())
+            .add(completedSegment.getSegmentName());
+      }
+      int numTaskConfigsForTable = 0;
+      for (Map.Entry<Integer, List<String>> entry : partitionToSegmentNames.entrySet()) {
+        List<String> completedSegmentNames = entry.getValue();
+        PinotTaskConfig pinotTaskConfig =
+            getPinotTaskConfig(tableNameWithType, compactionConfigs, completedSegmentNames);
+        pinotTaskConfigs.add(pinotTaskConfig);
+        numTaskConfigsForTable++;
+      }
+      LOGGER.info("Finished generating {} tasks configs for table: {} " + "for task: {}",
+          numTaskConfigsForTable, tableNameWithType, taskType);
+    }
+    return pinotTaskConfigs;
+  }
+
+  private static PinotTaskConfig getPinotTaskConfig(String tableNameWithType, Map<String, String> compactionConfigs,
+      List<String> completedSegmentNames) {
+    Map<String, String> configs = new HashMap<>();
+    configs.put(MinionConstants.TABLE_NAME_KEY, tableNameWithType);
+    configs.put(MinionConstants.SEGMENT_NAME_KEY,
+        StringUtils.join(completedSegmentNames, MinionConstants.SEGMENT_NAME_SEPARATOR));
+    configs.put(UpsertCompactionTask.BUCKET_TIME_PERIOD_KEY,
+        compactionConfigs.get(UpsertCompactionTask.BUCKET_TIME_PERIOD_KEY));
+    configs.put(UpsertCompactionTask.MAX_NUM_RECORDS_PER_SEGMENT,
+        compactionConfigs.get(UpsertCompactionTask.MAX_NUM_RECORDS_PER_SEGMENT));
+    configs.put(UpsertCompactionTask.INVALID_RECORDS_THRESHOLD,
+        compactionConfigs.get(UpsertCompactionTask.INVALID_RECORDS_THRESHOLD));
+    PinotTaskConfig pinotTaskConfig = new PinotTaskConfig(UpsertCompactionTask.TASK_TYPE, configs);
+    return pinotTaskConfig;
+  }
+
+  private static final String[] VALID_CONFIG_KEYS = {
+      UpsertCompactionTask.BUCKET_TIME_PERIOD_KEY,
+      UpsertCompactionTask.BUFFER_TIME_PERIOD_KEY,
+      UpsertCompactionTask.MAX_NUM_RECORDS_PER_SEGMENT,
+      UpsertCompactionTask.INVALID_RECORDS_THRESHOLD,
+  };
+
+  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) {

Review Comment:
   (minor) `taskType` seems not needed as this is under the scope of upsert compaction?



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