You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/07/10 08:47:44 UTC

[GitHub] [flink] qingwei91 commented on a diff in pull request #20235: Jdbc dialects

qingwei91 commented on code in PR #20235:
URL: https://github.com/apache/flink/pull/20235#discussion_r917362437


##########
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerDialect.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.connector.jdbc.dialect.sqlserver;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.jdbc.converter.JdbcRowConverter;
+import org.apache.flink.connector.jdbc.dialect.AbstractDialect;
+import org.apache.flink.connector.jdbc.internal.converter.SqlServerRowConverter;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** JDBC dialect for SqlServer. */
+@Internal
+public class SqlServerDialect extends AbstractDialect {
+    @Override
+    public String dialectName() {
+        return "SqlServer";
+    }
+
+    /**
+     * The maximum precision is supported by datetime2.
+     * https://docs.microsoft.com/en-us/sql/t-sql/data-types/datetime2-transact-sql?view=sql-server-ver16
+     */
+    @Override
+    public Optional<Range> timestampPrecisionRange() {
+        return Optional.of(Range.of(0, 7));
+    }
+
+    /**
+     * The maximum precision is supported by decimal.
+     * https://docs.microsoft.com/en-us/sql/t-sql/data-types/precision-scale-and-length-transact-sql?view=sql-server-ver16
+     *
+     * @return
+     */
+    @Override
+    public Optional<Range> decimalPrecisionRange() {
+        return Optional.of(Range.of(0, 38));
+    }
+
+    @Override
+    public Optional<String> defaultDriverName() {
+        return Optional.of("com.microsoft.sqlserver.jdbc.SQLServerDriver");
+    }
+
+    @Override
+    public String quoteIdentifier(String identifier) {
+        return identifier;
+    }
+
+    @Override
+    public Optional<String> getUpsertStatement(
+            String tableName, String[] fieldNames, String[] uniqueKeyFields) {
+        List<String> nonUniqueKeyFields =
+                Arrays.stream(fieldNames)
+                        .filter(f -> !Arrays.asList(uniqueKeyFields).contains(f))
+                        .collect(Collectors.toList());
+        String fieldsProjection =
+                Arrays.stream(fieldNames)
+                        .map(this::quoteIdentifier)
+                        .collect(Collectors.joining(", "));
+
+        String valuesBinding =
+                Arrays.stream(fieldNames)
+                        .map(f -> ":" + f + " " + quoteIdentifier(f))
+                        .collect(Collectors.joining(", "));
+
+        String usingClause = String.format("SELECT %s", valuesBinding);
+        String onConditions =
+                Arrays.stream(uniqueKeyFields)
+                        .map(
+                                f ->
+                                        "[TARGET]."
+                                                + quoteIdentifier(f)
+                                                + "=[SOURCE]."
+                                                + quoteIdentifier(f))
+                        .collect(Collectors.joining(" AND "));
+        String updateSetClause =
+                nonUniqueKeyFields.stream()
+                        .map(
+                                f ->
+                                        "[TARGET]."
+                                                + quoteIdentifier(f)
+                                                + "=[SOURCE]."
+                                                + quoteIdentifier(f))
+                        .collect(Collectors.joining(", "));
+
+        String insertValues =
+                Arrays.stream(fieldNames)
+                        .map(f -> "[SOURCE]." + quoteIdentifier(f))
+                        .collect(Collectors.joining(", "));
+        return Optional.of(
+                String.format(
+                        "MERGE INTO %s AS [TARGET]"
+                                + " USING (%s) AS [SOURCE]"
+                                + " ON (%s)"
+                                + " WHEN MATCHED THEN"
+                                + " UPDATE SET %s"
+                                + " WHEN NOT MATCHED THEN"
+                                + " INSERT (%s) VALUES (%s);",
+                        quoteIdentifier(tableName),
+                        usingClause,
+                        onConditions,
+                        updateSetClause,
+                        fieldsProjection,
+                        insertValues));
+    }
+
+    @Override
+    public JdbcRowConverter getRowConverter(RowType rowType) {
+        return new SqlServerRowConverter(rowType);
+    }
+
+    @Override
+    public String getLimitClause(long limit) {
+        throw new IllegalArgumentException("SqlServerDialect does not support limit clause");

Review Comment:
   Question to maintainer:
   
   Is it ok to not support limit for SQL Server?
   The reason is that SQL Server does not support limit natively, it supports by OFFSET syntax, but that syntax requires ORDER BY to be valid, with the current structure, I don't think we can support it properly.
   
   This method has no contextual knowledge of whether ORDER BY exists in the larger statement or not.
   
   If you think this is not acceptable, we might need to change code in `JdbcDynamicTableSource`, to support this special case



##########
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/SqlServerRowConverter.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.connector.jdbc.internal.converter;
+
+import org.apache.flink.connector.jdbc.converter.AbstractJdbcRowConverter;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+/**
+ * Runtime converter that responsible to convert between JDBC object and Flink internal object for
+ * MsSql.
+ */
+public class SqlServerRowConverter extends AbstractJdbcRowConverter {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public String converterName() {
+        return "SqlServer";
+    }
+
+    public SqlServerRowConverter(RowType rowType) {
+        super(rowType);
+    }
+
+    @Override
+    protected JdbcDeserializationConverter createInternalConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case TINYINT:
+                return val -> ((Short) val).byteValue();

Review Comment:
   TINY INT in SQL Server is Short, instead of Integer, thus we override here



##########
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerTableSourceITCase.java:
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.connector.jdbc.dialect.sqlserver;
+
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CollectionUtil;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.testcontainers.containers.MSSQLServerContainer;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** The Table Source ITCase for {@link SqlServerDialect}. */
+public class SqlServerTableSourceITCase extends AbstractTestBase {
+
+    private static final MSSQLServerContainer container =
+            new MSSQLServerContainer("mcr.microsoft.com/mssql/server:2019-GA-ubuntu-16.04")
+                    .acceptLicense();
+    private static String containerUrl;
+    private static final String INPUT_TABLE = "sql_test_table";
+
+    private static StreamExecutionEnvironment env;
+    private static TableEnvironment tEnv;
+
+    @BeforeClass
+    public static void beforeAll() throws ClassNotFoundException, SQLException {
+        container.start();
+        containerUrl = container.getJdbcUrl();
+        Class.forName(container.getDriverClassName());
+        try (Connection conn =
+                        DriverManager.getConnection(
+                                containerUrl, container.getUsername(), container.getPassword());
+                Statement statement = conn.createStatement()) {
+            statement.executeUpdate(
+                    "CREATE TABLE "
+                            + INPUT_TABLE
+                            + " ("
+                            + "id INT NOT NULL,"
+                            + "tiny_int TINYINT,"
+                            + "small_int SMALLINT,"
+                            + "big_int BIGINT,"
+                            + "float_col REAL,"
+                            + "double_col FLOAT ,"
+                            + "decimal_col DECIMAL(10, 4) NOT NULL,"
+                            + "bool BIT NOT NULL,"
+                            + "date_col DATE NOT NULL,"
+                            + "time_col TIME(5) NOT NULL,"
+                            + "datetime_col DATETIME,"
+                            + "datetime2_col DATETIME2,"
+                            + "char_col CHAR NOT NULL,"
+                            + "nchar_col NCHAR(3) NOT NULL,"
+                            + "varchar2_col VARCHAR(30) NOT NULL,"
+                            + "nvarchar2_col NVARCHAR(30) NOT NULL,"
+                            + "text_col TEXT,"
+                            + "ntext_col NTEXT,"
+                            + "binary_col BINARY(10)"
+                            + ")");

Review Comment:
   I've covered all types claimed to be supported in the doc



##########
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerTableSinkITCase.java:
##########
@@ -0,0 +1,492 @@
+/*
+ * 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.connector.jdbc.dialect.sqlserver;
+
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.jdbc.internal.GenericJdbcSinkFunction;
+import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.SinkContextUtil;
+import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor;
+import org.apache.flink.streaming.util.MockStreamingRuntimeContext;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.TableResult;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkFunctionProvider;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.table.planner.runtime.utils.TestData;
+import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.testcontainers.containers.MSSQLServerContainer;
+
+import java.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.connector.jdbc.internal.JdbcTableOutputFormatTest.check;
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
+
+/** The Table Sink ITCase for {@link SqlServerDialect}. */
+public class SqlServerTableSinkITCase extends AbstractTestBase {

Review Comment:
   This test is adapted from OracleTableSinkITCase



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org