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

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

                Author: ASF GitHub Bot
            Created on: 27/Apr/21 17:24
            Start Date: 27/Apr/21 17:24
    Worklog Time Spent: 10m 
      Work Description: ernvalentino commented on a change in pull request 
#14644:
URL: https://github.com/apache/beam/pull/14644#discussion_r621435078



##########
File path: 
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -1210,6 +1221,13 @@ public void open(WritableByteChannel channel) throws 
IOException {
               .withWriteMode(OVERWRITE)
               
.withConf(SerializableConfiguration.newConfiguration(getConfiguration()))
               .withRowGroupSize(getRowGroupSize());
+      if (getAvroDataModel() != null) {
+        try {
+          builder.withDataModel(buildModelObject(getAvroDataModel()));
+        } catch (ReflectiveOperationException e) {
+          LOG.warn("Couldn't set the model: " + e.getMessage());

Review comment:
       Which could be a non-sense since if a user specify a model this is 
silently failing and setting a different model. So I think you are right this 
should cause the pipeline to fail




-- 
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:
us...@infra.apache.org


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

    Worklog Id:     (was: 589890)
    Time Spent: 2h  (was: 1h 50m)

> ParquetIO sink should allow to pass an Avro data model
> ------------------------------------------------------
>
>                 Key: BEAM-12165
>                 URL: https://issues.apache.org/jira/browse/BEAM-12165
>             Project: Beam
>          Issue Type: Bug
>          Components: io-java-parquet
>            Reporter: Ning Kang
>            Assignee: Ernesto Valentino
>            Priority: P2
>          Time Spent: 2h
>  Remaining Estimate: 0h
>
> AvroParquetWriter instantiated in ParquetIO [1] does not specify the data 
> model.
> The default is SpecificData model [2], while the AvroParquetReader is reading 
> with a GenericData model [3].
> ParquetIO should pass in the correct data model.
> [1] 
> https://github.com/apache/beam/blob/v2.28.0/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java#L1052
> [2] 
> https://github.com/apache/parquet-mr/blob/master/parquet-avro/src/main/java/org/apache/parquet/avro/AvroParquetWriter.java#L163
> [3] 
> https://github.com/apache/beam/blob/v2.28.0/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java#L704



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

Reply via email to