ankitsultana commented on code in PR #14477: URL: https://github.com/apache/pinot/pull/14477#discussion_r1868761767
########## pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompactmerge/UpsertCompactMergeTaskExecutor.java: ########## @@ -0,0 +1,166 @@ +/** + * 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.upsertcompactmerge; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadataCustomMapModifier; +import org.apache.pinot.common.restlet.resources.ValidDocIdsType; +import org.apache.pinot.common.utils.SegmentUtils; +import org.apache.pinot.core.common.MinionConstants; +import org.apache.pinot.core.minion.PinotTaskConfig; +import org.apache.pinot.core.segment.processing.framework.SegmentProcessorConfig; +import org.apache.pinot.core.segment.processing.framework.SegmentProcessorFramework; +import org.apache.pinot.minion.MinionConf; +import org.apache.pinot.plugin.minion.tasks.BaseMultipleSegmentsConversionExecutor; +import org.apache.pinot.plugin.minion.tasks.MinionTaskUtils; +import org.apache.pinot.plugin.minion.tasks.SegmentConversionResult; +import org.apache.pinot.segment.local.segment.readers.CompactedPinotSegmentRecordReader; +import org.apache.pinot.segment.spi.creator.name.UploadedRealtimeSegmentNameGenerator; +import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.data.readers.RecordReader; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.roaringbitmap.RoaringBitmap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class UpsertCompactMergeTaskExecutor extends BaseMultipleSegmentsConversionExecutor { + + private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactMergeTaskExecutor.class); + + public UpsertCompactMergeTaskExecutor(MinionConf minionConf) { + super(minionConf); + } + + @Override + protected List<SegmentConversionResult> convert(PinotTaskConfig pinotTaskConfig, List<File> segmentDirs, + File workingDir) + throws Exception { + int numInputSegments = segmentDirs.size(); + List<SegmentConversionResult> results = new ArrayList<>(); + _eventObserver.notifyProgress(pinotTaskConfig, "Converting segments: " + numInputSegments); + String taskType = pinotTaskConfig.getTaskType(); + Map<String, String> configs = pinotTaskConfig.getConfigs(); + LOGGER.info("Starting task: {} with configs: {}", taskType, configs); + long startMillis = System.currentTimeMillis(); + + String tableNameWithType = configs.get(MinionConstants.TABLE_NAME_KEY); + TableConfig tableConfig = getTableConfig(tableNameWithType); + Schema schema = getSchema(tableNameWithType); + + SegmentProcessorConfig.Builder segmentProcessorConfigBuilder = + new SegmentProcessorConfig.Builder().setTableConfig(tableConfig).setSchema(schema); + + // Progress observer + segmentProcessorConfigBuilder.setProgressObserver(p -> _eventObserver.notifyProgress(_pinotTaskConfig, p)); + + List<RecordReader> recordReaders = new ArrayList<>(numInputSegments); + int count = 1; + int partitionId = -1; + long maxCreationTimeOfMergingSegments = 0; + List<String> originalSegmentCrcFromTaskGenerator = + List.of(configs.get(MinionConstants.ORIGINAL_SEGMENT_CRC_KEY).split(",")); + for (int i = 0; i < numInputSegments; i++) { + File segmentDir = segmentDirs.get(i); + _eventObserver.notifyProgress(_pinotTaskConfig, + String.format("Creating RecordReader for: %s (%d out of %d)", segmentDir, count++, numInputSegments)); + + SegmentMetadataImpl segmentMetadata = new SegmentMetadataImpl(segmentDir); + String segmentName = segmentMetadata.getName(); + Integer segmentPartitionId = SegmentUtils.getPartitionIdFromRealtimeSegmentName(segmentName); + if (segmentPartitionId == null) { + throw new IllegalStateException(String.format("Partition id not found for %s", segmentName)); + } + if (partitionId != -1 && partitionId != segmentPartitionId) { + throw new IllegalStateException(String.format("Partition id mismatched for %s, expected partition id: %d", + segmentName, partitionId)); + } + partitionId = segmentPartitionId; + maxCreationTimeOfMergingSegments = Math.max(maxCreationTimeOfMergingSegments, + segmentMetadata.getIndexCreationTime()); + + String crcFromDeepStorageSegment = segmentMetadata.getCrc(); + if (!originalSegmentCrcFromTaskGenerator.get(i).equals(crcFromDeepStorageSegment)) { + String message = String.format("Crc mismatched between ZK and deepstore copy of segment: %s. Expected crc " Review Comment: Can you remind me what are the known cases where this can happen? I remember we have seen this happen in production before. ########## pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompactmerge/UpsertCompactMergeTaskExecutor.java: ########## @@ -0,0 +1,166 @@ +/** + * 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.upsertcompactmerge; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadataCustomMapModifier; +import org.apache.pinot.common.restlet.resources.ValidDocIdsType; +import org.apache.pinot.common.utils.SegmentUtils; +import org.apache.pinot.core.common.MinionConstants; +import org.apache.pinot.core.minion.PinotTaskConfig; +import org.apache.pinot.core.segment.processing.framework.SegmentProcessorConfig; +import org.apache.pinot.core.segment.processing.framework.SegmentProcessorFramework; +import org.apache.pinot.minion.MinionConf; +import org.apache.pinot.plugin.minion.tasks.BaseMultipleSegmentsConversionExecutor; +import org.apache.pinot.plugin.minion.tasks.MinionTaskUtils; +import org.apache.pinot.plugin.minion.tasks.SegmentConversionResult; +import org.apache.pinot.segment.local.segment.readers.CompactedPinotSegmentRecordReader; +import org.apache.pinot.segment.spi.creator.name.UploadedRealtimeSegmentNameGenerator; +import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.data.readers.RecordReader; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.roaringbitmap.RoaringBitmap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class UpsertCompactMergeTaskExecutor extends BaseMultipleSegmentsConversionExecutor { Review Comment: Brief javadoc that describes the task at a high-level would be great ########## pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompactmerge/UpsertCompactMergeTaskExecutor.java: ########## @@ -0,0 +1,166 @@ +/** + * 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.upsertcompactmerge; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadataCustomMapModifier; +import org.apache.pinot.common.restlet.resources.ValidDocIdsType; +import org.apache.pinot.common.utils.SegmentUtils; +import org.apache.pinot.core.common.MinionConstants; +import org.apache.pinot.core.minion.PinotTaskConfig; +import org.apache.pinot.core.segment.processing.framework.SegmentProcessorConfig; +import org.apache.pinot.core.segment.processing.framework.SegmentProcessorFramework; +import org.apache.pinot.minion.MinionConf; +import org.apache.pinot.plugin.minion.tasks.BaseMultipleSegmentsConversionExecutor; +import org.apache.pinot.plugin.minion.tasks.MinionTaskUtils; +import org.apache.pinot.plugin.minion.tasks.SegmentConversionResult; +import org.apache.pinot.segment.local.segment.readers.CompactedPinotSegmentRecordReader; +import org.apache.pinot.segment.spi.creator.name.UploadedRealtimeSegmentNameGenerator; +import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.data.readers.RecordReader; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.roaringbitmap.RoaringBitmap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class UpsertCompactMergeTaskExecutor extends BaseMultipleSegmentsConversionExecutor { + + private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactMergeTaskExecutor.class); + + public UpsertCompactMergeTaskExecutor(MinionConf minionConf) { + super(minionConf); + } + + @Override + protected List<SegmentConversionResult> convert(PinotTaskConfig pinotTaskConfig, List<File> segmentDirs, + File workingDir) + throws Exception { + int numInputSegments = segmentDirs.size(); + List<SegmentConversionResult> results = new ArrayList<>(); + _eventObserver.notifyProgress(pinotTaskConfig, "Converting segments: " + numInputSegments); + String taskType = pinotTaskConfig.getTaskType(); + Map<String, String> configs = pinotTaskConfig.getConfigs(); + LOGGER.info("Starting task: {} with configs: {}", taskType, configs); + long startMillis = System.currentTimeMillis(); + + String tableNameWithType = configs.get(MinionConstants.TABLE_NAME_KEY); + TableConfig tableConfig = getTableConfig(tableNameWithType); + Schema schema = getSchema(tableNameWithType); + + SegmentProcessorConfig.Builder segmentProcessorConfigBuilder = + new SegmentProcessorConfig.Builder().setTableConfig(tableConfig).setSchema(schema); + + // Progress observer + segmentProcessorConfigBuilder.setProgressObserver(p -> _eventObserver.notifyProgress(_pinotTaskConfig, p)); + + List<RecordReader> recordReaders = new ArrayList<>(numInputSegments); + int count = 1; + int partitionId = -1; + long maxCreationTimeOfMergingSegments = 0; + List<String> originalSegmentCrcFromTaskGenerator = + List.of(configs.get(MinionConstants.ORIGINAL_SEGMENT_CRC_KEY).split(",")); + for (int i = 0; i < numInputSegments; i++) { Review Comment: nit: we are doing a bunch of different things in this for loop. I think it might help to create smaller methods to make it easier to read. e.g. * You can get a list of SegmentMetadataImpl from a method such as `getSegmentMetadata(List<File> segmentDirs)` * Similarly this. method can return a set or list `getPartitionIdForSegments(List<SegmentMetadata>)`, and then you can check the size of the set. (or something like that) * Similarly `segmentMetadata.stream().map(x -> x.getIndexCreationTime()).max` * Similarly `validateCrc` * And finally, `getRecordReaders(..)` ########## pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompactmerge/UpsertCompactMergeTaskGenerator.java: ########## @@ -0,0 +1,419 @@ +/** + * 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.upsertcompactmerge; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.BiMap; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.apache.helix.task.TaskState; +import org.apache.pinot.common.exception.InvalidConfigException; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.restlet.resources.ValidDocIdsMetadataInfo; +import org.apache.pinot.common.restlet.resources.ValidDocIdsType; +import org.apache.pinot.common.utils.SegmentUtils; +import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; +import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator; +import org.apache.pinot.controller.helix.core.minion.generator.TaskGeneratorUtils; +import org.apache.pinot.controller.util.ServerSegmentMetadataReader; +import org.apache.pinot.core.common.MinionConstants; +import org.apache.pinot.core.minion.PinotTaskConfig; +import org.apache.pinot.spi.annotations.minion.TaskGenerator; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.config.table.UpsertConfig; +import org.apache.pinot.spi.utils.TimeUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +@TaskGenerator +public class UpsertCompactMergeTaskGenerator extends BaseTaskGenerator { + + private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactMergeTaskGenerator.class); + private static final String DEFAULT_BUFFER_PERIOD = "7d"; Review Comment: Could we set it to 2 days instead? 7d default seems a bit high. ########## pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompactmerge/UpsertCompactMergeTaskExecutor.java: ########## @@ -0,0 +1,166 @@ +/** + * 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.upsertcompactmerge; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadataCustomMapModifier; +import org.apache.pinot.common.restlet.resources.ValidDocIdsType; +import org.apache.pinot.common.utils.SegmentUtils; +import org.apache.pinot.core.common.MinionConstants; +import org.apache.pinot.core.minion.PinotTaskConfig; +import org.apache.pinot.core.segment.processing.framework.SegmentProcessorConfig; +import org.apache.pinot.core.segment.processing.framework.SegmentProcessorFramework; +import org.apache.pinot.minion.MinionConf; +import org.apache.pinot.plugin.minion.tasks.BaseMultipleSegmentsConversionExecutor; +import org.apache.pinot.plugin.minion.tasks.MinionTaskUtils; +import org.apache.pinot.plugin.minion.tasks.SegmentConversionResult; +import org.apache.pinot.segment.local.segment.readers.CompactedPinotSegmentRecordReader; +import org.apache.pinot.segment.spi.creator.name.UploadedRealtimeSegmentNameGenerator; +import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.data.readers.RecordReader; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.roaringbitmap.RoaringBitmap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class UpsertCompactMergeTaskExecutor extends BaseMultipleSegmentsConversionExecutor { + + private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactMergeTaskExecutor.class); + + public UpsertCompactMergeTaskExecutor(MinionConf minionConf) { + super(minionConf); + } + + @Override + protected List<SegmentConversionResult> convert(PinotTaskConfig pinotTaskConfig, List<File> segmentDirs, + File workingDir) + throws Exception { + int numInputSegments = segmentDirs.size(); + List<SegmentConversionResult> results = new ArrayList<>(); Review Comment: nit: move this declaration towards the end of the function since it is not used until the last few lines. ########## pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompactmerge/UpsertCompactMergeTaskGenerator.java: ########## @@ -0,0 +1,419 @@ +/** + * 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.upsertcompactmerge; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.BiMap; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.apache.helix.task.TaskState; +import org.apache.pinot.common.exception.InvalidConfigException; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.restlet.resources.ValidDocIdsMetadataInfo; +import org.apache.pinot.common.restlet.resources.ValidDocIdsType; +import org.apache.pinot.common.utils.SegmentUtils; +import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; +import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator; +import org.apache.pinot.controller.helix.core.minion.generator.TaskGeneratorUtils; +import org.apache.pinot.controller.util.ServerSegmentMetadataReader; +import org.apache.pinot.core.common.MinionConstants; +import org.apache.pinot.core.minion.PinotTaskConfig; +import org.apache.pinot.spi.annotations.minion.TaskGenerator; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.config.table.UpsertConfig; +import org.apache.pinot.spi.utils.TimeUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +@TaskGenerator +public class UpsertCompactMergeTaskGenerator extends BaseTaskGenerator { + + private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactMergeTaskGenerator.class); + private static final String DEFAULT_BUFFER_PERIOD = "7d"; + private static final int DEFAULT_NUM_SEGMENTS_BATCH_PER_SERVER_REQUEST = 500; + + public static class SegmentMergerMetadata { + SegmentZKMetadata _segmentZKMetadata; + long _validDocIds; + long _invalidDocIds; + + SegmentMergerMetadata(SegmentZKMetadata segmentZKMetadata, long validDocIds, long invalidDocIds) { + _segmentZKMetadata = segmentZKMetadata; + _validDocIds = validDocIds; + _invalidDocIds = invalidDocIds; + } + + public SegmentZKMetadata getSegmentZKMetadata() { + return _segmentZKMetadata; + } + + public long getValidDocIds() { + return _validDocIds; + } + + public long getInvalidDocIds() { + return _invalidDocIds; + } + } + + public static class SegmentSelectionResult { + + private final Map<Integer, List<List<SegmentMergerMetadata>>> _segmentsForCompactMerge; + + private final List<String> _segmentsForDeletion; + + SegmentSelectionResult(Map<Integer, List<List<SegmentMergerMetadata>>> segmentsForCompactMerge, + List<String> segmentsForDeletion) { + _segmentsForCompactMerge = segmentsForCompactMerge; + _segmentsForDeletion = segmentsForDeletion; + } + + public Map<Integer, List<List<SegmentMergerMetadata>>> getSegmentsForCompactMerge() { + return _segmentsForCompactMerge; + } + + public List<String> getSegmentsForDeletion() { + return _segmentsForDeletion; + } + } + + @Override + public String getTaskType() { + return MinionConstants.UpsertCompactMergeTask.TASK_TYPE; + } + + @Override + public List<PinotTaskConfig> generateTasks(List<TableConfig> tableConfigs) { + String taskType = MinionConstants.UpsertCompactMergeTask.TASK_TYPE; + List<PinotTaskConfig> pinotTaskConfigs = new ArrayList<>(); + for (TableConfig tableConfig : tableConfigs) { + + String tableNameWithType = tableConfig.getTableName(); + LOGGER.info("Start generating task configs for table: {}", tableNameWithType); + + if (tableConfig.getTaskConfig() == null) { + LOGGER.warn("Task config is null for table: {}", tableNameWithType); + continue; + } + + // Only schedule 1 task of this type, per table + Map<String, TaskState> incompleteTasks = + TaskGeneratorUtils.getIncompleteTasks(taskType, tableNameWithType, _clusterInfoAccessor); + if (!incompleteTasks.isEmpty()) { + LOGGER.warn("Found incomplete tasks: {} for same table: {} and task type: {}. Skipping task generation.", + incompleteTasks.keySet(), tableNameWithType, taskType); + continue; + } + + Map<String, String> taskConfigs = tableConfig.getTaskConfig().getConfigsForTaskType(taskType); + List<SegmentZKMetadata> allSegments = _clusterInfoAccessor.getSegmentsZKMetadata(tableNameWithType); + + // Get completed segments and filter out the segments based on the buffer time configuration + List<SegmentZKMetadata> candidateSegments = + getCandidateSegments(taskConfigs, allSegments, System.currentTimeMillis()); + + if (candidateSegments.isEmpty()) { + LOGGER.info("No segments were eligible for compactMerge task for table: {}", tableNameWithType); + continue; + } + + // get server to segment mappings + PinotHelixResourceManager pinotHelixResourceManager = _clusterInfoAccessor.getPinotHelixResourceManager(); + Map<String, List<String>> serverToSegments = pinotHelixResourceManager.getServerToSegmentsMap(tableNameWithType); + BiMap<String, String> serverToEndpoints; + try { + serverToEndpoints = pinotHelixResourceManager.getDataInstanceAdminEndpoints(serverToSegments.keySet()); + } catch (InvalidConfigException e) { + throw new RuntimeException(e); + } + + ServerSegmentMetadataReader serverSegmentMetadataReader = + new ServerSegmentMetadataReader(_clusterInfoAccessor.getExecutor(), + _clusterInfoAccessor.getConnectionManager()); + + // Number of segments to query per server request. If a table has a lot of segments, then we might send a + // huge payload to pinot-server in request. Batching the requests will help in reducing the payload size. + int numSegmentsBatchPerServerRequest = Integer.parseInt( + taskConfigs.getOrDefault(MinionConstants.UpsertCompactMergeTask.NUM_SEGMENTS_BATCH_PER_SERVER_REQUEST, + String.valueOf(DEFAULT_NUM_SEGMENTS_BATCH_PER_SERVER_REQUEST))); + + Map<String, List<ValidDocIdsMetadataInfo>> validDocIdsMetadataList = + serverSegmentMetadataReader.getSegmentToValidDocIdsMetadataFromServer(tableNameWithType, serverToSegments, + serverToEndpoints, null, 60_000, ValidDocIdsType.SNAPSHOT.toString(), + numSegmentsBatchPerServerRequest); + + Map<String, SegmentZKMetadata> candidateSegmentsMap = + candidateSegments.stream().collect(Collectors.toMap(SegmentZKMetadata::getSegmentName, Function.identity())); + + List<String> alreadyMergedSegments = getAlreadyMergedSegments(allSegments); + + SegmentSelectionResult segmentSelectionResult = + processValidDocIdsMetadata(taskConfigs, candidateSegmentsMap, validDocIdsMetadataList, alreadyMergedSegments); + + if (!segmentSelectionResult.getSegmentsForDeletion().isEmpty()) { + pinotHelixResourceManager.deleteSegments(tableNameWithType, segmentSelectionResult.getSegmentsForDeletion(), + "0d"); + LOGGER.info( + "Deleted segments containing only invalid records for table: {}, number of segments to be deleted: {}", + tableNameWithType, segmentSelectionResult.getSegmentsForDeletion()); + } + + int numTasks = 0; + int maxTasks = Integer.parseInt(taskConfigs.getOrDefault(MinionConstants.TABLE_MAX_NUM_TASKS_KEY, + String.valueOf(MinionConstants.DEFAULT_TABLE_MAX_NUM_TASKS))); + for (Map.Entry<Integer, List<List<SegmentMergerMetadata>>> entry + : segmentSelectionResult.getSegmentsForCompactMerge().entrySet()) { + if (numTasks == maxTasks) { + break; + } + List<List<SegmentMergerMetadata>> groups = entry.getValue(); + // no valid groups found in the partition to merge + if (groups.isEmpty()) { + continue; + } + // there are no groups with more than 1 segment to merge + // this can be later removed if we want to just do single-segment compaction from this task + if (groups.get(0).size() <= 1) { + continue; + } + // TODO see if multiple groups of same partition can be added + Map<String, String> configs = new HashMap<>(getBaseTaskConfigs(tableConfig, + groups.get(0).stream() + .map(x -> x.getSegmentZKMetadata().getSegmentName()).collect(Collectors.toList()))); + configs.put(MinionConstants.DOWNLOAD_URL_KEY, + StringUtils.join(groups.get(0).stream().map(x -> x.getSegmentZKMetadata().getDownloadUrl()) Review Comment: Let's move the StringUtils stuff to a separate function? Not only improves readability but you can also write a quick UT. Same for `ORIGINAL_SEGMENT_CRC_KEY` ########## pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompactmerge/UpsertCompactMergeTaskGenerator.java: ########## @@ -0,0 +1,419 @@ +/** + * 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.upsertcompactmerge; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.BiMap; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.apache.helix.task.TaskState; +import org.apache.pinot.common.exception.InvalidConfigException; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.restlet.resources.ValidDocIdsMetadataInfo; +import org.apache.pinot.common.restlet.resources.ValidDocIdsType; +import org.apache.pinot.common.utils.SegmentUtils; +import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; +import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator; +import org.apache.pinot.controller.helix.core.minion.generator.TaskGeneratorUtils; +import org.apache.pinot.controller.util.ServerSegmentMetadataReader; +import org.apache.pinot.core.common.MinionConstants; +import org.apache.pinot.core.minion.PinotTaskConfig; +import org.apache.pinot.spi.annotations.minion.TaskGenerator; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.config.table.UpsertConfig; +import org.apache.pinot.spi.utils.TimeUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +@TaskGenerator +public class UpsertCompactMergeTaskGenerator extends BaseTaskGenerator { + + private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactMergeTaskGenerator.class); + private static final String DEFAULT_BUFFER_PERIOD = "7d"; + private static final int DEFAULT_NUM_SEGMENTS_BATCH_PER_SERVER_REQUEST = 500; + + public static class SegmentMergerMetadata { + SegmentZKMetadata _segmentZKMetadata; Review Comment: nit: final member vars ########## pinot-core/src/main/java/org/apache/pinot/core/common/MinionConstants.java: ########## @@ -199,4 +200,59 @@ public static class UpsertCompactionTask { */ public static final String NUM_SEGMENTS_BATCH_PER_SERVER_REQUEST = "numSegmentsBatchPerServerRequest"; } + + public static class UpsertCompactMergeTask { + public static final String TASK_TYPE = "UpsertCompactMergeTask"; + + /** + * 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"; + + /** + * number of segments to query in one batch to fetch valid doc id metadata, by default 500 + */ + public static final String NUM_SEGMENTS_BATCH_PER_SERVER_REQUEST = "numSegmentsBatchPerServerRequest"; + + /** + * prefix for the new segment name that is created, + * {@link org.apache.pinot.segment.spi.creator.name.UploadedRealtimeSegmentNameGenerator} will add __ as delimiter + * so not adding _ as a suffix here. + */ + public static final String MERGED_SEGMENT_NAME_PREFIX = "compactmerged"; Review Comment: How about simply `compacted`, since compaction job anyways doesn't alter the name. ########## pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompactmerge/UpsertCompactMergeTaskExecutor.java: ########## @@ -0,0 +1,166 @@ +/** + * 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.upsertcompactmerge; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadataCustomMapModifier; +import org.apache.pinot.common.restlet.resources.ValidDocIdsType; +import org.apache.pinot.common.utils.SegmentUtils; +import org.apache.pinot.core.common.MinionConstants; +import org.apache.pinot.core.minion.PinotTaskConfig; +import org.apache.pinot.core.segment.processing.framework.SegmentProcessorConfig; +import org.apache.pinot.core.segment.processing.framework.SegmentProcessorFramework; +import org.apache.pinot.minion.MinionConf; +import org.apache.pinot.plugin.minion.tasks.BaseMultipleSegmentsConversionExecutor; +import org.apache.pinot.plugin.minion.tasks.MinionTaskUtils; +import org.apache.pinot.plugin.minion.tasks.SegmentConversionResult; +import org.apache.pinot.segment.local.segment.readers.CompactedPinotSegmentRecordReader; +import org.apache.pinot.segment.spi.creator.name.UploadedRealtimeSegmentNameGenerator; +import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.data.readers.RecordReader; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.roaringbitmap.RoaringBitmap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class UpsertCompactMergeTaskExecutor extends BaseMultipleSegmentsConversionExecutor { + + private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactMergeTaskExecutor.class); + + public UpsertCompactMergeTaskExecutor(MinionConf minionConf) { + super(minionConf); + } + + @Override + protected List<SegmentConversionResult> convert(PinotTaskConfig pinotTaskConfig, List<File> segmentDirs, + File workingDir) + throws Exception { + int numInputSegments = segmentDirs.size(); + List<SegmentConversionResult> results = new ArrayList<>(); + _eventObserver.notifyProgress(pinotTaskConfig, "Converting segments: " + numInputSegments); + String taskType = pinotTaskConfig.getTaskType(); + Map<String, String> configs = pinotTaskConfig.getConfigs(); + LOGGER.info("Starting task: {} with configs: {}", taskType, configs); + long startMillis = System.currentTimeMillis(); + + String tableNameWithType = configs.get(MinionConstants.TABLE_NAME_KEY); + TableConfig tableConfig = getTableConfig(tableNameWithType); + Schema schema = getSchema(tableNameWithType); + + SegmentProcessorConfig.Builder segmentProcessorConfigBuilder = + new SegmentProcessorConfig.Builder().setTableConfig(tableConfig).setSchema(schema); + + // Progress observer + segmentProcessorConfigBuilder.setProgressObserver(p -> _eventObserver.notifyProgress(_pinotTaskConfig, p)); + + List<RecordReader> recordReaders = new ArrayList<>(numInputSegments); + int count = 1; Review Comment: count is always `i + 1`? if so let's remove? ########## pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompactmerge/UpsertCompactMergeTaskExecutor.java: ########## @@ -0,0 +1,166 @@ +/** + * 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.upsertcompactmerge; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadataCustomMapModifier; +import org.apache.pinot.common.restlet.resources.ValidDocIdsType; +import org.apache.pinot.common.utils.SegmentUtils; +import org.apache.pinot.core.common.MinionConstants; +import org.apache.pinot.core.minion.PinotTaskConfig; +import org.apache.pinot.core.segment.processing.framework.SegmentProcessorConfig; +import org.apache.pinot.core.segment.processing.framework.SegmentProcessorFramework; +import org.apache.pinot.minion.MinionConf; +import org.apache.pinot.plugin.minion.tasks.BaseMultipleSegmentsConversionExecutor; +import org.apache.pinot.plugin.minion.tasks.MinionTaskUtils; +import org.apache.pinot.plugin.minion.tasks.SegmentConversionResult; +import org.apache.pinot.segment.local.segment.readers.CompactedPinotSegmentRecordReader; +import org.apache.pinot.segment.spi.creator.name.UploadedRealtimeSegmentNameGenerator; +import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.data.readers.RecordReader; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.roaringbitmap.RoaringBitmap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class UpsertCompactMergeTaskExecutor extends BaseMultipleSegmentsConversionExecutor { + + private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactMergeTaskExecutor.class); + + public UpsertCompactMergeTaskExecutor(MinionConf minionConf) { + super(minionConf); + } + + @Override + protected List<SegmentConversionResult> convert(PinotTaskConfig pinotTaskConfig, List<File> segmentDirs, + File workingDir) + throws Exception { + int numInputSegments = segmentDirs.size(); + List<SegmentConversionResult> results = new ArrayList<>(); + _eventObserver.notifyProgress(pinotTaskConfig, "Converting segments: " + numInputSegments); + String taskType = pinotTaskConfig.getTaskType(); + Map<String, String> configs = pinotTaskConfig.getConfigs(); + LOGGER.info("Starting task: {} with configs: {}", taskType, configs); + long startMillis = System.currentTimeMillis(); + + String tableNameWithType = configs.get(MinionConstants.TABLE_NAME_KEY); + TableConfig tableConfig = getTableConfig(tableNameWithType); + Schema schema = getSchema(tableNameWithType); + + SegmentProcessorConfig.Builder segmentProcessorConfigBuilder = + new SegmentProcessorConfig.Builder().setTableConfig(tableConfig).setSchema(schema); + + // Progress observer + segmentProcessorConfigBuilder.setProgressObserver(p -> _eventObserver.notifyProgress(_pinotTaskConfig, p)); + + List<RecordReader> recordReaders = new ArrayList<>(numInputSegments); + int count = 1; + int partitionId = -1; + long maxCreationTimeOfMergingSegments = 0; + List<String> originalSegmentCrcFromTaskGenerator = + List.of(configs.get(MinionConstants.ORIGINAL_SEGMENT_CRC_KEY).split(",")); + for (int i = 0; i < numInputSegments; i++) { + File segmentDir = segmentDirs.get(i); + _eventObserver.notifyProgress(_pinotTaskConfig, + String.format("Creating RecordReader for: %s (%d out of %d)", segmentDir, count++, numInputSegments)); + + SegmentMetadataImpl segmentMetadata = new SegmentMetadataImpl(segmentDir); + String segmentName = segmentMetadata.getName(); + Integer segmentPartitionId = SegmentUtils.getPartitionIdFromRealtimeSegmentName(segmentName); + if (segmentPartitionId == null) { + throw new IllegalStateException(String.format("Partition id not found for %s", segmentName)); + } + if (partitionId != -1 && partitionId != segmentPartitionId) { + throw new IllegalStateException(String.format("Partition id mismatched for %s, expected partition id: %d", + segmentName, partitionId)); + } + partitionId = segmentPartitionId; + maxCreationTimeOfMergingSegments = Math.max(maxCreationTimeOfMergingSegments, + segmentMetadata.getIndexCreationTime()); + + String crcFromDeepStorageSegment = segmentMetadata.getCrc(); + if (!originalSegmentCrcFromTaskGenerator.get(i).equals(crcFromDeepStorageSegment)) { + String message = String.format("Crc mismatched between ZK and deepstore copy of segment: %s. Expected crc " + + "from ZK: %s, crc from deepstore: %s", segmentName, originalSegmentCrcFromTaskGenerator.get(i), + crcFromDeepStorageSegment); + LOGGER.error(message); + throw new IllegalStateException(message); + } + RoaringBitmap validDocIds = MinionTaskUtils.getValidDocIdFromServerMatchingCrc(tableNameWithType, segmentName, + ValidDocIdsType.SNAPSHOT.name(), MINION_CONTEXT, crcFromDeepStorageSegment); + if (validDocIds == null) { + // no valid crc match found or no validDocIds obtained from all servers + // error out the task instead of silently failing so that we can track it via task-error metrics + String message = String.format("No validDocIds found from all servers. They either failed to download " + + "or did not match crc from segment copy obtained from deepstore / servers. " + "Expected crc: %s", + ""); + LOGGER.error(message); + throw new IllegalStateException(message); + } + + recordReaders.add(new CompactedPinotSegmentRecordReader(segmentDir, validDocIds)); + } + + segmentProcessorConfigBuilder.setSegmentNameGenerator( + new UploadedRealtimeSegmentNameGenerator(TableNameBuilder.extractRawTableName(tableNameWithType), partitionId, + System.currentTimeMillis(), MinionConstants.UpsertCompactMergeTask.MERGED_SEGMENT_NAME_PREFIX, null)); + if (maxCreationTimeOfMergingSegments != 0) { Review Comment: If it is 0, we shouldn't continue ahead and throw an error? ########## pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompactmerge/UpsertCompactMergeTaskGenerator.java: ########## @@ -0,0 +1,419 @@ +/** + * 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.upsertcompactmerge; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.BiMap; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.apache.helix.task.TaskState; +import org.apache.pinot.common.exception.InvalidConfigException; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.restlet.resources.ValidDocIdsMetadataInfo; +import org.apache.pinot.common.restlet.resources.ValidDocIdsType; +import org.apache.pinot.common.utils.SegmentUtils; +import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; +import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator; +import org.apache.pinot.controller.helix.core.minion.generator.TaskGeneratorUtils; +import org.apache.pinot.controller.util.ServerSegmentMetadataReader; +import org.apache.pinot.core.common.MinionConstants; +import org.apache.pinot.core.minion.PinotTaskConfig; +import org.apache.pinot.spi.annotations.minion.TaskGenerator; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.config.table.UpsertConfig; +import org.apache.pinot.spi.utils.TimeUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +@TaskGenerator +public class UpsertCompactMergeTaskGenerator extends BaseTaskGenerator { + + private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactMergeTaskGenerator.class); + private static final String DEFAULT_BUFFER_PERIOD = "7d"; + private static final int DEFAULT_NUM_SEGMENTS_BATCH_PER_SERVER_REQUEST = 500; + + public static class SegmentMergerMetadata { + SegmentZKMetadata _segmentZKMetadata; + long _validDocIds; + long _invalidDocIds; + + SegmentMergerMetadata(SegmentZKMetadata segmentZKMetadata, long validDocIds, long invalidDocIds) { + _segmentZKMetadata = segmentZKMetadata; + _validDocIds = validDocIds; + _invalidDocIds = invalidDocIds; + } + + public SegmentZKMetadata getSegmentZKMetadata() { + return _segmentZKMetadata; + } + + public long getValidDocIds() { + return _validDocIds; + } + + public long getInvalidDocIds() { + return _invalidDocIds; + } + } + + public static class SegmentSelectionResult { + + private final Map<Integer, List<List<SegmentMergerMetadata>>> _segmentsForCompactMerge; + + private final List<String> _segmentsForDeletion; + + SegmentSelectionResult(Map<Integer, List<List<SegmentMergerMetadata>>> segmentsForCompactMerge, + List<String> segmentsForDeletion) { + _segmentsForCompactMerge = segmentsForCompactMerge; + _segmentsForDeletion = segmentsForDeletion; + } + + public Map<Integer, List<List<SegmentMergerMetadata>>> getSegmentsForCompactMerge() { + return _segmentsForCompactMerge; + } + + public List<String> getSegmentsForDeletion() { + return _segmentsForDeletion; + } + } + + @Override + public String getTaskType() { + return MinionConstants.UpsertCompactMergeTask.TASK_TYPE; + } + + @Override + public List<PinotTaskConfig> generateTasks(List<TableConfig> tableConfigs) { + String taskType = MinionConstants.UpsertCompactMergeTask.TASK_TYPE; + List<PinotTaskConfig> pinotTaskConfigs = new ArrayList<>(); + for (TableConfig tableConfig : tableConfigs) { + + String tableNameWithType = tableConfig.getTableName(); + LOGGER.info("Start generating task configs for table: {}", tableNameWithType); + + if (tableConfig.getTaskConfig() == null) { + LOGGER.warn("Task config is null for table: {}", tableNameWithType); + continue; + } + + // Only schedule 1 task of this type, per table + Map<String, TaskState> incompleteTasks = + TaskGeneratorUtils.getIncompleteTasks(taskType, tableNameWithType, _clusterInfoAccessor); + if (!incompleteTasks.isEmpty()) { + LOGGER.warn("Found incomplete tasks: {} for same table: {} and task type: {}. Skipping task generation.", + incompleteTasks.keySet(), tableNameWithType, taskType); + continue; + } + + Map<String, String> taskConfigs = tableConfig.getTaskConfig().getConfigsForTaskType(taskType); + List<SegmentZKMetadata> allSegments = _clusterInfoAccessor.getSegmentsZKMetadata(tableNameWithType); + + // Get completed segments and filter out the segments based on the buffer time configuration + List<SegmentZKMetadata> candidateSegments = + getCandidateSegments(taskConfigs, allSegments, System.currentTimeMillis()); + + if (candidateSegments.isEmpty()) { + LOGGER.info("No segments were eligible for compactMerge task for table: {}", tableNameWithType); + continue; + } + + // get server to segment mappings + PinotHelixResourceManager pinotHelixResourceManager = _clusterInfoAccessor.getPinotHelixResourceManager(); + Map<String, List<String>> serverToSegments = pinotHelixResourceManager.getServerToSegmentsMap(tableNameWithType); + BiMap<String, String> serverToEndpoints; + try { + serverToEndpoints = pinotHelixResourceManager.getDataInstanceAdminEndpoints(serverToSegments.keySet()); + } catch (InvalidConfigException e) { + throw new RuntimeException(e); + } + + ServerSegmentMetadataReader serverSegmentMetadataReader = + new ServerSegmentMetadataReader(_clusterInfoAccessor.getExecutor(), + _clusterInfoAccessor.getConnectionManager()); + + // Number of segments to query per server request. If a table has a lot of segments, then we might send a + // huge payload to pinot-server in request. Batching the requests will help in reducing the payload size. + int numSegmentsBatchPerServerRequest = Integer.parseInt( + taskConfigs.getOrDefault(MinionConstants.UpsertCompactMergeTask.NUM_SEGMENTS_BATCH_PER_SERVER_REQUEST, + String.valueOf(DEFAULT_NUM_SEGMENTS_BATCH_PER_SERVER_REQUEST))); + + Map<String, List<ValidDocIdsMetadataInfo>> validDocIdsMetadataList = + serverSegmentMetadataReader.getSegmentToValidDocIdsMetadataFromServer(tableNameWithType, serverToSegments, + serverToEndpoints, null, 60_000, ValidDocIdsType.SNAPSHOT.toString(), + numSegmentsBatchPerServerRequest); + + Map<String, SegmentZKMetadata> candidateSegmentsMap = + candidateSegments.stream().collect(Collectors.toMap(SegmentZKMetadata::getSegmentName, Function.identity())); + + List<String> alreadyMergedSegments = getAlreadyMergedSegments(allSegments); + + SegmentSelectionResult segmentSelectionResult = + processValidDocIdsMetadata(taskConfigs, candidateSegmentsMap, validDocIdsMetadataList, alreadyMergedSegments); + + if (!segmentSelectionResult.getSegmentsForDeletion().isEmpty()) { + pinotHelixResourceManager.deleteSegments(tableNameWithType, segmentSelectionResult.getSegmentsForDeletion(), + "0d"); + LOGGER.info( + "Deleted segments containing only invalid records for table: {}, number of segments to be deleted: {}", + tableNameWithType, segmentSelectionResult.getSegmentsForDeletion()); + } + + int numTasks = 0; + int maxTasks = Integer.parseInt(taskConfigs.getOrDefault(MinionConstants.TABLE_MAX_NUM_TASKS_KEY, + String.valueOf(MinionConstants.DEFAULT_TABLE_MAX_NUM_TASKS))); + for (Map.Entry<Integer, List<List<SegmentMergerMetadata>>> entry + : segmentSelectionResult.getSegmentsForCompactMerge().entrySet()) { + if (numTasks == maxTasks) { + break; + } + List<List<SegmentMergerMetadata>> groups = entry.getValue(); + // no valid groups found in the partition to merge + if (groups.isEmpty()) { + continue; + } + // there are no groups with more than 1 segment to merge + // this can be later removed if we want to just do single-segment compaction from this task + if (groups.get(0).size() <= 1) { + continue; + } + // TODO see if multiple groups of same partition can be added + Map<String, String> configs = new HashMap<>(getBaseTaskConfigs(tableConfig, + groups.get(0).stream() + .map(x -> x.getSegmentZKMetadata().getSegmentName()).collect(Collectors.toList()))); + configs.put(MinionConstants.DOWNLOAD_URL_KEY, + StringUtils.join(groups.get(0).stream().map(x -> x.getSegmentZKMetadata().getDownloadUrl()) + .collect(Collectors.toList()), ",")); + configs.put(MinionConstants.UPLOAD_URL_KEY, _clusterInfoAccessor.getVipUrl() + "/segments"); + configs.put(MinionConstants.ORIGINAL_SEGMENT_CRC_KEY, StringUtils.join(groups.get(0).stream() + .map(x -> String.valueOf(x.getSegmentZKMetadata().getCrc())).collect(Collectors.toList()), ",")); + pinotTaskConfigs.add(new PinotTaskConfig(MinionConstants.UpsertCompactMergeTask.TASK_TYPE, configs)); + numTasks++; + } + LOGGER.info("Finished generating {} tasks configs for table: {}", numTasks, tableNameWithType); + } + return pinotTaskConfigs; + } + + @VisibleForTesting + public static SegmentSelectionResult processValidDocIdsMetadata( + Map<String, String> taskConfigs, + Map<String, SegmentZKMetadata> candidateSegmentsMap, + Map<String, List<ValidDocIdsMetadataInfo>> validDocIdsMetadataInfoMap, + List<String> alreadyMergedSegments) { + Map<Integer, List<SegmentMergerMetadata>> segmentsEligibleForCompactMerge = new HashMap<>(); + List<String> segmentsForDeletion = new ArrayList<>(); + for (String segmentName : validDocIdsMetadataInfoMap.keySet()) { + // check if segment is part of completed segments + if (!candidateSegmentsMap.containsKey(segmentName)) { + LOGGER.warn("Segment {} is not found in the completed segments list, skipping it for {}", segmentName, + MinionConstants.UpsertCompactMergeTask.TASK_TYPE); + continue; + } + SegmentZKMetadata segment = candidateSegmentsMap.get(segmentName); + for (ValidDocIdsMetadataInfo validDocIdsMetadata : validDocIdsMetadataInfoMap.get(segmentName)) { + long totalInvalidDocs = validDocIdsMetadata.getTotalInvalidDocs(); + long totalValidDocs = validDocIdsMetadata.getTotalValidDocs(); + + // Skip segments if the crc from zk metadata and server does not match. They may be getting reloaded. + if (segment.getCrc() != Long.parseLong(validDocIdsMetadata.getSegmentCrc())) { + LOGGER.warn("CRC mismatch for segment: {}, (segmentZKMetadata={}, validDocIdsMetadata={})", segmentName, + segment.getCrc(), validDocIdsMetadata.getSegmentCrc()); + continue; + } + + // segments eligible for deletion with no valid records + long totalDocs = validDocIdsMetadata.getTotalDocs(); + if (totalInvalidDocs == totalDocs) { + segmentsForDeletion.add(segmentName); + } else if (alreadyMergedSegments.contains(segmentName)) { + LOGGER.debug("Segment {} already merged. Skipping it for {}", segmentName, + MinionConstants.UpsertCompactMergeTask.TASK_TYPE); + continue; + } else { + Integer partitionID = SegmentUtils.getPartitionIdFromRealtimeSegmentName(segmentName); + if (partitionID == null) { + LOGGER.warn("Partition ID not found for segment: {}, skipping it for {}", segmentName, + MinionConstants.UpsertCompactMergeTask.TASK_TYPE); + continue; + } + segmentsEligibleForCompactMerge.computeIfAbsent(partitionID, k -> new ArrayList<>()) + .add(new SegmentMergerMetadata(segment, totalValidDocs, totalInvalidDocs)); + } + break; + } + } + + segmentsEligibleForCompactMerge.forEach((partitionID, segmentList) -> segmentList.sort((o1, o2) -> { + // Sort primarily by creationTime in ascending order + if (o1.getSegmentZKMetadata().getCreationTime() < o2.getSegmentZKMetadata().getCreationTime()) { Review Comment: We can use Comparator.comparingLong here? ########## pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompactmerge/UpsertCompactMergeTaskGenerator.java: ########## @@ -0,0 +1,419 @@ +/** + * 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.upsertcompactmerge; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.BiMap; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.apache.helix.task.TaskState; +import org.apache.pinot.common.exception.InvalidConfigException; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.restlet.resources.ValidDocIdsMetadataInfo; +import org.apache.pinot.common.restlet.resources.ValidDocIdsType; +import org.apache.pinot.common.utils.SegmentUtils; +import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; +import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator; +import org.apache.pinot.controller.helix.core.minion.generator.TaskGeneratorUtils; +import org.apache.pinot.controller.util.ServerSegmentMetadataReader; +import org.apache.pinot.core.common.MinionConstants; +import org.apache.pinot.core.minion.PinotTaskConfig; +import org.apache.pinot.spi.annotations.minion.TaskGenerator; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.config.table.UpsertConfig; +import org.apache.pinot.spi.utils.TimeUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +@TaskGenerator +public class UpsertCompactMergeTaskGenerator extends BaseTaskGenerator { + + private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactMergeTaskGenerator.class); + private static final String DEFAULT_BUFFER_PERIOD = "7d"; + private static final int DEFAULT_NUM_SEGMENTS_BATCH_PER_SERVER_REQUEST = 500; + + public static class SegmentMergerMetadata { + SegmentZKMetadata _segmentZKMetadata; + long _validDocIds; + long _invalidDocIds; + + SegmentMergerMetadata(SegmentZKMetadata segmentZKMetadata, long validDocIds, long invalidDocIds) { + _segmentZKMetadata = segmentZKMetadata; + _validDocIds = validDocIds; + _invalidDocIds = invalidDocIds; + } + + public SegmentZKMetadata getSegmentZKMetadata() { + return _segmentZKMetadata; + } + + public long getValidDocIds() { + return _validDocIds; + } + + public long getInvalidDocIds() { + return _invalidDocIds; + } + } + + public static class SegmentSelectionResult { + + private final Map<Integer, List<List<SegmentMergerMetadata>>> _segmentsForCompactMerge; + + private final List<String> _segmentsForDeletion; + + SegmentSelectionResult(Map<Integer, List<List<SegmentMergerMetadata>>> segmentsForCompactMerge, + List<String> segmentsForDeletion) { + _segmentsForCompactMerge = segmentsForCompactMerge; + _segmentsForDeletion = segmentsForDeletion; + } + + public Map<Integer, List<List<SegmentMergerMetadata>>> getSegmentsForCompactMerge() { + return _segmentsForCompactMerge; + } + + public List<String> getSegmentsForDeletion() { + return _segmentsForDeletion; + } + } + + @Override + public String getTaskType() { + return MinionConstants.UpsertCompactMergeTask.TASK_TYPE; + } + + @Override + public List<PinotTaskConfig> generateTasks(List<TableConfig> tableConfigs) { + String taskType = MinionConstants.UpsertCompactMergeTask.TASK_TYPE; + List<PinotTaskConfig> pinotTaskConfigs = new ArrayList<>(); + for (TableConfig tableConfig : tableConfigs) { + + String tableNameWithType = tableConfig.getTableName(); + LOGGER.info("Start generating task configs for table: {}", tableNameWithType); + + if (tableConfig.getTaskConfig() == null) { + LOGGER.warn("Task config is null for table: {}", tableNameWithType); + continue; + } + + // Only schedule 1 task of this type, per table + Map<String, TaskState> incompleteTasks = + TaskGeneratorUtils.getIncompleteTasks(taskType, tableNameWithType, _clusterInfoAccessor); + if (!incompleteTasks.isEmpty()) { + LOGGER.warn("Found incomplete tasks: {} for same table: {} and task type: {}. Skipping task generation.", + incompleteTasks.keySet(), tableNameWithType, taskType); + continue; + } + + Map<String, String> taskConfigs = tableConfig.getTaskConfig().getConfigsForTaskType(taskType); + List<SegmentZKMetadata> allSegments = _clusterInfoAccessor.getSegmentsZKMetadata(tableNameWithType); + + // Get completed segments and filter out the segments based on the buffer time configuration + List<SegmentZKMetadata> candidateSegments = + getCandidateSegments(taskConfigs, allSegments, System.currentTimeMillis()); + + if (candidateSegments.isEmpty()) { + LOGGER.info("No segments were eligible for compactMerge task for table: {}", tableNameWithType); + continue; + } + + // get server to segment mappings + PinotHelixResourceManager pinotHelixResourceManager = _clusterInfoAccessor.getPinotHelixResourceManager(); + Map<String, List<String>> serverToSegments = pinotHelixResourceManager.getServerToSegmentsMap(tableNameWithType); + BiMap<String, String> serverToEndpoints; + try { + serverToEndpoints = pinotHelixResourceManager.getDataInstanceAdminEndpoints(serverToSegments.keySet()); + } catch (InvalidConfigException e) { + throw new RuntimeException(e); + } + + ServerSegmentMetadataReader serverSegmentMetadataReader = + new ServerSegmentMetadataReader(_clusterInfoAccessor.getExecutor(), + _clusterInfoAccessor.getConnectionManager()); + + // Number of segments to query per server request. If a table has a lot of segments, then we might send a + // huge payload to pinot-server in request. Batching the requests will help in reducing the payload size. + int numSegmentsBatchPerServerRequest = Integer.parseInt( + taskConfigs.getOrDefault(MinionConstants.UpsertCompactMergeTask.NUM_SEGMENTS_BATCH_PER_SERVER_REQUEST, + String.valueOf(DEFAULT_NUM_SEGMENTS_BATCH_PER_SERVER_REQUEST))); + + Map<String, List<ValidDocIdsMetadataInfo>> validDocIdsMetadataList = + serverSegmentMetadataReader.getSegmentToValidDocIdsMetadataFromServer(tableNameWithType, serverToSegments, + serverToEndpoints, null, 60_000, ValidDocIdsType.SNAPSHOT.toString(), + numSegmentsBatchPerServerRequest); + + Map<String, SegmentZKMetadata> candidateSegmentsMap = + candidateSegments.stream().collect(Collectors.toMap(SegmentZKMetadata::getSegmentName, Function.identity())); + + List<String> alreadyMergedSegments = getAlreadyMergedSegments(allSegments); + + SegmentSelectionResult segmentSelectionResult = + processValidDocIdsMetadata(taskConfigs, candidateSegmentsMap, validDocIdsMetadataList, alreadyMergedSegments); + + if (!segmentSelectionResult.getSegmentsForDeletion().isEmpty()) { + pinotHelixResourceManager.deleteSegments(tableNameWithType, segmentSelectionResult.getSegmentsForDeletion(), + "0d"); + LOGGER.info( + "Deleted segments containing only invalid records for table: {}, number of segments to be deleted: {}", + tableNameWithType, segmentSelectionResult.getSegmentsForDeletion()); + } + + int numTasks = 0; + int maxTasks = Integer.parseInt(taskConfigs.getOrDefault(MinionConstants.TABLE_MAX_NUM_TASKS_KEY, + String.valueOf(MinionConstants.DEFAULT_TABLE_MAX_NUM_TASKS))); + for (Map.Entry<Integer, List<List<SegmentMergerMetadata>>> entry + : segmentSelectionResult.getSegmentsForCompactMerge().entrySet()) { + if (numTasks == maxTasks) { + break; + } + List<List<SegmentMergerMetadata>> groups = entry.getValue(); + // no valid groups found in the partition to merge + if (groups.isEmpty()) { + continue; + } + // there are no groups with more than 1 segment to merge + // this can be later removed if we want to just do single-segment compaction from this task + if (groups.get(0).size() <= 1) { + continue; + } + // TODO see if multiple groups of same partition can be added + Map<String, String> configs = new HashMap<>(getBaseTaskConfigs(tableConfig, + groups.get(0).stream() + .map(x -> x.getSegmentZKMetadata().getSegmentName()).collect(Collectors.toList()))); + configs.put(MinionConstants.DOWNLOAD_URL_KEY, + StringUtils.join(groups.get(0).stream().map(x -> x.getSegmentZKMetadata().getDownloadUrl()) + .collect(Collectors.toList()), ",")); + configs.put(MinionConstants.UPLOAD_URL_KEY, _clusterInfoAccessor.getVipUrl() + "/segments"); + configs.put(MinionConstants.ORIGINAL_SEGMENT_CRC_KEY, StringUtils.join(groups.get(0).stream() + .map(x -> String.valueOf(x.getSegmentZKMetadata().getCrc())).collect(Collectors.toList()), ",")); + pinotTaskConfigs.add(new PinotTaskConfig(MinionConstants.UpsertCompactMergeTask.TASK_TYPE, configs)); + numTasks++; + } + LOGGER.info("Finished generating {} tasks configs for table: {}", numTasks, tableNameWithType); + } + return pinotTaskConfigs; + } + + @VisibleForTesting + public static SegmentSelectionResult processValidDocIdsMetadata( + Map<String, String> taskConfigs, + Map<String, SegmentZKMetadata> candidateSegmentsMap, + Map<String, List<ValidDocIdsMetadataInfo>> validDocIdsMetadataInfoMap, + List<String> alreadyMergedSegments) { + Map<Integer, List<SegmentMergerMetadata>> segmentsEligibleForCompactMerge = new HashMap<>(); + List<String> segmentsForDeletion = new ArrayList<>(); + for (String segmentName : validDocIdsMetadataInfoMap.keySet()) { + // check if segment is part of completed segments + if (!candidateSegmentsMap.containsKey(segmentName)) { + LOGGER.warn("Segment {} is not found in the completed segments list, skipping it for {}", segmentName, + MinionConstants.UpsertCompactMergeTask.TASK_TYPE); + continue; + } + SegmentZKMetadata segment = candidateSegmentsMap.get(segmentName); + for (ValidDocIdsMetadataInfo validDocIdsMetadata : validDocIdsMetadataInfoMap.get(segmentName)) { + long totalInvalidDocs = validDocIdsMetadata.getTotalInvalidDocs(); + long totalValidDocs = validDocIdsMetadata.getTotalValidDocs(); + + // Skip segments if the crc from zk metadata and server does not match. They may be getting reloaded. + if (segment.getCrc() != Long.parseLong(validDocIdsMetadata.getSegmentCrc())) { + LOGGER.warn("CRC mismatch for segment: {}, (segmentZKMetadata={}, validDocIdsMetadata={})", segmentName, + segment.getCrc(), validDocIdsMetadata.getSegmentCrc()); + continue; + } + + // segments eligible for deletion with no valid records + long totalDocs = validDocIdsMetadata.getTotalDocs(); + if (totalInvalidDocs == totalDocs) { + segmentsForDeletion.add(segmentName); + } else if (alreadyMergedSegments.contains(segmentName)) { + LOGGER.debug("Segment {} already merged. Skipping it for {}", segmentName, + MinionConstants.UpsertCompactMergeTask.TASK_TYPE); + continue; + } else { + Integer partitionID = SegmentUtils.getPartitionIdFromRealtimeSegmentName(segmentName); + if (partitionID == null) { + LOGGER.warn("Partition ID not found for segment: {}, skipping it for {}", segmentName, + MinionConstants.UpsertCompactMergeTask.TASK_TYPE); + continue; + } + segmentsEligibleForCompactMerge.computeIfAbsent(partitionID, k -> new ArrayList<>()) + .add(new SegmentMergerMetadata(segment, totalValidDocs, totalInvalidDocs)); + } + break; + } + } + + segmentsEligibleForCompactMerge.forEach((partitionID, segmentList) -> segmentList.sort((o1, o2) -> { + // Sort primarily by creationTime in ascending order + if (o1.getSegmentZKMetadata().getCreationTime() < o2.getSegmentZKMetadata().getCreationTime()) { + return -1; + } else if (o1.getSegmentZKMetadata().getCreationTime() == o2.getSegmentZKMetadata().getCreationTime()) { + return 0; + } + return 1; + })); + + // Map to store the result: each key (partition) will have a list of groups + Map<Integer, List<List<SegmentMergerMetadata>>> groupedSegments = new HashMap<>(); + + // Iterate over each partition and process its segments list + for (Map.Entry<Integer, List<SegmentMergerMetadata>> entry : segmentsEligibleForCompactMerge.entrySet()) { + int partitionID = entry.getKey(); + List<SegmentMergerMetadata> segments = entry.getValue(); + // task config thresholds + long validDocsThreshold = Long.parseLong(taskConfigs.getOrDefault(MinionConstants + .UpsertCompactMergeTask.MAX_NUM_RECORDS_PER_SEGMENT_KEY, + String.valueOf(MinionConstants.UpsertCompactMergeTask.DEFAULT_MAX_NUM_RECORDS_PER_SEGMENT))); + long maxRecordsPerTask = Long.parseLong(taskConfigs.getOrDefault(MinionConstants Review Comment: Let's leave a TODO for finalizing the semantics. Specifically, we should rely on segment size as our primary guide instead of max rows per task. We could do it in a separate PR which can help drive a discussion too. ########## pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/upsertcompactmerge/UpsertCompactMergeTaskGenerator.java: ########## @@ -0,0 +1,419 @@ +/** + * 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.upsertcompactmerge; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.BiMap; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.apache.helix.task.TaskState; +import org.apache.pinot.common.exception.InvalidConfigException; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.restlet.resources.ValidDocIdsMetadataInfo; +import org.apache.pinot.common.restlet.resources.ValidDocIdsType; +import org.apache.pinot.common.utils.SegmentUtils; +import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; +import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator; +import org.apache.pinot.controller.helix.core.minion.generator.TaskGeneratorUtils; +import org.apache.pinot.controller.util.ServerSegmentMetadataReader; +import org.apache.pinot.core.common.MinionConstants; +import org.apache.pinot.core.minion.PinotTaskConfig; +import org.apache.pinot.spi.annotations.minion.TaskGenerator; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.config.table.UpsertConfig; +import org.apache.pinot.spi.utils.TimeUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +@TaskGenerator +public class UpsertCompactMergeTaskGenerator extends BaseTaskGenerator { + + private static final Logger LOGGER = LoggerFactory.getLogger(UpsertCompactMergeTaskGenerator.class); + private static final String DEFAULT_BUFFER_PERIOD = "7d"; + private static final int DEFAULT_NUM_SEGMENTS_BATCH_PER_SERVER_REQUEST = 500; + + public static class SegmentMergerMetadata { + SegmentZKMetadata _segmentZKMetadata; + long _validDocIds; + long _invalidDocIds; + + SegmentMergerMetadata(SegmentZKMetadata segmentZKMetadata, long validDocIds, long invalidDocIds) { + _segmentZKMetadata = segmentZKMetadata; + _validDocIds = validDocIds; + _invalidDocIds = invalidDocIds; + } + + public SegmentZKMetadata getSegmentZKMetadata() { + return _segmentZKMetadata; + } + + public long getValidDocIds() { + return _validDocIds; + } + + public long getInvalidDocIds() { + return _invalidDocIds; + } + } + + public static class SegmentSelectionResult { + + private final Map<Integer, List<List<SegmentMergerMetadata>>> _segmentsForCompactMerge; + + private final List<String> _segmentsForDeletion; + + SegmentSelectionResult(Map<Integer, List<List<SegmentMergerMetadata>>> segmentsForCompactMerge, + List<String> segmentsForDeletion) { + _segmentsForCompactMerge = segmentsForCompactMerge; + _segmentsForDeletion = segmentsForDeletion; + } + + public Map<Integer, List<List<SegmentMergerMetadata>>> getSegmentsForCompactMerge() { + return _segmentsForCompactMerge; + } + + public List<String> getSegmentsForDeletion() { + return _segmentsForDeletion; + } + } + + @Override + public String getTaskType() { + return MinionConstants.UpsertCompactMergeTask.TASK_TYPE; + } + + @Override + public List<PinotTaskConfig> generateTasks(List<TableConfig> tableConfigs) { + String taskType = MinionConstants.UpsertCompactMergeTask.TASK_TYPE; + List<PinotTaskConfig> pinotTaskConfigs = new ArrayList<>(); + for (TableConfig tableConfig : tableConfigs) { + + String tableNameWithType = tableConfig.getTableName(); + LOGGER.info("Start generating task configs for table: {}", tableNameWithType); + + if (tableConfig.getTaskConfig() == null) { + LOGGER.warn("Task config is null for table: {}", tableNameWithType); + continue; + } + + // Only schedule 1 task of this type, per table + Map<String, TaskState> incompleteTasks = + TaskGeneratorUtils.getIncompleteTasks(taskType, tableNameWithType, _clusterInfoAccessor); + if (!incompleteTasks.isEmpty()) { + LOGGER.warn("Found incomplete tasks: {} for same table: {} and task type: {}. Skipping task generation.", + incompleteTasks.keySet(), tableNameWithType, taskType); + continue; + } + + Map<String, String> taskConfigs = tableConfig.getTaskConfig().getConfigsForTaskType(taskType); + List<SegmentZKMetadata> allSegments = _clusterInfoAccessor.getSegmentsZKMetadata(tableNameWithType); + + // Get completed segments and filter out the segments based on the buffer time configuration + List<SegmentZKMetadata> candidateSegments = + getCandidateSegments(taskConfigs, allSegments, System.currentTimeMillis()); + + if (candidateSegments.isEmpty()) { + LOGGER.info("No segments were eligible for compactMerge task for table: {}", tableNameWithType); + continue; + } + + // get server to segment mappings + PinotHelixResourceManager pinotHelixResourceManager = _clusterInfoAccessor.getPinotHelixResourceManager(); + Map<String, List<String>> serverToSegments = pinotHelixResourceManager.getServerToSegmentsMap(tableNameWithType); + BiMap<String, String> serverToEndpoints; + try { + serverToEndpoints = pinotHelixResourceManager.getDataInstanceAdminEndpoints(serverToSegments.keySet()); + } catch (InvalidConfigException e) { + throw new RuntimeException(e); + } + + ServerSegmentMetadataReader serverSegmentMetadataReader = + new ServerSegmentMetadataReader(_clusterInfoAccessor.getExecutor(), + _clusterInfoAccessor.getConnectionManager()); + + // Number of segments to query per server request. If a table has a lot of segments, then we might send a + // huge payload to pinot-server in request. Batching the requests will help in reducing the payload size. + int numSegmentsBatchPerServerRequest = Integer.parseInt( + taskConfigs.getOrDefault(MinionConstants.UpsertCompactMergeTask.NUM_SEGMENTS_BATCH_PER_SERVER_REQUEST, + String.valueOf(DEFAULT_NUM_SEGMENTS_BATCH_PER_SERVER_REQUEST))); + + Map<String, List<ValidDocIdsMetadataInfo>> validDocIdsMetadataList = + serverSegmentMetadataReader.getSegmentToValidDocIdsMetadataFromServer(tableNameWithType, serverToSegments, + serverToEndpoints, null, 60_000, ValidDocIdsType.SNAPSHOT.toString(), + numSegmentsBatchPerServerRequest); + + Map<String, SegmentZKMetadata> candidateSegmentsMap = + candidateSegments.stream().collect(Collectors.toMap(SegmentZKMetadata::getSegmentName, Function.identity())); + + List<String> alreadyMergedSegments = getAlreadyMergedSegments(allSegments); + + SegmentSelectionResult segmentSelectionResult = + processValidDocIdsMetadata(taskConfigs, candidateSegmentsMap, validDocIdsMetadataList, alreadyMergedSegments); + + if (!segmentSelectionResult.getSegmentsForDeletion().isEmpty()) { + pinotHelixResourceManager.deleteSegments(tableNameWithType, segmentSelectionResult.getSegmentsForDeletion(), + "0d"); + LOGGER.info( + "Deleted segments containing only invalid records for table: {}, number of segments to be deleted: {}", + tableNameWithType, segmentSelectionResult.getSegmentsForDeletion()); + } + + int numTasks = 0; + int maxTasks = Integer.parseInt(taskConfigs.getOrDefault(MinionConstants.TABLE_MAX_NUM_TASKS_KEY, + String.valueOf(MinionConstants.DEFAULT_TABLE_MAX_NUM_TASKS))); + for (Map.Entry<Integer, List<List<SegmentMergerMetadata>>> entry + : segmentSelectionResult.getSegmentsForCompactMerge().entrySet()) { + if (numTasks == maxTasks) { + break; + } + List<List<SegmentMergerMetadata>> groups = entry.getValue(); + // no valid groups found in the partition to merge + if (groups.isEmpty()) { + continue; + } + // there are no groups with more than 1 segment to merge + // this can be later removed if we want to just do single-segment compaction from this task Review Comment: let's leave a TODO -- 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