lnbest0707-uber commented on code in PR #15782: URL: https://github.com/apache/pinot/pull/15782#discussion_r2249497076
########## pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeOffsetAutoResetManager.java: ########## @@ -0,0 +1,248 @@ +/** + * 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 backfill job required info + // Examples of properties: + // resetOffsetTopicName=topicName + // resetOffsetTopicPartition=0 + // resetOffsetFrom=0 + // resetOffsetTo=1000 + if (periodicTaskProperties.containsKey(context._backfillJobPropertyKeys.toArray()[0])) { Review Comment: > For periodic task, periodicTaskProperties applies to all tables. How do you use it to setup the topic info for one single partition? Not sure if I understand correctly. When the task is triggered by the "periodic", the `periodicTaskProperties` is empty and the manager/handler would try to check if the registered backfill job is running or complete. If complete, it would remove the backfill topic. When the task is triggered by 'PinotLLCRealtimeSegmentManager` during segment commit, then `periodicTaskProperties` would have required info to "trigger" the backfill. ########## pinot-controller/src/main/java/org/apache/pinot/controller/validation/RealtimeOffsetAutoResetManager.java: ########## @@ -0,0 +1,248 @@ +/** + * 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 backfill job required info + // Examples of properties: + // resetOffsetTopicName=topicName + // resetOffsetTopicPartition=0 + // resetOffsetFrom=0 + // resetOffsetTo=1000 + if (periodicTaskProperties.containsKey(context._backfillJobPropertyKeys.toArray()[0])) { Review Comment: > For periodic task, periodicTaskProperties applies to all tables. How do you use it to setup the topic info for one single partition? Not sure if I understand correctly. When the task is triggered by the "periodic", the `periodicTaskProperties` is empty and the manager/handler would try to check if the registered backfill job is running or complete. If complete, it would remove the backfill topic. When the task is triggered by 'PinotLLCRealtimeSegmentManager` during segment commit, then `periodicTaskProperties` would have required info to "trigger" the backfill. -- 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]
