Skip to content

KAFKA-10888: Sticky partition leads to uneven produce msg#12049

Merged
junrao merged 7 commits intoapache:trunkfrom
artemlivshits:KAFKA-10888
May 6, 2022
Merged

KAFKA-10888: Sticky partition leads to uneven produce msg#12049
junrao merged 7 commits intoapache:trunkfrom
artemlivshits:KAFKA-10888

Conversation

@artemlivshits
Copy link
Contributor

The design is described in detail in KIP-794
https://cwiki.apache.org/confluence/display/KAFKA/KIP-794%3A+Strictly+Uniform+Sticky+Partitioner.

Implementation notes:

The default partitioning logic is moved to the BuiltInPartitioner class
(there is one object per topic). The object keeps track of how many
bytes are produced per-partition and once the amount exceeds batch.size,
switches to the next partition (note that partition switch decision is
decoupled from batching). The object also keeps track of probability
weights that are based on the queue sizes (the larger the queue size
is the less chance for the next partition to be chosen). The queue
sizes are calculated in the RecordAccumulator in the ready method,
the method already enumerates all partitions so we just add some extra
logic into the existing O(N) method. The partition switch decision may
take O(logN), where N is the number partitions per topic, but it happens
only once per batch.size (and the logic is avoided when all queues are
of equal size). Produce bytes accounting logic is lock-free.

When partitioner.availability.timeout.ms is non-0, RecordAccumulator
keeps stats on "node latency" which is defined as the difference between
the last time the node had a batch waiting to be send and the last time
the node was ready to take a new batch. If this difference exceeds
partitioner.availability.timeout.ms we don't switch to that partition
until the node is ready.

The corresponding unit tests are added / modified.

The perf test results are in the KIP-794.

Committer Checklist (excluded from commit message)

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

Copy link
Contributor Author

@artemlivshits artemlivshits left a comment

Choose a reason for hiding this comment

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

Added some annotation to explain code movements.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Just added recordInfo here.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Instead of flat map {tp -> batches}, it's now a map of maps {topic -> {partition -> {batches, etc.}}, for a couple reasons:

  1. For queue size statistics calculation is more efficient if partitions are grouped by topic.
  2. There are other per-topic data (partitioner info).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is mostly just moving part of the logic from append to a separate function.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Just a new argument.

Comment on lines +432 to +438
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Now it's a different loop because it's map of maps, but the body is the same.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Changes here is just refactoring of existing logic.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Needed to extend the mock to call the setPartition callback.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This and testAdaptiveBuiltInPartitioner are new additions, the rest is just tweaks to reflect changes in function signature.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is a new addition, the rest is just updates to reflect function signatures / callback changes.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Changes here are just to reflect function signature changes.

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@artemlivshits : Thanks for the PR. Made a pass of non-testing files. A few comments below.

Copy link
Contributor

Choose a reason for hiding this comment

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

This can be a bit confusing since we don't run the full default partitioning logic here. Also, we probably want to refer to it as built-in partitioning since we haven't removed DefaultPartitioner.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Rephrased.

Copy link
Contributor

Choose a reason for hiding this comment

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

Could we just fold the logic in ProducerInterceptors.extractTopicPartition to topicPartition()?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sure.

Copy link
Contributor

Choose a reason for hiding this comment

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

Could we add a comment that this is for testing purpose?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sure.

Copy link
Contributor

Choose a reason for hiding this comment

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

We typically don't use getter. So this can just be loadStatsRangeEnd.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ok.

Copy link
Contributor

Choose a reason for hiding this comment

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

Could we explain the difference between length and queueSizes.length here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sure,

Copy link
Contributor

Choose a reason for hiding this comment

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

We are adding additional logic to estimate the accumulated bytes, which duplicates some of the code in RecordAccumulator. I am wondering if we could track the accumulated bytes by reusing the estimation logic inside RecordAccumulator. All appends to a partition are already synchronized on the corresponding dq in RecordAccumulator. Maybe we could do the following.

  1. To append to RecordAccumulator, we find the current sticky partition instance and call tryAppend under dq lock.
  2. After the successful append, we update the accumulated bytes for this partition based on MemoryRecordsBuilder.estimatedBytesWritten(). If the accumulated bytes exceeds the sticky batch size, we call BuiltInPartitioner.nextPartition() to switch partition.
  3. A concurrent send() call may try to append to the previous partition after the partition is switched. We could have the caller pass in a reference to the expected partition instance to tryAppend(). Inside tryAppend), we could then check if the expected partition is the same as the current sticky partition. If not, the caller needs to obtain the current sticky partition and retry tryAppend().

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think it could work, let me try to implement it. In the meanwhile, I've addressed all other comments.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Re-implemented this logic as suggested 0013801.

Copy link
Contributor

Choose a reason for hiding this comment

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

Could we annotate this method as deprecated?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

Copy link
Contributor

Choose a reason for hiding this comment

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

extra new line

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ok

Copy link
Contributor

Choose a reason for hiding this comment

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

extra new line

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ok

Copy link
Contributor

Choose a reason for hiding this comment

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

Since this is a public class, could we annotate it as deprecated?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

You've probably meant to annotate the DefaultPartitioner.

Copy link
Contributor Author

@artemlivshits artemlivshits left a comment

Choose a reason for hiding this comment

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

@junrao thank you for review. I've addressed most comments except for changing record size estimator, I'll experiment and see if it works out.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Rephrased.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sure.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

You've probably meant to annotate the DefaultPartitioner.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sure.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ok.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Renamed to cumulativeFrequencyTable.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ok

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ok

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think it could work, let me try to implement it. In the meanwhile, I've addressed all other comments.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This may go away if changing estimation logic works out.

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@artemlivshits : Thanks for the updated PR. Made a pass of all files. A few more comments.

Copy link
Contributor

Choose a reason for hiding this comment

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

This comment can be a bit confusing since we always call deallocate in the finally block.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Rephrased.

Copy link
Contributor

Choose a reason for hiding this comment

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

It seems this method doesn't throw InterruptedException?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Removed.

Copy link
Contributor

Choose a reason for hiding this comment

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

We used to check if the producer is closed here. Is that still needed?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This check is moved to tryAppend.

Copy link
Contributor

Choose a reason for hiding this comment

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

Should we assert that partitionInfo is not null?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It can be null in the case we didn't use built-in partitioner.

Copy link
Contributor

Choose a reason for hiding this comment

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

Should we assert that partitionInfo is not null?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It can be null in the case we didn't use built-in partitioner.

Copy link
Contributor

Choose a reason for hiding this comment

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

EMPTY_HEADERS is no longer referenced.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Removed.

Copy link
Contributor

Choose a reason for hiding this comment

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

In the adaptive case, it seems that we should also avoid selecting partitions with no leader, if possible?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

When we build the table, we exclude partitions with no leaders. Added a comment.

Copy link
Contributor

Choose a reason for hiding this comment

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

Do we need an array since we only return one partition?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Array is a way to capture the result from the callback.

Copy link
Contributor

Choose a reason for hiding this comment

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

Perhaps it's more intuitive to use an AtomicInteger?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sure

Copy link
Contributor

Choose a reason for hiding this comment

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

extra empty line.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Removed.

Copy link
Contributor

Choose a reason for hiding this comment

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

Node 1 not ready?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed.

Copy link
Contributor Author

@artemlivshits artemlivshits left a comment

Choose a reason for hiding this comment

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

@junrao thank you for comments, addressed.

Copy link
Contributor Author

@artemlivshits artemlivshits Apr 28, 2022

Choose a reason for hiding this comment

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

Yeah, that's a good question. The MockProducer doesn't have RecordAccumulator, so I just left the code as it was. Interestingly, this code never calls "onNewBatch", so the default partitioner never switches partition. But I think we should fix it as a separate issue.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

When we build the table, we exclude partitions with no leaders. Added a comment.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It can be null in the case we didn't use built-in partitioner.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It can be null in the case we didn't use built-in partitioner.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Removed.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Updated javadoc to use fully qualified name.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I presume this would only work for keyed messages (partition is determined by the hash) because onNewBatch is never called, so a sticky partition would never switch. We should probably fix it, but as a separate change.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good point, added some tracing.

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@artemlivshits : Thanks for the updated PR. Just a few more minor comments. A couple of other things.

  1. Could you rebase?
  2. It would be useful to see if we need to add any new metrics. For example, it might be useful to have a metric that tracks the number of nodes excluded due to high latency. This can be done in a separate PR.

Copy link
Contributor

Choose a reason for hiding this comment

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

Was this comment addressed?

Copy link
Contributor

Choose a reason for hiding this comment

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

It seems that by default, kstreams just forwards the partitioning logic to DefaultPartitioner (which implements onNewBatch) and it should work for both keyed and non-keyed messages. Since the new built-in partition is strictly better, it seems that we should just switch the default to the built-in partitioner. I agree that we could do that in a separate PR.

Copy link
Contributor

Choose a reason for hiding this comment

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

DefaultPartitioner implements onNewBatch, right? I agree that we could address this in a separate PR.

Copy link
Contributor

Choose a reason for hiding this comment

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

Perhaps it's better to do the logging once inside nextPartition() instead of here and updatePartitionInfo()?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

Copy link
Contributor

Choose a reason for hiding this comment

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

Could we log what the meaning of the two lengths?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Added.

Copy link
Contributor

Choose a reason for hiding this comment

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

Could we add a comment that partitions w/o leader are excluded from the input arrays?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Added comments in params.

Copy link
Contributor

Choose a reason for hiding this comment

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

Perhaps it's more intuitive to use an AtomicInteger?

The design is described in detail in KIP-794
https://cwiki.apache.org/confluence/display/KAFKA/KIP-794%3A+Strictly+Uniform+Sticky+Partitioner.

Implementation notes:

The default partitioning logic is moved to the BuiltInPartitioner class
(there is one object per topic).  The object keeps track of how many
bytes are produced per-partition and once the amount exceeds batch.size,
switches to the next partition (note that partition switch decision is
decoupled from batching).  The object also keeps track of probability
weights that are based on the queue sizes (the larger the queue size
is the less chance for the next partition to be chosen).  The queue
sizes are calculated in the RecordAccumulator in the `ready` method,
the method already enumerates all partitions so we just add some extra
logic into the existing O(N) method.  The partition switch decision may
take O(logN), where N is the number partitions per topic, but it happens
only once per batch.size (and the logic is avoided when all queues are
of equal size).  Produce bytes accounting logic is lock-free.

When partitioner.availability.timeout.ms is non-0, RecordAccumulator
keeps stats on "node latency" which is defined as the difference between
the last time the node had a batch waiting to be send and the last time
the node was ready to take a new batch.  If this difference exceeds
partitioner.availability.timeout.ms we don't switch to that partition
until the node is ready.
Address review comments except for size estimation.
Re-implement appendedSize accounting as suggested by Jun.
Copy link
Contributor Author

@artemlivshits artemlivshits left a comment

Choose a reason for hiding this comment

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

@junrao thank you for review, addressed comments and rebased. Will file a ticket to add metrics once this PR is merged.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Missed, now rephrased.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

DefaultPartitioner implements onNewBatch, but MockProducer doesn't call it. It only calls .partition, which for unkeyed messages returns the same partition until the onNewBatch is called.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

DefaultPartitioner implements onNewBatch, but DefaultStreamPartitioner doesn't seem to ever call it (the DefaultPartitioner is a private object). Without onNewBatch, the DefaultPartitioner.partition would return the same partition for unkeyed messages.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Added comments in params.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Added.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sure

@artemlivshits
Copy link
Contributor Author

artemlivshits commented May 4, 2022

Hmm, for some reason the update didn't trigger any build & test jobs [Update: they started after some time]

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@artemlivshits : Thanks for the latest PR. LGTM. Do you have additional test results to share?

Copy link
Contributor

Choose a reason for hiding this comment

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

Hmm, onNewBatch() in DefaultPartitioner is not called by the application code. It's called inside Sender.

Converting DefaultStreamPartitioner to the new built-in partitioner is a bit tricky since kstreams calls the partitioner explicitly and then passes in the partition during send. We need to change kstreams code so that it lets the producer determine the partition. This can be done in a separate PR. cc @guozhangwang

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

One more minor comment.

}

public TopicPartition topicPartition() {
if (record == null)
Copy link
Contributor

Choose a reason for hiding this comment

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

We have a bunch of code in send() that depends on record being not null. Perhaps it's better to assert non-null record early in send()?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I've added it because there is a test that passes a null and expects some error handling. From that perspective, adding an assert would contradict the expectation that there is defined and tested error handling (in my mind, an assert means the behavior is undefined and all bets are off).

Copy link
Contributor

Choose a reason for hiding this comment

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

Hmm, by returning a null, we are breaking the contract that RecordMetadata.topic() and RecordMetadata.partition() is never null in producer callback and interceptor. Is that a new or existing test?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is an existing test https://github.com/apache/kafka/blob/trunk/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala#L1041
It passes in null and expects that the error goes through the interceptor's callback, but topicPartition() is called to supply an argument to the callback (which can handle null topicPartition) and it was throwing NPE.
I personally would just declare record as non-null and treat it as a bug in the application (undefined behavior), but as defined currently, it's a tested run-time error condition with some expected behavior.

Copy link
Contributor

Choose a reason for hiding this comment

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

Thanks for the explanation. Since this is the existing behavior, we could keep the code as it is in the PR and improve it in the future.

* @return sticky partition info object
*/
StickyPartitionInfo peekCurrentPartitionInfo(Cluster cluster) {
StickyPartitionInfo partitionInfo = stickyPartitionInfo.get();
Copy link
Contributor

Choose a reason for hiding this comment

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

This is an existing issue with the DefaultPartitioner. It seems that if the sticky partition loses the leader, we will continue sending new records to this partition. Perhaps it's better to force the partition selection in that case. This can be addressed separately.

Copy link
Contributor Author

@artemlivshits artemlivshits left a comment

Choose a reason for hiding this comment

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

@junrao I've re-done perf testing, the results are pretty much the same as published in the KIP-794.

}

public TopicPartition topicPartition() {
if (record == null)
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I've added it because there is a test that passes a null and expects some error handling. From that perspective, adding an assert would contradict the expectation that there is defined and tested error handling (in my mind, an assert means the behavior is undefined and all bets are off).

@junrao junrao merged commit f7db603 into apache:trunk May 6, 2022
jnh5y added a commit to jnh5y/kafka that referenced this pull request Jun 15, 2022
vvcephei added a commit that referenced this pull request Jun 24, 2022
vvcephei added a commit that referenced this pull request Jun 24, 2022
vvcephei added a commit to confluentinc/kafka that referenced this pull request Jun 24, 2022
assert partition != RecordMetadata.UNKNOWN_PARTITION;

// Update the current time in case the buffer allocation blocked above.
long nowMs = time.milliseconds();
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hmm, this call now happens under the queue lock, which might be the root cause for KAFKA-14020. Will move it outside, just after allocation of new buffer at line 312.

Copy link
Contributor

Choose a reason for hiding this comment

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

It seems this line doesn't need to be in the synchronized part. It just need to be after the free.allocate() call. However, this code is only called very time when a new batch is created.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yep, the previous code had it just after the allocate and while moving code around I accidentally moved it under the lock. Fix #12365.

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