Skip to content
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

feat: added support for message timestamp #915

Merged
merged 16 commits into from
Jul 23, 2024
Merged
Show file tree
Hide file tree
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
3 changes: 3 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,9 @@
## 0.30.0

* Dependency updates (Kafka 3.7.1, Vert.x 4.5.9, Netty 4.1.111.Final, Prometheus JMX Collector 1.0.1, Prometheus Client 1.3.1)
* Added support for message timestamp.
* Setting the timestamp on a message sent via the `send` API.
* Getting the timestamp on receiving a message via the `poll` API.

## 0.29.0

Expand Down
4 changes: 4 additions & 0 deletions documentation/book/api/definitions.adoc
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,8 @@ __optional__|<<_kafkaheaderlist,KafkaHeaderList>>
__optional__|integer (int64)
|**partition** +
__optional__|integer (int32)
|**timestamp** +
__optional__|integer (int64)
|**topic** +
__optional__|string
|===
Expand Down Expand Up @@ -222,6 +224,8 @@ __optional__|< <<_partition,Partition>> > array
__optional__|<<_kafkaheaderlist,KafkaHeaderList>>
|**partition** +
__optional__|integer (int32)
|**timestamp** +
__optional__|integer (int64)
|===


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,14 @@
/**
* Implementation of a message converter to deal with the "binary" embedded data format
*/
@SuppressWarnings("checkstyle:NPathComplexity")
public class HttpBinaryMessageConverter implements MessageConverter<byte[], byte[], byte[], byte[]> {

@Override
public ProducerRecord<byte[], byte[]> toKafkaRecord(String kafkaTopic, Integer partition, byte[] message) {

Integer partitionFromBody = null;
Long timestamp = null;
byte[] key = null;
byte[] value = null;
Headers headers = new RecordHeaders();
Expand All @@ -43,6 +45,9 @@ public ProducerRecord<byte[], byte[]> toKafkaRecord(String kafkaTopic, Integer p
if (json.has("value")) {
value = DatatypeConverter.parseBase64Binary(json.get("value").asText());
}
if (json.has("timestamp")) {
timestamp = json.get("timestamp").asLong();
}
if (json.has("headers")) {
ArrayNode jsonArray = (ArrayNode) json.get("headers");
for (JsonNode jsonObject: jsonArray) {
Expand All @@ -59,7 +64,7 @@ public ProducerRecord<byte[], byte[]> toKafkaRecord(String kafkaTopic, Integer p
partitionFromBody = partition;
}
}
return new ProducerRecord<>(kafkaTopic, partitionFromBody, key, value, headers);
return new ProducerRecord<>(kafkaTopic, partitionFromBody, timestamp, key, value, headers);
}

@Override
Expand Down Expand Up @@ -98,6 +103,7 @@ public byte[] toMessages(ConsumerRecords<byte[], byte[]> records) {
DatatypeConverter.printBase64Binary(record.value()) : null);
jsonObject.put("partition", record.partition());
jsonObject.put("offset", record.offset());
jsonObject.put("timestamp", record.timestamp());

ArrayNode headers = JsonUtils.createArrayNode();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,14 @@
/**
* Implementation of a message converter to deal with the "json" embedded data format
*/
@SuppressWarnings("checkstyle:NPathComplexity")
public class HttpJsonMessageConverter implements MessageConverter<byte[], byte[], byte[], byte[]> {

@Override
public ProducerRecord<byte[], byte[]> toKafkaRecord(String kafkaTopic, Integer partition, byte[] message) {

Integer partitionFromBody = null;
Long timestamp = null;
byte[] key = null;
byte[] value = null;
Headers headers = new RecordHeaders();
Expand All @@ -43,9 +45,12 @@ public ProducerRecord<byte[], byte[]> toKafkaRecord(String kafkaTopic, Integer p
if (json.has("value")) {
value = JsonUtils.jsonToBytes(json.get("value"));
}
if (json.has("timestamp")) {
timestamp = json.get("timestamp").asLong();
}
if (json.has("headers")) {
ArrayNode jsonArray = (ArrayNode) json.get("headers");
for (JsonNode jsonObject: jsonArray) {
for (JsonNode jsonObject : jsonArray) {
headers.add(new RecordHeader(jsonObject.get("key").asText(), DatatypeConverter.parseBase64Binary(jsonObject.get("value").asText())));
}
}
Expand All @@ -59,7 +64,7 @@ public ProducerRecord<byte[], byte[]> toKafkaRecord(String kafkaTopic, Integer p
partitionFromBody = partition;
}
}
return new ProducerRecord<>(kafkaTopic, partitionFromBody, key, value, headers);
return new ProducerRecord<>(kafkaTopic, partitionFromBody, timestamp, key, value, headers);
}

@Override
Expand Down Expand Up @@ -97,10 +102,11 @@ public byte[] toMessages(ConsumerRecords<byte[], byte[]> records) {
JsonUtils.bytesToJson(record.value()) : null);
jsonObject.put("partition", record.partition());
jsonObject.put("offset", record.offset());
jsonObject.put("timestamp", record.timestamp());

ArrayNode headers = JsonUtils.createArrayNode();

for (Header kafkaHeader: record.headers()) {
for (Header kafkaHeader : record.headers()) {
ObjectNode header = JsonUtils.createObjectNode();

header.put("key", kafkaHeader.key());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ public class HttpTextMessageConverter implements MessageConverter<byte[], byte[]
public ProducerRecord<byte[], byte[]> toKafkaRecord(String kafkaTopic, Integer partition, byte[] message) {

Integer partitionFromBody = null;
Long timestamp = null;
byte[] key = null;
byte[] value = null;
Headers headers = new RecordHeaders();
Expand All @@ -51,6 +52,9 @@ public ProducerRecord<byte[], byte[]> toKafkaRecord(String kafkaTopic, Integer p
}
value = valueNode.asText().getBytes();
}
if (json.has("timestamp")) {
timestamp = json.get("timestamp").asLong();
}
if (json.has("headers")) {
ArrayNode jsonArray = (ArrayNode) json.get("headers");
for (JsonNode jsonObject : jsonArray) {
Expand All @@ -67,7 +71,7 @@ public ProducerRecord<byte[], byte[]> toKafkaRecord(String kafkaTopic, Integer p
partitionFromBody = partition;
}
}
return new ProducerRecord<>(kafkaTopic, partitionFromBody, key, value, headers);
return new ProducerRecord<>(kafkaTopic, partitionFromBody, timestamp, key, value, headers);
}

@Override
Expand Down Expand Up @@ -101,6 +105,7 @@ public byte[] toMessages(ConsumerRecords<byte[], byte[]> records) {
jsonObject.set("value", record.value() != null ? new TextNode(new String(record.value())) : null);
jsonObject.put("partition", record.partition());
jsonObject.put("offset", record.offset());
jsonObject.put("timestamp", record.timestamp());

ArrayNode headers = JsonUtils.createArrayNode();

Expand Down
22 changes: 18 additions & 4 deletions src/main/resources/openapi.json
Original file line number Diff line number Diff line change
Expand Up @@ -778,7 +778,8 @@
"foo": "bar"
},
"partition": 0,
"offset": 2
"offset": 2,
"timestamp": 1591897790000
},
{
"topic": "topic",
Expand All @@ -788,7 +789,8 @@
"bar2"
],
"partition": 1,
"offset": 3
"offset": 3,
"timestamp": 1591897790002
}
]
}
Expand Down Expand Up @@ -1516,6 +1518,7 @@
"partition": 0,
"topic": "topic",
"value": "value1",
"timestamp": 1591897790000,
"headers": [
{
"key": "key1",
Expand Down Expand Up @@ -1566,6 +1569,10 @@
},
"headers": {
"$ref": "#/components/schemas/KafkaHeaderList"
},
"timestamp": {
"format": "int64",
"type": "integer"
}
},
"title": "ConsumerRecord",
Expand All @@ -1583,14 +1590,16 @@
"key": "key1",
"value": "value1",
"partition": 0,
"offset": 2
"offset": 2,
"timestamp": 1591897790000
},
{
"topic": "topic",
"key": "key2",
"value": "value2",
"partition": 1,
"offset": 3
"offset": 3,
"timestamp": 1591897790000
}
]
},
Expand Down Expand Up @@ -1846,6 +1855,10 @@
"format": "int32",
"type": "integer"
},
"timestamp": {
"format": "int64",
"type": "integer"
},
"value": {
"oneOf": [
{
Expand Down Expand Up @@ -1882,6 +1895,7 @@
"example": {
"key": "key1",
"partition": 0,
"timestamp": 1591897790000,
"value": "value1",
"headers": [
{
Expand Down
16 changes: 14 additions & 2 deletions src/main/resources/openapiv2.json
Original file line number Diff line number Diff line change
Expand Up @@ -1351,6 +1351,7 @@
"partition": 0,
"topic": "topic",
"value": "value1",
"timestamp": 1591897790000,
"headers": [
{
"key": "key1",
Expand Down Expand Up @@ -1391,6 +1392,10 @@
},
"headers": {
"$ref": "#/definitions/KafkaHeaderList"
},
"timestamp": {
"format": "int64",
"type": "integer"
}
},
"title": "ConsumerRecord",
Expand All @@ -1408,14 +1413,16 @@
"key": "key1",
"value": "value1",
"partition": 0,
"offset": 2
"offset": 2,
"timestamp": 1591897790000
},
{
"topic": "topic",
"key": "key2",
"value": "value2",
"partition": 1,
"offset": 3
"offset": 3,
"timestamp": 1591897790002
}
]
},
Expand Down Expand Up @@ -1660,6 +1667,10 @@
"format": "int32",
"type": "integer"
},
"timestamp": {
"format": "int64",
"type": "integer"
},
"value": {
"type": [
"array",
Expand All @@ -1683,6 +1694,7 @@
"example": {
"key": "key1",
"partition": 0,
"timestamp": 1591897790000,
"value": "value1",
"headers": [
{
Expand Down
Loading
Loading