Skip to content

Increase acceptable duration time for ReassignPartitionsClusterTest#shouldExecuteThrottledReassignment#5887

Merged
ijuma merged 5 commits intoapache:trunkfrom
stanislavkozlovski:shouldExecuteThrottledReassignment-flaky-test
Feb 2, 2019
Merged

Increase acceptable duration time for ReassignPartitionsClusterTest#shouldExecuteThrottledReassignment#5887
ijuma merged 5 commits intoapache:trunkfrom
stanislavkozlovski:shouldExecuteThrottledReassignment-flaky-test

Conversation

@stanislavkozlovski
Copy link
Copy Markdown
Contributor

We've seen this test fail in Jenkins (https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/) with 10400ms.
Running locally 50 times, I had two instances where it took 8.2s and 9.3s. Since Jenkins is typically running on a slower machine, I think that it is reasonable to increase the acceptable duration here in order to reduce failed builds due to test flakiness.

took > expectedDurationSecs * 0.9 * 1000)
assertTrue(s"Expected replication to be < ${expectedDurationSecs * 2 * 1000} but was $took",
assertTrue(s"Expected replication to be < ${expectedDurationSecs * 3 * 1000} but was $took",
took < expectedDurationSecs * 2 * 1000)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is the changing the error message, not the actual check :-) In any case, I am not sure this test makes any sense any more if we can't even get it to work with double the expected value. We should see if we can change message size or number of messages to get it to work consistently with a duration that is between 0.9t and 2t.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh whoops, hehe :)
I will try to tweak it a bit but it is worth mentioning that these slow runs are definitely outliers. Less than 1 in 50 locally

@stanislavkozlovski
Copy link
Copy Markdown
Contributor Author

I spent a fair bit of time debugging this. I ran the test what feels like a thousand times.

One thing I noticed is that in runs that timed out, we would hit the There are no active partitions. Back off for 1000 ms before sending a fetch request (kafka.server.ReplicaFetcherThread:74) trace log

trace(s"There are no active partitions. Back off for $fetchBackOffMs ms before sending a fetch request")

before starting to produce messages in the test.

I have no conclusive thoughts on why this is happening.
Changing the backoff ms to 100 has made me unable to reproduce the timeouts, though.

Another interesting thing is that the backoff would be called only once when it was 1000. Now that it is 100, I see it much more frequently in the logs. It gets called when we have no active partitions eligible for fetching - and that is expected if all are throttled.
I'm puzzled as to why this is happening.

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Nov 27, 2018

@stanislavkozlovski that makes sense right? If the backoff is large enough, it can cause us to wait for too long before restarting fetches given the test workload. The backoff is configurable (replica.fetch.backoff.ms), so maybe we just reduce it to 100ms for this test?

saslSslPort: Int = RandomPort,
rack: Option[String] = None,
logDirCount: Int = 1,
replicaFetchBackoff: Int = 1000,
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is not common enough to be here. I suggest setting the config on the returned Properties instance.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's fair. updated.

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Nov 27, 2018

Ah, I see that you have done that in the PR, just left a minor comment. Might be worth explaining why we set the config too.

def startBrokers(brokerIds: Seq[Int]) {
servers = brokerIds.map(i => createBrokerConfig(i, zkConnect, enableControlledShutdown = false, logDirCount = 3))
.map(c => createServer(KafkaConfig.fromProps(c)))
servers = brokerIds.map(i => {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Style nit: broker.ids.map { i => ....

Copy link
Copy Markdown
Member

@ijuma ijuma left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the PR, LGTM.

@ijuma ijuma merged commit 66129b1 into apache:trunk Feb 2, 2019
mjsax pushed a commit that referenced this pull request Feb 10, 2019
…st (#5887)

The default backoff of 1000ms when there are no partitions to fetch can cause `shouldExecuteThrottledReassignment` to fail due to it taking too long. So we reduce
it to 100ms.

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Ismael Juma <ismael@juma.me.uk
@mjsax
Copy link
Copy Markdown
Member

mjsax commented Feb 10, 2019

Cherry-picked to 2.2, as discussed on #6222

jarekr pushed a commit to confluentinc/kafka that referenced this pull request Apr 18, 2019
* AK/trunk:
  fix typo (apache#5150)
  MINOR: Reduce replica.fetch.backoff.ms in ReassignPartitionsClusterTest (apache#5887)
  KAFKA-7766: Fail fast PR builds (apache#6059)
  KAFKA-7798: Expose embedded clientIds (apache#6107)
  KAFKA-7641; Introduce "group.max.size" config to limit group sizes (apache#6163)
  KAFKA-7433; Introduce broker options in TopicCommand to use AdminClient (KIP-377)
  MINOR: Fix some field definitions for ListOffsetReponse (apache#6214)
  KAFKA-7873; Always seek to beginning in KafkaBasedLog (apache#6203)
  KAFKA-7719: Improve fairness in SocketServer processors (KIP-402) (apache#6022)
  MINOR: fix checkstyle suppressions for generated RPC code to work on Windows
  KAFKA-7859: Use automatic RPC generation in LeaveGroups (apache#6188)
  KAFKA-7652: Part II; Add single-point query for SessionStore and use for flushing / getter (apache#6161)
  KAFKA-3522: Add RocksDBTimestampedStore (apache#6149)
  KAFKA-3522: Replace RecordConverter with TimestampedBytesStore (apache#6204)
pengxiaolong pushed a commit to pengxiaolong/kafka that referenced this pull request Jun 14, 2019
…st (apache#5887)

The default backoff of 1000ms when there are no partitions to fetch can cause `shouldExecuteThrottledReassignment` to fail due to it taking too long. So we reduce
it to 100ms.

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Ismael Juma <ismael@juma.me.uk
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants