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

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

                Author: ASF GitHub Bot
            Created on: 27/Apr/21 15:33
            Start Date: 27/Apr/21 15:33
    Worklog Time Spent: 10m 
      Work Description: aromanenko-dev commented on a change in pull request 
#14644:
URL: https://github.com/apache/beam/pull/14644#discussion_r621316454



##########
File path: 
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -1166,6 +1168,8 @@ public static Sink sink(Schema schema) {
 
       abstract Builder setRowGroupSize(int rowGroupSize);
 
+      abstract Builder setAvroDataModel(Class<? extends GenericData> 
modelClass);

Review comment:
       The similar question - can we just pass `GenericData` here because of 
`withAvroDataModel(GenericData)`?

##########
File path: 
sdks/java/io/parquet/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOTest.java
##########
@@ -506,6 +506,53 @@ public void 
testWriteAndReadwithSplitUsingReflectDataSchemaWithDataModel() {
     readPipeline.run().waitUntilFinish();
   }
 
+  @Test
+  public void testWriteAndReadUsingGenericDataSchemaWithDataModel() {
+    Schema schema = new Schema.Parser().parse(SCHEMA_STRING);
+
+    List<GenericRecord> records = generateGenericRecords(1000);
+    mainPipeline
+        .apply(Create.of(records).withCoder(AvroCoder.of(schema)))
+        .apply(
+            FileIO.<GenericRecord>write()
+                
.via(ParquetIO.sink(schema).withAvroDataModel(GenericData.get()))
+                .to(temporaryFolder.getRoot().getAbsolutePath()));
+    mainPipeline.run().waitUntilFinish();
+
+    PCollection<GenericRecord> readBack =
+        readPipeline.apply(
+            ParquetIO.read(schema)
+                .withAvroDataModel(GenericData.get())
+                .from(temporaryFolder.getRoot().getAbsolutePath() + "/*"));
+
+    PAssert.that(readBack).containsInAnyOrder(records);
+    readPipeline.run().waitUntilFinish();
+  }
+
+  @Test
+  public void testWriteAndReadwithSplitUsingGenericDataSchemaWithDataModel() {

Review comment:
       Can we expect a different behaviour using `.withSplit()`? If not then 
probably one of the tests is unneeded.

##########
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:
       Don't we need to fail a pipeline here?

##########
File path: 
sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
##########
@@ -1154,6 +1154,8 @@ public static Sink sink(Schema schema) {
 
     abstract int getRowGroupSize();
 
+    abstract @Nullable Class<? extends GenericData> getAvroDataModel();

Review comment:
       Can we just return `GenericData` here?




-- 
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: 589815)
    Time Spent: 1h 20m  (was: 1h 10m)

> 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: 1h 20m
>  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