You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "libenchao (via GitHub)" <gi...@apache.org> on 2023/03/30 14:46:05 UTC

[GitHub] [flink] libenchao commented on a diff in pull request #22284: [FLINK-31547][jdbc-driver] Introduce FlinkResultSetMetaData for jdbc driver

libenchao commented on code in PR #22284:
URL: https://github.com/apache/flink/pull/22284#discussion_r1153358320


##########
flink-table/flink-sql-jdbc-driver/src/main/java/org/apache/flink/table/jdbc/ColumnInfo.java:
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.table.jdbc;
+
+import org.apache.flink.table.types.logical.ArrayType;

Review Comment:
   These are from `flink-table-common`, and we do not include them in the jdbc-driver-bundle yet. I've logged FLINK-31673, IMO, we'd better have some e2e tests to ensure the bundled jdbc driver works.



##########
flink-table/flink-sql-jdbc-driver/src/main/java/org/apache/flink/table/jdbc/ColumnInfo.java:
##########
@@ -0,0 +1,287 @@
+/*
+ * 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.table.jdbc;
+
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.flink.table.types.logical.ZonedTimestampType;
+
+import java.sql.ResultSetMetaData;
+import java.sql.Types;
+
+import static org.apache.flink.table.jdbc.utils.DriverUtils.checkNotNull;
+
+/** Column info for {@link ResultSetMetaData}, it is converted from {@link LogicalType}. */
+public class ColumnInfo {
+    private static final int VARBINARY_MAX = 1024 * 1024 * 1024;
+    private static final int TIME_ZONE_MAX = 40; // current longest time zone is 32
+    private static final int TIME_MAX = "HH:mm:ss.SSS".length();
+    private static final int TIMESTAMP_MAX = "yyyy-MM-dd HH:mm:ss.SSS".length();
+    private static final int TIMESTAMP_WITH_TIME_ZONE_MAX = TIMESTAMP_MAX + TIME_ZONE_MAX;
+    private static final int DATE_MAX = "yyyy-MM-dd".length();
+    private static final int STRUCT_MAX = 100 * 1024 * 1024;
+
+    private final int columnType;
+    private final String columnTypeName;
+    private final boolean nullable;
+    private final boolean signed;
+    private final int precision;
+    private final int scale;
+    private final int columnDisplaySize;
+    private final String columnName;
+
+    public ColumnInfo(

Review Comment:
   Is `private` enough since we already have a `Builder` for it?



##########
flink-table/flink-sql-jdbc-driver/src/test/java/org/apache/flink/table/jdbc/FlinkResultSetMetaDataTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.table.jdbc;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+
+import org.junit.jupiter.api.Test;
+
+import java.math.BigDecimal;
+import java.sql.Array;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.time.OffsetDateTime;
+import java.util.Map;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrowsExactly;
+
+/** Tests for {@link FlinkResultSetMetaData}. */
+public class FlinkResultSetMetaDataTest {
+    @Test
+    public void testResultSetMetaData() throws Exception {
+        ResolvedSchema schema =
+                ResolvedSchema.of(
+                        Column.physical("v1", DataTypes.BOOLEAN()),

Review Comment:
   Nit: what's the meaning for `v`? (I'm assuming you mean "column 1", and should be shortened as 'c1'?)



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