* Update KafkaAdminClient#describeTopics to throw UnknownTopicOrPartitionException.
* Remove unused method: WorkerUtils#getMatchingTopicPartitions.
* Add some JavaDoc.
Reviewed-by: Colin P. McCabe <cmccabe@apache.org>, Ryanne Dolan <ryannedolan@gmail.com>
The Trogdor Coordinator now overwrites a task's startMs to the time it received it if startMs is in the past.
The Trogdor Agent now correctly expires a task after the expiry time (startMs + durationMs) passes. Previously, it would ignore startMs and expire after durationMs milliseconds of local start of the task.
Reviewed-by: Colin P. McCabe <cmccabe@apache.org>
When I originally refactored the streams_upgrade_test#upgrade_downgrade_brokers test I removed the wait call which would wait for up 24 minutes for the SmokeTestDriver class to publish and verify all of its records.
Since most of the tests run in two minutes or less I set the monitor_log timeout to three minutes. However, the SmokeTestDriver#verify method allows up to six minutes to consume all records before verifying the monitor_log timeout needs to be greater than 6 minutes. I've set the timeout to 8 minutes.
Additionally, the steps needed to update the streams_upgrade_test should be documented as there are several components that need to get updated. So I've documented those steps here on the test as a giant comment.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
* KAFKA-6627: Prevent config default values overriding ones specified through --producer-property on command line.
In Console{Producer,Consumer}, extraProducerProps (options specified in
--producer-property) is applied first, then overriden unconditionally,
even if the value is not specified explicitly (and default value is
used). This patch fixes it so that it doesn't override the existing
value set by --producer-property if it is not explicitly specified.
The contribution is my original work and I license the work to the
project under the project's open source license.
Reviewers: Sriharsha Chintalapani <sriharsha@apache.org>
As documented in https://issues.apache.org/jira/browse/KAFKA-7741,
the javax dependency we receive transitively from connect is incompatible
with SBT builds.
Streams doesn't use the portion of Connect that needs the dependency,
so we can fix the builds by simply excluding it.
Reviewers: Bill Bejeck <bill@confluent.io>, Guozhang Wang <guozhang@confluent.io>
We recently improved the handling of the InternalTopicManager retries with #6085. The AdminClient will throw an InvalidTopicException if the topic is not found. We need to ignore that exception as when calling AdminClient#describe we may not have had a chance to create the topic yet, especially with the case of internal topics
I've created a new test asserting that when an InvalidTopicException is thrown when the topic is not found we continue on.
Reviewers: John Roesler <john@confluent.io>, Guozhang Wang <guozhang@confluent.io>
There is a race condition in ReplicaFetcherThread, where we can update PartitionFetchState with the new leader epoch (same leader) before handling the OffsetsForLeaderEpoch response with FENCED_LEADER_EPOCH error which causes removing partition from partitionStates, which in turn causes no fetching until the next LeaderAndIsr.
This patch adds logic to ensure that the leader epoch doesn't change while an OffsetsForLeaderEpoch request is in flight (which could happen with back-to-back leader elections). If it has changed, we ignore the response.
Also added toString() implementation to PartitionData, because some log messages did not show useful info which I found while investigating the above system test failure.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
Use `info` for failed authentications and `debug` for successful ones.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Ismael Juma <ismael@juma.me.uk>
Reviewers: Bill Bejeck <bill@confluent.io>, John Roesler <john@confluent.io>, Ryanne Dolan <ryannedolan@gmail.com>, Guozhang Wang <guozhang@confluent.io>
The test always fails if testOptionsDoesNotIncludeWadlOutput is executed before testCORSEnabled. It seems the problem is the use of the system property. Perhaps there is some static caching somewhere.
Reviewers: Randall Hauch <rhauch@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
This patch changes the behavior of KafkaProducer.waitOnMetadata to wait up to max.block.ms when the partition specified in the produce request is out of the range of partitions present in the metadata. This improves the user experience in the case when partitions are added to a topic and a client attempts to produce to one of the new partitions before the metadata has propagated to the brokers. Tested with unit tests.
Reviewers: Arjun Satish <arjun@confluent.io>, Jason Gustafson <jason@confluent.io>
The null map returned from the current snapshot causes the null type in response. The connector class name can be taken from the config of request instead since we require the config should contain the connector class name.
Reviewers: Jason Gustafson <jason@confluent.io>
This commit creates an EndToEndTest base class which relies on the verifiable consumer. This will ultimately replace ProduceConsumeValidateTest which depends on the console consumer. The advantage is that the verifiable consumer exposes more information to use for validation. It also allows for a nicer shutdown pattern. Rather than relying on the console consumer idle timeout, which requires a minimum wait time, we can halt consumption after we have reached the last acked offsets. This should be more reliable and faster. The downside is that the verifiable consumer only works with the new consumer, so we cannot yet convert the upgrade tests. This commit converts only the replication tests and a flaky security test to use EndToEndTest.
Lookup client host name after every full iteration through the addresses returned.
Reviewers: Loïc Monney <loicmonney@github.com>, Edoardo Comar <ecomar@uk.ibm.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
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>
Some Streams system tests have failed during the setup phase
due to the producer having retries disabled and getting some
transient error from the broker.
This patch adds a retries parameter to the VerifiableProducer
(default unchanged), and sets retries to 10 for Streams tests.
It also sets acks equal to the number of brokers for Streams tests.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <guozhang@confluent.io>
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 commits sets ACL on /kafka-acl-extended
- Extended ZkAuthorizationTest to check ACL on /kafka-acl-extended
- Using zookeeper-security-migration.sh tool on a Kerberized test cluster, I verified the changes: secured and unsecured Kafka znodes and examined ACL on /kafka-acl-extended with zookeeper client
Author: Attila Sasvari <asasvari@apache.org>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Andras Katona <41361962+akatona84@users.noreply.github.com>
Closes#6072 from asasvari/KAFKA-7752
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>
* Shard purgatory to reduce lock contention
* put constant into Object, use foldLeft instead of for loop
* watchersForKey -> watchersByKey
* Incorporate Jun's comments: use named arguments instead of _, and remove
an unnecessary lock
Reviewers: Sriharsha Chintalapani <sriharsha@apache.org>, Jun Rao <junrao@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
Check `running` in `Sender.maybeWaitForProducerId` to ensure that the producer can be closed while awaiting initialization of the producerId.
Reviewers: Jason Gustafson <jason@confluent.io>
Replace `channel` by `fileRecords` in potentially thrown KafkaException
descriptions when loading/writing `FileChannelRecordBatch`. This makes exception
messages more readable (channel only shows an object hashcode, fileRecords shows
the path of the file being read and start/end positions in the file).
Reviewers: Jason Gustafson <jason@confluent.io>
When using the Connect `JsonConverter`, it's impossible to produce tombstone messages, thus impacting the compaction of the topic. This patch allows the converter with and without schemas to output a NULL byte value in order to have a proper tombstone message. When it's regarding to get this data into a connect record, the approach is the same as when the payload looks like `"{ "schema": null, "payload": null }"`, this way the sink connectors can maintain their functionality and reduces the BCC.
Reviewers: Gunnar Morling <gunnar.morling@googlemail.com>, Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
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>
Previous PR #6043 reduced throughput for VerifiableProducer in base class, but the streams_standby_replica_test needs higher throughput for consumer to complete verification in 60 seconds
Update system test and kicked off branch builder with 25 repeats https://jenkins.confluent.io/job/system-test-kafka-branch-builder/2201/
Reviewers: Guozhang Wang <wangguoz@gmail.com>
**User Interface Improvement :** If topic doesn't exist then Kafka describe command should throw topic doesn't exist exception, like alter and delete commands
Author: Manohar Vanam <manohar.crazy09@gmail.com>
Reviewers: Vahid Hashemian <vahid.hashemian@gmail.com>, Jason Gustafson <jason@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#5211 from ManoharVanam/KAFKA-7054
Switch to lambda when ever possible instead of old anonymous way
in tools module
Author: Srinivas Reddy <srinivas96alluri@gmail.com>
Author: Srinivas Reddy <mrsrinivas@users.noreply.github.com>
Reviewers: Ryanne Dolan <ryannedolan@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#6013 from mrsrinivas/tools-switch-to-java8