Implementing perpetually running tasks for Streaming Ingestion#18466
Implementing perpetually running tasks for Streaming Ingestion#18466uds5501 wants to merge 49 commits intoapache:masterfrom
Conversation
| this.ioConfig = req.getIoConfig(); | ||
| this.stream = ioConfig.getStartSequenceNumbers().getStream(); | ||
| this.endOffsets = new ConcurrentHashMap<>(ioConfig.getEndSequenceNumbers().getPartitionSequenceNumberMap()); | ||
| minMessageTime = Configs.valueOrDefault(ioConfig.getMinimumMessageTime(), DateTimes.MIN); | ||
| maxMessageTime = Configs.valueOrDefault(ioConfig.getMaximumMessageTime(), DateTimes.MAX); |
There was a problem hiding this comment.
Maybe put this part in a new method so that constructor can also reuse the code.
| public Response updateConfig(TaskConfigUpdateRequest<PartitionIdType, SequenceOffsetType> req) throws InterruptedException | ||
| { | ||
| try { | ||
| requestPause(); |
There was a problem hiding this comment.
We should call pause() instead of requestPause(), since the latter only requests a pause but doesn't ensure that we reach a paused state.
If the pause() call returns non-OK response, we should return the same response immediately.
There was a problem hiding this comment.
Before starting with pause, we can log the new config that we are trying to update to.
After the update finishes, let's log the new config and also emit an event.
| * Creates new sequences for the ingestion process. It currently accepts the ioConfig given by the request as the correct offsets | ||
| * and ignores the offsets it may have stored in currOffsets and endOffsets. | ||
| */ | ||
| private void createNewSequenceFromIoConfig(SeekableStreamIndexTaskIOConfig<PartitionIdType, SequenceOffsetType> ioConfig) |
There was a problem hiding this comment.
For the 2 new methods, the SeekableStreamIndexTaskRunner must already be performing these actions.
Let's try to put them in common methods so that we can use the same method in the normal flow as well as on update config.
There was a problem hiding this comment.
Consider looking at the Kafka StickyAssignor implementation for inspiration here.
The Kafka StickyAssignor is a partition assignment strategy for Kafka consumers within a consumer group. Its primary goal is to achieve both a balanced distribution of partitions among consumers and to minimize the movement of partitions during rebalances.
|
@uds5501 , we should probably also add an API to get the latest config from the task since it would have diverged from the original task payload. |
f9c98ee to
aa67322
Compare
|
Edit : This has now been included in the original approach :) There's a flaw in the original approach. When the autoscaler event is triggered, the offsets are of time Instead, the chronology has to be:
Concerns:
|
|
New approach will perform the following:
|
9f608bd to
dcd3549
Compare
| final int numSegments = Integer.parseInt( | ||
| cluster.runSql("SELECT COUNT(*) FROM sys.segments WHERE datasource = '%s'", dataSource) | ||
| ); |
Check notice
Code scanning / CodeQL
Missing catch of NumberFormatException
| final int numRows = Integer.parseInt( | ||
| cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) | ||
| ); |
Check notice
Code scanning / CodeQL
Missing catch of NumberFormatException
| final int numSegments = Integer.parseInt( | ||
| cluster.runSql("SELECT COUNT(*) FROM sys.segments WHERE datasource = '%s'", dataSource) | ||
| ); |
Check notice
Code scanning / CodeQL
Missing catch of NumberFormatException
| final int numRows = Integer.parseInt( | ||
| cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) | ||
| ); |
Check notice
Code scanning / CodeQL
Missing catch of NumberFormatException
| final int numSegments = Integer.parseInt( | ||
| cluster.runSql("SELECT COUNT(*) FROM sys.segments WHERE datasource = '%s'", dataSource) | ||
| ); |
Check notice
Code scanning / CodeQL
Missing catch of NumberFormatException
| final int numRows = Integer.parseInt( | ||
| cluster.runSql("SELECT COUNT(*) FROM %s", dataSource) | ||
| ); |
Check notice
Code scanning / CodeQL
Missing catch of NumberFormatException
|
|
||
| SeekableStreamStartSequenceNumbers<KafkaTopicPartition, Long> startSequenceNumbers = | ||
| new SeekableStreamStartSequenceNumbers<>( | ||
| spec.getIoConfig().getStream(), |
Check notice
Code scanning / CodeQL
Deprecated method or constructor invocation
| existingTaskGroup.getMaximumMessageTime(), | ||
| spec.getIoConfig().getInputFormat(), | ||
| spec.getIoConfig().getConfigOverrides(), | ||
| spec.getIoConfig().isMultiTopic(), |
Check notice
Code scanning / CodeQL
Deprecated method or constructor invocation
| spec.getIoConfig().getInputFormat(), | ||
| spec.getIoConfig().getConfigOverrides(), | ||
| spec.getIoConfig().isMultiTopic(), | ||
| spec.getIoConfig().getTaskDuration().getStandardMinutes() |
Check notice
Code scanning / CodeQL
Deprecated method or constructor invocation
| } | ||
|
|
||
| SeekableStreamIndexTaskTuningConfig ss = spec.getSpec().getTuningConfig().convertToTaskTuningConfig(); | ||
| SeekableStreamIndexTaskTuningConfig ss = spec.getSpec().getTuningConfig().convertToTaskTuningConfig(spec.usePerpetuallyRunningTasks()); |
Check notice
Code scanning / CodeQL
Unread local variable
| /** | ||
| * Test implementation of OrderedSequenceNumber for Long values | ||
| */ | ||
| private static class TestSequenceNumber extends OrderedSequenceNumber<Long> |
Check warning
Code scanning / CodeQL
Inconsistent compareTo
| /** | ||
| * Test implementation that throws exceptions on comparison | ||
| */ | ||
| private static class TestExceptionSequenceNumber extends OrderedSequenceNumber<Long> |
Check warning
Code scanning / CodeQL
Inconsistent compareTo
There was a problem hiding this comment.
Thanks for the changes, @uds5501 !
I have taken a pass through the SeekableStreamIndexTaskRunner, left some initial feedback.
Will take a look at the supervisor side changes after this.
Also, as discussed offline, I think we should not be trying to update all the task payloads whenever there is a scaling event.
Instead, we should perhaps do the following:
- Add a new nullable field, say
taskIoConfigBuilder(or some other better name) toSeekableStreamSupervisorSpec - this field should have enough info to create task ioConfigs as needed
- include a String version inside the
taskIoConfigBuilder - this same version should be sent to the tasks in
TaskConfigUpdateRequest - update the supervisor spec whenever there is a scaling event
Persisting the update in the supervisor spec has multiple benefits:
- Supervisor table is already versioned. So we have the history at our disposal.
- We need not update all the task payloads in metadata store every time there is a scaling event.
(There can be a large number of tasks. Also this breaks the current model of immutable Task payloads). - The version provides the supervisor with an easy way to quickly check if a task
is running the same spec/ioConfig or not.
The only drawback is that changes made to the supervisor spec due to scaling
might be difficult to distinguish from changes made by the user.
But we can remedy that with some update message (The audit logs could also be used for this).
Haven't thought this through completely, will try to give it some more thought.
| final int compareToEnd = this.compareTo(end); | ||
| return isEndOffsetExclusive ? compareToEnd < 0 : compareToEnd <= 0; | ||
| } | ||
| catch (Exception e) { |
There was a problem hiding this comment.
What kind of exception can happen here? I don't think we should be catching it.
There was a problem hiding this comment.
Fair enough, we've already handled the end's sequence number to be null, we don't need to capture this exception.
|
|
||
| @Test | ||
| @Timeout(60) | ||
| public void test_ingest20kRows_ofSelfClusterMetricsWithScaleOuts_andVerifyValues() |
There was a problem hiding this comment.
I think it would make more sense to put these test methods in a new test class which is more focused on scaling, task duration etc. This test class KafkaClusterMetricsTest was mostly about ingesting metrics of a cluster.
The new test class can be called something like KafkaTaskScalingTest and it can use data directly published to Kafka topic (similar to KafkaSupervisorTest) rather than self cluster metrics.
There was a problem hiding this comment.
Some typical test cases can be something like:
(all of these cases should be added to a new class KafkaTaskScalingTest)
Case 1:
- No auto-scaling
- Task duration = 0.5s, perpetual = false
- Start the supervisor
- Do not publish any data to Kafka topic
- Verify that tasks finish within 1s (+ completion time might take the total duration to 2 or 3s but definitely < 5s)
- Stop the supervisor
Case 2:
- No auto-scaling
- Task duration = 0.5s, perpetual = true
- Start the supervisor
- Do not publish any data to Kafka topic
- Verify that tasks do not finish even after 10s
- Stop the supervisor
Case 3:
- Auto-scaling enabled
- Task duration = 0.5s, perpetual = true
- Start the supervisor
- Do not publish any data to Kafka topic
- Verify that tasks are scaled down after some time
- Stop the supervisor
Case 4:
- Auto-scaling enabled
- Task duration = 0.5s, perpetual = true
- Start the supervisor
- Publish some data and force the task to enter into some lag that triggers scale up action
(you may usedruid.unsafe.cluster.testingfor this, seeClusterTestingModule) - Verify that tasks are scaled up after some time.
- Verify all kinds of events here - that the tasks were paused and checkpointed, then they waited for config update, and then they were finally resumed
- Don't send any more data
- Verify that the tasks are scaled down
- Verify the events again
- Stop the supervisor
| @Nullable | ||
| private final InputRowParser<ByteBuffer> parser; | ||
| private final String stream; | ||
| private String stream; |
There was a problem hiding this comment.
The input source stream never changes for a supervisor. See SeekableStreamSupervisorSpec.validateSpecUpdateTo().
| private volatile DateTime minMessageTime; | ||
| private volatile DateTime maxMessageTime; | ||
| private final ScheduledExecutorService rejectionPeriodUpdaterExec; | ||
| private AtomicBoolean waitForConfigUpdate = new AtomicBoolean(false); |
There was a problem hiding this comment.
| private AtomicBoolean waitForConfigUpdate = new AtomicBoolean(false); | |
| private final AtomicBoolean waitForConfigUpdate = new AtomicBoolean(false); |
| //milliseconds waited for created segments to be handed off | ||
| long handoffWaitMs = 0L; | ||
|
|
||
| log.info("Task perpetually running: %s", task.isPerpetuallyRunning()); |
There was a problem hiding this comment.
Move this log line to runInternal.
There was a problem hiding this comment.
It's already in runInternal()
There was a problem hiding this comment.
👍🏻
Move this log line to the start of this method.
| log.info("Task perpetually running: %s", task.isPerpetuallyRunning()); | |
| log.info("Running task[%s] in persisted[%s] mode.", task.getId(), task.isPerpetuallyRunning()); |
| return Response.status(409).entity("Task must be paused for checkpoint completion before updating config").build(); | ||
| } | ||
| try { | ||
| log.info("Attempting to update config to [%s]", request.getIoConfig()); |
There was a problem hiding this comment.
Please move the logic inside the try into a separate private method, and add a short javadoc to it outlining the steps involved.
| log.info("Attempting to update config to [%s]", request.getIoConfig()); | ||
|
|
||
| SeekableStreamIndexTaskIOConfig<PartitionIdType, SequenceOffsetType> newIoConfig = (SeekableStreamIndexTaskIOConfig<PartitionIdType, SequenceOffsetType>) | ||
| toolbox.getJsonMapper().convertValue(request.getIoConfig(), SeekableStreamIndexTaskIOConfig.class); |
There was a problem hiding this comment.
Why convert? Isn't the payload already a SeekableStreamTaskIOConfig?
There was a problem hiding this comment.
My bad, I thought it required conversion to understand the generics, going to remove the conversion.
| */ | ||
| public class TaskConfigUpdateRequest | ||
| { | ||
| private final SeekableStreamIndexTaskIOConfig ioConfig; |
There was a problem hiding this comment.
Should this field and the class have generic args for partition id type and offset type?
There was a problem hiding this comment.
I remember some discussion earlier where if we set the partition id types etc here, the supervisor may change the partition id type and that will be invalid later, so decided to strip generics from here.
There was a problem hiding this comment.
No, let's retain the generics. It will work as expected.
| seekToStartingSequence(recordSupplier, assignment); | ||
| } else { | ||
| // if there is no assignment, It means that there was no partition assigned to this task after scaling down. | ||
| pause(); |
There was a problem hiding this comment.
Wouldn't the task already be in a paused state?
There was a problem hiding this comment.
It would be, fair enough
| createNewSequenceFromIoConfig(newIoConfig); | ||
|
|
||
| assignment = assignPartitions(recordSupplier); | ||
| boolean shouldResume = true; |
There was a problem hiding this comment.
Rather than this boolean, just call resume() inside the if.
| spec.getSpec().getIOConfig().getConfigOverrides(), | ||
| spec.getSpec().getIOConfig().isMultiTopic() |
There was a problem hiding this comment.
I don't think this change is needed. spec.getIoConfig() already does the right thing.
There was a problem hiding this comment.
It's deprecated and IDE + github seems to be complaining about that.
There was a problem hiding this comment.
Okay. Although, let's postpone it for a later PR if possible.
| if (spec.usePerpetuallyRunningTasks()) { | ||
| int taskGroupId = getRangeBasedTaskGroupId(partitionId, taskCount); | ||
| log.debug("Range-based assignment for partition [%s]: taskGroupId [%d] when taskCount is [%d]", partitionId, taskGroupId, taskCount); | ||
| return taskGroupId; | ||
| } else { | ||
| if (partitionId.isMultiTopicPartition()) { | ||
| return Math.abs(31 * partitionId.topic().hashCode() + partitionId.partition()) % taskCount; | ||
| } else { | ||
| return partitionId.partition() % taskCount; | ||
| } | ||
| } |
There was a problem hiding this comment.
Please simplify this if:
| if (spec.usePerpetuallyRunningTasks()) { | |
| int taskGroupId = getRangeBasedTaskGroupId(partitionId, taskCount); | |
| log.debug("Range-based assignment for partition [%s]: taskGroupId [%d] when taskCount is [%d]", partitionId, taskGroupId, taskCount); | |
| return taskGroupId; | |
| } else { | |
| if (partitionId.isMultiTopicPartition()) { | |
| return Math.abs(31 * partitionId.topic().hashCode() + partitionId.partition()) % taskCount; | |
| } else { | |
| return partitionId.partition() % taskCount; | |
| } | |
| } | |
| if (partitionId.isMultiTopicPartition()) { | |
| return Math.abs(31 * partitionId.topic().hashCode() + partitionId.partition()) % taskCount; | |
| } else if (spec.usePerpetuallyRunningTasks()) { | |
| return getRangeBasedTaskGroupId(partitionId, taskCount); | |
| } else { | |
| return partitionId.partition() % taskCount; | |
| } |
There was a problem hiding this comment.
removed this altogether.
| { | ||
| int minPartitionsPerTaskGroup = totalPartitions / taskCount; | ||
|
|
||
| if (partitionId.isMultiTopicPartition()) { |
There was a problem hiding this comment.
This method shouldn't need to handle multi topic stuff right now.
| return suspended; | ||
| } | ||
|
|
||
| public Optional<String> getVersion() |
Check notice
Code scanning / CodeQL
Missing Override annotation
| } | ||
| } | ||
|
|
||
| private Void updateEntryWithHandle( |
There was a problem hiding this comment.
nit: perhaps add some javadoc heading like insertEntryWithHandle.
| @JsonCreator | ||
| public TaskConfigUpdateRequest( | ||
| @JsonProperty("ioConfig") @Nullable SeekableStreamIndexTaskIOConfig<PartitionIdType, SequenceOffsetType> ioConfig, | ||
| @JsonProperty("supervisorSpecVersion") String supervisorSpecVersion |
There was a problem hiding this comment.
nit: can we assert this is non-null like we do for other task actions' required params?
| ) | ||
| { | ||
| this.ioConfig = ioConfig; | ||
| this.supervisorSpecVersion = supervisorSpecVersion; |
| } | ||
| if (isDynamicAllocationOngoing.get()) { | ||
| checkpointsToWaitFor -= setEndOffsetFutures.size(); | ||
| if (checkpointsToWaitFor <= 0) { |
| checkpointsToWaitFor -= setEndOffsetFutures.size(); | ||
| if (checkpointsToWaitFor <= 0) { | ||
| log.info("All tasks in current task groups have been checkpointed, resuming dynamic allocation"); | ||
| pendingConfigUpdateHook.call(); |
There was a problem hiding this comment.
Is pendingConfigUpdateHook written/read across threads? Might want to make this volatile or an atomic ref
| // For end sequences, use NOT_SET to indicate open-ended reading | ||
| Map<KafkaTopicPartition, Long> endingSequences = new HashMap<>(); | ||
| for (KafkaTopicPartition partition : partitions) { | ||
| endingSequences.put(partition, END_OF_PARTITION); |
There was a problem hiding this comment.
END_OF_PARTITION or NOT_SET? Comment is a bit misleading
| log.info("Task [%s] paused successfully & Checkpoint requested successffully", id); | ||
| return deserializeOffsetsMap(r.getContent()); | ||
| } else if (r.getStatus().equals(HttpResponseStatus.ACCEPTED)) { | ||
| return null; |
There was a problem hiding this comment.
Do we expect to see this kind of response?
| //milliseconds waited for created segments to be handed off | ||
| long handoffWaitMs = 0L; | ||
|
|
||
| log.info("Task perpetually running: %s", task.isPerpetuallyRunning()); |
There was a problem hiding this comment.
nit: add task ID to this log (unless thread ID is there).
| return Response.ok().entity("Task is already paused for checkpoint completion").build(); | ||
| } | ||
| Response pauseResponse = pause(); | ||
| if (pauseResponse.getStatus() == 409) { |
There was a problem hiding this comment.
nit:
.getStatus().equals(HttpResponseStatus.CONFLICT)| Response pauseResponse = pause(); | ||
| if (pauseResponse.getStatus() == 409) { | ||
| waitForConfigUpdate.set(false); | ||
| return pauseResponse; |
There was a problem hiding this comment.
possibly naive: do we need to resume() here?
| } | ||
|
|
||
| @Override | ||
| public void update(String id, Task entry) |
| } | ||
|
|
||
| @Test | ||
| public void testUpdateTask() |
There was a problem hiding this comment.
I don't think this is needed anymore.
| // The default implementation does not do any validation checks. | ||
| } | ||
|
|
||
| default Optional<String> getVersion() |
There was a problem hiding this comment.
We shouldn't put version into the SupervisorSpec since there is already a VersionedSupervisorSpec.
You can maintain the version as a separate variable in the supervisor or just use a VersionedSupervisorSpec.
| this.interval = interval; | ||
| } | ||
|
|
||
| private TestTask(String id, String dataSource, Interval interval, Map<String, Object> context) |
| TestSequenceNumber current = new TestSequenceNumber(null, false); | ||
| TestSequenceNumber end = new TestSequenceNumber(10L, false); | ||
|
|
||
| Assert.assertThrows(NullPointerException.class, () -> current.isMoreToReadBeforeReadingRecord(end, false)); |
There was a problem hiding this comment.
Please put args in separate lines.
| private boolean changeTaskCountForPerpetualTasks(int desiredActiveTaskCount, | ||
| Runnable successfulScaleAutoScalerCallback | ||
| ) |
| private boolean changeTaskCountForPerpetualTasks(int desiredActiveTaskCount, | ||
| Runnable successfulScaleAutoScalerCallback | ||
| ) |
| public void runInternal() | ||
| { | ||
| if (isDynamicAllocationOngoing.get()) { | ||
| log.info("Skipping run because dynamic allocation is ongoing."); |
There was a problem hiding this comment.
This can get noisy as task scaling can be a slow operation.
| .get() | ||
| .withDataSchema(schema -> schema.withTimestamp(new TimestampSpec("timestamp", "iso", null))) | ||
| .withTuningConfig(tuningConfig -> tuningConfig.withMaxRowsPerSegment(maxRowsPerSegment)) | ||
| .withTuningConfig(tuningConfig -> tuningConfig |
There was a problem hiding this comment.
Please revert all the changes to this file.
| * Embedded test to verify task scaling behaviour of {@code KafkaSupervisor} ingesting from a custom kafka topic. | ||
| */ | ||
| @SuppressWarnings("resource") | ||
| public class KafkaTaskScalingTest extends EmbeddedClusterTestBase |
There was a problem hiding this comment.
Why is this class separate from KafkaTaskAutoScalingTest. We can merge the two classes into one.
| .build(); | ||
| } else { | ||
| try { | ||
| // Don't acquire a lock if the task is already paused for checkpoint completion, avoiding deadlock |
There was a problem hiding this comment.
This comment is not valid anymore
| public Response setEndOffsets( | ||
| Map<PartitionIdType, SequenceOffsetType> sequenceNumbers, | ||
| boolean finish // this field is only for internal purposes, shouldn't be usually set by users | ||
| boolean finish, // this field is only for internal purposes, shouldn't be usually set by users |
There was a problem hiding this comment.
This old comment does not add any value. Please remove it.
| boolean finish, // this field is only for internal purposes, shouldn't be usually set by users | |
| boolean finish, |
| @JsonCreator | ||
| public TaskConfigResponse( | ||
| @JsonProperty("ioConfig") @Nullable SeekableStreamIndexTaskIOConfig<PartitionIdType, SequenceOffsetType> ioConfig, | ||
| @JsonProperty("supervisorSpecVersion") String supervisorSpecVersion |
There was a problem hiding this comment.
We might as well just call it supervisorVersion. The version always denotes the spec anyway.
Please make the same change in all the relevant places.
Proposed changes
As part of this PR, I aim to introduce perpetually running tasks for seekable ingestion (tasks that should never shut down). To enable this feature, users need to specify
usePerpetuallyRunningTasks:truein the supervisor spec.Once users have enabled this flag, they can expect the following changes in
SupervisorandSeekableStreamIndexTaskRunnerbehaviours:Task rollovers
pendingTaskGroupand eventually shut down.ioConfig.taskDurationis ignored).Auto Scaling
To felicitate the coordination of this flow, there will be a flag each on the supervisor and task runner end.
isDynamicAllocationOngoingflag ensures that another scale event is not accepted until the existing one is finished and it determines whether the checkpoint action needs to call the update config to the other tasks.waitForConfigUpdateis set true whenpauseAndCheckpointis triggered that can only be unset on/configUpdate.Checkpoint mechanism
/offsets/endused to resume the tasks with new sequence added in the task runner./offsets/endwill update the end of the latest sequence but not add a new sequence in thesequenceslist if the task runner is waiting for a config update. The task resume responsibility will now land on/configUpdateAPI in the event of auto scaling.Druid cluster upgrades
Partition <> Task Mapping assignment
Misc.
This will cover bits that are necessary to ensure that the supervisor don't end up killing tasks that are valid / other race conditions
Supervisor
TaskQueue.update()so that in the nextrunInternal()loop, supervisor knows about the change of ioConfig that took place during task discovery.activelyRuningTaskGroups.isCheckpointSignatureValid()has been introduced to returntruefor now , earlier the check verified that partitions and offsets are the same in the checkpoints however, with the possible change of partition mapping in tasks, this no longer holds true.Task Runner
What the overall changes look like:
/pauseAndCheckpointAPI./offsets/endPOST API ./configsAPI./updateConfigwith updated partitions is called across task groups. (this needs to be guarded behind a toggle).