You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by zi...@apache.org on 2022/05/25 07:04:43 UTC
[incubator-inlong] branch master updated: [INLONG-4346][Sort] Enhance upsert capability for SqlServer (#4348)
This is an automated email from the ASF dual-hosted git repository.
zirui pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-inlong.git
The following commit(s) were added to refs/heads/master by this push:
new 14f534a73 [INLONG-4346][Sort] Enhance upsert capability for SqlServer (#4348)
14f534a73 is described below
commit 14f534a733b7ba2fa198b01cdcdfb18063791968
Author: ganfengtan <Ga...@users.noreply.github.com>
AuthorDate: Wed May 25 15:04:38 2022 +0800
[INLONG-4346][Sort] Enhance upsert capability for SqlServer (#4348)
---
.../sort/protocol/node/load/SqlServerLoadNode.java | 14 ++++--
.../protocol/node/load/SqlServerLoadNodeTest.java | 50 ++++++++++++++++++++++
.../inlong/sort/jdbc/dialect/SqlServerDialect.java | 3 +-
.../sort/parser/SqlServerNodeSqlParseTest.java | 16 +++----
4 files changed, 70 insertions(+), 13 deletions(-)
diff --git a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/load/SqlServerLoadNode.java b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/load/SqlServerLoadNode.java
index 081546f1c..59a91be16 100644
--- a/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/load/SqlServerLoadNode.java
+++ b/inlong-sort/sort-common/src/main/java/org/apache/inlong/sort/protocol/node/load/SqlServerLoadNode.java
@@ -46,20 +46,23 @@ public class SqlServerLoadNode extends LoadNode implements Serializable {
private static final long serialVersionUID = 3083735489161823965L;
/**
- * jdbc:sqlserver://host:port
+ * jdbc:sqlserver://host:port;databaseName=database
*/
@JsonProperty("url")
private String url;
+
@JsonProperty("username")
private String username;
+
@JsonProperty("password")
private String password;
- @JsonProperty("database")
- private String database;
+
@JsonProperty(value = "schemaName", defaultValue = "dbo")
private String schemaName;
+
@JsonProperty("tableName")
private String tableName;
+
@JsonProperty("primaryKey")
private String primaryKey;
@@ -99,6 +102,11 @@ public class SqlServerLoadNode extends LoadNode implements Serializable {
return options;
}
+ @Override
+ public String getPrimaryKey() {
+ return primaryKey;
+ }
+
@Override
public String genTableName() {
return String.format("table_%s", super.getId());
diff --git a/inlong-sort/sort-common/src/test/java/org/apache/inlong/sort/protocol/node/load/SqlServerLoadNodeTest.java b/inlong-sort/sort-common/src/test/java/org/apache/inlong/sort/protocol/node/load/SqlServerLoadNodeTest.java
new file mode 100644
index 000000000..e99328cf4
--- /dev/null
+++ b/inlong-sort/sort-common/src/test/java/org/apache/inlong/sort/protocol/node/load/SqlServerLoadNodeTest.java
@@ -0,0 +1,50 @@
+/*
+ * 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.inlong.sort.protocol.node.load;
+
+import org.apache.inlong.sort.SerializeBaseTest;
+import org.apache.inlong.sort.formats.common.LongFormatInfo;
+import org.apache.inlong.sort.formats.common.StringFormatInfo;
+import org.apache.inlong.sort.protocol.FieldInfo;
+import org.apache.inlong.sort.protocol.transformation.FieldRelationShip;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Test for {@link SqlServerLoadNode} serialize
+ */
+public class SqlServerLoadNodeTest extends SerializeBaseTest<SqlServerLoadNode> {
+
+ @Override
+ public SqlServerLoadNode getTestObject() {
+ List<FieldInfo> fields = Arrays.asList(new FieldInfo("id", new LongFormatInfo()),
+ new FieldInfo("name", new StringFormatInfo()));
+ List<FieldRelationShip> relations = Arrays
+ .asList(new FieldRelationShip(new FieldInfo("id", new LongFormatInfo()),
+ new FieldInfo("id", new LongFormatInfo())),
+ new FieldRelationShip(new FieldInfo("name", new StringFormatInfo()),
+ new FieldInfo("name", new StringFormatInfo()))
+ );
+ return new SqlServerLoadNode("1", "sqlserver_out", fields, relations, null, null, 1,
+ null, "jdbc:sqlserver://localhost:1433;databaseName=column_type_test", "SA",
+ "INLONG*123", "dbo", "work1", "id");
+
+ }
+}
diff --git a/inlong-sort/sort-connectors/jdbc/src/main/java/org/apache/inlong/sort/jdbc/dialect/SqlServerDialect.java b/inlong-sort/sort-connectors/jdbc/src/main/java/org/apache/inlong/sort/jdbc/dialect/SqlServerDialect.java
index 41fae531f..678a980ac 100644
--- a/inlong-sort/sort-connectors/jdbc/src/main/java/org/apache/inlong/sort/jdbc/dialect/SqlServerDialect.java
+++ b/inlong-sort/sort-connectors/jdbc/src/main/java/org/apache/inlong/sort/jdbc/dialect/SqlServerDialect.java
@@ -144,7 +144,6 @@ public class SqlServerDialect extends AbstractJdbcDialect {
sb.append(" WHEN MATCHED THEN UPDATE SET ")
.append(updateSql);
}
-
sb.append(" WHEN NOT MATCHED THEN " + "INSERT (")
.append(
Arrays
@@ -167,7 +166,7 @@ public class SqlServerDialect extends AbstractJdbcDialect {
String collect =
Arrays
.stream(column)
- .map(col -> " ? " + quoteIdentifier(col))
+ .map(col -> " :".concat(col).concat(" ") + quoteIdentifier(col))
.collect(Collectors.joining(", "));
sb.append(collect);
return sb.toString();
diff --git a/inlong-sort/sort-core/src/test/java/org/apache/inlong/sort/parser/SqlServerNodeSqlParseTest.java b/inlong-sort/sort-core/src/test/java/org/apache/inlong/sort/parser/SqlServerNodeSqlParseTest.java
index 234d64cdf..43b5a2e50 100644
--- a/inlong-sort/sort-core/src/test/java/org/apache/inlong/sort/parser/SqlServerNodeSqlParseTest.java
+++ b/inlong-sort/sort-core/src/test/java/org/apache/inlong/sort/parser/SqlServerNodeSqlParseTest.java
@@ -108,8 +108,8 @@ public class SqlServerNodeSqlParseTest extends AbstractTestBase {
new FieldInfo("name", new StringFormatInfo()))
);
return new SqlServerLoadNode(id, "sqlserver_out", fields, relations, null, null, 1,
- null, "jdbc:sqlserver://localhost:1433", "SA",
- "INLONG*123", "column_type_test.dbo", "work1", null);
+ null, "jdbc:sqlserver://localhost:1433;databaseName=column_type_test", "SA",
+ "INLONG*123", "dbo", "work1", "id");
}
/**
@@ -137,11 +137,11 @@ public class SqlServerNodeSqlParseTest extends AbstractTestBase {
StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env, settings);
Node mySqlExtractNode = buildMySQLExtractNode("1");
Node sqlServerLoadNode = buildSqlServerLoadNode("2");
- StreamInfo streamInfoToHDFS = new StreamInfo("1L", Arrays.asList(mySqlExtractNode, sqlServerLoadNode),
+ StreamInfo streamInfo = new StreamInfo("1L", Arrays.asList(mySqlExtractNode, sqlServerLoadNode),
Collections.singletonList(buildNodeRelation(Collections.singletonList(mySqlExtractNode),
Collections.singletonList(sqlServerLoadNode))));
- GroupInfo groupInfoToHDFS = new GroupInfo("1", Collections.singletonList(streamInfoToHDFS));
- FlinkSqlParser parser = FlinkSqlParser.getInstance(tableEnv, groupInfoToHDFS);
+ GroupInfo groupInf = new GroupInfo("1", Collections.singletonList(streamInfo));
+ FlinkSqlParser parser = FlinkSqlParser.getInstance(tableEnv, groupInf);
Assert.assertTrue(parser.parse().tryExecute());
}
@@ -161,11 +161,11 @@ public class SqlServerNodeSqlParseTest extends AbstractTestBase {
StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env, settings);
Node sqlServerExtractNode = buildSqlServerExtractNode("1");
Node kafkaLoadNode = buildKafkaNode("2");
- StreamInfo streamInfoToHDFS = new StreamInfo("1L", Arrays.asList(sqlServerExtractNode, kafkaLoadNode),
+ StreamInfo streamInfo = new StreamInfo("1L", Arrays.asList(sqlServerExtractNode, kafkaLoadNode),
Collections.singletonList(buildNodeRelation(Collections.singletonList(sqlServerExtractNode),
Collections.singletonList(kafkaLoadNode))));
- GroupInfo groupInfoToHDFS = new GroupInfo("1", Collections.singletonList(streamInfoToHDFS));
- FlinkSqlParser parser = FlinkSqlParser.getInstance(tableEnv, groupInfoToHDFS);
+ GroupInfo groupInfo = new GroupInfo("1", Collections.singletonList(streamInfo));
+ FlinkSqlParser parser = FlinkSqlParser.getInstance(tableEnv, groupInfo);
Assert.assertTrue(parser.parse().tryExecute());
}