Instead of waiting until Tasks are assigned to us, we pre-emptively
create a StandbyTask for each non-empty Task directory found on-disk.
We do this before starting any StreamThreads, and on our first
assignment (after joining the consumer group), we recycle any of these
StandbyTasks that were assigned to us, either as an Active or a
Standby.
We can't just use these "initial Standbys" as-is, because they were
constructed outside the context of a StreamThread, so we first have to
update them with the context (log context, ChangelogReader, and source
topics) of the thread that it has been assigned to.
The motivation for this is to (in a later commit) read StateStore
offsets for unowned Tasks from the StateStore itself, rather than the
.checkpoint file, which we plan to deprecate and remove.
There are a few additional benefits:
Initializing these Tasks on start-up, instead of on-assignment, will
reduce the time between a member joining the consumer group and beginning
processing. This is especially important when active tasks are being moved
over, for example, as part of a rolling restart.
If a Task has corrupt data on-disk, it will be discovered on startup and
wiped under EOS. This is preferable to wiping the state after being
assigned the Task, because another instance may have non-corrupt data and
would not need to restore (as much).
There is a potential performance impact: we open all on-disk Task
StateStores, and keep them all open until we have our first assignment.
This could require large amounts of memory, in particular when there are
a large number of local state stores on-disk.
However, since old local state for Tasks we don't own is automatically
cleaned up after a period of time, in practice, we will almost always
only be dealing with the state that was last assigned to the local
instance.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Bruno Cadonna <cadonna@apache.org>, Matthias Sax <mjsax@apache.org>
This PR augments Streams messages with leader epoch. In case of empty buffer queues, the last offset and leader epoch are retrieved from the streams task 's cache of nextOffsets.
Co-authored-by: Lucas Brutschy <lbrutschy@confluent.io>
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Matthias J. Sax <matthias@confluent.io>
This PR implements KIP-1094.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Kirk True <ktrue@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
With EOSv1 removed, we don't need to collect producer's clientInstanceIds per task any longer. While we never completed this feature, we can remove the corresponding scaffolding code.
Reviewers: Bill Bejeck <bill@confluent.io>
This PR leverages the updates brought by KIP-1033 to get the name of the processor node which raised a processing exception and display it in the stacktrace instead of the source node.
Reviewer: Bruno Cadonna <cadonna@apache.org>
Integration tests should run with either at-least-once or exactly-once.
There is no need to run them twice.
This PR removes the corresponding test parameters and picks either one of both.
Reviewers: Matthias J. Sax <matthias@confluent.io>
Deprecate for 4.0. This feature will be phased out, with new functionality generally not supporting named topologies, so users are encouraged to migrate off of named topologies as soon as possible.
Though who are using this experimental feature are encouraged to reach out by filing a jira ticket so we can better understand your use case and how to support it going forward.
Reviewers: Matthias Sax <mjsax@apache.org>
This PR disables metrics push for the AdminClient as the default. KafkaStreams enables metrics push for its internal AdminClient.
Tests are included that assert an error if a user disables either the main consumer or admin client metrics push but Kafka Streams metrics push config is enabled.
Reviewers: Matthias Sax <mjsax@apache.org>
Unit test shouldRestoreSingleActiveStatefulTask() in DefaultStateUpdaterTest is flaky.
The flakiness comes from the fact that the state updater thread could call the first time changelogReader.allChangelogsCompleted() before it calls the first time changelogReader.completedChangelogs(). That happens, if runOnce() is run before the state updater thread reads a task from the input queue.
This commit fixes the flakiness, by making stubbing changelogReader.allChangelogsCompleted() depend on stubbing changelogReader.completedChangelogs().
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Matthias J. Sax <matthias@confluent.io>
With EOSv1 removal, we don't have producer-per-task any longer,
and thus can remove the corresponding code which handles task producers.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Bill Bejeck <bill@confluent.io>
We don't pass in a client-supplier into `StreamsProducer` any longer,
so we can simplify `StreamsProducerTest` and remove client-supplier.
Reviewers: Bill Bejeck <bill@confluent.io>
This PR adds a Reporter instance that will add streams client metrics to the telemetry pipeline.
For testing, the PR adds a unit test.
Reviewers: Matthias Sax <mjsax@apache.org>
This PR adds a Reporter instance that will add streams thread metrics to the telemetry pipeline.
For testing, the PR adds a unit test.
Reviewers: Matthias Sax <mjsax@apache.org>
This patch completely removes the compile-time dependency on core for both test and main sources by introducing two new modules.
1) `test-common` include all the common test implementation code (including dependency on :core for BrokerServer, ControllerServer, etc)
2) `test-common:api` new sub-module that just includes interfaces including our junit extension
Reviewers: David Arthur <mumrah@gmail.com>
When calling KafkaStreams#close from teardown methods in integration tests, we need to pass timeout to avoid potentially blocking forever during teardown.
Reviewers: Matthias J. Sax <matthias@confluent.io>
This PR implements exponential backoff for failed initializations of tasks due to lock exceptions. It increases the time between two consecutive attempts of initializing the tasks.
Reviewer: Bruno Cadonna <cadonna@apache.org>
Implements KIP-1087
Reviewers: Matthias J. Sax <matthias@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
With EOSv1 removed, we don't need to create a producer per task, and thus can simplify the code by removing KafkaClientSupplier from the deeply nested StreamsProducer, to simplify the code.
Reviewers: Bill Bejeck <bill@confluent.io>
The existing check is not correct, because `byte` range is from -128...127.
This PR fixes the check to use `< 0`.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Found two issues in the flaky tests: (Put the log analysis under Jira comments.)
1) The error "java.nio.file.DirectoryNotEmptyException" occurs if the flush() of kafkaStreams.close() and purgeLocalStreamsState() are triggered in the same time. (The current timeout is 5 sec, which is too short since the CI is unstable and slow).
2) Racing issue: Task to-be restored in ks-1 are rebalanced to ks-2 before entering active restoring state. So no onRestoreSuspend() was triggered.
To solve the issues:
1) Remove the timeout in kafkaStreams.close()
2) Ensure all tasks in ks-1 are active restoring before start second KafkaStreams(ks-2)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Failed tasks discovered when removed from the state updater during assignment or revocation are added to the task registry. From there they are retrieved and handled as normal tasks. This leads to a couple of IllegalStateExceptions because it breaks some invariants that ensure that only good tasks are assigned and processed.
This commit solves this bug by distinguish failed from non-failed tasks in the task registry.
Reviewer: Lucas Brutschy <lbrutschy@confluent.io>
This PR implements exponential backoff for state directory lock to increase the time between two consecutive attempts of acquiring the lock.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
Implements KIP-1056:
- deprecates default.deserialization.exception.handler in favor of deserialization.exception.handler
- deprecates default.production.exception.handler in favor of production.exception.handler
Reviewers: Matthias J. Sax <matthias@confluent.io>
As described in KAFKA-14460, one of the functional requirements of KeyValueStore is that "The returned iterator must not return null values" on methods which return iterator.
This is not completely the case today for InMemoryKeyValueStore. To iterate over the store, we copy the keySet in order not to block access for other threads. However, entries that are removed from the store after initializing the iterator will be returned with null values by the iterator.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
This test has a tricky race condition. We want the restoration to go slow enough so that when a second Kafka Streams instance starts, the restoration of a given TopicPartition pauses due to task re-assignment. But after that point, we'd like the test to proceed faster to avoid any timeout assertions. To that end, here are the changes in this PR:
Increase the restore pause to 2 seconds; this should slow the restoration enough so that the process is still in progress once the second instance starts. But once tasks are re-assigned and onRestorePause is called, the restore pause is decremented to zero, allowing the test to proceed faster.
Increase the restore batch to its original value of 5 - otherwise, the test moved too slowly.
Decrease the number of test records produced to the original value of 100. By increasing the time of restoring each batch until Kafka Streams calls onRestorePause and removing the intentional restoration slowness, 100 records proved good enough in local testing.
Reviewers: Matthias J. Sax <mjsax@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>,
Yu-LinChen <kh87313@gmail.com>
Part of KIP-1033.
Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
Reviewers: Matthias J. Sax <matthias@confluent.io>
Increased the number of records while decreasing the restore batch size to ensure the restoration does not complete before the second Kafka Streams instance starts up.
Reviewers: Matthias J. Sax <mjsax@apache.org>
KAFKA-17100 changed the behavior of GlobalStreamThread introducing a race condition for state changes, that was exposed by failing (flaky) tests in GlobalStreamThreadTest.
This PR moves the state transition to fix the race condition.
Reviewers: Bill Bejeck <bbejeck@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
We currently use a `CountDownLatch` to signal when a thread has
completed shutdown to the blocking `shutdown` method. However, this
latch triggers _before_ the thread has fully exited.
Dependent on the OS thread scheduling, it's possible that this thread
will still be "alive" after the latch has unblocked the `shutdown`
method.
In practice, this is mostly a problem for `StreamThreadTest`, which now
checks that there are no `TaskExecutor` or `StateUpdater` threads
immediately after shutting them down.
Sometimes, after shutdown returns, we find that these threads are still
"alive", usually completing execution of the "thread shutdown" log
message, or even the `Thread#exit` JVM method that's invoked to clean up
threads just before they exit. This causes sporadic test failures, even
though these threads did indeed shutdown correctly.
Instead of using a `CountDownLatch`, let's just await the thread to exit
directly, using `Thread#join`. Just as before, we set a timeout, and if
the Thread is still alive after the timeout, we throw a
`StreamsException`, maintaining the contract of the `shutdown` method.
There should be no measurable impact on production code here. This will
mostly just improve the reliability of tests that require these threads
have fully exited after calling `shutdown`.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>, Bruno Cadonna <cadonna@apache.org>
Streams is not compatible with the new consumer rebalance protocol proposed in KIP-848. Thus, Streams should set/override config group.protocol to classic at startup to ensure that the classic protocol is used.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
testStateGlobalThreadClose() does fail sometimes, with unclear root
cause. This PR is an attempt to fix it, by cleaning up and improving the
test code across the board.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
The method branch in both Java and Scala KStream class was deprecated in version 2.8:
1) org.apache.kafka.streams.scala.kstream.KStream#branch
2) org.apache.kafka.streams.kstream.KStream#branch(org.apache.kafka.streams.kstream.Predicate<? super K,? super V>...)
3) org.apache.kafka.streams.kstream.KStream#branch(org.apache.kafka.streams.kstream.Named, org.apache.kafka.streams.kstream.Predicate<? super K,? super V>...)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
- StreamsConfig#RETRIES_CONFIG was deprecated in AK 2.7 and is no longer in use.
- StreamsConfig#DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS and
- StreamsConfig#DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS were deprecated in AK 3.0.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Follow up code cleanup for KIP-1033.
This PR unifies the handling of both error cases for exception handlers:
- handler throws an exception
- handler returns null
The unification happens for all 5 handler cases:
- deserialzation
- production / serialization
- production / send
- processing
- punctuation
Reviewers: Sebastien Viale <sebastien.viale@michelin.com>, Loic Greffier <loic.greffier@michelin.com>, Bill Bejeck <bill@confluent.io>
This PR is part of KIP-1033 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing.
This PR actually catches processing exceptions from punctuate.
Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
Reviewers: Bruno Cadonna <bruno@confluent.io>, Matthias J. Sax <matthias@confluent.io>
This PR is part of KIP-1033 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing.
This PR catch the exceptions thrown while handling a processing exception
Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
Reviewers: Bruno Cadonna <bruno@confluent.io>, Matthias J. Sax <matthias@confluent.io>
As discussed in #16657 (comment) , we should make logger as static to avoid creating multiple logger instances.
I use the regex private.*Logger.*LoggerFactory to search and check all the results if certain logs need to be static.
There are some exceptions that loggers don't need to be static:
1) The logger in the inner class. Since java8 doesn't support static field in the inner class.
https://github.com/apache/kafka/blob/trunk/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java#L3676
2) Custom loggers for each instance (non-static + non-final). In this case, multiple logger instances is actually really needed.
https://github.com/apache/kafka/blob/trunk/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorage.java#L166
3) The logger is initialized in constructor by LogContext. Many non-static but with final modifier loggers are in this category, that's why I use .*LoggerFactory to only check the loggers that are assigned initial value when declaration.
4) protected final Logger log = Logger.getLogger(getClass())
This is for subclass can do logging with subclass name instead of superclass name.
But in this case, if the log access modifier is private, the purpose cannot be achieved since subclass cannot access the log defined in superclass. So if access modifier is private, we can replace getClass() with <className>.class
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This PR reduces the MAX_BLOCK_MS config which defaults to 60sec to
10sec, to avoid a race condition with the 60sec test timeout.
Reviewers: Bill Bejeck <bill@confluent.io>
This PR is part of KIP1033 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing.
This PR expose the new ErrorHandlerContext as a parameter to the Deserialization exception handlers and deprecate the previous handle signature.
Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
Reviewers: Bruno Cadonna <bruno@confluent.io>, Matthias J. Sax <matthias@confluent.io>
This PR is part of KIP-1033 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing.
This PR expose the new ErrorHandlerContext as a parameter to the Production exception handler and deprecate the previous handle signature.
Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
Reviewers: Bruno Cadonna <bruno@confluent.io>, Matthias J. Sax <matthias@confluent.io>
To avoid a resource leak, we need to close the AdminClient after the test.
Reviewers: Lianet Magrans <lianetmr@gmail.com>, Matthias J. Sax <matthias@confluent.io>
Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
Minor code improvements across different classed, related to the `ProcessingExceptionHandler` implementation (KIP-1033).
Reviewers: Bruno Cadonna <bruno@confluent.io>, Matthias J. Sax <matthias@confluent.io>
This PR is part of KAFKA-16448 (KIP-1033) which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing.
This PR actually catches processing exceptions.
Authors:
@Dabz
@sebastienviale
@loicgreffier
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Bruno Cadonna <cadonna@apache.org>
When Streams tries to transit a restored active task to RUNNING, the
first thing it does is getting the committed offsets for this task.
If getting the offsets expires a timeout, Streams does not re-throw
the error initially, but tries to get the committed offsets later
until a Streams-specific timeout is hit.
Restored active tasks from the state updater are removed from the
output queue of the restored tasks in the state updater. If a
timeout occurs, the restored task is neither added to the
task registry nor re-added to the state updater. The task is lost
since it is not maintained anywhere. This means the task is also
not closed. When the same task is created again on the same
stream thread since the stream thread does not know about this
lost task, the state stores are opened again and RocksDB will
throw the "No locks available" error.
This commit re-adds the task to the state updater if the
committed request times out.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
When a active tasks are revoked they land as suspended tasks in the
task registry. If they are then reassigned, the tasks are resumed
and put into restoration. On assignment, we first handle the tasks
in the task registry and then the tasks in the state updater. That
means that if a task is re-assigned after a revocation, we remove
the suspended task from the task registry, resume it, add it
to the state updater, and then remove it from the list of tasks
to create. After that we iterate over the tasks in the state
updater and remove from there the tasks that are not in the list
of tasks to create. However, now the state updater contains the
resumed tasks that we removed from the task registry before but
are no more in the list of tasks to create. In other words, we
remove the resumed tasks from the state updater and close them
although we just got them assigned.
This commit ensures that we first handle the tasks in the
state updater and then the tasks in the task registry.
Cherry-pick of 4ecbb75
Reviewer: Lucas Brutschy <lbrutschy@confluent.io>
With enabled state updater tasks that are created but not initialized
are stored in a set. In each poll iteration the stream thread drains
that set, intializes the tasks, and adds them to the state updater.
On partition lost, all active tasks are closed.
This commit ensures that active tasks pending initialization in
the set mentioned above are closed cleanly on partition lost.
Reviewer: Lucas Brutschy <lbrutschy@confluent.io>
When the PurgeRepartitionTopicintegrationTest was written, the InitialTaskDelayMs was hard-coded on the broker requiring setting a timeout in the test to wait for the delay to expire. But I believe this creates a race condition where the test times out before the broker deletes the inactive segment. PR #15719 introduced an internal config to control the IntitialTaskDelayMs config for speeding up tests, and this PR leverages this internal config to reduce the task delay to 0 to eliminate this race condition.
For a non-existing output topic, Kafka Streams ends up in an infinite retry loop, because the returned TimeoutException extends RetriableException.
This PR updates the error handling pass for this case and instead of retrying calls the ProductionExceptionHandler to allow breaking the infinite retry loop.
Reviewers: Matthias J. Sax <matthias@confluent.io>
Foreign key joins have an additional "internal" state store used for subscriptions, which is not exposed for configuration via Materialized or StoreBuilder which means there is no way to plug in a different store implementation via the DSL operator. However, we should respect the configured default dsl store supplier if one is configured, to allow these stores to be customized and conform to the store type selection logic used for other DSL operator stores
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
Each KafkaStreams instance maintains a map from threadId to state
to use to aggregate to a KafkaStreams app state. The map is updated
on every state change, and when a new thread is created. State change
updates are done in a synchronized blocks, however the update that
happens on thread creation is not, which can raise
ConcurrentModificationException. This patch moves this update
into the listener object and protects it using the object's lock.
It also moves ownership of the state map into the listener so that
its less likely that future changes access it without locking
Reviewers: Matthias J. Sax <matthias@confluent.io>
The new default standby task assignment in TaskAssignment should only assign standby tasks for changelogged tasks, not all stateful tasks.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
KStreamKStreamJoin was recently refactored to be type safe with regard to left and right value-types. This PR updates the corresponding test to use different value types instead of the same one to improve test coverage.
Reviewers: Bruno Cadonna <bruno@confluent.io>, Matthias J. Sax <matthias@confluent.io>
To avoid confusion in 3.8/until we fully remove all the old task assignors and internal config, we should rename the old internal assignor classes like the StickyTaskAssignor so that they won't be mixed up with the new version of the assignor (which is also named StickyTaskAssignor)
Reviewers: Bruno Cadonna <cadonna@apache.org>, Josep Prat <josep.prat@aiven.io>
Since the new public API for TaskAssignor shared a name, this rename will prevent users from confusing the internal definition with the public one.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
Also moved the assignment validation test from StreamsPartitionAssignorTest to TaskAssignmentUtilsTest.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
We have already enabled the state updater by default once.
However, we ran into issues that forced us to disable it again.
We think that we fixed those issues. So we want to enable the
state updater again by default.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Matthias J. Sax <matthias@confluent.io>
We now provide a way to more easily customize the rack aware
optimizations that we provide by way of a configuration class called
RackAwareOptimizationParams.
We also simplified the APIs for the optimizeXYZ utility functions since
they were mutating the inputs anyway.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
This PR is part of KAFKA-16448 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing.
This PR brings ProcessingExceptionHandler in Streams configuration.
Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: sebastienviale <sebastien.viale@michelin.com>
Reviewer: Bruno Cadonna <cadonna@apache.org>
The StreamsConfig class was not parsing the new task assignment
configuration flag properly, which made it impossible to properly
configure a custom task assignor.
This PR fixes this and adds a bit of INFO logging to help users diagnose
assignor misconfiguration issues.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
Added more testing for the StickyTaskAssignor, which includes a large-scale test with rack aware enabled.
Also added a no-op change to StreamsAssignmentScaleTest.java to allow for rack aware optimization testing.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
This PR updates all of the streams task assignment code to use the new AssignmentConfigs public class.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
A task does not know anything about a produce error thrown
by a different task. That might lead to a InvalidTxnStateException
when a task attempts to do a transactional operation on a producer
that failed due to a different task.
This commit stores the produce exception in the streams producer
on completion of a send instead of the record collector since the
record collector is on task level whereas the stream producer
is on stream thread level. Since all tasks use the same streams
producer the error should be correctly propagated across tasks
of the same stream thread.
For EOS alpha, this commit does not change anything because
each task uses its own producer. The send error is still
on task level but so is also the transaction.
Reviewers: Matthias J. Sax <matthias@confluent.io>
This PR is part of KAFKA-16448 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing.
This PR brings ProcessingExceptionHandler interface and default implementations.
Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: sebastienviale <sebastien.viale@michelin.com>
Reviewer: Bruno Cadonna <cadonna@apache.org>
Fixed the calculation of the store name list based on the subtopology being accessed.
Also added a new test to make sure this new functionality works as intended.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
This PR takes care of making the call back toTaskAssignor.onAssignmentComputed.
It also contains a change to the public AssignmentConfigs API, as well as some simplifications of the StickyTaskAssignor.
This PR also changes the rack information fetching to happen lazily in the case where the TaskAssignor makes its decisions without said rack information.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
This fills in the implementation details of the standby task assignment utility functions within TaskAssignmentUtils.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
This PR is part of KAFKA-16448 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing.
This PR brings ProcessingExceptionHandler in Streams configuration.
Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: sebastienviale <sebastien.viale@michelin.com>
Reviewer: Bruno Cadonna <cadonna@apache.org>
This PR implements the rack aware optimization of active tasks that can be used by the assignors themselves. It takes in the full output of the assignment and tries to reorganize it so as to minimize cross-rack traffic.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
This PR adds the logic and wiring necessary to make the callback to
TaskAssignor::onAssignmentComputed with the necessary parameters.
We also fixed some log statements in the actual assignment error
computation, as well as modified the ApplicationState::allTasks method
to return a Map instead of a Set of TaskInfos.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
This PR is part of KAFKA-16448 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing.
This PR brings ProcessingExceptionHandler interface and default implementations.
Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: sebastienviale <sebastien.viale@michelin.com>
Reviewer: Bruno Cadonna <cadonna@apache.org>
The introduced changes provide a cleaner definition of the join side in KStreamKStreamJoin. Before, this was done by using a Boolean flag, which led to returning a raw LeftOrRightValue without generic arguments because the generic type arguments depended on the boolean input.
Reviewers: Greg Harris <greg.harris@aiven.io>, Bruno Cadonna <cadonna@apache.org>
This PR adds the post-processing of the TaskAssignment to figure out if the new assignment is valid, and return an AssignmentError otherwise.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
This PR creates the new public config of KIP-924 in StreamsConfig and uses it to instantiate user-created TaskAssignors. If such a TaskAssignor is found and successfully created we then use that assignor to perform the task assignment, otherwise we revert back to the pre KIP-924 world with the internal task assignors.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Almog Gavra <almog@responsive.dev>
Part of [KIP-989](https://cwiki.apache.org/confluence/x/9KCzDw).
This new `StateStore` metric tracks the timestamp that the oldest
surviving Iterator was created.
This timestamp should continue to climb, and closely track the current
time, as old iterators are closed and new ones created. If the timestamp
remains very low (i.e. old), that suggests an Iterator has leaked, which
should enable users to isolate the affected store.
It will report no data when there are no currently open Iterators.
Reviewers: Matthias J. Sax <matthias@confluent.io>
`LongAdder` performs better than `AtomicInteger` when under contention
from many threads. Since it's possible that many Interactive Query
threads could create a large number of `KeyValueIterator`s, we don't
want contention on a metric to be a performance bottleneck.
The trade-off is memory, as `LongAdder` uses more memory to space out
independent counters across different cache lines. In practice, I don't
expect this to cause too many problems, as we're only constructing 1
per-store.
Reviewers: Matthias J. Sax <matthias@confluent.io>
This PR uses the new TaskTopicPartition structure to simplify the build
process for the ApplicationState, which is the input to the new
TaskAssignor#assign call.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
Part of [KIP-989](https://cwiki.apache.org/confluence/x/9KCzDw).
This new `StateStore` metric tracks the average and maximum amount of
time between creating and closing Iterators.
Iterators with very high durations can indicate to users performance
problems that should be addressed.
If a store reports no data for these metrics, despite the user opening
Iterators on the store, it suggests those iterators are not being
closed, and have therefore leaked.
Reviewers: Matthias J. Sax <matthias@confluent.io>
For task assignment purposes, the user needs to have a set of information available for each topic partition affecting the desired tasks.
This PR introduces a new interface for a read-only container class that allows all the important and relevant information to be found in one place.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
Rack aware graphs don't actually need any topology information about the system, but rather require a simple ordered (not sorted) grouping of tasks.
This PR changes the internal constructors and some interface signatures of RackAwareGraphConstructor and its implementations to allow reuse by future components that may not have access to the actual subtopology information.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
This rack information is required to compute rack-aware assignments, which many of the current assigners do.
The internal ClientMetadata class was also edited to pass around this rack information.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
Part of [KIP-989](https://cwiki.apache.org/confluence/x/9KCzDw).
This new `StateStore` metric tracks the number of `Iterator` instances
that have been created, but not yet closed (via `AutoCloseable#close()`).
This will aid users in detecting leaked iterators, which can cause major
performance problems.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Matthias J. Sax <matthias@confluent.io>
This PR creates the required methods to post-process the result of TaskAssignor.assign into the required ClientMetadata map. This allows most of the internal logic to remain intact after the user's assignment code runs.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
Added unit tests of two processors included in foreignKey join : SubscriptionSendProcessorSupplier and ForeignTableJoinProcessorSupplier.
Renamed ForeignTableJoinProcessorSupplierTest to SubscriptionJoinProcessorSupplierTest as that's the processor which the test class is testing.
Reviewers: Walker Carlson <wcarlson@apache.org>
The test shouldCloseAllTaskProducersOnCloseIfEosEnabled() in
StreamThreadTest is flaky with a concurrent modification exception.
The concurrent modification exception is caused by the test itself
because it starts a stream thread and at the same time the thread
that executes the test calls methods on the stream thread. The stream
thread was not designed for such a concurrency.
The tests verifies that under EOS the streams producer are closed
during shutdown. Actually the test is not needed since we already
have a test that verifies that when the stream thread shuts down
also the task manager shuts down and for the tasks manager we have
tests that verify that the producers are closed when the task manager
shuts down.
This commit verifies that those tests are run under EOS and ALOS.
Reviewer: Matthias J. Sax <matthias@confluent.io>
The graph solving utilities are currently hardcoded to work with ClientState, but don't actually depend on anything in those state classes.
This change allows the MinTrafficGraphConstructor and BalanceSubtopologyGraphConstructor to be reused with KafkaStreamsStates instead.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Almog Gavra <almog@responsive.dev>
Removes the unused remove operation from the state updater
that asynchronously removed tasks and put them into an
output queue.
Reviewer: Lucas Brutschy <lbrutschy@confluent.io>