Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
Already on GitHub? Sign in to your account
Filtered Reader Implementation, Avro Specific Support #68
5f0f929
ef5c143
61239a0
48bb48e
8285b62
ac5cbd1
be8e4e9
1d7a5c3
c4b14fb
f52a26e
80a449d
| @@ -16,7 +16,7 @@ | ||
| package parquet.avro; | ||
| import org.apache.avro.Schema; | ||
| -import org.apache.avro.generic.GenericRecord; | ||
| +import org.apache.avro.generic.IndexedRecord; | ||
| import org.apache.hadoop.mapreduce.Job; | ||
| import parquet.avro.AvroWriteSupport; | ||
| import parquet.hadoop.ParquetOutputFormat; | ||
| @@ -25,7 +25,7 @@ | ||
| /** | ||
| * A Hadoop {@link org.apache.hadoop.mapreduce.OutputFormat} for Parquet files. | ||
| */ | ||
| -public class AvroParquetOutputFormat extends ParquetOutputFormat<GenericRecord> { | ||
| +public class AvroParquetOutputFormat extends ParquetOutputFormat<IndexedRecord> { | ||
dvryaboy
Contributor
|
||
| public static void setSchema(Job job, Schema schema) { | ||
| AvroWriteSupport.setSchema(ContextUtil.getConfiguration(job), schema); | ||
| @@ -17,6 +17,7 @@ | ||
| import java.io.IOException; | ||
| import org.apache.avro.Schema; | ||
| +import org.apache.avro.generic.IndexedRecord; | ||
| import org.apache.hadoop.fs.Path; | ||
| import parquet.hadoop.ParquetWriter; | ||
| import parquet.hadoop.api.WriteSupport; | ||
| @@ -25,7 +26,7 @@ | ||
| /** | ||
| * Write Avro records to a Parquet file. | ||
| */ | ||
| -public class AvroParquetWriter<T> extends ParquetWriter<T> { | ||
| +public class AvroParquetWriter<T extends IndexedRecord> extends ParquetWriter<T> { | ||
JacobMetcalf
Contributor
|
||
| /** Create a new {@link AvroParquetWriter}. | ||
| * | ||
| @@ -39,21 +40,38 @@ | ||
| public AvroParquetWriter(Path file, Schema avroSchema, | ||
| CompressionCodecName compressionCodecName, int blockSize, | ||
| int pageSize) throws IOException { | ||
| + super(file, (WriteSupport<T>)new AvroWriteSupport(new AvroSchemaConverter().convert(avroSchema), avroSchema), | ||
| + compressionCodecName, blockSize, pageSize); | ||
| + } | ||
| + | ||
| + /** Create a new {@link AvroParquetWriter}. | ||
| + * | ||
| + * @param file The file name to write to. | ||
| + * @param avroSchema The schema to write with. | ||
| + * @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. | ||
| + * @throws IOException | ||
| + */ | ||
| + public AvroParquetWriter(Path file, Schema avroSchema, | ||
| + CompressionCodecName compressionCodecName, int blockSize, | ||
| + int pageSize, boolean enableDictionary) throws IOException { | ||
| super(file, (WriteSupport<T>) | ||
| - new AvroWriteSupport(new AvroSchemaConverter().convert(avroSchema), avroSchema), | ||
| - compressionCodecName, blockSize, pageSize); | ||
| + new AvroWriteSupport(new AvroSchemaConverter().convert(avroSchema),avroSchema), | ||
| + compressionCodecName, blockSize, pageSize, enableDictionary, false); | ||
| } | ||
| /** Create a new {@link AvroParquetWriter}. The default block size is 50 MB.The default | ||
| * page size is 1 MB. Default compression is no compression. (Inherited from {@link ParquetWriter}) | ||
| * | ||
| - * @param file | ||
| - * @param avroSchema | ||
| + * @param file The file name to write to. | ||
| + * @param avroSchema The schema to write with. | ||
| * @throws IOException | ||
| */ | ||
| public AvroParquetWriter(Path file, Schema avroSchema) throws IOException { | ||
| this(file, avroSchema, CompressionCodecName.UNCOMPRESSED, | ||
| - DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE); | ||
| + DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE); | ||
| } | ||
| } | ||
in what case is enumClass null ?
Please add a comment about this