deemoliu commented on code in PR #15782: URL: https://github.com/apache/pinot/pull/15782#discussion_r2240673800
########## pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeOffsetAutoResetManager.java: ########## @@ -0,0 +1,240 @@ +/** + * 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.controller.validation; + +import com.google.common.annotations.VisibleForTesting; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.pinot.common.metrics.ControllerMetrics; +import org.apache.pinot.controller.ControllerConf; +import org.apache.pinot.controller.LeadControllerManager; +import org.apache.pinot.controller.api.resources.Constants; +import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; +import org.apache.pinot.controller.helix.core.periodictask.ControllerPeriodicTask; +import org.apache.pinot.controller.helix.core.periodictask.RealtimeOffsetAutoResetHandler; +import org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.stream.StreamConfig; +import org.apache.pinot.spi.utils.IngestionConfigUtils; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class RealtimeOffsetAutoResetManager extends ControllerPeriodicTask<RealtimeOffsetAutoResetManager.Context> { + + private static final Logger LOGGER = LoggerFactory.getLogger(RealtimeOffsetAutoResetManager.class); + private final PinotLLCRealtimeSegmentManager _llcRealtimeSegmentManager; + private final PinotHelixResourceManager _pinotHelixResourceManager; + private final Map<String, RealtimeOffsetAutoResetHandler> _tableToHandler; + private final Map<String, Set<String>> _tableTopicsUnderBackfill; + private final Map<String, Set<String>> _tableEphemeralTopics; + + public RealtimeOffsetAutoResetManager(ControllerConf config, PinotHelixResourceManager pinotHelixResourceManager, + LeadControllerManager leadControllerManager, PinotLLCRealtimeSegmentManager llcRealtimeSegmentManager, + ControllerMetrics controllerMetrics) { + super("RealtimeOffsetAutoResetManager", config.getRealtimeOffsetAutoResetFrequencyInSeconds(), + config.getRealtimeOffsetAutoResetInitialDelaySeconds(), pinotHelixResourceManager, + leadControllerManager, controllerMetrics); + _llcRealtimeSegmentManager = llcRealtimeSegmentManager; + _pinotHelixResourceManager = pinotHelixResourceManager; + _tableToHandler = new ConcurrentHashMap<>(); + _tableTopicsUnderBackfill = new ConcurrentHashMap<>(); + _tableEphemeralTopics = new ConcurrentHashMap<>(); + } + + @Override + protected RealtimeOffsetAutoResetManager.Context preprocess(Properties periodicTaskProperties) { + RealtimeOffsetAutoResetManager.Context context = new RealtimeOffsetAutoResetManager.Context(); + // Fill offset back fill job required info + if (periodicTaskProperties.containsKey(context._backfillJobPropertyKeys.toArray()[0])) { + context._shouldTriggerBackfillJobs = true; + for (String key : context._backfillJobPropertyKeys) { + context._backfillJobProperties.put(key, periodicTaskProperties.getProperty(key)); + } + } + return context; + } + + @VisibleForTesting + protected RealtimeOffsetAutoResetHandler getTableHandler(String tableNameWithType) { + return _tableToHandler.get(tableNameWithType); + } + + @Override + protected void processTable(String tableNameWithType, RealtimeOffsetAutoResetManager.Context context) { + if (!TableNameBuilder.isRealtimeTableResource(tableNameWithType)) { + return; + } + LOGGER.info("Processing offset auto reset backfill for table {}, with context {}", tableNameWithType, context); + + TableConfig tableConfig = _pinotHelixResourceManager.getTableConfig(tableNameWithType); + if (tableConfig == null) { + LOGGER.error("Failed to find table config for table: {}, skipping auto reset periodic job", tableNameWithType); + return; + } + RealtimeOffsetAutoResetHandler handler = getOrConstructHandler(tableConfig); + if (handler == null) { + return; + } + + if (context._shouldTriggerBackfillJobs) { + _tableTopicsUnderBackfill.putIfAbsent(tableNameWithType, ConcurrentHashMap.newKeySet()); + String topicName = context._backfillJobProperties.get(Constants.RESET_OFFSET_TOPIC_NAME); + _tableTopicsUnderBackfill.get(tableNameWithType).add(topicName); + + StreamConfig topicStreamConfig = IngestionConfigUtils.getStreamConfigs(tableConfig).stream() + .filter(config -> topicName.equals(config.getTopicName())) + .findFirst().orElseThrow(() -> new RuntimeException("No matching topic found")); + LOGGER.info("Trigger backfill jobs with StreamConfig {}, topicName {}, properties {}", + topicStreamConfig, topicName, context._backfillJobProperties); + _tableToHandler.get(tableNameWithType).triggerBackfillJob(tableNameWithType, + topicStreamConfig, + topicName, + Integer.valueOf(context._backfillJobProperties.get(Constants.RESET_OFFSET_TOPIC_PARTITION)), Review Comment: try catch numberFormatException? ########## pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeOffsetAutoResetManager.java: ########## @@ -0,0 +1,240 @@ +/** + * 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.controller.validation; + +import com.google.common.annotations.VisibleForTesting; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.pinot.common.metrics.ControllerMetrics; +import org.apache.pinot.controller.ControllerConf; +import org.apache.pinot.controller.LeadControllerManager; +import org.apache.pinot.controller.api.resources.Constants; +import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; +import org.apache.pinot.controller.helix.core.periodictask.ControllerPeriodicTask; +import org.apache.pinot.controller.helix.core.periodictask.RealtimeOffsetAutoResetHandler; +import org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.stream.StreamConfig; +import org.apache.pinot.spi.utils.IngestionConfigUtils; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class RealtimeOffsetAutoResetManager extends ControllerPeriodicTask<RealtimeOffsetAutoResetManager.Context> { + + private static final Logger LOGGER = LoggerFactory.getLogger(RealtimeOffsetAutoResetManager.class); + private final PinotLLCRealtimeSegmentManager _llcRealtimeSegmentManager; + private final PinotHelixResourceManager _pinotHelixResourceManager; + private final Map<String, RealtimeOffsetAutoResetHandler> _tableToHandler; + private final Map<String, Set<String>> _tableTopicsUnderBackfill; + private final Map<String, Set<String>> _tableEphemeralTopics; + + public RealtimeOffsetAutoResetManager(ControllerConf config, PinotHelixResourceManager pinotHelixResourceManager, + LeadControllerManager leadControllerManager, PinotLLCRealtimeSegmentManager llcRealtimeSegmentManager, + ControllerMetrics controllerMetrics) { + super("RealtimeOffsetAutoResetManager", config.getRealtimeOffsetAutoResetFrequencyInSeconds(), + config.getRealtimeOffsetAutoResetInitialDelaySeconds(), pinotHelixResourceManager, + leadControllerManager, controllerMetrics); + _llcRealtimeSegmentManager = llcRealtimeSegmentManager; + _pinotHelixResourceManager = pinotHelixResourceManager; + _tableToHandler = new ConcurrentHashMap<>(); + _tableTopicsUnderBackfill = new ConcurrentHashMap<>(); + _tableEphemeralTopics = new ConcurrentHashMap<>(); + } + + @Override + protected RealtimeOffsetAutoResetManager.Context preprocess(Properties periodicTaskProperties) { + RealtimeOffsetAutoResetManager.Context context = new RealtimeOffsetAutoResetManager.Context(); + // Fill offset back fill job required info + if (periodicTaskProperties.containsKey(context._backfillJobPropertyKeys.toArray()[0])) { Review Comment: can you add pointers for a test or a comment/javaDoc for an example of periodTaskProperties. ########## pinot-spi/src/main/java/org/apache/pinot/spi/utils/IngestionConfigUtils.java: ########## @@ -124,6 +126,25 @@ public static int getStreamConfigIndexFromPinotPartitionId(int partitionId) { return partitionId / PARTITION_PADDING_OFFSET; } + /** + * Getting the StreamConfig from StreamConfigs list based on topicName and partitionId. + * @param partitionId the segment partition id on Pinot + */ + public static StreamConfig getStreamConfigFromStreamConfigList( + int partitionId, String topicName, List<StreamConfig> streamConfigs) { + int rawTopicIndex = getStreamConfigIndexFromPinotPartitionId(partitionId); + if (topicName != null && !topicName.isEmpty()) { + return streamConfigs.stream().filter(c -> topicName.equals(c.getTopicName())) Review Comment: If multiple StreamConfigs match a same topic name, only the first is returned. we should clarify we don't support multiple "same topic different kafka cluster" in the streamConfigs. ########## pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/periodictask/RealtimeOffsetAutoResetKafkaHandler.java: ########## @@ -0,0 +1,135 @@ +/** + * 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.controller.helix.core.periodictask; + +import com.google.common.base.Preconditions; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; +import org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.stream.OffsetCriteria; +import org.apache.pinot.spi.stream.StreamConfig; +import org.apache.pinot.spi.stream.StreamConfigProperties; +import org.apache.pinot.spi.utils.IngestionConfigUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public abstract class RealtimeOffsetAutoResetKafkaHandler extends RealtimeOffsetAutoResetHandler { + + private static final Logger LOGGER = LoggerFactory.getLogger(RealtimeOffsetAutoResetKafkaHandler.class); + private static final String STREAM_TYPE = "kafka"; + + public RealtimeOffsetAutoResetKafkaHandler(PinotLLCRealtimeSegmentManager llcRealtimeSegmentManager, + PinotHelixResourceManager pinotHelixResourceManager) { + super(llcRealtimeSegmentManager, pinotHelixResourceManager); + } + + /** + * Trigger the job to backfill the skipped interval due to offset auto reset. + * It is expected to backfill the [fromOffset, toOffset) interval. + * @return if successfully started the backfill job and its ingestion + */ + @Override + public boolean triggerBackfillJob( + String tableNameWithType, StreamConfig streamConfig, String topicName, int partitionId, long fromOffset, + long toOffset) { + Map<String, String> newTopicStreamConfig = triggerDataReplicationAndGetTopicInfo( + tableNameWithType, streamConfig, topicName, partitionId, fromOffset, toOffset); + if (newTopicStreamConfig == null) { + return false; + } + try { + TableConfig currentTableConfig = _pinotHelixResourceManager.getTableConfig(tableNameWithType); + addNewTopicToTableConfig(newTopicStreamConfig, currentTableConfig); + _pinotHelixResourceManager.setExistingTableConfig(currentTableConfig); + } catch (IOException e) { + LOGGER.error("Cannot add backfill topic to the table config", e); + return false; + } + return true; + } + + /** + * Override this method to trigger Kafka Ecosystem APIs and replicate skipped offsets to the new topic. + * Then refer to the lagged topic's StreamConfig and return the new topic's stream config map. + */ + protected abstract Map<String, String> triggerDataReplicationAndGetTopicInfo( + String tableNameWithType, StreamConfig streamConfig, String topicName, int partitionId, long fromOffset, + long toOffset); + + public abstract void ensureBackfillJobsRunning(String tableNameWithType, List<String> topicNames); + + public Collection<String> cleanupCompletedBackfillJobs(String tableNameWithType, Collection<String> topicNames) { + Collection<String> cleanedUpTopics = new ArrayList<>(); + for (String topicName : topicNames) { + if (isTopicBackfillJobComplete(tableNameWithType, topicName)) { + cleanedUpTopics.add(topicName); + } + } + TableConfig currentTableConfig = _pinotHelixResourceManager.getTableConfig(tableNameWithType); + for (String topicName : cleanedUpTopics) { + removeTopicFromTableConfig(tableNameWithType, topicName, currentTableConfig); + } + try { + _pinotHelixResourceManager.setExistingTableConfig(currentTableConfig); + } catch (IOException e) { + LOGGER.error("Cannot remove backfill topics {} from the table config", topicNames, e); + cleanedUpTopics.clear(); + } + return cleanedUpTopics; + } + + public abstract boolean isTopicBackfillJobComplete(String tableNameWithType, String topicName); + + private void addNewTopicToTableConfig(Map<String, String> streamConfig, TableConfig tableConfig) { + List<Map<String, String>> streamConfigs = IngestionConfigUtils.getStreamConfigMaps(tableConfig); + String topicNameKey = StreamConfigProperties.constructStreamProperty( + STREAM_TYPE, StreamConfigProperties.STREAM_TOPIC_NAME); + String topicName = streamConfig.get(topicNameKey); + Preconditions.checkNotNull(topicName); + for (Map<String, String> config : streamConfigs) { + if (topicName.equals(config.get(topicNameKey))) { + LOGGER.info("Topic {} already added to table {}", topicName, tableConfig.getTableName()); + return; + } + } + streamConfig.put(StreamConfigProperties.EPHEMERAL_BACKFILL_TOPIC, String.valueOf(true)); + streamConfig.put( + StreamConfigProperties.constructStreamProperty( + STREAM_TYPE, StreamConfigProperties.STREAM_CONSUMER_OFFSET_CRITERIA), + OffsetCriteria.SMALLEST_OFFSET_CRITERIA.getOffsetString()); + IngestionConfigUtils.getStreamConfigMaps(tableConfig).add(streamConfig); + } + + private void removeTopicFromTableConfig(String tableNameWithType, String topicName, TableConfig tableConfig) { + List<Map<String, String>> streamConfigMaps = IngestionConfigUtils.getStreamConfigMaps(tableConfig); + for (int i = 0; i < streamConfigMaps.size(); i++) { Review Comment: Major: Q1: Does streamConfigMaps maintain the order? Q2: I feel it might be a concurrent modification exception here. ########## pinot-spi/src/main/java/org/apache/pinot/spi/stream/PartitionGroupConsumptionStatus.java: ########## @@ -35,15 +35,18 @@ * This information is needed by the stream, when grouping the partitions/shards into new partition groups. */ public class PartitionGroupConsumptionStatus { + + private final String _topicName; Review Comment: topicName seems to be a notNull value, let's verify this carefully in the integration tests. ########## pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeOffsetAutoResetManager.java: ########## @@ -0,0 +1,240 @@ +/** + * 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.controller.validation; + +import com.google.common.annotations.VisibleForTesting; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.pinot.common.metrics.ControllerMetrics; +import org.apache.pinot.controller.ControllerConf; +import org.apache.pinot.controller.LeadControllerManager; +import org.apache.pinot.controller.api.resources.Constants; +import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; +import org.apache.pinot.controller.helix.core.periodictask.ControllerPeriodicTask; +import org.apache.pinot.controller.helix.core.periodictask.RealtimeOffsetAutoResetHandler; +import org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.stream.StreamConfig; +import org.apache.pinot.spi.utils.IngestionConfigUtils; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class RealtimeOffsetAutoResetManager extends ControllerPeriodicTask<RealtimeOffsetAutoResetManager.Context> { + + private static final Logger LOGGER = LoggerFactory.getLogger(RealtimeOffsetAutoResetManager.class); + private final PinotLLCRealtimeSegmentManager _llcRealtimeSegmentManager; + private final PinotHelixResourceManager _pinotHelixResourceManager; + private final Map<String, RealtimeOffsetAutoResetHandler> _tableToHandler; + private final Map<String, Set<String>> _tableTopicsUnderBackfill; + private final Map<String, Set<String>> _tableEphemeralTopics; + + public RealtimeOffsetAutoResetManager(ControllerConf config, PinotHelixResourceManager pinotHelixResourceManager, + LeadControllerManager leadControllerManager, PinotLLCRealtimeSegmentManager llcRealtimeSegmentManager, + ControllerMetrics controllerMetrics) { + super("RealtimeOffsetAutoResetManager", config.getRealtimeOffsetAutoResetFrequencyInSeconds(), + config.getRealtimeOffsetAutoResetInitialDelaySeconds(), pinotHelixResourceManager, + leadControllerManager, controllerMetrics); + _llcRealtimeSegmentManager = llcRealtimeSegmentManager; + _pinotHelixResourceManager = pinotHelixResourceManager; + _tableToHandler = new ConcurrentHashMap<>(); + _tableTopicsUnderBackfill = new ConcurrentHashMap<>(); + _tableEphemeralTopics = new ConcurrentHashMap<>(); + } + + @Override + protected RealtimeOffsetAutoResetManager.Context preprocess(Properties periodicTaskProperties) { + RealtimeOffsetAutoResetManager.Context context = new RealtimeOffsetAutoResetManager.Context(); + // Fill offset back fill job required info + if (periodicTaskProperties.containsKey(context._backfillJobPropertyKeys.toArray()[0])) { + context._shouldTriggerBackfillJobs = true; + for (String key : context._backfillJobPropertyKeys) { + context._backfillJobProperties.put(key, periodicTaskProperties.getProperty(key)); + } + } + return context; + } + + @VisibleForTesting + protected RealtimeOffsetAutoResetHandler getTableHandler(String tableNameWithType) { + return _tableToHandler.get(tableNameWithType); + } + + @Override + protected void processTable(String tableNameWithType, RealtimeOffsetAutoResetManager.Context context) { + if (!TableNameBuilder.isRealtimeTableResource(tableNameWithType)) { + return; + } + LOGGER.info("Processing offset auto reset backfill for table {}, with context {}", tableNameWithType, context); + + TableConfig tableConfig = _pinotHelixResourceManager.getTableConfig(tableNameWithType); + if (tableConfig == null) { + LOGGER.error("Failed to find table config for table: {}, skipping auto reset periodic job", tableNameWithType); + return; + } + RealtimeOffsetAutoResetHandler handler = getOrConstructHandler(tableConfig); + if (handler == null) { + return; + } + + if (context._shouldTriggerBackfillJobs) { + _tableTopicsUnderBackfill.putIfAbsent(tableNameWithType, ConcurrentHashMap.newKeySet()); + String topicName = context._backfillJobProperties.get(Constants.RESET_OFFSET_TOPIC_NAME); + _tableTopicsUnderBackfill.get(tableNameWithType).add(topicName); + + StreamConfig topicStreamConfig = IngestionConfigUtils.getStreamConfigs(tableConfig).stream() + .filter(config -> topicName.equals(config.getTopicName())) + .findFirst().orElseThrow(() -> new RuntimeException("No matching topic found")); + LOGGER.info("Trigger backfill jobs with StreamConfig {}, topicName {}, properties {}", + topicStreamConfig, topicName, context._backfillJobProperties); + _tableToHandler.get(tableNameWithType).triggerBackfillJob(tableNameWithType, + topicStreamConfig, + topicName, + Integer.valueOf(context._backfillJobProperties.get(Constants.RESET_OFFSET_TOPIC_PARTITION)), + Long.valueOf(context._backfillJobProperties.get(Constants.RESET_OFFSET_FROM)), + Long.valueOf(context._backfillJobProperties.get(Constants.RESET_OFFSET_TO))); + } + + ensureBackfillJobsRunning(tableNameWithType); + ensureCompletedBackfillJobsCleanedUp(tableConfig); + } + + /** + * Get the list of tables & topics being backfilled and ensure the backfill jobs are running. + */ + private void ensureBackfillJobsRunning(String tableNameWithType) { + // Recover state from ephemeral multi-topics ingestion + // TODO: refactor or add other recover methods when other backfill approaches are ready + TableConfig tableConfig = _pinotHelixResourceManager.getTableConfig(tableNameWithType); + for (StreamConfig streamConfig : IngestionConfigUtils.getStreamConfigs(tableConfig)) { + if (streamConfig.isEphemeralBackfillTopic()) { + _tableEphemeralTopics.putIfAbsent(tableNameWithType, ConcurrentHashMap.newKeySet()); + _tableEphemeralTopics.get(tableNameWithType).add(streamConfig.getTopicName()); + } + } + if (!_tableTopicsUnderBackfill.containsKey(tableNameWithType) + || _tableTopicsUnderBackfill.get(tableNameWithType).isEmpty()) { + return; + } + RealtimeOffsetAutoResetHandler handler = getOrConstructHandler(tableConfig); + if (handler == null) { + return; + } + handler.ensureBackfillJobsRunning(tableNameWithType, _tableTopicsUnderBackfill.get(tableNameWithType)); + } + + private void ensureCompletedBackfillJobsCleanedUp(TableConfig tableConfig) { + String tableNameWithType = tableConfig.getTableName(); + if (!_tableEphemeralTopics.containsKey(tableNameWithType)) { + return; + } + LOGGER.info("Trying to clean up backfill jobs on {}", tableNameWithType); + RealtimeOffsetAutoResetHandler handler = getOrConstructHandler(tableConfig); + Collection<String> cleanedUpTopics = handler.cleanupCompletedBackfillJobs( + tableNameWithType, _tableEphemeralTopics.get(tableNameWithType)); + if (cleanedUpTopics.size() >= _tableEphemeralTopics.get(tableNameWithType).size()) { Review Comment: suggest: if (cleanedUpTopics.containsAll(_tableEphemeralTopics.get(tableNameWithType))) { ... } if that possible we have uncleaned topics from the previous round? ########## pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeOffsetAutoResetManager.java: ########## @@ -0,0 +1,240 @@ +/** + * 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.controller.validation; + +import com.google.common.annotations.VisibleForTesting; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.pinot.common.metrics.ControllerMetrics; +import org.apache.pinot.controller.ControllerConf; +import org.apache.pinot.controller.LeadControllerManager; +import org.apache.pinot.controller.api.resources.Constants; +import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; +import org.apache.pinot.controller.helix.core.periodictask.ControllerPeriodicTask; +import org.apache.pinot.controller.helix.core.periodictask.RealtimeOffsetAutoResetHandler; +import org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.stream.StreamConfig; +import org.apache.pinot.spi.utils.IngestionConfigUtils; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class RealtimeOffsetAutoResetManager extends ControllerPeriodicTask<RealtimeOffsetAutoResetManager.Context> { + + private static final Logger LOGGER = LoggerFactory.getLogger(RealtimeOffsetAutoResetManager.class); + private final PinotLLCRealtimeSegmentManager _llcRealtimeSegmentManager; + private final PinotHelixResourceManager _pinotHelixResourceManager; + private final Map<String, RealtimeOffsetAutoResetHandler> _tableToHandler; + private final Map<String, Set<String>> _tableTopicsUnderBackfill; + private final Map<String, Set<String>> _tableEphemeralTopics; + + public RealtimeOffsetAutoResetManager(ControllerConf config, PinotHelixResourceManager pinotHelixResourceManager, + LeadControllerManager leadControllerManager, PinotLLCRealtimeSegmentManager llcRealtimeSegmentManager, + ControllerMetrics controllerMetrics) { + super("RealtimeOffsetAutoResetManager", config.getRealtimeOffsetAutoResetFrequencyInSeconds(), + config.getRealtimeOffsetAutoResetInitialDelaySeconds(), pinotHelixResourceManager, + leadControllerManager, controllerMetrics); + _llcRealtimeSegmentManager = llcRealtimeSegmentManager; + _pinotHelixResourceManager = pinotHelixResourceManager; + _tableToHandler = new ConcurrentHashMap<>(); + _tableTopicsUnderBackfill = new ConcurrentHashMap<>(); + _tableEphemeralTopics = new ConcurrentHashMap<>(); + } + + @Override + protected RealtimeOffsetAutoResetManager.Context preprocess(Properties periodicTaskProperties) { + RealtimeOffsetAutoResetManager.Context context = new RealtimeOffsetAutoResetManager.Context(); + // Fill offset back fill job required info + if (periodicTaskProperties.containsKey(context._backfillJobPropertyKeys.toArray()[0])) { + context._shouldTriggerBackfillJobs = true; + for (String key : context._backfillJobPropertyKeys) { + context._backfillJobProperties.put(key, periodicTaskProperties.getProperty(key)); + } + } + return context; + } + + @VisibleForTesting + protected RealtimeOffsetAutoResetHandler getTableHandler(String tableNameWithType) { + return _tableToHandler.get(tableNameWithType); + } + + @Override + protected void processTable(String tableNameWithType, RealtimeOffsetAutoResetManager.Context context) { + if (!TableNameBuilder.isRealtimeTableResource(tableNameWithType)) { + return; + } + LOGGER.info("Processing offset auto reset backfill for table {}, with context {}", tableNameWithType, context); + + TableConfig tableConfig = _pinotHelixResourceManager.getTableConfig(tableNameWithType); + if (tableConfig == null) { + LOGGER.error("Failed to find table config for table: {}, skipping auto reset periodic job", tableNameWithType); + return; + } + RealtimeOffsetAutoResetHandler handler = getOrConstructHandler(tableConfig); + if (handler == null) { + return; + } + + if (context._shouldTriggerBackfillJobs) { + _tableTopicsUnderBackfill.putIfAbsent(tableNameWithType, ConcurrentHashMap.newKeySet()); + String topicName = context._backfillJobProperties.get(Constants.RESET_OFFSET_TOPIC_NAME); + _tableTopicsUnderBackfill.get(tableNameWithType).add(topicName); + + StreamConfig topicStreamConfig = IngestionConfigUtils.getStreamConfigs(tableConfig).stream() + .filter(config -> topicName.equals(config.getTopicName())) + .findFirst().orElseThrow(() -> new RuntimeException("No matching topic found")); + LOGGER.info("Trigger backfill jobs with StreamConfig {}, topicName {}, properties {}", + topicStreamConfig, topicName, context._backfillJobProperties); + _tableToHandler.get(tableNameWithType).triggerBackfillJob(tableNameWithType, + topicStreamConfig, + topicName, + Integer.valueOf(context._backfillJobProperties.get(Constants.RESET_OFFSET_TOPIC_PARTITION)), + Long.valueOf(context._backfillJobProperties.get(Constants.RESET_OFFSET_FROM)), + Long.valueOf(context._backfillJobProperties.get(Constants.RESET_OFFSET_TO))); + } + + ensureBackfillJobsRunning(tableNameWithType); + ensureCompletedBackfillJobsCleanedUp(tableConfig); + } + + /** + * Get the list of tables & topics being backfilled and ensure the backfill jobs are running. + */ + private void ensureBackfillJobsRunning(String tableNameWithType) { + // Recover state from ephemeral multi-topics ingestion + // TODO: refactor or add other recover methods when other backfill approaches are ready + TableConfig tableConfig = _pinotHelixResourceManager.getTableConfig(tableNameWithType); + for (StreamConfig streamConfig : IngestionConfigUtils.getStreamConfigs(tableConfig)) { + if (streamConfig.isEphemeralBackfillTopic()) { + _tableEphemeralTopics.putIfAbsent(tableNameWithType, ConcurrentHashMap.newKeySet()); + _tableEphemeralTopics.get(tableNameWithType).add(streamConfig.getTopicName()); + } + } + if (!_tableTopicsUnderBackfill.containsKey(tableNameWithType) + || _tableTopicsUnderBackfill.get(tableNameWithType).isEmpty()) { + return; + } + RealtimeOffsetAutoResetHandler handler = getOrConstructHandler(tableConfig); + if (handler == null) { + return; + } + handler.ensureBackfillJobsRunning(tableNameWithType, _tableTopicsUnderBackfill.get(tableNameWithType)); + } + + private void ensureCompletedBackfillJobsCleanedUp(TableConfig tableConfig) { + String tableNameWithType = tableConfig.getTableName(); + if (!_tableEphemeralTopics.containsKey(tableNameWithType)) { + return; + } + LOGGER.info("Trying to clean up backfill jobs on {}", tableNameWithType); + RealtimeOffsetAutoResetHandler handler = getOrConstructHandler(tableConfig); + Collection<String> cleanedUpTopics = handler.cleanupCompletedBackfillJobs( + tableNameWithType, _tableEphemeralTopics.get(tableNameWithType)); + if (cleanedUpTopics.size() >= _tableEphemeralTopics.get(tableNameWithType).size()) { + _tableTopicsUnderBackfill.remove(tableNameWithType); + _tableEphemeralTopics.remove(tableNameWithType); + if (_tableToHandler.get(tableNameWithType) != null) { + _tableToHandler.get(tableNameWithType).close(); + _tableToHandler.remove(tableNameWithType); + } + } else { + _tableEphemeralTopics.get(tableNameWithType).removeAll(cleanedUpTopics); + } + if (cleanedUpTopics.size() > 0) { + LOGGER.info("Cleaned up complete backfill topics {} for table {}", cleanedUpTopics, tableNameWithType); + } + } + + @Override + protected void nonLeaderCleanup(List<String> tableNamesWithType) { + for (String tableNameWithType : tableNamesWithType) { Review Comment: do we need to cleanup _tableEphemeralTopics in this setup -- 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