You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2021/12/23 11:47:24 UTC

[GitHub] [beam] mosche commented on a change in pull request #16284: [BEAM-13400] JDBC IO does not support UUID and JSONB PostgreSQL types…

mosche commented on a change in pull request #16284:
URL: https://github.com/apache/beam/pull/16284#discussion_r774395188



##########
File path: sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcUtil.java
##########
@@ -121,6 +123,26 @@ static String generateStatement(String tableName, List<Schema.Field> fields) {
       case LOGICAL_TYPE:
         {
           String logicalTypeName = fieldType.getLogicalType().getIdentifier();
+          // handle other jdbc types

Review comment:
       I'm aware, none of the `PreparedStatementSetCaller`s is covered with unit tests in `JdbcUtilTest` so far. But would it make sense to start adding tests for the new types? Testing the effect of the setter should be straight forward using a mocked `PreparedStatement`

##########
File path: sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcUtil.java
##########
@@ -121,6 +123,26 @@ static String generateStatement(String tableName, List<Schema.Field> fields) {
       case LOGICAL_TYPE:
         {
           String logicalTypeName = fieldType.getLogicalType().getIdentifier();
+          // handle other jdbc types
+          if (logicalTypeName.startsWith("beam:")) {
+            switch (logicalTypeName) {
+              case UuidLogicalType.IDENTIFIER:
+                return (element, ps, i, fieldWithIndex) ->
+                    ps.setObject(
+                        i + 1, element.getLogicalTypeValue(fieldWithIndex.getIndex(), UUID.class));

Review comment:
       I would suggest to also add the sql type like below to prevent issues with nulls on some dbs.

##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/UuidLogicalType.java
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.beam.sdk.schemas.logicaltypes;
+
+import java.util.UUID;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/** Base class for types representing UUID as String. */
+@Experimental(Kind.SCHEMAS)
+@SuppressWarnings({
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})

Review comment:
       Please don't silence nullness warnings on new code. 

##########
File path: sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcUtil.java
##########
@@ -121,6 +123,26 @@ static String generateStatement(String tableName, List<Schema.Field> fields) {
       case LOGICAL_TYPE:
         {
           String logicalTypeName = fieldType.getLogicalType().getIdentifier();
+          // handle other jdbc types
+          if (logicalTypeName.startsWith("beam:")) {
+            switch (logicalTypeName) {
+              case UuidLogicalType.IDENTIFIER:
+                return (element, ps, i, fieldWithIndex) ->
+                    ps.setObject(
+                        i + 1, element.getLogicalTypeValue(fieldWithIndex.getIndex(), UUID.class));
+              case LogicalTypes.JSON_IDENTIFIER:
+              case LogicalTypes.JSONB_IDENTIFIER:
+                return (element, ps, i, fieldWithIndex) ->
+                    ps.setObject(
+                        i + 1,
+                        element.getBaseValue(fieldWithIndex.getIndex()),

Review comment:
       Just curiosity, is there cases that require `getBaseValue` here? Isn't `getValue` (or even `getString`) sufficient?

##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/UuidLogicalType.java
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.beam.sdk.schemas.logicaltypes;
+
+import java.util.UUID;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/** Base class for types representing UUID as String. */
+@Experimental(Kind.SCHEMAS)
+@SuppressWarnings({
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
+public class UuidLogicalType implements Schema.LogicalType<UUID, String> {

Review comment:
       For storage I would rather suggest using long[] as base type to store as two longs for the most significant bits & least significant bits.

##########
File path: sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/SchemaUtil.java
##########
@@ -157,6 +161,24 @@ private static BeamFieldConverter jdbcTypeToBeamFieldConverter(JDBCType jdbcType
         return beamLogicalField(VARBINARY.getName(), LogicalTypes.VariableLengthBytes::of);
       case VARCHAR:
         return beamLogicalField(VARCHAR.getName(), LogicalTypes.VariableLengthString::of);
+      case OTHER:

Review comment:
       Please make sure to cover the changes in this class in `SchemaUtilTest`




-- 
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: github-unsubscribe@beam.apache.org

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