Skip to content

Commit 41f0c87

Browse files
author
dapeng
committed
kafka 按照time消费
1 parent 56df6ec commit 41f0c87

File tree

8 files changed

+26
-6
lines changed

8 files changed

+26
-6
lines changed

kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/AbstractKafkaSource.java

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -54,6 +54,8 @@ protected Properties getKafkaProperties(KafkaSourceTableInfo kafkaSourceTableInf
5454

5555
if (DtStringUtil.isJson(kafkaSourceTableInfo.getOffsetReset())) {
5656
props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, EKafkaOffset.NONE.name().toLowerCase());
57+
} else if(StringUtils.equalsIgnoreCase(EKafkaOffset.TIMESTAMP.name().toLowerCase(), kafkaSourceTableInfo.getOffsetReset())){
58+
props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, EKafkaOffset.EARLIEST.name().toLowerCase());
5759
} else {
5860
props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, kafkaSourceTableInfo.getOffsetReset());
5961
}
@@ -95,10 +97,12 @@ protected void setParallelism(Integer parallelism, DataStreamSource kafkaSource)
9597
}
9698
}
9799

98-
protected void setStartPosition(String offset, String topicName, FlinkKafkaConsumerBase<Row> kafkaSrc) {
100+
protected void setStartPosition(String offset, String topicName, FlinkKafkaConsumerBase<Row> kafkaSrc, Runnable runnable) {
99101
if (StringUtils.equalsIgnoreCase(offset, EKafkaOffset.EARLIEST.name())) {
100102
kafkaSrc.setStartFromEarliest();
101-
} else if (DtStringUtil.isJson(offset)) {
103+
} else if(StringUtils.equalsIgnoreCase(offset, EKafkaOffset.TIMESTAMP.name())) {
104+
runnable.run();
105+
}else if (DtStringUtil.isJson(offset)) {
102106
Map<KafkaTopicPartition, Long> specificStartupOffsets = buildOffsetMap(offset, topicName);
103107
kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets);
104108
} else {

kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/enums/EKafkaOffset.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,5 +27,6 @@ public enum EKafkaOffset {
2727

2828
LATEST,
2929
EARLIEST,
30+
TIMESTAMP,
3031
NONE
3132
}

kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -56,6 +56,9 @@ public AbstractTableInfo getTableInfo(String tableName, String fieldsInfo, Map<S
5656
kafkaSourceTableInfo.setFieldDelimiter(MathUtil.getString(props.getOrDefault(KafkaSourceTableInfo.CSV_FIELD_DELIMITER_KEY.toLowerCase(), "|")));
5757
kafkaSourceTableInfo.setSourceDataType(MathUtil.getString(props.getOrDefault(KafkaSourceTableInfo.SOURCE_DATA_TYPE_KEY.toLowerCase(), FormatType.DT_NEST.name())));
5858

59+
if(props.containsKey(KafkaSourceTableInfo.TIMESTAMP_OFFSET.toLowerCase())){
60+
kafkaSourceTableInfo.setTimestampOffset(MathUtil.getLongVal(props.getOrDefault(KafkaSourceTableInfo.TIMESTAMP_OFFSET.toLowerCase(), System.currentTimeMillis())));
61+
}
5962
Map<String, String> kafkaParams = props.keySet().stream()
6063
.filter(key -> !key.isEmpty() && key.startsWith("kafka."))
6164
.collect(Collectors.toMap(

kafka-base/kafka-base-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -55,6 +55,8 @@ public class KafkaSourceTableInfo extends AbstractSourceTableInfo {
5555

5656
public static final String CHARSET_NAME_KEY = "charsetName";
5757

58+
public static final String TIMESTAMP_OFFSET = "timestampOffset";
59+
5860
private String bootstrapServers;
5961

6062
private String topic;
@@ -75,6 +77,8 @@ public class KafkaSourceTableInfo extends AbstractSourceTableInfo {
7577

7678
public String charsetName;
7779

80+
private Long timestampOffset;
81+
7882
public String getBootstrapServers() {
7983
return bootstrapServers;
8084
}
@@ -159,6 +163,14 @@ public void setCharsetName(String charsetName) {
159163
this.charsetName = charsetName;
160164
}
161165

166+
public Long getTimestampOffset() {
167+
return timestampOffset;
168+
}
169+
170+
public void setTimestampOffset(Long timestampOffset) {
171+
this.timestampOffset = timestampOffset;
172+
}
173+
162174
@Override
163175
public boolean check() {
164176
Preconditions.checkNotNull(getType(), "kafka of type is required");

kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -51,7 +51,7 @@ public Table genStreamSource(AbstractSourceTableInfo sourceTableInfo, StreamExec
5151
DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation);
5252

5353
setParallelism(kafkaSourceTableInfo.getParallelism(), kafkaSource);
54-
setStartPosition(kafkaSourceTableInfo.getOffsetReset(), topicName, kafkaSrc);
54+
setStartPosition(kafkaSourceTableInfo.getOffsetReset(), topicName, kafkaSrc, () -> kafkaSrc.setStartFromTimestamp(kafkaSourceTableInfo.getTimestampOffset()));
5555
String fields = StringUtils.join(kafkaSourceTableInfo.getFields(), ",");
5656

5757
return tableEnv.fromDataStream(kafkaSource, fields);

kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -52,7 +52,7 @@ public Table genStreamSource(AbstractSourceTableInfo sourceTableInfo, StreamExec
5252
DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation);
5353

5454
setParallelism(kafkaSourceTableInfo.getParallelism(), kafkaSource);
55-
setStartPosition(kafkaSourceTableInfo.getOffsetReset(), topicName, kafkaSrc);
55+
setStartPosition(kafkaSourceTableInfo.getOffsetReset(), topicName, kafkaSrc, () -> kafkaSrc.setStartFromLatest());
5656
String fields = StringUtils.join(kafkaSourceTableInfo.getFields(), ",");
5757

5858
return tableEnv.fromDataStream(kafkaSource, fields);

kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -55,7 +55,7 @@ public Table genStreamSource(AbstractSourceTableInfo sourceTableInfo, StreamExec
5555
DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation);
5656

5757
setParallelism(kafkaSourceTableInfo.getParallelism(), kafkaSource);
58-
setStartPosition(kafkaSourceTableInfo.getOffsetReset(), topicName, kafkaSrc);
58+
setStartPosition(kafkaSourceTableInfo.getOffsetReset(), topicName, kafkaSrc, () -> kafkaSrc.setStartFromTimestamp(kafkaSourceTableInfo.getTimestampOffset()));
5959
String fields = StringUtils.join(kafkaSourceTableInfo.getFields(), ",");
6060

6161
return tableEnv.fromDataStream(kafkaSource, fields);

kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -54,7 +54,7 @@ public Table genStreamSource(AbstractSourceTableInfo sourceTableInfo, StreamExec
5454
DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation);
5555

5656
setParallelism(kafkaSourceTableInfo.getParallelism(), kafkaSource);
57-
setStartPosition(kafkaSourceTableInfo.getOffsetReset(), topicName, kafkaSrc);
57+
setStartPosition(kafkaSourceTableInfo.getOffsetReset(), topicName, kafkaSrc, () -> kafkaSrc.setStartFromTimestamp(kafkaSourceTableInfo.getTimestampOffset()));
5858
String fields = StringUtils.join(kafkaSourceTableInfo.getFields(), ",");
5959

6060
return tableEnv.fromDataStream(kafkaSource, fields);

0 commit comments

Comments
 (0)