Skip to content

KAFKA-9700:Fix negative estimatedCompressionRatio issue#8285

Merged
ijuma merged 7 commits intoapache:trunkfrom
jiameixie:negative
Mar 25, 2020
Merged

KAFKA-9700:Fix negative estimatedCompressionRatio issue#8285
ijuma merged 7 commits intoapache:trunkfrom
jiameixie:negative

Conversation

@jiameixie
Copy link
Copy Markdown
Contributor

There are cases that currentEstimation is smaller than
COMPRESSION_RATIO_IMPROVING_STEP and it will get negative
estimatedCompressionRatio,which leads to misjudgment
about if there is no room and MESSAGE_TOO_LARGE might occur.

Change-Id: I0932a2a6ca669f673ab5d862d3fe7b2bb6d96ff6
Signed-off-by: Jiamei Xie jiamei.xie@arm.com

More detailed description of your change,
if necessary. The PR title and PR message become
the squashed commit message, so use a separate
comment to ping reviewers.

Summary of testing strategy (including rationale)
for the feature or bug fix. Unit and/or integration
tests are expected for any behaviour change and
system tests should be considered for larger changes.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

There are cases that currentEstimation is smaller than
COMPRESSION_RATIO_IMPROVING_STEP and it will get negative
estimatedCompressionRatio,which leads to misjudgment
about if there is no room and MESSAGE_TOO_LARGE might occur.

Change-Id: I0932a2a6ca669f673ab5d862d3fe7b2bb6d96ff6
Signed-off-by: Jiamei Xie <jiamei.xie@arm.com>
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. Can we please add a test?

@jiameixie
Copy link
Copy Markdown
Contributor Author

@ijuma I tracked this issue because of MESSAGE_TOO_LARGE by running command
bin/kafka-producer-perf-test.sh --topic test --num-records 50000000 --throughput -1 --record-size 5000 --producer-props bootstrap.servers=server04:9092 acks=1 buffer.memory=67108864 batch.size 65536 compression.type=zstd
But MESSAGE_TOO_LARGE doesn't occur every time. Could you please give me some suggestions about how to write the test? Thanks.

There are cases that currentEstimation is smaller than
COMPRESSION_RATIO_IMPROVING_STEP and it will get negative
estimatedCompressionRatio,which leads to misjudgment
about if there is no room and MESSAGE_TOO_LARGE might occur.

Change-Id: I0932a2a6ca669f673ab5d862d3fe7b2bb6d96ff6
Signed-off-by: Jiamei Xie <jiamei.xie@arm.com>
@jiameixie
Copy link
Copy Markdown
Contributor Author

@ijuma A test has been added. Please review it. Thanks.

@jiameixie jiameixie requested a review from ijuma March 13, 2020 09:58
@ijuma
Copy link
Copy Markdown
Member

ijuma commented Mar 13, 2020

ok to test

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 test. A few comments below.

// 2. currentEstimation < observedRatio && (currentEstimation + COMPRESSION_RATIO_DETERIORATE_STEP) > observedRatio
// 3. currentEstimation > observedRatio && (currentEstimation - COMPRESSION_RATIO_IMPROVING_STEP) > observedRatio
// 4. currentEstimation > observedRatio && (currentEstimation - COMPRESSION_RATIO_IMPROVING_STEP) < observedRatio
// In all cases, updatedCompressionRatio shouldn't smaller than observedRatio
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.

It seems like we are duplicating a lot of the logic of the code in this comment. This is likely to get stale over time. Can we make the comment more concise and refer to the non test code for more detail?

new EstimationsObservedRatios(0.8f, 0.84f, 0.84f),
new EstimationsObservedRatios(0.6f, 0.7f, 0.7f),
new EstimationsObservedRatios(0.6f, 0.4f, 0.4f),
new EstimationsObservedRatios(0.004f, 0.001f, 0.001f)
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.

It seems that expected is always the same as observed? If so, why do we have 3 parameters instead of 2?

// 3. currentEstimation > observedRatio && (currentEstimation - COMPRESSION_RATIO_IMPROVING_STEP) > observedRatio
// 4. currentEstimation > observedRatio && (currentEstimation - COMPRESSION_RATIO_IMPROVING_STEP) < observedRatio
// In all cases, updatedCompressionRatio shouldn't smaller than observedRatio
EstimationsObservedRatios[] currentEstimationsObservedRatios = new EstimationsObservedRatios[] {
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.

If we use a List, we should be able to simplify the logic below by using an enhanced for loop.

Signed-off-by: Jiamei Xie <jiamei.xie@arm.com>
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 update, a few minor comments below.

estimationsObservedRatios.add(new EstimationsObservedRatios(0.8f, 0.84f));
estimationsObservedRatios.add(new EstimationsObservedRatios(0.6f, 0.7f));
estimationsObservedRatios.add(new EstimationsObservedRatios(0.6f, 0.4f));
estimationsObservedRatios.add(new EstimationsObservedRatios(0.004f, 0.001f));
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.

Nit: this can be written more concisely by using Arrays.asList.

estimationsObservedRatios.add(new EstimationsObservedRatios(0.004f, 0.001f));
float updatedCompressionRatio;
for(EstimationsObservedRatios estimationsObservedRatio:estimationsObservedRatios)
{
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.

Nit: this brace should be on the previous line.

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Mar 20, 2020

ok to test

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Mar 21, 2020

retest this please

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Mar 21, 2020

ok to test

1 similar comment
@ijuma
Copy link
Copy Markdown
Member

ijuma commented Mar 23, 2020

ok to test

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Mar 23, 2020

retest this please

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Mar 23, 2020

Hmm, looks like org.apache.kafka.common.record.CompressionRatioEstimatorTest.testUpdateEstimation is failing. @jiameixie, can you please take a look?

Signed-off-by: Jiamei Xie <jiamei.xie@arm.com>
@jiameixie
Copy link
Copy Markdown
Contributor Author

@ijuma In my second commit, it was ">=". I am really sorry for my careless.

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Mar 24, 2020

retest this please

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Mar 24, 2020

retest this please

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Mar 25, 2020

One job passed and the other had a flaky test failure:

kafka.api.PlaintextConsumerTest.testLowMaxFetchSizeForRequestAndPartition

@ijuma ijuma merged commit b5409b9 into apache:trunk Mar 25, 2020
@ijuma
Copy link
Copy Markdown
Member

ijuma commented Mar 25, 2020

Merged to trunk and cherry picked to 2.5.

ijuma pushed a commit that referenced this pull request Mar 25, 2020
There are cases where `currentEstimation` is less than
`COMPRESSION_RATIO_IMPROVING_STEP` causing
`estimatedCompressionRatio` to be negative. This, in turn,
may result in `MESSAGE_TOO_LARGE`.

Reviewers: Ismael Juma <ismael@juma.me.uk>
@jiameixie jiameixie deleted the negative branch March 27, 2020 01:22
@ijuma
Copy link
Copy Markdown
Member

ijuma commented Apr 2, 2020

@jiameixie Quick question, were you seeing the MESSAGE_TOO_LARGE in the broker?

@jiameixie
Copy link
Copy Markdown
Contributor Author

@ijuma I saw MESSAGE_TOO_LARGE in producer client

MaximGonnissen added a commit to MaximGonnissen/kafka that referenced this pull request May 28, 2022
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.

2 participants