-
Notifications
You must be signed in to change notification settings - Fork 2.1k
[Feature][Kafka source] Inject Kafka record timestamp as EventTime metadata #9994
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
Open
Adamyuanyuan
wants to merge
10
commits into
apache:dev
Choose a base branch
from
Adamyuanyuan:dev-kafka-EventTime-1028
base: dev
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
22f9504
BDPL-25768 KafkaRecordEmitter
96199b8
BDPL-25768 Add UT for kafka event time
408a033
BDPL-25768 fix KafkaIT spotless
1e67117
BDPL-25768 fix KafkaIT e2e, adds that missing piece in KafkaSourceConfig
ea6b9cb
BDPL-25768 fix spotless
d28d529
BDPL-25768 disable SPARK engine e2e.
ff278d2
Merge branch 'apache:dev' into dev-kafka-EventTime-1028
Adamyuanyuan 896e8bb
BDPL-25768 Attach Kafka record timestamp into metadata only if the ro…
2de4141
Merge remote-tracking branch 'origin/dev-kafka-EventTime-1028' into d…
70d9a5d
Merge branch 'apache:dev' into dev-kafka-EventTime-1028
Adamyuanyuan File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
163 changes: 163 additions & 0 deletions
163
...t/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaRecordEmitterTest.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,163 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.seatunnel.connectors.seatunnel.kafka.source; | ||
|
|
||
| import org.apache.seatunnel.api.serialization.DeserializationSchema; | ||
| import org.apache.seatunnel.api.source.Collector; | ||
| import org.apache.seatunnel.api.table.catalog.TablePath; | ||
| import org.apache.seatunnel.api.table.type.BasicType; | ||
| import org.apache.seatunnel.api.table.type.CommonOptions; | ||
| import org.apache.seatunnel.api.table.type.SeaTunnelDataType; | ||
| import org.apache.seatunnel.api.table.type.SeaTunnelRow; | ||
| import org.apache.seatunnel.api.table.type.SeaTunnelRowType; | ||
| import org.apache.seatunnel.connectors.seatunnel.kafka.config.MessageFormatErrorHandleWay; | ||
|
|
||
| import org.apache.kafka.common.TopicPartition; | ||
|
|
||
| import org.junit.jupiter.api.Assertions; | ||
| import org.junit.jupiter.api.Test; | ||
| import org.mockito.Mockito; | ||
|
|
||
| import java.io.IOException; | ||
| import java.nio.charset.StandardCharsets; | ||
| import java.util.ArrayList; | ||
| import java.util.HashMap; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
|
|
||
| class KafkaRecordEmitterTest { | ||
|
|
||
| @Test | ||
| void emitRecordShouldAttachKafkaTimestampAsEventTime() throws Exception { | ||
| long kafkaTimestamp = 1690000000000L; | ||
|
|
||
| // Prepare a simple deserialization schema that creates a single-field row from bytes | ||
| SeaTunnelRowType rowType = | ||
| new SeaTunnelRowType( | ||
| new String[] {"f0"}, new SeaTunnelDataType[] {BasicType.STRING_TYPE}); | ||
| DeserializationSchema<SeaTunnelRow> schema = new SimpleStringRowSchema(rowType); | ||
|
|
||
| // Build ConsumerMetadata map for the table | ||
| ConsumerMetadata metadata = new ConsumerMetadata(); | ||
| metadata.setDeserializationSchema(schema); | ||
| Map<TablePath, ConsumerMetadata> map = new HashMap<>(); | ||
| TablePath tablePath = TablePath.DEFAULT; | ||
| map.put(tablePath, metadata); | ||
|
|
||
| KafkaRecordEmitter emitter = new KafkaRecordEmitter(map, MessageFormatErrorHandleWay.FAIL); | ||
|
|
||
| // Mock ConsumerRecord<byte[], byte[]> | ||
| org.apache.kafka.clients.consumer.ConsumerRecord<byte[], byte[]> record = | ||
| Mockito.mock(org.apache.kafka.clients.consumer.ConsumerRecord.class); | ||
| Mockito.when(record.timestamp()).thenReturn(kafkaTimestamp); | ||
| Mockito.when(record.value()).thenReturn("hello".getBytes(StandardCharsets.UTF_8)); | ||
| Mockito.when(record.offset()).thenReturn(100L); | ||
|
|
||
| // Prepare split state | ||
| KafkaSourceSplit split = new KafkaSourceSplit(tablePath, new TopicPartition("t", 0)); | ||
| KafkaSourceSplitState splitState = new KafkaSourceSplitState(split); | ||
|
|
||
| // Capture outputs | ||
| List<SeaTunnelRow> out = new ArrayList<>(); | ||
| Collector<SeaTunnelRow> collector = new TestCollector(out); | ||
|
|
||
| emitter.emitRecord(record, collector, splitState); | ||
|
|
||
| Assertions.assertEquals(1, out.size()); | ||
| SeaTunnelRow row = out.get(0); | ||
| Object eventTime = row.getOptions().get(CommonOptions.EVENT_TIME.getName()); | ||
| Assertions.assertEquals(kafkaTimestamp, eventTime); | ||
|
|
||
| // Also verify split state offset advanced | ||
| Assertions.assertEquals(101L, splitState.getCurrentOffset()); | ||
| } | ||
|
|
||
| @Test | ||
| void emitRecordShouldNotAttachEventTimeWhenTimestampNegative() throws Exception { | ||
| long kafkaTimestamp = -1L; // invalid timestamp | ||
|
|
||
| SeaTunnelRowType rowType = | ||
| new SeaTunnelRowType( | ||
| new String[] {"f0"}, new SeaTunnelDataType[] {BasicType.STRING_TYPE}); | ||
| DeserializationSchema<SeaTunnelRow> schema = new SimpleStringRowSchema(rowType); | ||
|
|
||
| ConsumerMetadata metadata = new ConsumerMetadata(); | ||
| metadata.setDeserializationSchema(schema); | ||
| Map<TablePath, ConsumerMetadata> map = new HashMap<>(); | ||
| TablePath tablePath = TablePath.DEFAULT; | ||
| map.put(tablePath, metadata); | ||
|
|
||
| KafkaRecordEmitter emitter = new KafkaRecordEmitter(map, MessageFormatErrorHandleWay.FAIL); | ||
|
|
||
| org.apache.kafka.clients.consumer.ConsumerRecord<byte[], byte[]> record = | ||
| Mockito.mock(org.apache.kafka.clients.consumer.ConsumerRecord.class); | ||
| Mockito.when(record.timestamp()).thenReturn(kafkaTimestamp); | ||
| Mockito.when(record.value()).thenReturn("world".getBytes(StandardCharsets.UTF_8)); | ||
| Mockito.when(record.offset()).thenReturn(5L); | ||
|
|
||
| KafkaSourceSplit split = new KafkaSourceSplit(tablePath, new TopicPartition("t2", 1)); | ||
| KafkaSourceSplitState splitState = new KafkaSourceSplitState(split); | ||
|
|
||
| List<SeaTunnelRow> out = new ArrayList<>(); | ||
| Collector<SeaTunnelRow> collector = new TestCollector(out); | ||
|
|
||
| emitter.emitRecord(record, collector, splitState); | ||
|
|
||
| Assertions.assertEquals(1, out.size()); | ||
| SeaTunnelRow row = out.get(0); | ||
| Assertions.assertFalse(row.getOptions().containsKey(CommonOptions.EVENT_TIME.getName())); | ||
| Assertions.assertEquals(6L, splitState.getCurrentOffset()); | ||
| } | ||
|
|
||
| private static class SimpleStringRowSchema implements DeserializationSchema<SeaTunnelRow> { | ||
| private final SeaTunnelRowType producedType; | ||
|
|
||
| private SimpleStringRowSchema(SeaTunnelRowType producedType) { | ||
| this.producedType = producedType; | ||
| } | ||
|
|
||
| @Override | ||
| public SeaTunnelRow deserialize(byte[] message) throws IOException { | ||
| String v = new String(message, StandardCharsets.UTF_8); | ||
| return new SeaTunnelRow(new Object[] {v}); | ||
| } | ||
|
|
||
| @Override | ||
| public SeaTunnelDataType<SeaTunnelRow> getProducedType() { | ||
| return producedType; | ||
| } | ||
| } | ||
|
|
||
| private static class TestCollector implements Collector<SeaTunnelRow> { | ||
| private final List<SeaTunnelRow> out; | ||
|
|
||
| private TestCollector(List<SeaTunnelRow> out) { | ||
| this.out = out; | ||
| } | ||
|
|
||
| @Override | ||
| public void collect(SeaTunnelRow record) { | ||
| out.add(record); | ||
| } | ||
|
|
||
| @Override | ||
| public Object getCheckpointLock() { | ||
| return this; | ||
| } | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
80 changes: 80 additions & 0 deletions
80
...afka-e2e/src/test/resources/textFormatIT/kafka_source_text_with_event_time_to_assert.conf
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,80 @@ | ||
| # | ||
| # Licensed to the Apache Software Foundation (ASF) under one or more | ||
| # contributor license agreements. See the NOTICE file distributed with | ||
| # this work for additional information regarding copyright ownership. | ||
| # The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| # (the "License"); you may not use this file except in compliance with | ||
| # the License. You may obtain a copy of the License at | ||
| # | ||
| # http://www.apache.org/licenses/LICENSE-2.0 | ||
| # | ||
| # Unless required by applicable law or agreed to in writing, software | ||
| # distributed under the License is distributed on an "AS IS" BASIS, | ||
| # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| # See the License for the specific language governing permissions and | ||
| # limitations under the License. | ||
| # | ||
|
|
||
| env { | ||
| parallelism = 1 | ||
| job.mode = "BATCH" | ||
| } | ||
|
|
||
| source { | ||
| Kafka { | ||
| bootstrap.servers = "kafkaCluster:9092" | ||
| topic = "test_topic_text_eventtime" | ||
| plugin_output = "kafka_table" | ||
| start_mode = "earliest" | ||
| format_error_handle_way = fail | ||
| schema = { | ||
| fields { | ||
| id = bigint | ||
| c_map = "map<string, smallint>" | ||
| c_array = "array<tinyint>" | ||
| c_string = string | ||
| c_boolean = boolean | ||
| c_tinyint = tinyint | ||
| c_smallint = smallint | ||
| c_int = int | ||
| c_bigint = bigint | ||
| c_float = float | ||
| c_double = double | ||
| c_decimal = "decimal(2, 1)" | ||
| c_bytes = bytes | ||
| c_date = date | ||
| c_timestamp = timestamp | ||
| } | ||
| } | ||
| format = text | ||
| field_delimiter = "," | ||
| } | ||
| } | ||
|
|
||
| transform { | ||
| Metadata { | ||
| plugin_input = "kafka_table" | ||
| plugin_output = "kafka_table_with_meta" | ||
| metadata_fields = { | ||
| EventTime = event_time_ms | ||
| } | ||
| } | ||
| } | ||
|
|
||
| sink { | ||
| Assert { | ||
| plugin_input = "kafka_table_with_meta" | ||
| rules = { | ||
| field_rules = [ | ||
| { | ||
| field_name = event_time_ms | ||
| field_type = bigint | ||
| field_value = [ | ||
| { rule_type = NOT_NULL } | ||
| ] | ||
| } | ||
| ] | ||
| } | ||
| } | ||
| } | ||
|
|
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
why not do it in
DeserializationSchema?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.
Thanks for the reminder. I'll check it later.