eskabetxe commented on code in PR #72:
URL: 
https://github.com/apache/flink-connector-jdbc/pull/72#discussion_r1409480831


##########
flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/oceanbase/dialect/OceanBaseRowConverter.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.databases.oceanbase.dialect;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.jdbc.converter.AbstractJdbcRowConverter;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.oceanbase.jdbc.Blob;
+import com.oceanbase.jdbc.extend.datatype.BINARY_DOUBLE;
+import com.oceanbase.jdbc.extend.datatype.BINARY_FLOAT;
+import com.oceanbase.jdbc.extend.datatype.NUMBER;
+import com.oceanbase.jdbc.extend.datatype.RAW;
+import com.oceanbase.jdbc.extend.datatype.TIMESTAMP;
+import com.oceanbase.jdbc.extend.datatype.TIMESTAMPTZ;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDateTime;
+
+/**
+ * Runtime converter that responsible to convert between JDBC object and Flink 
internal object for
+ * OceanBase.
+ */
+@Internal
+public class OceanBaseRowConverter extends AbstractJdbcRowConverter {
+
+    private static final long serialVersionUID = 1L;
+
+    private final String compatibleMode;
+
+    @Override
+    public String converterName() {
+        return "OceanBase";
+    }
+
+    public OceanBaseRowConverter(RowType rowType, String compatibleMode) {
+        super(rowType);
+        this.compatibleMode = compatibleMode;
+    }
+
+    public JdbcDeserializationConverter createInternalConverter(LogicalType 
type) {
+        if ("mysql".equalsIgnoreCase(compatibleMode)) {

Review Comment:
   Why not use MySqlRowConverter and OracleRowConverter as you use on dialect?



##########
docs/content/docs/connectors/table/jdbc.md:
##########
@@ -139,6 +140,14 @@ Connector Options
       <td>String</td>
       <td>The class name of the JDBC driver to use to connect to this URL, if 
not set, it will automatically be derived from the URL.</td>
     </tr>
+    <tr>
+      <td><h5>compatible-mode</h5></td>
+      <td>optional</td>
+      <td>yes</td>
+      <td style="word-wrap: break-word;">(none)</td>
+      <td>String</td>
+      <td>The compatible mode of database, required if the database supports 
multiple compatible modes. For OceanBase, it should be <code>'mysql'</code> or 
<code>'oracle'</code>.</td>

Review Comment:
   on the configuration we should avoid related it to one database..
   
   this: For OceanBase, it should be <code>'mysql'</code> or 
<code>'oracle'</code>
   
   should be on OceanBase related documentation



##########
flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/JdbcDialectFactory.java:
##########
@@ -46,4 +46,14 @@ public interface JdbcDialectFactory {
 
     /** @return Creates a new instance of the {@link JdbcDialect}. */
     JdbcDialect create();
+
+    /**
+     * Creates a new instance of the {@link JdbcDialect} based on compatible 
mode.
+     *
+     * @param compatibleMode the compatible mode of database
+     * @return a new instance of {@link JdbcDialect}
+     */
+    default JdbcDialect create(String compatibleMode) {
+        return create();

Review Comment:
   dont know if this is better option..
   for example on Postgres if we define the config compatible-mode, should we 
fail or go on..



##########
flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oceanbase/table/OceanBaseMySqlDynamicTableSourceITCase.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.databases.oceanbase.table;
+
+import org.apache.flink.connector.jdbc.databases.oceanbase.OceanBaseTestBase;
+import org.apache.flink.connector.jdbc.table.JdbcDynamicTableSourceITCase;
+import org.apache.flink.connector.jdbc.testutils.tables.TableRow;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.types.Row;
+
+import java.math.BigDecimal;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.List;
+
+import static 
org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.dbType;
+import static 
org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.field;
+import static 
org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.pkField;
+
+/** The Table Source ITCase for OceanBase MySql mode. */
+public class OceanBaseMySqlDynamicTableSourceITCase extends 
JdbcDynamicTableSourceITCase

Review Comment:
   missing Oracle compatibility test



##########
flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oceanbase/dialect/OceanBaseMysqlDialectTypeTest.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.databases.oceanbase.dialect;
+
+import org.apache.flink.connector.jdbc.dialect.JdbcDialectTypeTest;
+
+import java.util.Arrays;
+import java.util.List;
+
+/** The OceanBase MySql mode params for {@link JdbcDialectTypeTest}. */
+public class OceanBaseMysqlDialectTypeTest extends JdbcDialectTypeTest {

Review Comment:
   the types are the same on MySql or Oracle compatibility?
   if yes we could parametrize the test to have only one



##########
flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/options/InternalJdbcConnectionOptions.java:
##########
@@ -161,6 +162,12 @@ public Builder setDriverName(String driverName) {
             return this;
         }
 
+        /** optional, compatible mode, required by OceanBase database. */

Review Comment:
   remove any reference to a specific database



##########
flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oceanbase/table/OceanBaseMySqlDynamicTableSinkITCase.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.databases.oceanbase.table;
+
+import org.apache.flink.connector.jdbc.databases.oceanbase.OceanBaseTestBase;
+import org.apache.flink.connector.jdbc.table.JdbcDynamicTableSinkITCase;
+import org.apache.flink.connector.jdbc.testutils.tables.TableRow;
+import org.apache.flink.table.api.DataTypes;
+
+import java.util.Map;
+
+import static 
org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.dbType;
+import static 
org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.field;
+import static 
org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.pkField;
+
+/** The Table Sink ITCase for OceanBase MySql mode. */
+public class OceanBaseMySqlDynamicTableSinkITCase extends 
JdbcDynamicTableSinkITCase

Review Comment:
   missing Oracle compatibility test



##########
flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcConnectorOptions.java:
##########
@@ -62,6 +62,12 @@ public class JdbcConnectorOptions {
                             "The class name of the JDBC driver to use to 
connect to this URL. "
                                     + "If not set, it will automatically be 
derived from the URL.");
 
+    public static final ConfigOption<String> COMPATIBLE_MODE =
+            ConfigOptions.key("compatible-mode")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The compatible mode of database, 
required by OceanBase.");

Review Comment:
   remove any reference to a specific database.



-- 
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]

Reply via email to