You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by bl...@apache.org on 2015/03/05 03:24:24 UTC

incubator-parquet-mr git commit: PARQUET-134 patch - Support file write mode

Repository: incubator-parquet-mr
Updated Branches:
  refs/heads/master ea81e9aac -> 998d6507e


PARQUET-134 patch - Support file write mode

Julien,
   I changed the integer constants to enum as you requested.  Please review the patch.

Thanks.

Author: Mariappan Asokan <ma...@gmail.com>

Closes #111 from masokan/master and squashes the following commits:

7a8aa6f [Mariappan Asokan] PARQUET-134 patch - Support file write mode


Project: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/commit/998d6507
Tree: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/tree/998d6507
Diff: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/diff/998d6507

Branch: refs/heads/master
Commit: 998d6507ecabf025188d9f3e8c8367f810895a17
Parents: ea81e9a
Author: Mariappan Asokan <ma...@gmail.com>
Authored: Wed Mar 4 18:24:21 2015 -0800
Committer: Ryan Blue <bl...@apache.org>
Committed: Wed Mar 4 18:24:21 2015 -0800

----------------------------------------------------------------------
 .../java/parquet/hadoop/ParquetFileWriter.java  | 29 +++++++++++++---
 .../main/java/parquet/hadoop/ParquetWriter.java | 35 +++++++++++++++++++-
 .../parquet/hadoop/TestParquetFileWriter.java   | 33 ++++++++++++++++++
 3 files changed, 91 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/998d6507/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java
index 0420170..e211ce4 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetFileWriter.java
@@ -69,6 +69,12 @@ public class ParquetFileWriter {
   public static final byte[] MAGIC = "PAR1".getBytes(Charset.forName("ASCII"));
   public static final int CURRENT_VERSION = 1;
 
+  // File creation modes
+  public static enum Mode {
+    CREATE,
+    OVERWRITE
+  }
+
   private static final ParquetMetadataConverter metadataConverter = new ParquetMetadataConverter();
 
   private final MessageType schema;
@@ -144,17 +150,30 @@ public class ParquetFileWriter {
   private STATE state = STATE.NOT_STARTED;
 
   /**
-   *
+   * @param configuration Hadoop configuration
+   * @param schema the schema of the data
+   * @param file the file to write to
+   * @throws IOException if the file can not be created
+   */
+  public ParquetFileWriter(Configuration configuration, MessageType schema,
+      Path file) throws IOException {
+    this(configuration, schema, file, Mode.CREATE);
+  }
+
+  /**
+   * @param configuration Hadoop configuration
    * @param schema the schema of the data
-   * @param out the file to write to
-   * @param codec the codec to use to compress blocks
+   * @param file the file to write to
+   * @param mode file creation mode
    * @throws IOException if the file can not be created
    */
-  public ParquetFileWriter(Configuration configuration, MessageType schema, Path file) throws IOException {
+  public ParquetFileWriter(Configuration configuration, MessageType schema,
+      Path file, Mode mode) throws IOException {
     super();
     this.schema = schema;
     FileSystem fs = file.getFileSystem(configuration);
-    this.out = fs.create(file, false);
+    boolean overwriteFlag = (mode == Mode.OVERWRITE);
+    this.out = fs.create(file, overwriteFlag);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/998d6507/parquet-hadoop/src/main/java/parquet/hadoop/ParquetWriter.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetWriter.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetWriter.java
index 4754ee9..863132f 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetWriter.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetWriter.java
@@ -172,11 +172,44 @@ public class ParquetWriter<T> implements Closeable {
       boolean validating,
       WriterVersion writerVersion,
       Configuration conf) throws IOException {
+    this(file, ParquetFileWriter.Mode.CREATE, writeSupport,
+        compressionCodecName, blockSize, pageSize, dictionaryPageSize,
+        enableDictionary, validating, writerVersion, conf);
+  }
+  /**
+   * Create a new ParquetWriter.
+   *
+   * @param file the file to create
+   * @param mode file creation mode
+   * @param writeSupport the implementation to write a record to a RecordConsumer
+   * @param compressionCodecName the compression codec to use
+   * @param blockSize the block size threshold
+   * @param pageSize the page size threshold
+   * @param dictionaryPageSize the page size threshold for the dictionary pages
+   * @param enableDictionary to turn dictionary encoding on
+   * @param validating to turn on validation using the schema
+   * @param writerVersion version of parquetWriter from {@link ParquetProperties.WriterVersion}
+   * @param conf Hadoop configuration to use while accessing the filesystem
+   * @throws IOException
+   */
+  public ParquetWriter(
+      Path file,
+      ParquetFileWriter.Mode mode,
+      WriteSupport<T> writeSupport,
+      CompressionCodecName compressionCodecName,
+      int blockSize,
+      int pageSize,
+      int dictionaryPageSize,
+      boolean enableDictionary,
+      boolean validating,
+      WriterVersion writerVersion,
+      Configuration conf) throws IOException {
 
     WriteSupport.WriteContext writeContext = writeSupport.init(conf);
     MessageType schema = writeContext.getSchema();
 
-    ParquetFileWriter fileWriter = new ParquetFileWriter(conf, schema, file);
+    ParquetFileWriter fileWriter = new ParquetFileWriter(conf, schema, file,
+        mode);
     fileWriter.start();
 
     CodecFactory codecFactory = new CodecFactory(conf);

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/998d6507/parquet-hadoop/src/test/java/parquet/hadoop/TestParquetFileWriter.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/test/java/parquet/hadoop/TestParquetFileWriter.java b/parquet-hadoop/src/test/java/parquet/hadoop/TestParquetFileWriter.java
index 4152956..4a81b85 100644
--- a/parquet-hadoop/src/test/java/parquet/hadoop/TestParquetFileWriter.java
+++ b/parquet-hadoop/src/test/java/parquet/hadoop/TestParquetFileWriter.java
@@ -64,6 +64,39 @@ public class TestParquetFileWriter {
   private String writeSchema;
 
   @Test
+  public void testWriteMode() throws Exception {
+    File testDir = new File("target/test/TestParquetFileWriter/");
+    testDir.mkdirs();
+    File testFile = new File(testDir, "testParquetFile");
+    testFile = testFile.getAbsoluteFile();
+    testFile.createNewFile();
+    MessageType schema = MessageTypeParser.parseMessageType(
+        "message m { required group a {required binary b;} required group "
+        + "c { required int64 d; }}");
+    Configuration conf = new Configuration();
+
+    ParquetFileWriter writer = null;
+    boolean exceptionThrown = false;
+    Path path = new Path(testFile.toURI());
+    try {
+      writer = new ParquetFileWriter(conf, schema, path,
+          ParquetFileWriter.Mode.CREATE);
+    } catch(IOException ioe1) {
+      exceptionThrown = true;
+    }
+    assertTrue(exceptionThrown);
+    exceptionThrown = false;
+    try {
+      writer = new ParquetFileWriter(conf, schema, path,
+          ParquetFileWriter.Mode.OVERWRITE);
+    } catch(IOException ioe2) {
+      exceptionThrown = true;
+    }
+    assertTrue(!exceptionThrown);
+    testFile.delete();
+  }
+
+  @Test
   public void testWriteRead() throws Exception {
 
     File testFile = new File("target/test/TestParquetFileWriter/testParquetFile").getAbsoluteFile();