lnbest0707-uber commented on code in PR #15782: URL: https://github.com/apache/pinot/pull/15782#discussion_r2241540223
########## 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: revised -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
