Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

PARQUET-1292 Adding constructors to ProtoParquetWriter with writeSpecsCompliant flag #473

Open
wants to merge 2 commits into
base: master
Choose a base branch
from
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import com.google.protobuf.Message;
import com.google.protobuf.MessageOrBuilder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.api.WriteSupport;
Expand Down Expand Up @@ -68,17 +69,77 @@ public ProtoParquetWriter(Path file, Class<? extends Message> protoMessage,
compressionCodecName, blockSize, pageSize, enableDictionary, validating);
}

/**
* Create a new {@link ProtoParquetWriter}. The default block size is {@link ParquetWriter#DEFAULT_BLOCK_SIZE} bytes.The
* default page size is {@link ParquetWriter#DEFAULT_PAGE_SIZE} bytes. Default compression is no compression. (Inherited from
* {@link ParquetWriter})
*
* @param file The file name to write to.
* @param protoMessage Protobuf message class
* @throws IOException if there is an error while writing
*/
public ProtoParquetWriter(Path file, Class<? extends Message> protoMessage) throws IOException {
this(file, protoMessage, CompressionCodecName.UNCOMPRESSED,
DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE);
}

/**
* Create a new {@link ProtoParquetWriter}. The default block size is 50 MB.The default
* page size is 1 MB. Default compression is no compression. (Inherited from {@link ParquetWriter})
* Create a new {@link ProtoParquetWriter}.
*
* @param file The file name to write to.
* @param file The file name to write to.
* @param protoMessage Protobuf message class
* @param compressionCodecName Compression code to use, or CompressionCodecName.UNCOMPRESSED
* @param blockSize HDFS block size
* @param pageSize See parquet write up. Blocks are subdivided into pages for alignment and other purposes.
* @param enableDictionary Whether to use a dictionary to compress columns.
* @param validating to turn on validation using the schema
* @param writeSpecsCompliant Enable writing specs compliant schemas with parquet-protobuf
* @throws IOException if there is an error while writing
*/
public ProtoParquetWriter(Path file, Class<? extends Message> protoMessage) throws IOException {
public ProtoParquetWriter(Path file, Class<? extends Message> protoMessage,
CompressionCodecName compressionCodecName, int blockSize, int pageSize, boolean enableDictionary,
boolean validating, boolean writeSpecsCompliant) throws IOException {
super(file, new ProtoWriteSupport(protoMessage), compressionCodecName, blockSize, pageSize, pageSize,
enableDictionary, validating, DEFAULT_WRITER_VERSION,
getConfigWithWriteSpecsCompliant(writeSpecsCompliant));
}

/**
* Create a new {@link ProtoParquetWriter}.
*
* @param file The file name to write to.
* @param protoMessage Protobuf message class
* @param compressionCodecName Compression code to use, or CompressionCodecName.UNCOMPRESSED
* @param blockSize HDFS block size
* @param pageSize See parquet write up. Blocks are subdivided into pages for alignment and other purposes.
* @param writeSpecsCompliant Enable writing specs compliant schemas with parquet-protobuf
* @throws IOException if there is an error while writing
*/
public ProtoParquetWriter(Path file, Class<? extends Message> protoMessage,
CompressionCodecName compressionCodecName, int blockSize,
int pageSize, boolean writeSpecsCompliant) throws IOException {
this(file, protoMessage, compressionCodecName, blockSize, pageSize, DEFAULT_IS_DICTIONARY_ENABLED,
DEFAULT_IS_VALIDATING_ENABLED, writeSpecsCompliant);
}

/**
* Create a new {@link ProtoParquetWriter}. The default block size is {@link ParquetWriter#DEFAULT_BLOCK_SIZE} bytes.The
* default page size is {@link ParquetWriter#DEFAULT_PAGE_SIZE} bytes. Default compression is no compression.
*
* @param file The file name to write to.
* @param protoMessage Protobuf message class
* @param writeSpecsCompliant Enable writing specs compliant schemas with parquet-protobuf
* @throws IOException if there is an error while writing
*/
public ProtoParquetWriter(Path file, Class<? extends Message> protoMessage, boolean writeSpecsCompliant) throws IOException {
this(file, protoMessage, CompressionCodecName.UNCOMPRESSED,
DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE);
DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE, writeSpecsCompliant);
}

private static Configuration getConfigWithWriteSpecsCompliant(boolean writeSpecsCompliant) {
Configuration config = new Configuration();
ProtoWriteSupport.setWriteSpecsCompliant(config, writeSpecsCompliant);
return config;
}

}