This is an automated email from the ASF dual-hosted git repository.

gangwu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-mr.git


The following commit(s) were added to refs/heads/master by this push:
     new cfa054067 PARQUET-2265: Don't set default Model in AvroParquetWriter 
(#1049)
cfa054067 is described below

commit cfa05406780de4df2a32da387bfe9c34de7b6a7b
Author: Claire McGinty <[email protected]>
AuthorDate: Thu Apr 13 23:27:33 2023 -0400

    PARQUET-2265: Don't set default Model in AvroParquetWriter (#1049)
    
    - Don't set default Model in AvroParquetWriter
    - Test that data model is parsed from Configuration
---
 .../org/apache/parquet/avro/AvroParquetWriter.java |  2 +-
 .../org/apache/parquet/avro/TestReadWrite.java     | 85 ++++++++++++++++++++++
 2 files changed, 86 insertions(+), 1 deletion(-)

diff --git 
a/parquet-avro/src/main/java/org/apache/parquet/avro/AvroParquetWriter.java 
b/parquet-avro/src/main/java/org/apache/parquet/avro/AvroParquetWriter.java
index 9d514673e..94d8167b0 100644
--- a/parquet-avro/src/main/java/org/apache/parquet/avro/AvroParquetWriter.java
+++ b/parquet-avro/src/main/java/org/apache/parquet/avro/AvroParquetWriter.java
@@ -160,7 +160,7 @@ public class AvroParquetWriter<T> extends ParquetWriter<T> {
 
   public static class Builder<T> extends ParquetWriter.Builder<T, Builder<T>> {
     private Schema schema = null;
-    private GenericData model = SpecificData.get();
+    private GenericData model = null;
 
     private Builder(Path file) {
       super(file);
diff --git 
a/parquet-avro/src/test/java/org/apache/parquet/avro/TestReadWrite.java 
b/parquet-avro/src/test/java/org/apache/parquet/avro/TestReadWrite.java
index 66f166cfd..6484ab4a6 100644
--- a/parquet-avro/src/test/java/org/apache/parquet/avro/TestReadWrite.java
+++ b/parquet-avro/src/test/java/org/apache/parquet/avro/TestReadWrite.java
@@ -27,6 +27,8 @@ import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
+import java.time.LocalDate;
+import java.time.format.DateTimeFormatter;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -35,7 +37,10 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+
+import org.apache.avro.Conversion;
 import org.apache.avro.Conversions;
+import org.apache.avro.LogicalType;
 import org.apache.avro.LogicalTypes;
 import org.apache.avro.Schema;
 import org.apache.avro.SchemaBuilder;
@@ -775,6 +780,86 @@ public class TestReadWrite {
     }
   }
 
+  public static class CustomDataModel implements AvroDataSupplier {
+    @Override
+    public GenericData get() {
+      GenericData genericData = new GenericData();
+      genericData.addLogicalTypeConversion(new Conversion<LocalDate>() {
+        private final DateTimeFormatter dateTimeFormatter = 
DateTimeFormatter.ofPattern("yyyyMMdd");
+
+        @Override
+        public Class<LocalDate> getConvertedType() {
+          return LocalDate.class;
+        }
+
+        @Override
+        public String getLogicalTypeName() {
+          return "date";
+        }
+
+        public LocalDate fromInt(Integer localDate, Schema schema, LogicalType 
type) {
+          return LocalDate.parse(String.valueOf(localDate), dateTimeFormatter);
+        }
+
+        public Integer toInt(LocalDate date, Schema schema, LogicalType type) {
+          return Integer.parseInt(dateTimeFormatter.format(date));
+        }
+      });
+      return genericData;
+    }
+  }
+  @Test
+  public void testParsesDataModelFromConf() throws Exception {
+    Schema datetimeSchema = Schema.createRecord("myrecord", null, null, false);
+    Schema date = LogicalTypes.date().addToSchema(
+      Schema.create(Schema.Type.INT));
+    datetimeSchema.setFields(Collections.singletonList(
+      new Schema.Field("date", date, null, null)));
+
+    File file = temp.newFile("datetime.parquet");
+    file.delete();
+    Path path = new Path(file.toString());
+    List<GenericRecord> expected = Lists.newArrayList();
+
+    Configuration conf = new Configuration();
+    AvroWriteSupport.setAvroDataSupplier(conf, CustomDataModel.class);
+
+    // .withDataModel is not set; AvroWriteSupport should parse it from the 
Configuration
+    try(ParquetWriter<GenericRecord> writer = AvroParquetWriter
+      .<GenericRecord>builder(path)
+      .withConf(conf)
+      .withSchema(datetimeSchema)
+      .build()) {
+
+      GenericRecordBuilder builder = new GenericRecordBuilder(datetimeSchema);
+      for (int i = 0; i < 100; i += 1) {
+        builder.set("date", LocalDate.now().minusDays(i));
+
+        GenericRecord rec = builder.build();
+        expected.add(rec);
+        writer.write(builder.build());
+      }
+    }
+    List<GenericRecord> records = Lists.newArrayList();
+
+    AvroReadSupport.setAvroDataSupplier(conf, CustomDataModel.class);
+
+    try(ParquetReader<GenericRecord> reader = AvroParquetReader
+      .<GenericRecord>builder(path)
+      .disableCompatibility()
+      .withConf(conf)
+      .build()) {
+      GenericRecord rec;
+      while ((rec = reader.read()) != null) {
+        records.add(rec);
+      }
+    }
+
+    Assert.assertTrue("date field should be a LocalDate instance",
+      records.get(0).get("date") instanceof LocalDate);
+    Assert.assertEquals("Content should match", expected, records);
+  }
+
   private File createTempFile() throws IOException {
     File tmp = File.createTempFile(getClass().getSimpleName(), ".tmp");
     tmp.deleteOnExit();

Reply via email to