This test has been observed to have flaky failures.
Apparently, in the failed runs, Streams had entered a rebalance
before some of the assertions were made. We recently made
IQ a little stricter on whether it would return errors instead of
null responses in such cases:
KAFKA-10598: Improve IQ name and type checks (#9408)
As a result, we have started seeing failures now instead of
silently executing an invalid test (I.e., it was asserting the
return to be null, but the result was null for the wrong
reason).
Now, if the test discovers that Streams is no longer running,
it will repeat the verification until it actually gets a valid
positive or negative result.
Reviewers: Chia-Ping Tsai <chia7712@apache.org>
This reverts commit 21dc5231ce as we decide to use Envelope for redirection instead of initial principal.
Reviewers: Jason Gustafson <jason@confluent.io>
This patch fixes a test a test case in `QuorumStateTest`. The method name is "testVotedToUnattachedHigherEpoch," but the code initialized in the unattached state instead of the voted state.
Reviewers: Jason Gustafson <jason@confluent.io>
The patch adds `quorum.append.linger.ms` behavior to the raft implementation. This gives users a powerful knob to tune the impact of fsync. When an append is accepted from the state machine, it is held in an accumulator (similar to the producer) until the configured linger time is exceeded. This allows the implementation to amortize fsync overhead at the expense of some write latency.
The patch also improves our methodology for testing performance. Up to now, we have relied on the producer performance test, but it is difficult to simulate expected controller loads because producer performance is limited by other factors such as the number of producer clients and head-of-line blocking. Instead, this patch adds a workload generator which runs on the leader after election.
Finally, this patch brings us nearer to the write semantics expected by the KIP-500 controller. It makes the following changes:
- Introduce `RecordSerde<T>` interface which abstracts the underlying log implementation from `RaftClient`. The generic type is carried over to `RaftClient<T>` and is exposed through the read/write APIs.
- `RaftClient.append` is changed to `RaftClient.scheduleAppend` and returns the last offset of the expected log append.
- `RaftClient.scheduleAppend` accepts a list of records and ensures that the full set are included in a single batch.
- Introduce `RaftClient.Listener` with a single `handleCommit` API which will eventually replace `RaftClient.read` in order to surface committed data to the controller state machine. Currently `handleCommit` is only used for records appended by the leader.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Guozhang Wang <wangguoz@gmail.com>
A regression got introduced by 466f8fd21c. The owned partition field must be ignored for version < 1 otherwise the serialization fails with an unsupported version exception.
Reviewers: Jason Gustafson <jason@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
Scala uses NonLocalReturnException to implement the control flow of returning from a nested anonymous function. That is anti-pattern so we should avoid using it in the hot methods.
Reviewers: Ismael Juma <ismael@confluent.io>
Today for active tasks we the following active task suspension:
1. closeAndRevive in handleTaskCorruption.
2. closeClean in assignor#onAssignment.
3. closeClean in shutdown.
4. closeDirty in assignor#onAssignment.
5. closeDirty in listener#onPartitionsLost.
6. closeDirty in shutdown.
7. suspend in listener#onPartitionsRevoked.
Among those, 1/4/5/6 do not call prepareCommit which would stateManager#flushCache and may cause illegal state manager. This PR would require a prepareCommit triggered before suspend.
Reviewers: A. Sophie Blee-Goldman <ableegoldman@apache.org>
DescribeUserScramCredentialsRequest states that all users are described when Users is empty or null. null is not handled at the moment and throws an NPE.
Reviewers: Ron Dagostino <rdagostino@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
`KafkaAdminClient.describeUserScramCredentials` should not fail with a NPE when `users` is `null` as `null` means that all the users must be returned.
Reviewers: Ron Dagostino <rdagostino@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <djacot@confluent.io>
quota_test.py tests are failing with below error.
```
23:24:42 [INFO:2020-10-24 17:54:42,366]: RunnerClient: kafkatest.tests.client.quota_test.QuotaTest.test_quota.quota_type=user.override_quota=False: FAIL: not enough arguments for format string
23:24:42 Traceback (most recent call last):
23:24:42 File "/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/venv/lib/python3.6/site-packages/ducktape-0.8.0-py3.6.egg/ducktape/tests/runner_client.py", line 134, in run
23:24:42 data = self.run_test()
23:24:42 File "/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/venv/lib/python3.6/site-packages/ducktape-0.8.0-py3.6.egg/ducktape/tests/runner_client.py", line 192, in run_test
23:24:42 return self.test_context.function(self.test)
23:24:42 File "/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/venv/lib/python3.6/site-packages/ducktape-0.8.0-py3.6.egg/ducktape/mark/_mark.py", line 429, in wrapper
23:24:42 return functools.partial(f, *args, **kwargs)(*w_args, **w_kwargs)
23:24:42 File "/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/tests/kafkatest/tests/client/quota_test.py", line 141, in test_quota
23:24:42 self.quota_config = QuotaConfig(quota_type, override_quota, self.kafka)
23:24:42 File "/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/tests/kafkatest/tests/client/quota_test.py", line 60, in __init__
23:24:42 self.configure_quota(kafka, self.producer_quota, self.consumer_quota, ['users', None])
23:24:42 File "/home/jenkins/workspace/system-test-kafka-branch-builder/kafka/tests/kafkatest/tests/client/quota_test.py", line 83, in configure_quota
23:24:42 (kafka.kafka_configs_cmd_with_optional_security_settings(node, force_use_zk_conection), producer_byte_rate, consumer_byte_rate)
23:24:42 TypeError: not enough arguments for format string
23:24:42
```
ran thee tests locally.
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: David Jacot <djacot@confluent.io>, Ron Dagostino <rndgstn@gmail.com>
Closes#9496 from omkreddy/quota-tests
Fix vagrant for a system tests with a python3.
Author: Nikolay Izhikov <nizhikov@apache.org>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#9480 from nizhikov/KAFKA-10592
There is a lot of functionality in KafkaRaftClientTest that is useful for writing other tests. Refactor that functionality into another class that can be reused in other tests.
Reviewers: Jason Gustafson <jason@confluent.io>
* Replace quorum.bootstrap.servers and quorum.bootstrap.voters with
quorum.voters.
* Remove seemingly unused `verbose` config.
* Use constant to avoid unnecessary repeated concatenation.
Reviewers: Jason Gustafson <jason@confluent.io>
The transaction manager does currently not handle producer fenced errors returned from a offset commit request.
Adds the handling of the producer fenced errors.
Reviewers: Boyang Chen <boyang@apache.org>, John Roesler <vvcephei@apache.org>
TopologyTestDriver comes with a paper cut that it passes through a
config requirement that application.id and bootstrap.servers must be
configured. But these configs are not required in the context of
TopologyTestDriver specifically. This change relaxes the requirement.
Reviewers: Boyang Chen <boyang@apache.org>, Matthias J. Sax <mjsax@apache.org>
We currently stop polling in `Sender` in a transactional producer if there is only one broker in the bootstrap server list and `max.in.flight.requests.per.connection=1` and Metadata response is pending when InitProducerId request is ready to be sent. In this scenario, we attempt to send FindCoordinator to `leastLoadedNode`, but since that is blocked due to `max.in.flight=1` as a result of the pending metadata response, we never unblock unless we poll. This PR ensures we poll in this case.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jason Gustafson <jason@confluent.io>, David Jacot <djacot@confluent.io>
Introduce an ASSIGNMENT_ERROR code to distinguish from INCOMPLETE_SOURCE_TOPIC_METADATA and shut down all members in case of an unexpected exception during task assignment.
Reviewers: Matthias J. Sax <mjsax@apache.org>, John Roesler <vvcephei@apache.org>
In order to support topic IDs, we need to create a public UUID class. This class will be used in protocols. This PR creates the class, modifies code to use the class in the message protocol and changes the code surrounding the existing messages/json that used the old UUID class.
SimpleExampleMessage was used only for testing, so all usages of UUID have been switched to the new class.
SubscriptionInfoData uses UUID for processId extensively. It also utilizes java.util.UUID implementation of Comparable so that UUIDs can be ordered. This functionality was not necessary for the UUIDs used for topic IDs converted to java.util.UUID on the boundary of SubscriptionInfoData. Sorting was used only for testing, though, so this still may be changed.
Also added tests for the methods of the new UUID class. The existing SimpleExampleMessage tests should be sufficient for testing the new UUID class in message protocols.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Minor update to fix flaky state directory test by advancing the MockTime.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, A. Sophie Blee-Goldman <ableegoldman@apache.org>
Fix KAFKA-10454 bug
Main issue was that when optimization algorithm was removing repartition nodes, corresponding copartitionSourceGroups was never updated. As a result, copartition enforcer wasn't able to do the checks and set proper number of partitions.
Test ensures that whenever optimization is set, changelog topic for the table is not created. And whenever optimization is turned off, appropriate changelog topic for the table is created.
Reviewers: John Roesler <john@confluent.io>, Bill Bejeck <bbejeck@gmail.com>
Also introduced the notion of WrappingNullableSerdes (aligned to the concept
of WrappingNullableSerializer and WrappingNullableDeserializer) and centralized
initialization in WrappingNullables.
The added integeration test KTableKTableForeignKeyJoinDistributedTest tests
whether all serdes are now correctly set on all stream clients.
Reviewers: John Roesler <vvcephei@apache.org>
UpdateMetadataRequestTest.testVersionLogic's assertions must verify the deserialized request instead of the original one.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
`ClientQuotaManager.updateQuota` updates the in-memory quota before updating the configuration of the metric. Therefore, `quotaLimit` can return the updated value while the metric's configuration still returns the previous one. This patch updates the test to be resilient to this case.
Reviewers: David Jacot <djacot@confluent.io>
DistributedHerder goes to updateConfigsWithIncrementalCooperative() synchronized method and called configBackingStore.snapshot() which take a lock on internal object in KafkaConfigBackingStore class.
Meanwhile KafkaConfigBackingStore in ConsumeCallback inside synchronized block on internal object gets SESSION_KEY record and calls updateListener.onSessionKeyUpdate() which take a lock on DistributedHerder.
This results to a deadlock.
To avoid this, updateListener with new session key should be called outside synchronized block as it's done, for example, for updateListener.onTaskConfigUpdate(updatedTasks).
Co-authored-by: Taisiia Goltseva <tado0618@netcracker.com>
Reviewers: Konstantine Karantasis <k.karantasis@gmail.com>
Using a Set is not necessary as the caller only cares about having the list of timed out connections/nodes.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Add dummy data to subscriptionUserData to make sure that
it is different each time a member rejoins.
Reviewers: A. Sophie Blee-Goldman <ableegoldman@apache.org>, John Roesler <vvcephei@apache.org>