Skip to content
Closed
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
package org.apache.kafka.clients.producer;

import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.record.Record;

/**
* The metadata for a record that has been acknowledged by the server
Expand Down Expand Up @@ -51,6 +52,11 @@ private RecordMetadata(TopicPartition topicPartition, long offset, long timestam
this.topicPartition = topicPartition;
}

@Deprecated
public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relativeOffset) {
this(topicPartition, baseOffset, relativeOffset, Record.NO_TIMESTAMP, -1, -1, -1);
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.

Thanks. Any chance of adding record.NO_CHECKSUM and record.NO_SIZE?

We have several places in the code where we call the constructor with -1 and I think this will improve readability.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

I can add them if you like. Since they would only be used in the case of a deprecated method (or tests that don't care) I didn't add them thinking the would just be removed when the constructor is removed.

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.

Does this actually fix compatibility? I don't think we used relative offsets before, so even though it will compile, it won't behave correctly?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

@ijuma The behavior here matches that of 0.9 branch: https://github.com/apache/kafka/blob/0.9.0/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java#L35

I am not sure how or why people would be using this constructor, perhaps in unit tests or something, but I would prefer to keep it compatible and follow a deprecation cycle since its public.

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.

I agree we should keep it, I was just unsure if semantics had been maintained. Seems like it's fine. Thanks for the references.

}

public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relativeOffset,
long timestamp, long checksum, int serializedKeySize, int serializedValueSize) {
// ignore the relativeOffset if the base offset is -1,
Expand Down