-
Notifications
You must be signed in to change notification settings - Fork 14k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-3025 Added timetamp to Message and use relative offset. #764
Changes from all commits
ce01646
d16091c
3b23383
813e0dd
b424c45
e76aa20
60108b0
8606c41
78b0936
0a1a0b8
b0ba4a0
1824bfa
c691fd0
b8e65cb
fd1afde
56fc79f
67201a6
92a39ba
acc7189
b38877f
def2673
a0f76bb
89958b9
c3ca748
085958e
9bd7f88
095d88d
648a457
d0772a3
a4313f4
0ec15f9
200d69b
99a4a27
22c74cd
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -12,6 +12,8 @@ | |
*/ | ||
package org.apache.kafka.clients.consumer; | ||
|
||
import org.apache.kafka.common.record.TimestampType; | ||
|
||
/** | ||
* A key/value pair to be received from Kafka. This consists of a topic name and a partition number, from which the | ||
* record is being received and an offset that points to the record in a Kafka partition. | ||
|
@@ -20,6 +22,8 @@ public final class ConsumerRecord<K, V> { | |
private final String topic; | ||
private final int partition; | ||
private final long offset; | ||
private final long timestamp; | ||
private final TimestampType timestampType; | ||
private final K key; | ||
private final V value; | ||
|
||
|
@@ -29,15 +33,25 @@ public final class ConsumerRecord<K, V> { | |
* @param topic The topic this record is received from | ||
* @param partition The partition of the topic this record is received from | ||
* @param offset The offset of this record in the corresponding Kafka partition | ||
* @param timestamp The timestamp of the record. | ||
* @param timestampType The timestamp type | ||
* @param key The key of the record, if one exists (null is allowed) | ||
* @param value The record contents | ||
*/ | ||
public ConsumerRecord(String topic, int partition, long offset, K key, V value) { | ||
public ConsumerRecord(String topic, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. how would you feel about adding a testing constructor for this class that matches the existing signature, and would just use dummy values for the timestamp and There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do you mean the unit tests in projects other than Kafka? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes. It's a minor change to update anywhere these constructors are used in unit tests, but any test that instantiates a There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am not sure if we should add the testing constructor for other projects. It is a little bit weird to have a testing constructor which never used by Kafka but for some unknown external project. Technically speaking ConsumerRecord should only be constructed by Kafka, but not other projects. If we do so, arguably we should maintain the constructor backward compatibility for any public class, even though most of them are not supposed to be constructed by any user. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. if |
||
int partition, | ||
long offset, | ||
long timestamp, | ||
TimestampType timestampType, | ||
K key, | ||
V value) { | ||
if (topic == null) | ||
throw new IllegalArgumentException("Topic cannot be null"); | ||
this.topic = topic; | ||
this.partition = partition; | ||
this.offset = offset; | ||
this.timestamp = timestamp; | ||
this.timestampType = timestampType; | ||
this.key = key; | ||
this.value = value; | ||
} | ||
|
@@ -77,9 +91,23 @@ public long offset() { | |
return offset; | ||
} | ||
|
||
/** | ||
* The timestamp of this record | ||
*/ | ||
public long timestamp() { | ||
return timestamp; | ||
} | ||
|
||
/** | ||
* The timestamp type of this record | ||
*/ | ||
public TimestampType timestampType() { | ||
return timestampType; | ||
} | ||
|
||
@Override | ||
public String toString() { | ||
return "ConsumerRecord(topic = " + topic() + ", partition = " + partition() + ", offset = " + offset() | ||
+ ", key = " + key + ", value = " + value + ")"; | ||
+ ", " + timestampType + " = " + timestamp + ", key = " + key + ", value = " + value + ")"; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -24,18 +24,25 @@ | |
public final class RecordMetadata { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Also would be good to add a comment to RecordMetadata class description what timestamp is actually returned (either set by client, producer, or broker). |
||
|
||
private final long offset; | ||
// The timestamp of the message. | ||
// If LogAppendTime is used for the topic, the timestamp will be the timestamp returned by the broker. | ||
// If CreateTime is used for the topic, the timestamp is the timestamp in the corresponding ProducerRecord if the | ||
// user provided one. Otherwise, it will be the producer local time when the producer record was handed to the | ||
// producer. | ||
private final long timestamp; | ||
private final TopicPartition topicPartition; | ||
|
||
private RecordMetadata(TopicPartition topicPartition, long offset) { | ||
private RecordMetadata(TopicPartition topicPartition, long offset, long timestamp) { | ||
super(); | ||
this.offset = offset; | ||
this.timestamp = timestamp; | ||
this.topicPartition = topicPartition; | ||
} | ||
|
||
public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relativeOffset) { | ||
public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relativeOffset, long timestamp) { | ||
// ignore the relativeOffset if the base offset is -1, | ||
// since this indicates the offset is unknown | ||
this(topicPartition, baseOffset == -1 ? baseOffset : baseOffset + relativeOffset); | ||
this(topicPartition, baseOffset == -1 ? baseOffset : baseOffset + relativeOffset, timestamp); | ||
} | ||
|
||
/** | ||
|
@@ -45,6 +52,13 @@ public long offset() { | |
return this.offset; | ||
} | ||
|
||
/** | ||
* The timestamp of the record in the topic/partition. | ||
*/ | ||
public long timestamp() { | ||
return timestamp; | ||
} | ||
|
||
/** | ||
* The topic the record was appended to | ||
*/ | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Could we add the doc for timestampType?