Commit Graph

6558 Commits

Author SHA1 Message Date
Scott Hendricks 39afe9fe0e KAFKA-8853; Create sustained connections test for Trogdor
This creates a test that generates sustained connections against Kafka.  There
are three different components we can stress with this, KafkaConsumer,
KafkaProducer, and AdminClient.  This test tries use minimal bandwidth per
connection to reduce overhead impacts.

This test works by creating a threadpool that creates connections and then
maintains a central pool of connections at a specified keepalive rate.  The
keepalive action varies by which component is being stressed:

  * KafkaProducer:  Sends one single produce record.  The configuration for
    the produce request uses the same key/value generator as the ProduceBench
    test.

  * KafkaConsumer: Subscribes to a single partition, seeks to the end, and
    then polls a minimal number of records.  Each consumer connection is its
    own consumer group, and defaults to 1024 bytes as FETCH_MAX_BYTES to keep
    traffic to a minimum.

  * AdminClient: Makes an API call to get the nodes in the cluster.

NOTE: This test is designed to be run alongside a ProduceBench test for a
specific topic, due to the way the Consumer test polls a single partition.
There may be no data returned by the consumer test if this is run on its own.
The connection should still be kept alive, but with no data returned.

Author: Scott Hendricks <scott.hendricks@confluent.io>

Reviewers: Stanislav Kozlovski, Gwen Shapira

Closes #7289 from scott-hendricks/trunk
2019-09-08 19:49:13 -07:00
John Roesler 0f177ea6b8 MINOR: Clean up partition assignment logic (#7249)
These are just some "tidying up" changes I made when I was preparing to start working on KIP-441.

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2019-09-08 17:06:06 -07:00
Jason Gustafson 7b62248c62
MINOR: Add cause to thrown exception when deleting topic in TopicCommand (#7301)
Unexpected exceptions are caught during topic deletion in `TopicCommand`. The caught exception is currently lost and we raise `AdminOperationException`. This patch fixes the problem by chaining the caught exception. 

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2019-09-06 09:44:15 -07:00
Jason Gustafson c65e19ae95
MINOR: Use toString in Records implementations only for summary metadata (#7290)
This patch fixes a couple problems with logging of request/response objects which include records data. First, it adds a missing `toString` to `LazyDownConversionRecords`. Second, it changes the `toString` of `MemoryRecords` to not print record-level information. This was always a dangerous practice, but it was especially bad when these objects ended up in request logs. With this patch, implementations use `toString` only to print summary details.

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2019-09-06 00:54:53 -07:00
Boyang Chen c0019e6538 KAFKA-8590; Use automated TxnOffsetCommit type and add tests for OffsetCommit (#6994)
This PR changes the TxnOffsetCommit protocol to auto-generated types, and add more unit test coverage to the plain OffsetCommit protocol.

Reviewers: Jason Gustafson <jason@confluent.io>
2019-09-05 23:07:42 -07:00
Sean Glover ad3ccf8f31 KAFKA-8822; Consolidate `PartitionRecords` and `CompletedFetch` types in Fetcher (#7228)
The `CompletedFetch` and `PartitionRecords` types are somewhat redundant. This PR consolidates the two types.

Reviewers: Jason Gustafson <jason@confluent.io>
2019-09-05 23:03:13 -07:00
vinoth chandar ffef0871c2 KAFKA-7149 : Reducing streams assignment data size (#7185)
* Leader instance uses dictionary encoding on the wire to send topic partitions
* Topic names (most expensive component) are mapped to an integer using the dictionary
* Follower instances receive the dictionary, decode topic names back
* Purely an on-the-wire optimization, no in-memory structures changed
* Test case added for version 5 AssignmentInfo

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2019-09-05 13:50:55 -07:00
Jason Gustafson deac5d93ce
KAFKA-8724; Improve range checking when computing cleanable partitions (#7264)
This patch contains a few improvements on the offset range handling when computing the cleanable range of offsets.

1. It adds bounds checking to ensure the dirty offset cannot be larger than the log end offset. If it is, we reset to the log start offset.
2. It adds a method to get the non-active segments in the log while holding the lock. This ensures that a truncation cannot lead to an invalid segment range.
3. It improves exception messages in the case that an inconsistent segment range is provided so that we have more information to find the root cause.

The patch also fixes a few problems in `LogCleanerManagerTest` due to unintended reuse of the underlying log directory.

Reviewers: Vikas Singh <soondenana@users.noreply.github.com>, Jun Rao <junrao@gmail.com>
2019-09-05 09:10:52 -07:00
Rajini Sivaram c9318ced19
KAFKA-8857; Don't check synonyms while determining if config is readOnly (#7278)
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2019-09-05 09:23:11 +01:00
Rajini Sivaram b66ea87440
KAFKA-8866; Return exceptions as Optional<ApiException> in authorizer API (#7294)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Manikumar Reddy <manikumar.reddy@gmail.com>
2019-09-05 09:21:01 +01:00
康智冬 c5f2bd64d1 MINOR: Fix few typos in the javadocs/docs 2019-09-05 09:07:23 +05:30
Tu V. Tran 8d8e2fb5c9 KAFKA-8729, pt 1: Add 4 new metrics to keep track of various types of invalid record rejections (#7142)
Right now we only have very generic FailedProduceRequestsPerSec and FailedFetchRequestsPerSec metrics that mark whenever a record is failed on the broker side. To improve the debugging UX, I added 4 new metrics in BrokerTopicStats to log various scenarios when an InvalidRecordException is thrown when LogValidator fails to validate a record:

-- NoKeyCompactedTopicRecordsPerSec: counter of failures by compacted records with no key
-- InvalidMagicNumberRecordsPerSec: counter of failures by records with invalid magic number
-- InvalidMessageCrcRecordsPerSec: counter of failures by records with crc corruption
-- NonIncreasingOffsetRecordsPerSec: counter of failures by records with invalid offset

Reviewers: Robert Yokota <rayokota@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
2019-09-04 12:46:30 -07:00
Chia-Ping Tsai 18e6bb251b KAFKA-8861 Fix flaky RegexSourceIntegrationTest.testMultipleConsumersCanReadFromPartitionedTopic (#7281)
similar to https://issues.apache.org/jira/browse/KAFKA-8011 and https://issues.apache.org/jira/browse/KAFKA-8026

Reviewers:  Matthias J. Sax <mjsax@apache.org>,  Bill Bejeck <bbejeck@gmail.com>
2019-09-04 10:13:56 -04:00
khairy caaf253b63 MINOR: remove unnecessary nulllity check (#7282)
Minor code enhancement: remove unnecessary check of nullity.

Reviewers: Bill Bejeck <bbejeck@gmail.com>
2019-09-04 10:12:18 -04:00
Omar Al-Safi 8dc80e2297 KAFKA-7849: Fix the warning when using GlobalKTable (#7104)
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2019-09-03 23:23:59 -07:00
Jason Gustafson 3b8d7a661c
MINOR: Add system configuration to zk security exception messages (#7280)
This patch ensures that relevant system configurations are included in exception messages when zk security validation fails.

Reviewers: Vikas Singh <soondenana@users.noreply.github.com>,  José Armando García Sancio <jsancio@users.noreply.github.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
2019-09-03 23:16:33 -07:00
Konstantine Karantasis 74fc3323a5 MINOR: Add unit test for KAFKA-8676 to guard against unrequired task restarts (#7287)
Added unit test for recent fix of `KafkaConfigBackingStore`.

Author: Konstantine Karantasis <konstantine@confluent.io>
Reviewer: Randall Hauch <rhauch@gmail.com>
2019-09-03 15:48:19 -05:00
LuyingLiu 31a5f92b9f Changed for updatedTasks, avoids stopping and starting of unnecessary tasks (#7097)
Corrected the `KafkaConfigBackingStore` logic to notify of only the changed tasks, rather than all tasks. This was not noticed before because Connect always stopped and restarted all tasks during a rebalanced, but since 2.3 the incremental rebalance logic exposed this bug.

Author: Luying Liu <lyliu@lyliu-mac.freewheelmedia.net>
Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Randall Hauch <rhauch@gmail.com>
2019-09-03 15:47:17 -05:00
Matthias J. Sax 2c0157765f MINOR: Use new `Admin` interface instead of `KafkaAdminClient` class (#7232)
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Ismael Juma <ismael@confluent.io>, Bill Bejeck <bbejeck@gmail.com>
2019-09-03 10:27:03 -04:00
teebee 88d1b6de1f KAFKA-8860: Let SslPrincipalMapper split SSL principal mapping rules
Author: teebee <tb@teebee.de>

Reviewers: Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>

Closes #7140 from teebee/teebee/ssl-principal-mapping-rules-handling
2019-09-02 23:32:49 +05:30
Rajini Sivaram 364794866f
KAFKA-8760; New Java Authorizer API (KIP-504) (#7268)
New Java Authorizer API and a new out-of-the-box authorizer (AclAuthorizer) that implements the new interface.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2019-09-02 14:43:17 +01:00
Colin Patrick McCabe a225347ff2
KAFKA-8840: Fix bug where ClientCompatibilityFeaturesTest fails when running multiple iterations (#7260)
Fix a bug where ClientCompatibilityFeaturesTest fails when running multiple iterations.

Also, fix a typo in tests/docker/Dockerfile.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2019-08-30 16:07:59 -07:00
Guozhang Wang 40432e31f7
MONIR: Check for NULL in case of version probing (#7275)
In case of version probing we would skip the logic for setting cluster / assigned tasks; since these values are initialized as null they are vulnerable to NPE when code changes.

Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Bill Bejeck <bill@confluent.io>
2019-08-30 14:48:46 -07:00
Bruno Cadonna d18d6b033e MINOR: Refactor tag key for store level metrics (#7257)
The tag key for store level metrics specified in StreamsMetricsImpl
is unified with the tag keys on thread and task level.

Reviewers: Sophie Blee-Goldman <sophie@confluent.io>, Bill Bejeck <bbejeck@gmail.com>
2019-08-30 09:46:07 -04:00
Lucas Bradstreet fb381cb6c7 MINOR: Fix integer overflow in LRUCacheBenchmark (#7270)
The jmh LRUCacheBenchmark will exhibit an int overflow when run on a fast machine:

```
java.lang.ArrayIndexOutOfBoundsException: Index -3648 out of bounds for length 10000
	at org.apache.kafka.jmh.cache.LRUCacheBenchmark.testCachePerformance(LRUCacheBenchmark.java:70)
	at org.apache.kafka.jmh.cache.generated.LRUCacheBenchmark_testCachePerformance_jmhTest.testCachePerformance_thrpt_jmhStub(LRUCacheBenchmark_testCachePerformance_jmhTest.java:119)
	at org.apache.kafka.jmh.cache.generated.LRUCacheBenchmark_testCachePerformance_jmhTest.testCachePerformance_Throughput(LRUCacheBenchmark_testCachePerformance_jmhTest.java:83)
```

Reviewers: Jason Gustafson <jason@confluent.io>
2019-08-29 18:02:27 -07:00
Vikas Singh 09ad6b84c5 MINOR. Fix 2.3.0 streams systest dockerfile typo (#7272)
As part of commit 4d1ee26a13 streams
version 2.3.0 test jar was added, but there was a simple typo in the
path that specified the version.

`ducker-ak up` was failing because of that. Fixed that.

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2019-08-29 14:12:08 -07:00
Bruno Cadonna d2741e5cbf MINOR: Remove `activeTaskCheckpointableOffsets` from `AbstractTask` (#7253)
Reviewers: cpettitt-confluent <53191309+cpettitt-confluent@users.noreply.github.com>, A. Sophie Blee-Goldman <sophie@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2019-08-29 11:26:44 -07:00
Bill Bejeck fcfee618ee MINOR: Only send delete request if there are offsets in map (#7256)
Currently on commit streams will attempt to delete offsets from repartition topics. However, if a topology does not have any repartition topics, then the recordsToDelete map will be empty.

This PR adds a check that the recordsToDelete is not empty before executing the AdminClient#deleteRecords() method.

Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2019-08-28 09:22:36 -07:00
Anastasia Vela d32a2d1275 KAFKA-8837: KafkaMetricReporterClusterIdTest may not shutdown ZooKeeperTestHarness (#7255)
- Call `assertNoNonDaemonThreads` in test method instead of tear down method
to avoid situation where parent's class tear down is not invoked.
- Pass the thread prefix in tests that call `assertNoNonDaemonThreads` so that it
works correctly.
- Rename `verifyNonDaemonThreadsStatus` to `assertNoNonDaemonThreads` to
make it clear that it may throw.

Reviewers: Anna Povzner <anna@confluent.io>, Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>
2019-08-27 22:23:19 -07:00
Stanislav Kozlovski 826408e122 MINOR: Initialize BrokerTopicMetrics with no topic tag greedily (#7198)
This patch fixes the quota system test whose JMX tool relies on the existence
of these metrics.

Reviewers: Guozhang Wang <wangguoz@gmail.com>, Nikhil Bhatia <nikhil@confluent.io>, Tu V. Tran <tuvtran97@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2019-08-27 21:12:38 -07:00
A. Sophie Blee-Goldman cf32a1a6a0 KAFKA-8179: Part 4, add CooperativeStickyAssignor (#7130)
Splits the existing StickyAssignor logic into an AbstractStickyAssignor class, which is extended by the existing (eager) StickyAssignor and by the new CooperativeStickyAssignor which supports incremental cooperative rebalancing.

There is no actual change to the logic -- most methods from StickyAssignor were moved to AbstractStickyAssignor to be shared with CooperativeStickyAssignor, and the abstract MemberData memberData(Subscription) method converts the Subscription to the embedded list of owned partitions for each assignor.

The "generation" logic is left in, however this is always Optional.empty() for the CooperativeStickyAssignor as onPartitionsLost should always be called when a generation is missed.

Reviewers: Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2019-08-27 14:41:33 -07:00
James Cheng e23a7182d5 KAFKA-8745: DumpLogSegments doesn't show keys, when the message is null (#7152)
Make sure to show the message key, even when the message value is null.

This changes the output of one of the tools. Is the output of the tool considered a public API? Does this need a discussion or a KIP?

Testing: Ran the tool on a compacted topic. Previously, the tool did not show any message keys for tombstone messages (messages where the value is null). Now, the tool shows message keys.

Reviewers: Mickael Maison <mimaison@users.noreply.github.com>, Guozhang Wang <wangguoz@gmail.com>
2019-08-27 13:01:29 -07:00
Robert Yokota add662907d MINOR: fix ProduceBenchWorker not to fail on final produce (#7254)
When sending bad records, the Trogdor task will fail if the final record produced is bad. Instead we should catch the exception to allow the task to finish since sending bad records is a valid use case.

Reviewers: Tu V. Tran <tuvtran97@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
2019-08-27 09:55:51 -07:00
Bruno Cadonna 24547b810c KAFKA-8579: Expose RocksDB metrics (#7209)
RocksDB metrics are added to the Kafka metrics. For each segmented state store only
one set of metrics is exposed rather than one set of metrics for each segment.

The metrics are not computed yet.

Reviewers: John Roesler <john@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2019-08-26 16:28:22 -07:00
cpettitt-confluent 6b24b2e836 KAFKA-8816: Make offsets immutable to users of RecordCollector.offsets (#7223)
Make offsets immutable to users of RecordCollector.offsets. Fix up an
existing case where offsets could be modified in this way. Add a simple
test to verify offsets cannot be changed externally.

Reviewers: Bruno Cadonna <bruno@confluent.io>, Guozhang Wang <guozhang@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2019-08-26 13:59:49 -07:00
saisandeep d08bcae7f9 KAFKA-8669: Add security providers in kafka security config (#7090)
* Adds custom provider class to security config 
* Implementation of KIP-492
Reviewers: Sriharsha Chintalapani <sriharsha@apache.org> , Jeff Huang
2019-08-26 10:01:01 -07:00
cpettitt-confluent 7334222a71 KAFKA-8412: Fix nullpointer exception thrown on flushing before closing producers (#7207)
Prior to this change an NPE is raised when calling AssignedTasks.close
under the following conditions:

1. EOS is enabled
2. The task was in a suspended state

The cause for the NPE is that when a clean close is requested for a
StreamTask the StreamTask tries to commit. However, in the suspended
state there is no producer so ultimately an NPE is thrown for the
contained RecordCollector in flush.

The fix put forth in this commit is to have AssignedTasks call
closeSuspended when it knows the underlying StreamTask is suspended.

Note also that this test is quite involved. I could have just tested
that AssignedTasks calls closeSuspended when appropriate, but that is
testing, IMO, a detail of the implementation and doesn't actually verify
we reproduced the original problem as it was described. I feel much more
confident that we are reproducing the behavior - and we can test exactly
the conditions that lead to it - when testing across AssignedTasks and
StreamTask. I believe this is an additional support for the argument of
eventually consolidating the state split across classes.

Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2019-08-26 09:53:36 -07:00
Konstantine Karantasis d7f8ec8628 MINOR: Fix the doc of scheduled.rebalance.max.delay.ms config property (#7242)
Author: Konstantine Karantasis <konstantine@confluent.io>
Reviewer: Randall Hauch <rhauch@gmail.com>
2019-08-25 17:35:03 -05:00
Randall Hauch 5b4e749dc6 KAFKA-8391; Temporarily ignore flaky Connect rebalance integration tests
I've spent quite a bit of time on trying to discover the root cause, with no luck so far. I have been able to reproduce it locally by running the following 100 times:
```
./gradlew connect:runtime:clean connect:runtime:test --tests org.apache.kafka.connect.integration.RebalanceSourceConnectorsIntegrationTest
```
The `testReconfigConnector` test failed 28% of the time and the others failed 0%. This issue and KAFKA-8661 suggest that `testDeleteConnector` and `testStartTwoConnectors` are also flaky, though I've not seen those tests fail locally.

Because this flakiness is causing issues for the rest of the project, I'm going to temporarily ignore several of the flaky ITs while I continue to investigate:
* `RebalanceSourceConnectorsIntegrationTest.testReconfigConnector`
* `RebalanceSourceConnectorsIntegrationTest.testDeleteConnector`
* `RebalanceSourceConnectorsIntegrationTest.testStartTwoConnectors`

**This should be backported to the `2.3` branch, which is when these integration tests were first added.**

Author: Randall Hauch <rhauch@gmail.com>

Reviewers: Ismael Juma

Closes #7237 from rhauch/kafka-8391-temporary
2019-08-25 14:30:23 -07:00
Chris Egerton 237e83dea0 KAFKA-8586: Fail source tasks when producers fail to send records (#6993)
Changed Connect's `WorkerSourceTask` to capture non-retriable exceptions from the `producer.send(...)` (e.g., authentication or authorization errors) and to fail the connector task when such an error is encountered. Modified the existing unit tests to verify this functionality.

Note that most producer errors are retriable, and Connect will (by default) set up each producer with 1 max in-flight message and infinite retries. This change only affects non-retriable errors.
2019-08-25 15:54:00 -05:00
Guozhang Wang c6664e1d08
MINOR: Move the resetting from revoked to the thread loop (#7243)
Move the error code resetting logic from the onPartitionsRevoked callback into the streamthread directly after we've decided to rejoin the group, since onPartitionsRevoked are not guaranteed to be triggered.

Ran system tests on the originally failed StreamsUpgradeTest 10 times and passed.

Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Jun Rao <junrao@gmail.com>
2019-08-23 14:34:32 -07:00
David Arthur c1f2b0ffb8 KAFKA-8753; Expose controller topic deletion metrics (KIP-503) (#7156)
This is the implementation for [KIP-503](https://cwiki.apache.org/confluence/display/KAFKA/KIP-503%3A+Add+metric+for+number+of+topics+marked+for+deletion)

When deleting a large number of topics, the Controller can get quite bogged down. One problem with this is the lack of visibility into the progress of the Controller. We can look into the ZK path for topics marked for deletion, but in a production environment this is inconvenient. This PR adds a JMX metric `kafka.controller:type=KafkaController,name=TopicsToDeleteCount` to make it easier to see how many topics are being deleted.

Reviewers: Stanislav Kozlovski <stanislav@confluent.io>, Jun Rao <junrao@gmail.com>, Jason Gustafson <jason@confluent.io>
2019-08-23 13:22:41 -07:00
asutosh936 40b7166e81 KAFKA-8698: Fix typo in ListOffsetResponse v0 protocol field name
Author: asutosh936 <asutosh.pandya@hotmail.com>

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Vahid Hashemian <vahid.hashemian@gmail.com>, Jason Gustafson <jason@confluent.io>

Closes #7141 from asutosh936/KAFKA-8698
2019-08-23 18:42:57 +05:30
John Roesler e213608127 KAFKA-8824: bypass value serde on null (#7235)
In a KTable context, we should not pass null into a user-supplied serde.

Testing: I verified that the change to the test results in test failures without the patch.

Reviewers: Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>,
2019-08-22 13:35:33 -07:00
Bob Barrett e4215c1784 KAFKA-8325; Remove batch from in-flight requests on MESSAGE_TOO_LARGE errors (#7176)
This patch fixes a bug in the handling of MESSAGE_TOO_LARGE errors. The large batch is split, the smaller batches are re-added to the accumulator, and the batch is deallocated, but it was not removed from the list of in-flight batches. When the batch was eventually expired from the in-flight batches, the producer would try to deallocate it a second time, causing an error. This patch changes the behavior to correctly remove the batch from the list of in-flight requests.

Reviewers: Luke Stephenson, Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
2019-08-21 23:29:20 -07:00
Andy Coates 35bc53c55f KIP-476: Add new getAdmin method to KafkaClientSupplier (#7162)
Reviewers: Matthias J. Sax <matthias@confluent.io>, Ismael Juma <ismael@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2019-08-21 19:48:00 -07:00
Matthias J. Sax 4d1ee26a13
KAFKA-8594: Add version 2.3 to Streams system tests (#7131)
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Boyang Chen <boyang@confluent.io>, Bill Bejeck <bill@confluent.io>
2019-08-21 10:26:57 -07:00
Brian Bushree 6c8f654d5f MINOR: Upgrade ducktape to 0.7.6
Author: Brian Bushree <bbushree@confluent.io>

Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>

Closes #7138 from brianbushree/update-ducktape
2019-08-19 17:12:48 -07:00
Bob Barrett aa4ba8eee8 KAFKA-8041: Enable producer retries in log dir failure test to address flakiness (#7200)
`testProduceAfterLogDirFailureOnLeader` currently disables producer retries in
order to catch and validate the exception thrown by a failure, and then tries to
produce successfully once the leadership changes. This second produce can
intermittently fail, causing test flakiness. This patch splits these validations
into two tests in order to allow retries for the produce request after the
leadership change.

Reviewers: Colin P. Mccabe <cmccabe@confluent.io>, Ismael Juma <ismael@juma.me.uk>
2019-08-17 14:24:08 -07:00
Lee Dongjin 784f7e84ed KAFKA-8800: Increase poll timeout in poll[Records]UntilTrue (#7211)
If retrieving metadata during `poll(Duration)` repeatedly takes longer than the
poll timeout, we don't make progress and `waitUntilTrue` eventually times out
causing the test to fail. This behaviour differs from the older `poll(long)` that
would block until metadata retrieval had completed. The flakiness was likely
introduced when we switched from the latter to the former.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2019-08-17 09:27:35 -07:00