somandal commented on code in PR #16857:
URL: https://github.com/apache/pinot/pull/16857#discussion_r2412042193
##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java:
##########
@@ -239,40 +254,79 @@ public Map<String, String> createTask(String taskType,
String tableName, @Nullab
// Example usage in BaseTaskGenerator.getNumSubTasks()
taskConfigs.put(MinionConstants.TRIGGERED_BY,
CommonConstants.TaskTriggers.ADHOC_TRIGGER.name());
- List<PinotTaskConfig> pinotTaskConfigs =
taskGenerator.generateTasks(tableConfig, taskConfigs);
- if (pinotTaskConfigs.isEmpty()) {
- LOGGER.warn("No ad-hoc task generated for task type: {}", taskType);
- continue;
+ // Acquire distributed lock before proceeding with ad-hoc task generation
+ // Need locking to protect against:
+ // 1. Race conditions with periodic task generation
+ // 2. Multiple simultaneous ad-hoc requests
+ // 3. Leadership changes during task generation
+ DistributedTaskLockManager.TaskLock lock = null;
+ boolean taskCreationSuccess = true;
+ if (_distributedTaskLockManager != null) {
+ lock = _distributedTaskLockManager.acquireLock(tableNameWithType);
+ if (lock == null) {
+ String message = String.format("Could not acquire table level
distributed lock for ad-hoc task type: %s, "
+ + "table: %s. Another controller is likely generating tasks
for this type. Please try again later.",
+ taskType, tableNameWithType);
+ LOGGER.warn(message);
+ throw new RuntimeException(message);
+ }
+ LOGGER.info("Acquired table level distributed lock for ad-hoc task
type: {} on table: {}", taskType,
+ tableNameWithType);
}
- int maxNumberOfSubTasks = taskGenerator.getMaxAllowedSubTasksPerTask();
- if (pinotTaskConfigs.size() > maxNumberOfSubTasks) {
- String message = String.format(
- "Number of tasks generated for task type: %s for table: %s is %d,
which is greater than the "
- + "maximum number of tasks to schedule: %d. This is "
- + "controlled by the cluster config %s which is set based on
controller's performance.", taskType,
- tableName, pinotTaskConfigs.size(), maxNumberOfSubTasks,
MinionConstants.MAX_ALLOWED_SUB_TASKS_KEY);
- message += "Optimise the task config or reduce tableMaxNumTasks to
avoid the error";
- // We throw an exception to notify the user
- // This is to ensure that the user is aware of the task generation
limit
- throw new RuntimeException(message);
+
+ try {
+ List<PinotTaskConfig> pinotTaskConfigs =
taskGenerator.generateTasks(tableConfig, taskConfigs);
+ if (pinotTaskConfigs.isEmpty()) {
+ LOGGER.warn("No ad-hoc task generated for task type: {}, for table:
{}", taskType, tableNameWithType);
+ continue;
+ }
+ int maxNumberOfSubTasks = taskGenerator.getMaxAllowedSubTasksPerTask();
+ if (pinotTaskConfigs.size() > maxNumberOfSubTasks) {
+ String message = String.format(
+ "Number of tasks generated for task type: %s for table: %s is
%d, which is greater than the "
+ + "maximum number of tasks to schedule: %d. This is
controlled by the cluster config %s which is set "
+ + "based on controller's performance.", taskType,
tableNameWithType, pinotTaskConfigs.size(),
+ maxNumberOfSubTasks, MinionConstants.MAX_ALLOWED_SUB_TASKS_KEY);
+ message += "Optimise the task config or reduce tableMaxNumTasks to
avoid the error";
+ // We throw an exception to notify the user
+ // This is to ensure that the user is aware of the task generation
limit
+ taskCreationSuccess = false;
+ throw new RuntimeException(message);
+ }
+ pinotTaskConfigs.forEach(pinotTaskConfig ->
pinotTaskConfig.getConfigs()
+ .computeIfAbsent(MinionConstants.TRIGGERED_BY, k ->
CommonConstants.TaskTriggers.ADHOC_TRIGGER.name()));
+ addDefaultsToTaskConfig(pinotTaskConfigs);
+ LOGGER.info("Submitting ad-hoc task for task type: {} with task
configs: {}", taskType, pinotTaskConfigs);
+ _controllerMetrics.addMeteredTableValue(taskType,
ControllerMeter.NUMBER_ADHOC_TASKS_SUBMITTED, 1);
+ responseMap.put(tableNameWithType,
+ _helixTaskResourceManager.submitTask(parentTaskName,
pinotTaskConfigs, minionInstanceTag,
+ taskGenerator.getTaskTimeoutMs(minionInstanceTag),
+
taskGenerator.getNumConcurrentTasksPerInstance(minionInstanceTag),
+ taskGenerator.getMaxAttemptsPerTask(minionInstanceTag)));
+ } finally {
+ if (!responseMap.containsKey(tableNameWithType)) {
+ LOGGER.warn("No task submitted for tableNameWithType: {}",
tableNameWithType);
+ taskCreationSuccess = false;
+ }
+ if (lock != null) {
+ _distributedTaskLockManager.releaseLock(lock, taskCreationSuccess);
Review Comment:
removed this - was an artifact of the earlier -State node
##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/DistributedTaskLockManager.java:
##########
@@ -0,0 +1,423 @@
+/**
+ * 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.minion;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.UUID;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import javax.annotation.Nullable;
+import org.apache.helix.AccessOption;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Manages distributed locks for minion task generation using ZooKeeper
ephemeral sequential nodes.
+ * Uses ephemeral nodes that automatically disappear when the controller
session ends.
+ * This approach provides automatic cleanup and is suitable for long-running
task generation.
+ * Locks are held until explicitly released or the controller session
terminates.
+ * Locks are at the table level, to ensure that only one type of task can be
generated per table at any given time.
+ * <p>
+ * ZK EPHEMERAL_SEQUENTIAL Locks (see <a
href="https://zookeeper.apache.org/doc/current/recipes.html#sc_recipes_Locks">
+ * ZooKeeper Lock Recipe.</a> for more details):
+ * <ul>
+ * <li>Every lock is created with a lock prefix. Lock prefix used:
[controllerName]-lock-[UUID]. The UUID helps
+ * differentiate between requests originating from the same controller at
the same time
+ * <li>When ZK creates the ZNode, it appends a sequence number at the end.
E.g.
+ * [controllerName]-lock-[UUID]-00000001
+ * <li>The sequence number is used to identify the lock winner in case
more than one lock node is created at the
+ * same time. The smallest sequence number always wins
+ * <li>The locks are EPHEMERAL in nature, meaning that once the session
with ZK is lost, the lock is automatically
+ * cleaned up. Scenarios when the ZK session can be lost: a) controller
shutdown, b) controller crash, c) ZK session
+ * expiry (e.g. long GC pauses can cause this)
+ * <li>This implementation does not set up watches as described in the
recipe as the task lock is released whenever
+ * we identify that the lock is already acquired. Do not expect lock
ownership to automatically change for the
+ * time being. If such support is needed in the future, this can be
enhanced to add a watch on the neighboring
+ * lock node
+ * </ul>
+ * <p>
+ * Example of how the locks will work:
+ * <p>
+ * Say we have two controllers, and one controller happens to run 2 threads at
the same time, all of which need to take
+ * the distributed lock. Each thread will create a distributed lock node, and
the "-Lock" ZNode getChildren will return:
+ * <ul>
+ * <li>controller2-lock-xyzwx-00000002
+ * <li>controller1-lock-abcde-00000001
+ * <li>controller1-lock-ab345-00000003
+ * </ul>
+ * <p>
+ * In the above, the controller1 with UUID abcde will win the lock as it has
the smallest sequence number. The other
+ * two threads will clean up their locks and return error that the distributed
lock could not be acquired. Controller1
+ * will proceed with performing its tasks, and when done will release the lock.
+ */
+public class DistributedTaskLockManager {
+ private static final Logger LOGGER =
LoggerFactory.getLogger(DistributedTaskLockManager.class);
+
+ // Lock paths are constructed using ZKMetadataProvider
+ private static final String LOCK_OWNER_KEY = "lockOwner";
+ private static final String LOCK_UUID_KEY = "lockUuid";
+ private static final String LOCK_TIMESTAMP_MILLIS_KEY =
"lockTimestampMillis";
+
+ // Define a custom comparator to compare strings of format
'<controllerName>-lock-<uuid>-<sequenceNumber>' and sort
+ // them by the sequence number at the end
+ private static final Comparator<String> TASK_LOCK_SEQUENCE_ID_COMPARATOR =
(s1, s2) -> {
+ // Regex to find the trailing sequence of digits
+ Pattern p = Pattern.compile("\\d+$");
+
+ // Extract the number from the first string
+ Matcher m1 = p.matcher(s1);
+ long num1 = m1.find() ? Long.parseLong(m1.group()) : 0;
+
+ // Extract the number from the second string
+ Matcher m2 = p.matcher(s2);
+ long num2 = m2.find() ? Long.parseLong(m2.group()) : 0;
+
+ return Long.compare(num1, num2);
+ };
+
+ private final ZkHelixPropertyStore<ZNRecord> _propertyStore;
+ private final String _controllerInstanceId;
+
+ public DistributedTaskLockManager(ZkHelixPropertyStore<ZNRecord>
propertyStore, String controllerInstanceId) {
+ _propertyStore = propertyStore;
+ _controllerInstanceId = controllerInstanceId;
+
+ // Ensure base paths exist
+ ensureBasePaths();
+ }
+
+ /**
+ * Attempts to acquire a distributed lock at the table level for task
generation using session-based locking.
+ * The lock is held until explicitly released or the controller session ends.
Review Comment:
done
##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java:
##########
@@ -210,6 +224,7 @@ public Map<String, String> createTask(String taskType,
String tableName, @Nullab
if (tableNameWithTypes.isEmpty()) {
throw new TableNotFoundException("'tableName' " + tableName + " is not
found");
}
+ LOGGER.info("Generating tasks for {} tables, list: {}",
tableNameWithTypes.size(), tableNameWithTypes);
Review Comment:
done
--
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]