[ https://issues.apache.org/jira/browse/BEAM-13400?focusedWorklogId=705989&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-705989 ]
ASF GitHub Bot logged work on BEAM-13400: ----------------------------------------- Author: ASF GitHub Bot Created on: 10/Jan/22 11:01 Start Date: 10/Jan/22 11:01 Worklog Time Spent: 10m Work Description: mosche commented on a change in pull request #16284: URL: https://github.com/apache/beam/pull/16284#discussion_r780965560 ########## File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/UuidLogicalType.java ########## @@ -0,0 +1,80 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; + +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.apache.beam.sdk.values.Row; + +/** Base class for types representing UUID as two long values. */ +@Experimental(Kind.SCHEMAS) +public class UuidLogicalType implements Schema.LogicalType<UUID, Row> { + public static final String IDENTIFIER = "beam:logical_type:uuid:v1"; + public static final String LEAST_SIGNIFICANT_BITS_FIELD_NAME = "LeastSignificantBits"; + public static final String MOST_LEAST_SIGNIFICANT_BITS_FIELD_NAME = "MostSignificantBits"; Review comment: ```suggestion public static final String MOST_SIGNIFICANT_BITS_FIELD_NAME = "MostSignificantBits"; ``` ########## File path: sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java ########## @@ -706,7 +710,9 @@ public void testWriteWithoutPreparedStatement() throws Exception { stmt.append("column_time TIME,"); // Time stmt.append("column_timestamptz TIMESTAMP,"); // Timestamp stmt.append("column_timestamp TIMESTAMP,"); // Timestamp - stmt.append("column_short SMALLINT"); // short + stmt.append("column_short SMALLINT,"); // short + stmt.append("column_blob BLOB,"); // blob + stmt.append("column_clob CLOB"); // clob Review comment: Optionally, UUID could be tested here using a derby custom type. But leaving it to you ... ```sql CREATE TYPE UUID EXTERNAL NAME 'java.util.UUID' LANGUAGE JAVA ``` That would be of jdbc type `JAVA_OBJECT`. Supporting that would be simple enough. Though, in that case, the setter cannot specify the JDBC type (https://github.com/apache/beam/pull/16284#discussion_r774043316) ########## File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/UuidLogicalType.java ########## @@ -0,0 +1,80 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; + +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.apache.beam.sdk.values.Row; + +/** Base class for types representing UUID as two long values. */ +@Experimental(Kind.SCHEMAS) +public class UuidLogicalType implements Schema.LogicalType<UUID, Row> { Review comment: Could you add a unit test in `LogicalTypesTest` pls ########## File path: sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/SchemaUtil.java ########## @@ -105,7 +108,8 @@ Schema.Field create(int index, ResultSetMetaData md) throws SQLException; } - private static BeamFieldConverter jdbcTypeToBeamFieldConverter(JDBCType jdbcType) { + private static BeamFieldConverter jdbcTypeToBeamFieldConverter( + JDBCType jdbcType, String columnClassName) { Review comment: How about just naming it `className` for consistency?`jdbcType` is also in the context of column... ########## File path: sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/OtherJdbcTypesIT.java ########## @@ -0,0 +1,147 @@ +/* + * 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.io.jdbc; + +import static org.apache.beam.sdk.io.common.IOITHelper.executeWithRetry; +import static org.apache.beam.sdk.io.common.IOITHelper.readIOTestPipelineOptions; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.List; +import java.util.UUID; +import org.apache.beam.sdk.io.common.DatabaseTestHelper; +import org.apache.beam.sdk.io.common.PostgresIOTestPipelineOptions; +import org.apache.beam.sdk.schemas.transforms.Select; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.postgresql.ds.PGSimpleDataSource; + +@RunWith(JUnit4.class) +public class OtherJdbcTypesIT { Review comment: I would prefer if we could avoid adding a 2nd integration test for JdbcIO as run times will easily explode over time. But I do see your reasoning. It's not straight forward to add tests in JdbcIOTest at the moment or adjust the existing IT tests 😞 So this is probably fine for now 👍 -- 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-unsubscr...@beam.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 705989) Time Spent: 2h 50m (was: 2h 40m) > JDBC IO does not support UUID and JSONB PostgreSQL types and OTHER JDBC > types in general > ----------------------------------------------------------------------------------------- > > Key: BEAM-13400 > URL: https://issues.apache.org/jira/browse/BEAM-13400 > Project: Beam > Issue Type: Improvement > Components: io-java-jdbc > Affects Versions: 2.34.0 > Reporter: Vitaly Ivanov > Priority: P2 > Time Spent: 2h 50m > Remaining Estimate: 0h > > The following exception occurs when trying to read rows from table which > contains fields with type JSONB and UUID. They have JDBCType OTHER. > {noformat} > java.lang.UnsupportedOperationException: Converting OTHER to Beam schema type > is not supported > at > org.apache.beam.sdk.io.jdbc.SchemaUtil.jdbcTypeToBeamFieldConverter(SchemaUtil.java:161) > at > org.apache.beam.sdk.io.jdbc.SchemaUtil.toBeamSchema(SchemaUtil.java:172) > at > org.apache.beam.sdk.io.jdbc.JdbcIO$ReadRows.inferBeamSchema(JdbcIO.java:655) > at org.apache.beam.sdk.io.jdbc.JdbcIO$ReadRows.expand(JdbcIO.java:632) > at org.apache.beam.sdk.io.jdbc.JdbcIO$ReadRows.expand(JdbcIO.java:551) > at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:548) > at org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:499) > at org.apache.beam.sdk.values.PBegin.apply(PBegin.java:56){noformat} > I suppose the issue is quite important because the UUID type is widespread in > PostgeSQL. > > Actual for Oracle BLOB as well. -- This message was sent by Atlassian Jira (v8.20.1#820001)