The StreamsUpgradeTest::test_upgrade_downgrade_brokers used sleep calls in the test which led to flaky test performance and as a result, we placed an @ignore annotation on the test. This PR uses log events instead of the sleep calls hence we can now remove the @ignore setting.
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
While looking into KAFKA-7657, I found there are a few loopholes in this logic:
We kept a map of thread-name to thread-state and a global-thread state at the KafkaStreams instance-level, in addition to the instance state itself. stateLock is used when accessing the instance state, however when we are in the thread state change callback, we are accessing both the thread-states as well as the instance state at the same time in the callers of setState without a lock, which is vulnerable to concurrent multi-stream threads. The fix is a) introduce a threadStatesLock in addition to the stateLock, which should always be grabbed to modify the thread-states map before the stateLock for modifying the instance level; and we also defer the checking of the instance-level state inside the setState call.
When transiting to state.RUNNING, we check if all threads are either in RUNNING or DEAD state, this is because some threads maybe dead at the rebalance period but we should still proceed to RUNNING if the rest of threads are still transiting to RUNNING.
Added unit test for 2) above. Also simplified another test as a nit change.
Reviewers: John Roesler <vvcephei@users.noreply.github.com>, Matthias J. Sax <mjsax@apache.org>
While looking into KAFKA-7657, I found there are a few loopholes in this logic:
1. We kept a map of thread-name to thread-state and a global-thread state at the KafkaStreams instance-level, in addition to the instance state itself. stateLock is used when accessing the instance state, however when we are in the thread state change callback, we are accessing both the thread-states as well as the instance state at the same time in the callers of setState without a lock, which is vulnerable to concurrent multi-stream threads. The fix is a) introduce a threadStatesLock in addition to the stateLock, which should always be grabbed to modify the thread-states map before the stateLock for modifying the instance level; and we also defer the checking of the instance-level state inside the setState call.
2. When transiting to state.RUNNING, we check if all threads are either in RUNNING or DEAD state, this is because some threads maybe dead at the rebalance period but we should still proceed to RUNNING if the rest of threads are still transiting to RUNNING.
Added unit test for 2) above. Also simplified another test as a nit change.
Reviewers: John Roesler <vvcephei@users.noreply.github.com>, Matthias J. Sax <mjsax@apache.org>
Currently the TimeWindowedSerde does not deserialize the windowed keys from a changelog topic properly. There are a few assumptions made in the TimeWindowedDeserializer that prevents the changelog windowed keys from being correctly deserialized. This PR will introduce a new WindowSerde to allow proper deserialization of changelog windowed keys.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
1. The retry loop of the InternalTopicManager would just be: a) describe topics, and exclude those which already exist with the right num.partitions, b) for the remaining topics, try to create them. Remove any inner loops.
2. In CreateTopicResponse and MetadataResponse (for describe topic), handle the special error code of TopicExist and UnknownTopicOrPartition in order to retry in the next loop.
3. Do not handle TimeoutException since it should already been handled inside AdminClient.
Add corresponding unit tests for a) topic marked for deletion but not complete yet, in which case metadata response would not contain this topic, but create topic would return error TopicExists; b) request keep getting timed out.
Reviewers: Matthias J. Sax <matthias@confluent.io>
This pull request replaces HashMap with LinkedHashMap to guarantee ordering of metrics tags.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <guozhang@confluent.io>, John Roesler <vvcephei@users.noreply.github.com>
Right now if a repartition is required and users choose to name the repartition topic for an aggregation i.e. kGroupedStream = builder.<String, String>stream("topic").selectKey((k, v) -> k).groupByKey(Grouped.as("grouping")); The resulting KGroupedStream can't be reused
with optimizations are disabled, as Streams will attempt to create two repartiton topics with the same name.
However, if optimizations are enabled then the resulting KGroupedStream can be re-used
For example the following will work if optimizations are enabled.
This PR provides a unit test proving as much.
Reviewers: Matthias J. Sax <mjsax@apache.org>, Guozhang Wang <wangguoz@gmail.com>
Updating the documentation for table operation because I believe it is incorrect.
In PR #5163 the table operation stopped disabling the changelog topic by default and instead moved that optimization to a configuration that is not enabled by default. This PR updates the documentation to reflect the change in behavior and point to the new configuration for optimization.
Reviewers: Bill Bejeck <bbejeck@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
Reviewers: Guozhang Wang <guozhang@confluent.io>, Nikolay Izhikov <nIzhikov@gmail.com>, Ismael Juma <ismael@confluent.io>, Bill Bejeck <bill@confluent.io>
This is a follow-up PR from the previous PR #5779, where KTabeSource always get old values from the store even if sendOldValues. It gets me to make a pass over all the KTable/KStreamXXX processor to push the sendOldValues at the callers in order to avoid unnecessary store reads.
More details: ForwardingCacheFlushListener and TupleForwarder both need sendOldValues as parameters.
a. For ForwardingCacheFlushListener it is not needed at all, since its callers XXXCachedStore already use the sendOldValues values passed from TupleForwarder to avoid getting old values from underlying stores.
b. For TupleForwarder, it actually only need to pass the boolean flag to the cached store; and then it does not need to keep it as its own variable since the cached store already respects the boolean to pass null or the actual value..
The only other minor bug I found from the pass in on KTableJoinMerge, where we always pass old values and ignores sendOldValues.
Reviewers: Matthias J. Sax <mjsax@apache.org>
Refactor the materialization for source KTables in the way that:
If Materialized.as(queryableName) is specified, materialize;
If the downstream operator requires to fetch from this KTable via ValueGetters, materialize;
If the downstream operator requires to send old values, materialize.
Otherwise do not materialize the KTable. E.g. builder.table("topic").filter().toStream().to("topic") would not create any state stores.
There's a couple of minor changes along with PR as well:
KTableImpl's queryableStoreName and isQueryable are merged into queryableStoreName only, and if it is null it means not queryable. As long as it is not null, it should be queryable (i.e. internally generated names will not be used any more).
To achieve this, splitted MaterializedInternal.storeName() and MaterializedInternal.queryableName(). The former can be internally generated and will not be exposed to users. QueryableName can be modified to set to the internal store name if we decide to materialize it during the DSL parsing / physical topology generation phase. And only if queryableName is specified the corresponding KTable is determined to be materialized.
Found some overlapping unit tests among KTableImplTest, and KTableXXTest, removed them.
There are a few typing bugs found along the way, fixed them as well.
-----------------------
This PR is an illustration of experimenting a poc towards logical materializations.
Today we've logically materialized the KTable for filter / mapValues / transformValues if queryableName is not specified via Materialized, but whenever users specify queryableName we will still always materialize. My original goal is to also consider logically materialize for queryable stores, but when implementing it via a wrapped store to apply the transformations on the fly I realized it is tougher than I thought, because we not only need to support fetch or get, but also needs to support range queries, approximateNumEntries, and isOpen etc as well, which are not efficient to support. So in the end I'd suggest we still stick with the rule of always materializing if queryableName is specified, and only consider logical materialization otherwise.
Reviewers: Bill Bejeck <bill@confluent.io>, Matthias J. Sax <mjsax@apache.org>
We saw a log statement in which the cause of the failure to write a checkpoint was not properly logged.
This change logs the exception properly and also verifies the log message.
Reviewers: Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Changes made as part of this commit.
- Improved error message for better readability at millis validation utility
- Corrected java documentation on `AdvanceInterval` check.
- Added caller specific prefix text to make error message more clear to developers/users.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>, Jacek Laskowski <jacek@japila.pl>
This is a system test for doing a rolling upgrade of a topology with a named repartition topic.
1. An initial Kafka Streams application is started on 3 nodes. The topology has one operation forcing a repartition and the repartition topic is explicitly named.
2. Each node is started and processing of data is validated
3. Then one node is stopped (full stop is verified)
4. A property is set signaling the node to add operations to the topology before the repartition node which forces a renumbering of all operators (except repartition node)
5. Restart the node and confirm processing records
6. Repeat the steps for the other 2 nodes completing the rolling upgrade
I ran two runs of the system test with 25 repeats in each run for a total of 50 test runs.
All test runs passed
Reviewers: John Roesler <john@confluent.io>, Matthias J. Sax <matthias@confluent.io>
In StreamsMetricsImpl, the parentSensors map was keeping references to Sensors after the sensors themselves had been removed.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
This PR fixes an issue reported from a user. When we join a KStream with a GlobalKTable we should not reset the repartition flag as the stream may have previously changed its key, and the resulting stream could be used in an aggregation operation or join with another stream which may require a repartition for correct results.
I've added a test which fails without the fix.
Reviewers: John Roesler <john@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
* KAFKA-7367: Ensure stateless topologies don't require disk access
* KAFKA-7367: Streams should not create state store directories unless they are needed.
* Addressed the review comments.
* Addressed the review-2 comments.
* Fixed FileAlreadyExistsException
* Addressed the review-3 comments.
* Resolved the conflicts.
This is a new system test testing for optimizing an existing topology. This test takes the following steps
1. Start a Kafka Streams application that uses a selectKey then performs 3 groupByKey() operations and 1 join creating four repartition topics
2. Verify all instances start and process data
3. Stop all instances and verify stopped
4. For each stopped instance update the config for TOPOLOGY_OPTIMIZATION to all then restart the instance and verify the instance has started successfully also verifying Kafka Streams reduced the number of repartition topics from 4 to 1
5. Verify that each instance is processing data from the aggregation, reduce, and join operation
Stop all instances and verify the shut down is complete.
6. For testing I ran two passes of the system test with 25 repeats for a total of 50 test runs.
All test runs passed
Reviewers: Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Add the final batch of metrics from KIP-328
Reviewers: Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
In TopologyTestDriver constructor set non-null topic; and in unit test intentionally turn on caching to verify this case.
Reviewers: Bill Bejeck <bill@confluent.io>, Matthias J. Sax <matthias@confluent.io>
This pull request removes the final reference to KStreamWindowReducer and replaces it with KStreamWindowAggregate
Signed-off-by: Samuel Hawker sam.b.hawker@gmail.com
contribution is my original work and that I license the work to the project under the project's open source license.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
The removed tests have counterparts covered by SuppressScenarioTest using the TopologyTestDriver.
This will speed up the build and improve stability in the CPU-constrained Jenkins environment.
Reviewers: Bill Bejeck <bill@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Callers of 1) Windows#until, 2) Windows#of, 3) Serialized are replaced when possible with the new APIs.
Reviewers: Matthias J. Sax <mjsax@apache.org>, Bill Bejeck <bill@confluent.io>
- Use Xlint:all with 3 exclusions (filed KAFKA-7613 to remove the exclusions)
- Use the same javac options when compiling tests (seems accidental that
we didn't do this before)
- Replaced several deprecated method calls with non-deprecated ones:
- `KafkaConsumer.poll(long)` and `KafkaConsumer.close(long)`
- `Class.newInstance` and `new Integer/Long` (deprecated since Java 9)
- `scala.Console` (deprecated in Scala 2.11)
- `PartitionData` taking a timestamp (one of them seemingly a bug)
- `JsonMappingException` single parameter constructor
- Fix unnecessary usage of raw types in several places.
- Add @SuppressWarnings for deprecations, unchecked and switch fallthrough in
several places.
- Scala clean-ups (var -> val, ETA expansion warnings, avoid reflective calls)
- Use lambdas to simplify code in a few places
- Add @SafeVarargs, fix varargs usage and remove unnecessary `Utils.mkList` method
Reviewers: Matthias J. Sax <mjsax@apache.org>, Manikumar Reddy <manikumar.reddy@gmail.com>, Randall Hauch <rhauch@gmail.com>, Bill Bejeck <bill@confluent.io>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
This patch makes two improvements to internal metadata handling logic and testing:
1. It reduce dependence on the public object `Cluster` for internal metadata propagation since it is not easy to evolve. As an example, we need to propagate leader epochs from the metadata response to `Metadata`, but it is not straightforward to do this without exposing it in `PartitionInfo` since that is what `Cluster` uses internally. By doing this change, we are able to remove some redundant `Cluster` building logic.
2. We want to make the metadata handling in `MockClient` simpler and more consistent. Currently we have mix of metadata update mechanisms which are internally inconsistent with each other and do not match the implementation in `NetworkClient`.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Sets StreamsConfig.STATED_DIR_CONFIG to temp directory in
SuppressionIntegrationTest, to match StreamsTestUtils.
This is a similar fix to #5826.
Reviewers: Ismael Juma <ismael@juma.me.uk>
Set `StreamsConfig.STATED_DIR_CONFIG` in `SuppressScenarioTest`, as
with `StreamsTestUtils`. I have deliberately avoided using `StreamsTestUtils` as
this test sets bogus config parameters, but still fails if the default
`STATE_DIR_CONFIG` does not exist.
Reviewers: Colin Patrick McCabe <colin@cmccabe.xyz>, John Roesler <john@confluent.io>, Ismael Juma <ismael@juma.me.uk>