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 fadbe6e  Parquet-1860: Add missing Builder Class to ProtoParquetWriter 
Class (#791)
fadbe6e is described below

commit fadbe6ef326faa5984d4d8d6df581a91c8c6cca2
Author: Santosh Lade <[email protected]>
AuthorDate: Thu Jul 23 01:35:38 2020 +0530

    Parquet-1860: Add missing Builder Class to ProtoParquetWriter Class (#791)
    
    * Add missing Builder support
    
    ProtoParquetWriter only has basic constructors, that which call deprecated 
super constructors. We cannot set many other options (Write mode, encoding etc) 
as well. Extended the ParquetWriter.Builder class for builder support.
    
    * Add import for OutputFile
    
    Parquet-1860 checks failed due ambiguous constructor. Added a missing 
import, and proceed to recheck.
    
    * Add Configuration import
    
    Added missing import of class Configuration
---
 .../apache/parquet/proto/ProtoParquetWriter.java   | 39 ++++++++++++++++++++++
 1 file changed, 39 insertions(+)

diff --git 
a/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoParquetWriter.java
 
b/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoParquetWriter.java
index ef9a5ba..bef7436 100644
--- 
a/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoParquetWriter.java
+++ 
b/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoParquetWriter.java
@@ -24,6 +24,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.parquet.hadoop.ParquetWriter;
 import org.apache.parquet.hadoop.api.WriteSupport;
 import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+import org.apache.parquet.io.OutputFile;
+import org.apache.hadoop.conf.Configuration;
 
 import java.io.IOException;
 
@@ -80,5 +82,42 @@ public class ProtoParquetWriter<T extends MessageOrBuilder> 
extends ParquetWrite
     this(file, protoMessage, CompressionCodecName.UNCOMPRESSED,
             DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE);
   }
+  
+  public static <T> Builder<T> builder(Path file) {
+           return new Builder<T>(file);
+       }
 
+       public static <T> Builder<T> builder(OutputFile file) {
+           return new Builder<T>(file);
+       }
+       
+       private static <T extends MessageOrBuilder> WriteSupport<T> 
writeSupport(Class<? extends Message> protoMessage) {
+               return new ProtoWriteSupport<T>(protoMessage);
+       }
+         
+       public static class Builder<T> extends ParquetWriter.Builder<T, 
Builder<T>> {
+                 
+               Class<? extends Message> protoMessage = null;
+
+               private Builder(Path file) {
+                       super(file);
+               }
+
+               private Builder(OutputFile file) {
+                   super(file);
+               }
+
+               protected Builder<T> self() {
+                   return this;
+               }
+               
+               public Builder<T> withMessage(Class<? extends Message> 
protoMessage){
+                       this.protoMessage = protoMessage;
+                       return this;
+               }
+
+               protected WriteSupport<T> getWriteSupport(Configuration conf) {
+                   return (WriteSupport<T>) 
ProtoParquetWriter.writeSupport(protoMessage);
+               }    
+       }
 }

Reply via email to