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


##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java:
##########
@@ -0,0 +1,268 @@
+/**
+ * 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.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 = 30.0;
+  private static final long DEFAULT_MIN_RECORD_COUNT = 100_000;
+  @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);
+
+      // 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> segmentsForCompaction = new ArrayList<>();
+      List<String> segmentsForDeletion = new ArrayList<>();
+      for (Map.Entry<String, String> streamResponse : 
serviceResponse._httpResponses.entrySet()) {
+        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();
+          double invalidRecordCount = 
validDocIdMetadata.get("totalInvalidDocs").asDouble();
+          String segmentName = validDocIdMetadata.get("segmentName").asText();
+          SegmentZKMetadata segment = completedSegmentsMap.get(segmentName);
+          double invalidRecordPercent = (invalidRecordCount / 
segment.getTotalDocs()) * 100;
+          if (invalidRecordPercent == 100.0) {

Review Comment:
   (minor) to be more accurate
   ```suggestion
             if (invalidRecordCount == segment.getTotalDocs()) {
   ```



##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java:
##########
@@ -0,0 +1,268 @@
+/**
+ * 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.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 = 30.0;
+  private static final long DEFAULT_MIN_RECORD_COUNT = 100_000;
+  @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);
+
+      // 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> segmentsForCompaction = new ArrayList<>();
+      List<String> segmentsForDeletion = new ArrayList<>();
+      for (Map.Entry<String, String> streamResponse : 
serviceResponse._httpResponses.entrySet()) {
+        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();
+          double invalidRecordCount = 
validDocIdMetadata.get("totalInvalidDocs").asDouble();

Review Comment:
   (minor) read it as long and do the casting if needed. It should be a whole 
number



##########
pinot-core/src/main/java/org/apache/pinot/core/common/MinionConstants.java:
##########
@@ -136,4 +136,18 @@ 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 period to wait before picking segments for this task
+     * e.g. if set to "2d", no task will be scheduled for a time window 
younger than 2 days
+     */
+    public static final String BUFFER_TIME_PERIOD_KEY = "bufferTimePeriod";
+    /**
+     * The maximum percent of old records allowed for a completed segment.
+     * e.g. if the percent surpasses 30, then the segment will be compacted
+     */
+    public static final String INVALID_RECORDS_THRESHOLD_PERCENT = 
"invalidRecordsThresholdPercent";

Review Comment:
   @snleee @robertzych I still prefer using record count as the threshold so 
that each time when a segment is compacted, we know it will reduce enough 
records to justify the cost of the task. How about keeping both, and user can 
decide which one to use



##########
pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompaction/UpsertCompactionTaskGenerator.java:
##########
@@ -0,0 +1,268 @@
+/**
+ * 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.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 = 30.0;
+  private static final long DEFAULT_MIN_RECORD_COUNT = 100_000;
+  @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);
+
+      // 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> segmentsForCompaction = new ArrayList<>();
+      List<String> segmentsForDeletion = new ArrayList<>();

Review Comment:
   Seems this doesn't work because completed segments doesn't include the small 
segments. Let's add a test for it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

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


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

Reply via email to