This is an automated email from the ASF dual-hosted git repository.
lzljs3620320 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-paimon.git
The following commit(s) were added to refs/heads/master by this push:
new 3dd0b8436 [kafka-cdc] Fix Null point exception when kafka cdc
operation does not configure group id (#1747)
3dd0b8436 is described below
commit 3dd0b84366a913b023d238c39350f97e29dc78d1
Author: Kerwin <[email protected]>
AuthorDate: Tue Aug 8 13:27:27 2023 +0800
[kafka-cdc] Fix Null point exception when kafka cdc operation does not
configure group id (#1747)
---
.../flink/action/cdc/ComputedColumnUtils.java | 81 +++++++++++++++++++++
.../flink/action/cdc/kafka/KafkaActionUtils.java | 84 ++++++++--------------
.../paimon/flink/action/cdc/kafka/KafkaSchema.java | 6 +-
.../action/cdc/kafka/KafkaSyncDatabaseAction.java | 8 ---
.../action/cdc/kafka/KafkaSyncTableAction.java | 7 +-
.../flink/action/cdc/mysql/MySqlActionUtils.java | 51 -------------
.../action/cdc/mysql/MySqlSyncTableAction.java | 4 +-
.../kafka/KafkaCanalSyncDatabaseActionITCase.java | 2 +-
8 files changed, 120 insertions(+), 123 deletions(-)
diff --git
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/ComputedColumnUtils.java
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/ComputedColumnUtils.java
new file mode 100644
index 000000000..4cb80a137
--- /dev/null
+++
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/ComputedColumnUtils.java
@@ -0,0 +1,81 @@
+/*
+ * 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.paimon.flink.action.cdc;
+
+import org.apache.paimon.types.DataType;
+import org.apache.paimon.utils.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.paimon.utils.Preconditions.checkArgument;
+
+/** Utility methods for {@link ComputedColumn}, such as build. */
+public class ComputedColumnUtils {
+
+ public static List<ComputedColumn> buildComputedColumns(
+ List<String> computedColumnArgs, Map<String, DataType>
typeMapping) {
+ List<ComputedColumn> computedColumns = new ArrayList<>();
+ for (String columnArg : computedColumnArgs) {
+ String[] kv = columnArg.split("=");
+ if (kv.length != 2) {
+ throw new IllegalArgumentException(
+ String.format(
+ "Invalid computed column argument: %s. Please
use format 'column-name=expr-name(args, ...)'.",
+ columnArg));
+ }
+ String columnName = kv[0].trim();
+ String expression = kv[1].trim();
+ // parse expression
+ int left = expression.indexOf('(');
+ int right = expression.indexOf(')');
+ Preconditions.checkArgument(
+ left > 0 && right > left,
+ String.format(
+ "Invalid expression: %s. Please use format
'expr-name(args, ...)'.",
+ expression));
+
+ String exprName = expression.substring(0, left);
+ String[] args = expression.substring(left + 1, right).split(",");
+ checkArgument(args.length >= 1, "Computed column needs at least
one argument.");
+
+ String fieldReference = args[0].trim();
+ String[] literals =
+
Arrays.stream(args).skip(1).map(String::trim).toArray(String[]::new);
+ checkArgument(
+ typeMapping.containsKey(fieldReference),
+ String.format(
+ "Referenced field '%s' is not in given MySQL
fields: %s.",
+ fieldReference, typeMapping.keySet()));
+
+ computedColumns.add(
+ new ComputedColumn(
+ columnName,
+ Expression.create(
+ exprName,
+ fieldReference,
+ typeMapping.get(fieldReference),
+ literals)));
+ }
+
+ return computedColumns;
+ }
+}
diff --git
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionUtils.java
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionUtils.java
index 93330eb4e..3fae54f01 100644
---
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionUtils.java
+++
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionUtils.java
@@ -19,13 +19,11 @@
package org.apache.paimon.flink.action.cdc.kafka;
import org.apache.paimon.flink.action.cdc.ComputedColumn;
-import org.apache.paimon.flink.action.cdc.Expression;
import org.apache.paimon.flink.sink.cdc.UpdatedDataFieldsProcessFunction;
import org.apache.paimon.schema.Schema;
import org.apache.paimon.schema.TableSchema;
import org.apache.paimon.types.DataField;
import org.apache.paimon.types.DataType;
-import org.apache.paimon.utils.Preconditions;
import org.apache.paimon.utils.StringUtils;
import org.apache.flink.api.common.serialization.SimpleStringSchema;
@@ -38,11 +36,11 @@ import
org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions;
import
org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.util.CollectionUtil;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
import org.apache.kafka.common.TopicPartition;
-import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.LinkedHashMap;
@@ -158,12 +156,12 @@ class KafkaActionUtils {
}
static KafkaSource<String> buildKafkaSource(Configuration kafkaConfig) {
+ validateKafkaConfig(kafkaConfig);
KafkaSourceBuilder<String> kafkaSourceBuilder = KafkaSource.builder();
- String groupId = kafkaConfig.get(KafkaConnectorOptions.PROPS_GROUP_ID);
kafkaSourceBuilder
.setTopics(kafkaConfig.get(KafkaConnectorOptions.TOPIC))
.setValueOnlyDeserializer(new SimpleStringSchema())
- .setGroupId(StringUtils.isEmpty(groupId) ?
UUID.randomUUID().toString() : groupId);
+ .setGroupId(kafkaPropertiesGroupId(kafkaConfig));
Properties properties = new Properties();
for (Map.Entry<String, String> entry : kafkaConfig.toMap().entrySet())
{
String key = entry.getKey();
@@ -263,54 +261,6 @@ class KafkaActionUtils {
.collect(Collectors.joining(",")))));
}
- static List<ComputedColumn> buildComputedColumns(
- List<String> computedColumnArgs, Map<String, DataType>
typeMapping) {
- List<ComputedColumn> computedColumns = new ArrayList<>();
- for (String columnArg : computedColumnArgs) {
- String[] kv = columnArg.split("=");
- if (kv.length != 2) {
- throw new IllegalArgumentException(
- String.format(
- "Invalid computed column argument: %s. Please
use format 'column-name=expr-name(args, ...)'.",
- columnArg));
- }
- String columnName = kv[0].trim();
- String expression = kv[1].trim();
- // parse expression
- int left = expression.indexOf('(');
- int right = expression.indexOf(')');
- Preconditions.checkArgument(
- left > 0 && right > left,
- String.format(
- "Invalid expression: %s. Please use format
'expr-name(args, ...)'.",
- expression));
-
- String exprName = expression.substring(0, left);
- String[] args = expression.substring(left + 1, right).split(",");
- checkArgument(args.length >= 1, "Computed column needs at least
one argument.");
-
- String fieldReference = args[0].trim();
- String[] literals =
-
Arrays.stream(args).skip(1).map(String::trim).toArray(String[]::new);
- checkArgument(
- typeMapping.containsKey(fieldReference),
- String.format(
- "Referenced field '%s' is not in given MySQL
fields: %s.",
- fieldReference, typeMapping.keySet()));
-
- computedColumns.add(
- new ComputedColumn(
- columnName,
- Expression.create(
- exprName,
- fieldReference,
- typeMapping.get(fieldReference),
- literals)));
- }
-
- return computedColumns;
- }
-
/**
* Parses specificOffsets String to Map.
*
@@ -360,4 +310,32 @@ class KafkaActionUtils {
}
return offsetMap;
}
+
+ private static void validateKafkaConfig(Configuration kafkaConfig) {
+ checkArgument(
+ kafkaConfig.get(KafkaConnectorOptions.VALUE_FORMAT) != null,
+ String.format(
+ "kafka-conf [%s] must be specified.",
+ KafkaConnectorOptions.VALUE_FORMAT.key()));
+
+ checkArgument(
+
!CollectionUtil.isNullOrEmpty(kafkaConfig.get(KafkaConnectorOptions.TOPIC)),
+ String.format(
+ "kafka-conf [%s] must be specified.",
KafkaConnectorOptions.TOPIC.key()));
+
+ checkArgument(
+ kafkaConfig.get(KafkaConnectorOptions.PROPS_BOOTSTRAP_SERVERS)
!= null,
+ String.format(
+ "kafka-conf [%s] must be specified.",
+ KafkaConnectorOptions.PROPS_BOOTSTRAP_SERVERS.key()));
+ }
+
+ public static String kafkaPropertiesGroupId(Configuration kafkaConfig) {
+ String groupId = kafkaConfig.get(KafkaConnectorOptions.PROPS_GROUP_ID);
+ if (StringUtils.isEmpty(groupId)) {
+ groupId = UUID.randomUUID().toString();
+ kafkaConfig.set(KafkaConnectorOptions.PROPS_GROUP_ID, groupId);
+ }
+ return groupId;
+ }
}
diff --git
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchema.java
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchema.java
index 130bf2181..741ca0d57 100644
---
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchema.java
+++
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSchema.java
@@ -38,6 +38,8 @@ import java.util.Map;
import java.util.Objects;
import java.util.Properties;
+import static
org.apache.paimon.flink.action.cdc.kafka.KafkaActionUtils.kafkaPropertiesGroupId;
+
/** Utility class to load canal kafka schema. */
public class KafkaSchema {
@@ -81,9 +83,7 @@ public class KafkaSchema {
props.put(
ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG,
kafkaConfig.get(KafkaConnectorOptions.PROPS_BOOTSTRAP_SERVERS));
- props.put(
- ConsumerConfig.GROUP_ID_CONFIG,
- kafkaConfig.get(KafkaConnectorOptions.PROPS_GROUP_ID));
+ props.put(ConsumerConfig.GROUP_ID_CONFIG,
kafkaPropertiesGroupId(kafkaConfig));
props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
StringDeserializer.class.getName());
props.put(
ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
StringDeserializer.class.getName());
diff --git
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseAction.java
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseAction.java
index 6ec6b3864..0b570dd7a 100644
---
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseAction.java
+++
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseAction.java
@@ -35,7 +35,6 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.connector.kafka.source.KafkaSource;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions;
-import org.apache.flink.util.CollectionUtil;
import javax.annotation.Nullable;
@@ -119,13 +118,6 @@ public class KafkaSyncDatabaseAction extends ActionBase {
}
public void build(StreamExecutionEnvironment env) throws Exception {
- checkArgument(
- kafkaConfig.contains(KafkaConnectorOptions.VALUE_FORMAT),
- KafkaConnectorOptions.VALUE_FORMAT.key() + " cannot be null.");
- checkArgument(
-
!CollectionUtil.isNullOrEmpty(kafkaConfig.get(KafkaConnectorOptions.TOPIC)),
- KafkaConnectorOptions.TOPIC.key() + " cannot be null.");
-
boolean caseSensitive = catalog.caseSensitive();
if (!caseSensitive) {
diff --git
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableAction.java
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableAction.java
index 8d25e1c04..661c11d32 100644
---
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableAction.java
+++
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableAction.java
@@ -42,7 +42,7 @@ import java.util.Collections;
import java.util.List;
import java.util.Map;
-import static org.apache.paimon.utils.Preconditions.checkArgument;
+import static
org.apache.paimon.flink.action.cdc.ComputedColumnUtils.buildComputedColumns;
/**
* An {@link Action} which synchronize one kafka topic into one Paimon table.
@@ -130,9 +130,6 @@ public class KafkaSyncTableAction extends ActionBase {
}
public void build(StreamExecutionEnvironment env) throws Exception {
- checkArgument(
- kafkaConfig.contains(KafkaConnectorOptions.VALUE_FORMAT),
- KafkaConnectorOptions.VALUE_FORMAT.key() + " cannot be null ");
KafkaSource<String> source =
KafkaActionUtils.buildKafkaSource(kafkaConfig);
String topic = kafkaConfig.get(KafkaConnectorOptions.TOPIC).get(0);
KafkaSchema kafkaSchema = KafkaSchema.getKafkaSchema(kafkaConfig,
topic);
@@ -143,7 +140,7 @@ public class KafkaSyncTableAction extends ActionBase {
Identifier identifier = new Identifier(database, table);
FileStoreTable table;
List<ComputedColumn> computedColumns =
- KafkaActionUtils.buildComputedColumns(computedColumnArgs,
kafkaSchema.fields());
+ buildComputedColumns(computedColumnArgs, kafkaSchema.fields());
Schema fromCanal =
KafkaActionUtils.buildPaimonSchema(
kafkaSchema,
diff --git
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlActionUtils.java
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlActionUtils.java
index add673f8b..fff0fdaf2 100644
---
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlActionUtils.java
+++
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlActionUtils.java
@@ -20,13 +20,11 @@ package org.apache.paimon.flink.action.cdc.mysql;
import org.apache.paimon.catalog.Identifier;
import org.apache.paimon.flink.action.cdc.ComputedColumn;
-import org.apache.paimon.flink.action.cdc.Expression;
import org.apache.paimon.flink.sink.cdc.UpdatedDataFieldsProcessFunction;
import org.apache.paimon.schema.Schema;
import org.apache.paimon.schema.TableSchema;
import org.apache.paimon.types.DataField;
import org.apache.paimon.types.DataType;
-import org.apache.paimon.utils.Preconditions;
import com.ververica.cdc.connectors.mysql.source.MySqlSource;
import com.ververica.cdc.connectors.mysql.source.MySqlSourceBuilder;
@@ -50,7 +48,6 @@ import java.sql.DatabaseMetaData;
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
@@ -361,52 +358,4 @@ public class MySqlActionUtils {
"mysql-conf [%s] must be specified.",
MySqlSourceOptions.DATABASE_NAME.key()));
}
-
- static List<ComputedColumn> buildComputedColumns(
- List<String> computedColumnArgs, Map<String, DataType>
typeMapping) {
- List<ComputedColumn> computedColumns = new ArrayList<>();
- for (String columnArg : computedColumnArgs) {
- String[] kv = columnArg.split("=");
- if (kv.length != 2) {
- throw new IllegalArgumentException(
- String.format(
- "Invalid computed column argument: %s. Please
use format 'column-name=expr-name(args, ...)'.",
- columnArg));
- }
- String columnName = kv[0].trim();
- String expression = kv[1].trim();
- // parse expression
- int left = expression.indexOf('(');
- int right = expression.indexOf(')');
- Preconditions.checkArgument(
- left > 0 && right > left,
- String.format(
- "Invalid expression: %s. Please use format
'expr-name(args, ...)'.",
- expression));
-
- String exprName = expression.substring(0, left);
- String[] args = expression.substring(left + 1, right).split(",");
- checkArgument(args.length >= 1, "Computed column needs at least
one argument.");
-
- String fieldReference = args[0].trim();
- String[] literals =
-
Arrays.stream(args).skip(1).map(String::trim).toArray(String[]::new);
- checkArgument(
- typeMapping.containsKey(fieldReference),
- String.format(
- "Referenced field '%s' is not in given MySQL
fields: %s.",
- fieldReference, typeMapping.keySet()));
-
- computedColumns.add(
- new ComputedColumn(
- columnName,
- Expression.create(
- exprName,
- fieldReference,
- typeMapping.get(fieldReference),
- literals)));
- }
-
- return computedColumns;
- }
}
diff --git
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java
index b2a067b16..2a421c967 100644
---
a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java
+++
b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java
@@ -45,6 +45,7 @@ import java.util.function.Predicate;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
+import static
org.apache.paimon.flink.action.cdc.ComputedColumnUtils.buildComputedColumns;
import static
org.apache.paimon.flink.action.cdc.mysql.MySqlActionUtils.MYSQL_CONVERTER_TINYINT1_BOOL;
import static org.apache.paimon.utils.Preconditions.checkArgument;
@@ -163,8 +164,7 @@ public class MySqlSyncTableAction extends ActionBase {
Identifier identifier = new Identifier(database, table);
FileStoreTable table;
List<ComputedColumn> computedColumns =
- MySqlActionUtils.buildComputedColumns(
- computedColumnArgs, mySqlSchema.typeMapping());
+ buildComputedColumns(computedColumnArgs,
mySqlSchema.typeMapping());
Schema fromMySql =
MySqlActionUtils.buildPaimonSchema(
mySqlSchema,
diff --git
a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java
b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java
index a6ab2fdd4..a4df8149c 100644
---
a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java
+++
b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaCanalSyncDatabaseActionITCase.java
@@ -292,7 +292,7 @@ public class KafkaCanalSyncDatabaseActionITCase extends
KafkaActionITCaseBase {
assertThatThrownBy(() -> action.build(env))
.isInstanceOf(IllegalArgumentException.class)
- .hasMessage("topic cannot be null.");
+ .hasMessage("kafka-conf [topic] must be specified.");
}
@Test