ferenc-csaky commented on code in PR #250:
URL:
https://github.com/apache/flink-connector-kafka/pull/250#discussion_r3233132311
##########
.github/workflows/push_pr.yml:
##########
@@ -28,7 +28,7 @@ jobs:
compile_and_test:
strategy:
matrix:
- flink: &flink_versions [ 2.2.0, 2.1.1 ]
+ flink: &flink_versions [ 2.2.0 ]
Review Comment:
nit: 2.2.1 is released, i'd bump to that
##########
pom.xml:
##########
@@ -53,7 +53,7 @@ under the License.
<!-- Main Dependencies -->
<confluent.version>7.9.2</confluent.version>
- <flink.version>2.1.0</flink.version>
+ <flink.version>2.2.0</flink.version>
Review Comment:
nit: 2.2.1 is released, i'd bump to that
##########
flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/GetKafkaSourceOffsetsTableFunction.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.flink.streaming.connectors.kafka.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
+import
org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitSerializer;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.runtime.checkpoint.OperatorState;
+import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
+import org.apache.flink.runtime.checkpoint.metadata.CheckpointMetadata;
+import org.apache.flink.runtime.state.DefaultOperatorStateBackend;
+import org.apache.flink.runtime.state.DefaultOperatorStateBackendBuilder;
+import org.apache.flink.runtime.state.OperatorStateHandle;
+import org.apache.flink.state.api.OperatorIdentifier;
+import org.apache.flink.state.api.runtime.SavepointLoader;
+import org.apache.flink.table.annotation.DataTypeHint;
+import org.apache.flink.table.annotation.FunctionHint;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.functions.BuiltInFunctionDefinition;
+import org.apache.flink.table.functions.SpecializedFunction;
+import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.table.types.inference.TypeStrategies;
+import org.apache.flink.types.Row;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.StreamSupport;
+
+import static org.apache.flink.table.functions.FunctionKind.TABLE;
+
+/**
+ * Table function to extract Kafka source offsets from savepoints.
+ *
+ * <p>This function reads Kafka partition splits from a savepoint and extracts
the offset
+ * information for each partition. The function is designed to work with the
Flink state processing
+ * API and provides visibility into the Kafka source state.
+ */
+@Internal
+@FunctionHint(
+ output =
+ @DataTypeHint(
+ "ROW<topic STRING NOT NULL, "
+ + "partition INT NOT NULL, "
+ + "starting-offset BIGINT NOT NULL, "
+ + "stopping-offset BIGINT NULL>"))
+public class GetKafkaSourceOffsetsTableFunction extends TableFunction<Row> {
+
+ public static final BuiltInFunctionDefinition FUNCTION_DEFINITION =
+ BuiltInFunctionDefinition.newBuilder()
+ .name("savepoint_get_kafka_offsets")
+ .kind(TABLE)
+
.runtimeClass(GetKafkaSourceOffsetsTableFunction.class.getName())
+ .outputTypeStrategy(
+ TypeStrategies.explicit(
+ DataTypes.ROW(
+ DataTypes.FIELD("topic",
DataTypes.STRING().notNull()),
+ DataTypes.FIELD("partition",
DataTypes.INT().notNull()),
+ DataTypes.FIELD(
+ "starting-offset",
+
DataTypes.BIGINT().notNull()),
+ DataTypes.FIELD(
+ "stopping-offset",
+
DataTypes.BIGINT().nullable()))))
+ .build();
+
+ public
GetKafkaSourceOffsetsTableFunction(SpecializedFunction.SpecializedContext
context) {}
Review Comment:
Any reason for this ctor that does nothing?
##########
flink-connector-kafka/src/main/resources/META-INF/services/org.apache.flink.table.functions.DynamicBuiltInFunctionDefinitionFactory:
##########
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.flink.streaming.connectors.kafka.table.KafkaDynamicBuiltInFunctionDefinitionFactory
Review Comment:
nit: missing newline @ EOF.
##########
flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/SavepointKafkaOffsetsTableFunctionTest.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.flink.streaming.connectors.kafka.table;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.types.Row;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Unit tests for the Kafka offsets extraction from savepoints. */
+public class SavepointKafkaOffsetsTableFunctionTest {
+ @Test
+ public void testReadKafkaOffsets() throws Exception {
+ Configuration config = new Configuration();
+ config.set(RUNTIME_MODE, RuntimeExecutionMode.BATCH);
+ StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment(config);
+ StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+ tEnv.executeSql("LOAD MODULE state");
+ Table table =
+ tEnv.sqlQuery(
+ "SELECT * FROM
savepoint_get_kafka_offsets('src/test/resources/table-state-kafka-offsets',
'kafka-source-uid')");
+ List<Row> result = tEnv.toDataStream(table).executeAndCollect(100);
+ result.sort(Comparator.comparing(a -> ((Integer)
a.getField("partition"))));
+
+ assertThat(result.size()).isEqualTo(2);
+ Iterator<Row> it = result.iterator();
+ assertThat(it.next().toString()).isEqualTo("+I[test1, 0, -2, null]");
+ assertThat(it.next().toString()).isEqualTo("+I[test1, 1, 1, null]");
Review Comment:
We can skip converting to datastream, and can also make the assertions a bit
smarter:
```java
TableResult res =
tEnv.executeSql(
"SELECT * FROM
savepoint_get_kafka_offsets('src/test/resources/table-state-kafka-offsets',
'kafka-source-uid')");
List<Row> rows = new ArrayList<>();
try (CloseableIterator<Row> it = res.collect()) {
it.forEachRemaining(rows::add);
}
rows.sort(Comparator.comparing(a -> ((Integer) a.getField("partition"))));
assertThat(rows)
.extracting(Row::toString)
.containsExactly("+I[test1, 0, -2, null]", "+I[test1, 1, 1, null]");
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]