lnbest0707-uber commented on code in PR #15782:
URL: https://github.com/apache/pinot/pull/15782#discussion_r2241539166


##########
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:
   It is ordered. And it is using the 
`_pinotHelixResourceManager.setExistingTableConfig` method to set the config. 
That method has built-in retry and further using the helix API to ensure 
concurrency control. IIUC, the concurrent modification should not be the issue.



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

Review Comment:
   `isTopicBackfillJobComplete` method was used there.



##########
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:
   Added the duplicate check in `TableConfigUtil` to prevent duplicate topic 
name



##########
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:
   Yes, we do not use `Nullable` here. It should at least be empty string `""`



##########
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:
   no, we don't. The new leader controller of the table would take care of it



##########
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:
   added the example



##########
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:
   good point



##########
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:
   Good catch. This should be major. Made the change.



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


Reply via email to