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

fokko 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 9a1fbc4  PARQUET-1778: Do Not Consider Class for Avro Generic Record 
Reader (#751)
9a1fbc4 is described below

commit 9a1fbc4ee3f63284a675eeac6c62e96ffc973575
Author: belugabehr <[email protected]>
AuthorDate: Wed Jul 22 16:06:50 2020 -0400

    PARQUET-1778: Do Not Consider Class for Avro Generic Record Reader (#751)
---
 .../org/apache/parquet/avro/AvroParquetReader.java | 32 +++++++++++++++
 .../org/apache/parquet/avro/TestReadWrite.java     | 46 ++++++++++++++++++++++
 2 files changed, 78 insertions(+)

diff --git 
a/parquet-avro/src/main/java/org/apache/parquet/avro/AvroParquetReader.java 
b/parquet-avro/src/main/java/org/apache/parquet/avro/AvroParquetReader.java
index d26fb63..8970b66 100644
--- a/parquet-avro/src/main/java/org/apache/parquet/avro/AvroParquetReader.java
+++ b/parquet-avro/src/main/java/org/apache/parquet/avro/AvroParquetReader.java
@@ -21,6 +21,7 @@ package org.apache.parquet.avro;
 import java.io.IOException;
 
 import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.specific.SpecificData;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -53,6 +54,37 @@ public class AvroParquetReader<T> extends ParquetReader<T> {
   }
 
   /**
+   * Convenience method for creating a ParquetReader which uses Avro
+   * {@link GenericData} objects to store data from reads.
+   *
+   * @param file The location to read data from
+   * @return A {@code ParquetReader} which reads data as Avro
+   *         {@code GenericData}
+   * @throws IOException if the InputFile has been closed, or if some other I/O
+   *           error occurs
+   */
+  public static ParquetReader<GenericRecord> genericRecordReader(InputFile 
file) throws IOException {
+    return new 
Builder<GenericRecord>(file).withDataModel(GenericData.get()).build();
+  }
+
+  /**
+   * Convenience method for creating a ParquetReader which uses Avro
+   * {@link GenericData} objects to store data from reads.
+   *
+   * @param file The location to read data from
+   * @return A {@code ParquetReader} which reads data as Avro
+   *         {@code GenericData}
+   * @throws IOException if the InputFile has been closed, or if some other I/O
+   *           error occurs
+   *
+   * @deprecated will be removed in 2.0.0; use {@link 
#genericRecordReader(InputFile)} instead.
+   */
+  @Deprecated
+  public static ParquetReader<GenericRecord> genericRecordReader(Path file) 
throws IOException {
+    return new 
Builder<GenericRecord>(file).withDataModel(GenericData.get()).build();
+  }
+
+  /**
    * @param file a file path
    * @throws IOException if there is an error while reading
    * @deprecated will be removed in 2.0.0; use {@link #builder(InputFile)} 
instead.
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 6bdf61e..d4048ff 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
@@ -727,6 +727,52 @@ public class TestReadWrite {
     assertEquals(str("hello"), l2List.get(0));
   }
 
+  /**
+   * A test demonstrating the most simple way to write and read Parquet files
+   * using Avro {@link GenericRecord}.
+   */
+  @Test
+  public void testSimpleGeneric() throws IOException {
+    final Schema schema =
+        Schema.createRecord("Person", null, "org.apache.parquet", false);
+    schema.setFields(Arrays.asList(
+        new Schema.Field("name", Schema.create(Schema.Type.STRING), null, 
null),
+        new Schema.Field("weight", Schema.create(Schema.Type.INT), null,
+            null)));
+
+    final Path file = new Path(createTempFile().getPath());
+
+    try (final ParquetWriter<GenericData.Record> parquetWriter =
+        AvroParquetWriter.<GenericData.Record> builder(file).withSchema(schema)
+            .build()) {
+
+      final GenericData.Record fooRecord = new GenericData.Record(schema);
+      fooRecord.put("name", "foo");
+      fooRecord.put("weight", 123);
+
+      final GenericData.Record oofRecord = new GenericData.Record(schema);
+      oofRecord.put("name", "oof");
+      oofRecord.put("weight", 321);
+
+      parquetWriter.write(fooRecord);
+      parquetWriter.write(oofRecord);
+    }
+
+    // Read the file. String data is returned as org.apache.avro.util.Utf8 so 
it
+    // must be converting to a String before checking equality
+    try (ParquetReader<GenericRecord> reader =
+        AvroParquetReader.genericRecordReader(file)) {
+
+      final GenericRecord r1 = reader.read();
+      assertEquals("foo", r1.get("name").toString());
+      assertEquals(123, r1.get("weight"));
+
+      final GenericRecord r2 = reader.read();
+      assertEquals("oof", r2.get("name").toString());
+      assertEquals(321, r2.get("weight"));
+    }
+  }
+
   private File createTempFile() throws IOException {
     File tmp = File.createTempFile(getClass().getSimpleName(), ".tmp");
     tmp.deleteOnExit();

Reply via email to