eskabetxe commented on code in PR #29:
URL:
https://github.com/apache/flink-connector-jdbc/pull/29#discussion_r1151937024
##########
flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePath.java:
##########
@@ -36,8 +36,14 @@ public class PostgresTablePath {
private final String pgTableName;
public PostgresTablePath(String pgSchemaName, String pgTableName) {
- checkArgument(!StringUtils.isNullOrWhitespaceOnly(pgSchemaName));
- checkArgument(!StringUtils.isNullOrWhitespaceOnly(pgTableName));
+ checkArgument(!StringUtils.isNullOrWhitespaceOnly(pgSchemaName),
+ String.format(
+ "Schema name '%s' is not valid. Null or empty is not
allowed",
Review Comment:
putting the name on error message is not necessary no?
the pgSchemaName will be "null" or " ", creating messages as
Schema name 'null' is not valid. Null or empty is not allowed
Schema name ' ' is not valid. Null or empty is not allowed
##########
flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePath.java:
##########
@@ -36,8 +36,14 @@ public class PostgresTablePath {
private final String pgTableName;
public PostgresTablePath(String pgSchemaName, String pgTableName) {
- checkArgument(!StringUtils.isNullOrWhitespaceOnly(pgSchemaName));
- checkArgument(!StringUtils.isNullOrWhitespaceOnly(pgTableName));
+ checkArgument(!StringUtils.isNullOrWhitespaceOnly(pgSchemaName),
+ String.format(
+ "Schema name '%s' is not valid. Null or empty is not
allowed",
+ pgSchemaName));
+ checkArgument(!StringUtils.isNullOrWhitespaceOnly(pgTableName),
+ String.format(
+ "Table name '%s' is not valid. Null or empty is not
allowed",
Review Comment:
Same as Schema
##########
flink-connector-jdbc/archunit-violations/6b9ab1b0-c14d-4667-bab5-407b81fba98b:
##########
@@ -1,36 +0,0 @@
-org.apache.flink.connector.jdbc.JdbcITCase does not satisfy: only one of the
following predicates match:\
Review Comment:
I will say that you should not fix this.
This PR is for adding CrateDB, the fix should have another PR (we already
have one waiting approval)
##########
flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/cratedb/CrateDBTypeMapper.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.cratedb;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.jdbc.dialect.psql.PostgresTypeMapper;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.types.DataType;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+
+/** CrateDBTypeMapper util class. */
+@Internal
+public class CrateDBTypeMapper extends PostgresTypeMapper {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(CrateDBTypeMapper.class);
+
+ // CrateDB jdbc driver uses very similar mapping
+ // to PostgreSQL driver, and adds some extras:
+ private static final String PG_STRING = "string";
+ private static final String PG_STRING_ARRAY = "_string";
+
+ @Override
+ public DataType mapping(ObjectPath tablePath, ResultSetMetaData metadata,
int colIndex)
+ throws SQLException {
+ String pgType = metadata.getColumnTypeName(colIndex);
+
+ int precision = metadata.getPrecision(colIndex);
+ int scale = metadata.getScale(colIndex);
+
+ DataType dataType = getMapping(pgType, precision, scale);
+ if (dataType == null) {
+ throw new UnsupportedOperationException(
+ String.format("Doesn't support CrateDB type '%s' yet",
pgType));
Review Comment:
we override all method only to change error message no?
##########
flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/psql/PostgresTypeMapper.java:
##########
@@ -93,6 +93,15 @@ public DataType mapping(ObjectPath tablePath,
ResultSetMetaData metadata, int co
int precision = metadata.getPrecision(colIndex);
int scale = metadata.getScale(colIndex);
+ DataType dataType = getMapping(pgType, precision, scale);
+ if (dataType == null) {
Review Comment:
Can't be null a valid mapping?
##########
flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/cratedb/CrateDBTypeMapper.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.cratedb;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.jdbc.dialect.psql.PostgresTypeMapper;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.types.DataType;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+
+/** CrateDBTypeMapper util class. */
+@Internal
+public class CrateDBTypeMapper extends PostgresTypeMapper {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(CrateDBTypeMapper.class);
+
+ // CrateDB jdbc driver uses very similar mapping
+ // to PostgreSQL driver, and adds some extras:
+ private static final String PG_STRING = "string";
+ private static final String PG_STRING_ARRAY = "_string";
+
+ @Override
+ public DataType mapping(ObjectPath tablePath, ResultSetMetaData metadata,
int colIndex)
+ throws SQLException {
+ String pgType = metadata.getColumnTypeName(colIndex);
+
+ int precision = metadata.getPrecision(colIndex);
+ int scale = metadata.getScale(colIndex);
+
+ DataType dataType = getMapping(pgType, precision, scale);
+ if (dataType == null) {
+ throw new UnsupportedOperationException(
+ String.format("Doesn't support CrateDB type '%s' yet",
pgType));
+ }
+ return dataType;
+ }
+
+ @Override
+ protected DataType getMapping(String pgType, int precision, int scale) {
+ switch (pgType) {
+ case PG_BIGSERIAL:
+ return null;
Review Comment:
so this is not supported on Crate but is supported on Postgres? is correct?
##########
flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/cratedb/CrateDBDatabase.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.cratedb;
+
+import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.databases.DatabaseTest;
+
+import org.testcontainers.containers.JdbcDatabaseContainer;
+import org.testcontainers.containers.wait.strategy.Wait;
+import org.testcontainers.containers.wait.strategy.WaitStrategy;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+import org.testcontainers.utility.DockerImageName;
+
+import java.time.Duration;
+
+import static java.time.temporal.ChronoUnit.SECONDS;
+
+/** A CrateDB database for testing. */
+@Testcontainers
+public interface CrateDBDatabase extends DatabaseTest {
+
+ String CRATEDB = "crate:latest";
Review Comment:
should have a fixed version (same as dependency)
##########
flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/CrateDBRowConverter.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.table.data.GenericArrayData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+
+import io.crate.shade.org.postgresql.jdbc.PgArray;
+
+import java.lang.reflect.Array;
+
+/**
+ * Runtime converter that responsible to convert between JDBC object and Flink
internal object for
+ * CrateDB.
+ */
+public class CrateDBRowConverter extends PostgresRowConverter {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public String converterName() {
+ return "CrateDB";
+ }
+
+ public CrateDBRowConverter(RowType rowType) {
+ super(rowType);
+ }
+
+ @Override
+ protected JdbcDeserializationConverter
createPostgresArrayConverter(ArrayType arrayType) {
Review Comment:
this is the kind of things that makes me think that its better an abstract
class like 'AbstractPostgresCompatibleRowConverter', because that way we can
set the PgArray as a generic type.
##########
flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/psql/PostgresTypeMapper.java:
##########
@@ -49,41 +49,41 @@ public class PostgresTypeMapper implements
JdbcDialectTypeMapper {
// float <=> float8
// boolean <=> bool
// decimal <=> numeric
- private static final String PG_SMALLSERIAL = "smallserial";
- private static final String PG_SERIAL = "serial";
- private static final String PG_BIGSERIAL = "bigserial";
- private static final String PG_BYTEA = "bytea";
- private static final String PG_BYTEA_ARRAY = "_bytea";
- private static final String PG_SMALLINT = "int2";
- private static final String PG_SMALLINT_ARRAY = "_int2";
- private static final String PG_INTEGER = "int4";
- private static final String PG_INTEGER_ARRAY = "_int4";
- private static final String PG_BIGINT = "int8";
- private static final String PG_BIGINT_ARRAY = "_int8";
- private static final String PG_REAL = "float4";
- private static final String PG_REAL_ARRAY = "_float4";
- private static final String PG_DOUBLE_PRECISION = "float8";
- private static final String PG_DOUBLE_PRECISION_ARRAY = "_float8";
- private static final String PG_NUMERIC = "numeric";
- private static final String PG_NUMERIC_ARRAY = "_numeric";
- private static final String PG_BOOLEAN = "bool";
- private static final String PG_BOOLEAN_ARRAY = "_bool";
- private static final String PG_TIMESTAMP = "timestamp";
- private static final String PG_TIMESTAMP_ARRAY = "_timestamp";
- private static final String PG_TIMESTAMPTZ = "timestamptz";
- private static final String PG_TIMESTAMPTZ_ARRAY = "_timestamptz";
- private static final String PG_DATE = "date";
- private static final String PG_DATE_ARRAY = "_date";
- private static final String PG_TIME = "time";
- private static final String PG_TIME_ARRAY = "_time";
- private static final String PG_TEXT = "text";
- private static final String PG_TEXT_ARRAY = "_text";
- private static final String PG_CHAR = "bpchar";
- private static final String PG_CHAR_ARRAY = "_bpchar";
- private static final String PG_CHARACTER = "character";
- private static final String PG_CHARACTER_ARRAY = "_character";
- private static final String PG_CHARACTER_VARYING = "varchar";
- private static final String PG_CHARACTER_VARYING_ARRAY = "_varchar";
+ protected static final String PG_SMALLSERIAL = "smallserial";
Review Comment:
Is better only change to protected the cases you are using..
It gives scope of field in any future change
--
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]