[ 
https://issues.apache.org/jira/browse/BEAM-12385?focusedWorklogId=606636&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-606636
 ]

ASF GitHub Bot logged work on BEAM-12385:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 04/Jun/21 08:07
            Start Date: 04/Jun/21 08:07
    Worklog Time Spent: 10m 
      Work Description: anantdamle commented on a change in pull request #14858:
URL: https://github.com/apache/beam/pull/14858#discussion_r644665441



##########
File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java
##########
@@ -906,6 +906,23 @@ private void readObject(ObjectInputStream in) throws 
IOException, ClassNotFoundE
                         .map(x -> getFieldSchema(x.getType(), x.getName(), 
namespace))
                         .collect(Collectors.toList()));
             break;
+
+          case "NVARCHAR":
+          case "VARCHAR":
+          case "LONGNVARCHAR":
+          case "LONGVARCHAR":
+            baseType = org.apache.avro.Schema.create(Type.STRING);

Review comment:
       building the Hive way into the PR.

##########
File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java
##########
@@ -906,6 +906,23 @@ private void readObject(ObjectInputStream in) throws 
IOException, ClassNotFoundE
                         .map(x -> getFieldSchema(x.getType(), x.getName(), 
namespace))
                         .collect(Collectors.toList()));
             break;
+
+          case "NVARCHAR":
+          case "VARCHAR":
+          case "LONGNVARCHAR":
+          case "LONGVARCHAR":
+            baseType = org.apache.avro.Schema.create(Type.STRING);

Review comment:
       @iemejia Thanks for the Hive reference. Have updated the code, also 
added a small test in `SchemaUtilTest`.

##########
File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java
##########
@@ -1277,4 +1308,17 @@ private static void checkTypeName(Schema.TypeName got, 
Schema.TypeName expected,
     checkArgument(
         got.equals(expected), "Can't convert '%s' to %s, expected: %s", label, 
got, expected);
   }
+
+  /** Helper factory to build JDBC Logical types for AVRO Schema. */
+  private static org.apache.avro.Schema makeJdbcLogicalStringAvroType(
+      Schema.LogicalType<?, ?> logicalType) {
+    JDBCType jdbcType = JDBCType.valueOf(logicalType.getIdentifier());
+    Integer size = logicalType.getArgument();
+
+    String schemaJson =

Review comment:
       @RyanSkraba thanks for the suggestion, one specific reason I didn't use 
the `SchemaBuilder` is due to the need to use a an Integer (non-String) value 
for the property. The Schema produced by using `SchemaBuilder` looks like:
   `{"type":"string","logicalType":"LONGVARCHAR","maxLength": "50"}`
   vs. the expected:
   `{"type":"string","logicalType":"LONGVARCHAR","maxLength": 50}`
   
   I think its the same reason, Hive's 
[TypeInfoToSchema#L116](https://github.com/apache/hive/blob/5d268834a5f5278ea76399f8af0d0ab043ae0b45/serde/src/java/org/apache/hadoop/hive/serde2/avro/TypeInfoToSchema.java#L116)
 also uses the JSON based parsing approach.




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

For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 606636)
    Time Spent: 4.5h  (was: 4h 20m)

> AvroUtils exception when converting JDBC Row to GenericRecord
> -------------------------------------------------------------
>
>                 Key: BEAM-12385
>                 URL: https://issues.apache.org/jira/browse/BEAM-12385
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-java-core
>            Reporter: Anant Damle
>            Assignee: Anant Damle
>            Priority: P2
>             Fix For: 2.31.0
>
>          Time Spent: 4.5h
>  Remaining Estimate: 0h
>
> `{{AvroUtils.toAvroSchema()}}` and `{{AvroUtils.toGenericRecord()}}` throw 
> exception for JDBC specific logical types like: `{{DATE}}`, `{{NVARCHAR}}`, 
> `{{VARCHAR}}`, `{{LONGVARCHAR}}` etc.
>  
> {code:java}
> pipeline
>   .apply(JdbcIO.readRows()...)
>   .apply(MapElements.via(
>         (Row row) -> {
>          // This statement throws Exception "Unhandled Logical Type"
>           org.apache.avro.Schema schema = 
> AvroUtils.toAvroSchema(row.getSchema());
>         }));
> {code}
>  
>  
> This can be handled by adding additional case-statements in AvroUtils:



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to