@Override public BytesInput getBytes() { if (!dictionaryTooBig && dict.size() > 0) { // remember size of dictionary when we last wrote a page lastUsedDictionarySize = dict.size(); lastUsedDictionaryByteSize = dictionaryByteSize; int maxDicId = dict.size() - 1; if (DEBUG) LOG.debug("max dic id " + maxDicId); int bitWidth = BytesUtils.getWidthFromMaxInt(maxDicId); // TODO: what is a good initialCapacity? final RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(BytesUtils.getWidthFromMaxInt(maxDicId), 64 * 1024); IntIterator iterator = out.iterator(); try { while (iterator.hasNext()) { encoder.writeInt(iterator.next()); } // encodes the bit width byte[] bytesHeader = new byte[] {(byte) bitWidth}; BytesInput rleEncodedBytes = encoder.toBytes(); if (DEBUG) LOG.debug("rle encoded bytes " + rleEncodedBytes.size()); return concat(BytesInput.from(bytesHeader), rleEncodedBytes); } catch (IOException e) { throw new ParquetEncodingException("could not encode the values", e); } } return plainValuesWriter.getBytes(); }
/** * writes a single page * * @param valueCount count of values * @param uncompressedPageSize the size of the data once uncompressed * @param bytes the compressed data for the page without header * @param rlEncoding encoding of the repetition level * @param dlEncoding encoding of the definition level * @param valuesEncoding encoding of values */ public void writeDataPage( int valueCount, int uncompressedPageSize, BytesInput bytes, Statistics statistics, parquet.column.Encoding rlEncoding, parquet.column.Encoding dlEncoding, parquet.column.Encoding valuesEncoding) throws IOException { state = state.write(); long beforeHeader = out.getPos(); if (DEBUG) LOG.debug(beforeHeader + ": write data page: " + valueCount + " values"); int compressedPageSize = (int) bytes.size(); metadataConverter.writeDataPageHeader( uncompressedPageSize, compressedPageSize, valueCount, statistics, rlEncoding, dlEncoding, valuesEncoding, out); long headerSize = out.getPos() - beforeHeader; this.uncompressedLength += uncompressedPageSize + headerSize; this.compressedLength += compressedPageSize + headerSize; if (DEBUG) LOG.debug(out.getPos() + ": write data page content " + compressedPageSize); bytes.writeAllTo(out); currentStatistics.mergeStatistics(statistics); currentEncodings.add(rlEncoding); currentEncodings.add(dlEncoding); currentEncodings.add(valuesEncoding); }
private void checkRead() throws IOException { if (current == totalCountLoadedSoFar) { if (current != 0) { long timeAssembling = System.currentTimeMillis() - startedAssemblingCurrentBlockAt; totalTimeSpentProcessingRecords += timeAssembling; LOG.info( "Assembled and processed " + totalCountLoadedSoFar + " records from " + columnCount + " columns in " + totalTimeSpentProcessingRecords + " ms: " + ((float) totalCountLoadedSoFar / totalTimeSpentProcessingRecords) + " rec/ms, " + ((float) totalCountLoadedSoFar * columnCount / totalTimeSpentProcessingRecords) + " cell/ms"); long totalTime = totalTimeSpentProcessingRecords + totalTimeSpentReadingBytes; long percentReading = 100 * totalTimeSpentReadingBytes / totalTime; long percentProcessing = 100 * totalTimeSpentProcessingRecords / totalTime; LOG.info( "time spent so far " + percentReading + "% reading (" + totalTimeSpentReadingBytes + " ms) and " + percentProcessing + "% processing (" + totalTimeSpentProcessingRecords + " ms)"); } LOG.info("at row " + current + ". reading next block"); long t0 = System.currentTimeMillis(); PageReadStore pages = reader.readNextRowGroup(); if (pages == null) { throw new IOException( "expecting more rows but reached last block. Read " + current + " out of " + total); } long timeSpentReading = System.currentTimeMillis() - t0; totalTimeSpentReadingBytes += timeSpentReading; LOG.info( "block read in memory in " + timeSpentReading + " ms. row count = " + pages.getRowCount()); if (Log.DEBUG) LOG.debug("initializing Record assembly with requested schema " + requestedSchema); MessageColumnIO columnIO = columnIOFactory.getColumnIO(requestedSchema); recordReader = columnIO.getRecordReader(pages, recordConverter); startedAssemblingCurrentBlockAt = System.currentTimeMillis(); totalCountLoadedSoFar += pages.getRowCount(); } }
static void waitForJob(Job job) throws Exception { job.submit(); while (!job.isComplete()) { LOG.debug("waiting for job " + job.getJobName()); sleep(50); } LOG.debug( "status for job " + job.getJobName() + ": " + (job.isSuccessful() ? "SUCCESS" : "FAILURE")); if (!job.isSuccessful()) { throw new RuntimeException("job failed " + job.getJobName()); } }
/** * ends a block once all column chunks have been written * * @throws IOException */ public void endBlock() throws IOException { state = state.endBlock(); if (DEBUG) LOG.debug(out.getPos() + ": end block"); currentBlock.setRowCount(currentRecordCount); blocks.add(currentBlock); currentBlock = null; }
public int run(String[] args) throws Exception { if (args.length < 2) { LOG.error("Usage: " + getClass().getName() + " inputFileHDFS outputFileHDFS"); return 1; } String inputFile = args[0]; String outputFile = args[1] + System.nanoTime(); Configuration configuration = getConf(); Job job = new Job(configuration); job.setJarByClass(getClass()); job.setJobName(getClass().getName()); job.setMapperClass(ReadRequestMap.class); job.setNumReduceTasks(0); job.setInputFormatClass(TextInputFormat.class); job.setOutputFormatClass(TextOutputFormat.class); FileInputFormat.setInputPaths(job, new Path(inputFile)); FileOutputFormat.setOutputPath(job, new Path(outputFile)); job.waitForCompletion(true); return 0; }
/** * start a block * * @param recordCount the record count in this block * @throws IOException */ public void startBlock(long recordCount) throws IOException { state = state.startBlock(); if (DEBUG) LOG.debug(out.getPos() + ": start block"); // out.write(MAGIC); // TODO: add a magic delimiter currentBlock = new BlockMetaData(); currentRecordCount = recordCount; }
@Override public void writeBytes(Binary v) { if (!dictionaryTooBig) { writeBytesUsingDict(v); if (dictionaryByteSize > maxDictionaryByteSize || dict.size() > MAX_DICTIONARY_ENTRIES) { // if the dictionary reaches the max byte size or the values can not be encoded on two bytes // anymore. if (DEBUG) LOG.debug( "dictionary is now too big, falling back to plain: " + dictionaryByteSize + "B and " + dict.size() + " entries"); dictionaryTooBig = true; if (lastUsedDictionarySize == 0) { // if we never used the dictionary // we free dictionary encoded data dict = null; dictionaryByteSize = 0; out = null; } } } // write also to plain encoding if we need to fall back plainValuesWriter.writeBytes(v); }
/** * writes a number of pages at once * * @param bytes bytes to be written including page headers * @param uncompressedTotalPageSize total uncompressed size (without page headers) * @param compressedTotalPageSize total compressed size (without page headers) * @throws IOException */ void writeDataPages( BytesInput bytes, long uncompressedTotalPageSize, long compressedTotalPageSize, Statistics totalStats, List<parquet.column.Encoding> encodings) throws IOException { state = state.write(); if (DEBUG) LOG.debug(out.getPos() + ": write data pages"); long headersSize = bytes.size() - compressedTotalPageSize; this.uncompressedLength += uncompressedTotalPageSize + headersSize; this.compressedLength += compressedTotalPageSize + headersSize; if (DEBUG) LOG.debug(out.getPos() + ": write data pages content"); bytes.writeAllTo(out); currentEncodings.addAll(encodings); currentStatistics = totalStats; }
/** * ends a file once all blocks have been written. closes the file. * * @param extraMetaData the extra meta data to write in the footer * @throws IOException */ public void end(Map<String, String> extraMetaData) throws IOException { state = state.end(); if (DEBUG) LOG.debug(out.getPos() + ": end"); ParquetMetadata footer = new ParquetMetadata(new FileMetaData(schema, extraMetaData, Version.FULL_VERSION), blocks); serializeFooter(footer, out); out.close(); }
/** * @param bytes the bytes for this page * @param valueCount count of values in this page * @param uncompressedSize the uncompressed size of the page * @param encoding the encoding for this page */ public Page(BytesInput bytes, int valueCount, int uncompressedSize, Encoding encoding) { this.bytes = bytes; this.valueCount = valueCount; this.uncompressedSize = uncompressedSize; this.encoding = encoding; this.id = nextId++; if (DEBUG) LOG.debug("new Page #" + id + " : " + bytes.size() + " bytes and " + valueCount + " records"); }
public static void closeQuietly(Closeable res) { try { if (res != null) { res.close(); } } catch (IOException ioe) { LOG.warn("Exception closing reader " + res + ": " + ioe.getMessage()); } }
@Override public BytesInput getBytes() { try { bitPackingWriter.finish(); } catch (IOException e) { throw new ParquetEncodingException("could not write page", e); } if (Log.DEBUG) LOG.debug("writing a buffer of size " + out.size()); return BytesInput.from(out); }
public void run(Context context) throws IOException, InterruptedException { if (inputMessages == null || inputMessages.size() == 0) { throw new RuntimeException("No mock data given"); } else { for (Message msg : inputMessages) { context.write(null, msg); LOG.debug("Reading msg from mock writing mapper" + msg); } } }
private static void serializeFooter(ParquetMetadata footer, FSDataOutputStream out) throws IOException { long footerIndex = out.getPos(); parquet.format.FileMetaData parquetMetadata = new ParquetMetadataConverter().toParquetMetadata(CURRENT_VERSION, footer); writeFileMetaData(parquetMetadata, out); if (DEBUG) LOG.debug(out.getPos() + ": footer length = " + (out.getPos() - footerIndex)); BytesUtils.writeIntLittleEndian(out, (int) (out.getPos() - footerIndex)); out.write(MAGIC); }
/** {@inheritDoc} */ @Override public boolean nextKeyValue() throws IOException, InterruptedException { if (current < total) { checkRead(); currentValue = recordReader.read(); if (DEBUG) LOG.debug("read value: " + currentValue); current++; return true; } return false; }
/** * end a column (once all rep, def and data have been written) * * @throws IOException */ public void endColumn() throws IOException { state = state.endColumn(); if (DEBUG) LOG.debug(out.getPos() + ": end column"); currentBlock.addColumn( ColumnChunkMetaData.get( currentChunkPath, currentChunkType, currentChunkCodec, currentEncodings, currentStatistics, currentChunkFirstDataPage, currentChunkDictionaryPageOffset, currentChunkValueCount, compressedLength, uncompressedLength)); if (DEBUG) LOG.info("ended Column chumk: " + currentColumn); currentColumn = null; this.currentBlock.setTotalByteSize(currentBlock.getTotalByteSize() + uncompressedLength); this.uncompressedLength = 0; this.compressedLength = 0; }
/** * An implementation of the PLAIN encoding * * @author Julien Le Dem */ public class BooleanPlainValuesWriter extends ValuesWriter { private static final Log LOG = Log.getLog(BooleanPlainValuesWriter.class); private CapacityByteArrayOutputStream out; private BitPackingWriter bitPackingWriter; public BooleanPlainValuesWriter(int initialSize) { out = new CapacityByteArrayOutputStream(initialSize); bitPackingWriter = getBitPackingWriter(1, out); } @Override public final void writeBoolean(boolean v) { try { bitPackingWriter.write(v ? 1 : 0); } catch (IOException e) { throw new ParquetEncodingException("could not write boolean", e); } } @Override public long getBufferedSize() { return out.size(); } @Override public BytesInput getBytes() { try { bitPackingWriter.finish(); } catch (IOException e) { throw new ParquetEncodingException("could not write page", e); } if (Log.DEBUG) LOG.debug("writing a buffer of size " + out.size()); return BytesInput.from(out); } @Override public void reset() { out.reset(); bitPackingWriter = getBitPackingWriter(1, out); } @Override public long getAllocatedSize() { return out.getCapacity(); } @Override public Encoding getEncoding() { return PLAIN; } }
public class TestHDFSWrite extends Configured implements Tool { private static final Log LOG = Log.getLog(TestHDFSWrite.class); public static void main(String[] args) throws Exception { try { int res = ToolRunner.run(new Configuration(), new TestHDFSWrite(), args); System.exit(res); } catch (Exception e) { e.printStackTrace(); System.exit(255); } } public int run(String[] args) throws Exception { if (args.length < 2) { LOG.error("Usage: " + getClass().getName() + " inputFileHDFS outputFileHDFS"); return 1; } String inputFile = args[0]; String outputFile = args[1] + System.nanoTime(); Configuration configuration = getConf(); Job job = new Job(configuration); job.setJarByClass(getClass()); job.setJobName(getClass().getName()); job.setMapperClass(ReadRequestMap.class); job.setNumReduceTasks(0); job.setInputFormatClass(TextInputFormat.class); job.setOutputFormatClass(TextOutputFormat.class); FileInputFormat.setInputPaths(job, new Path(inputFile)); FileOutputFormat.setOutputPath(job, new Path(outputFile)); job.waitForCompletion(true); return 0; } public static class ReadRequestMap extends Mapper<LongWritable, Text, Void, Text> { private MessageType schema = null; @Override public void map(LongWritable key, Text value, Context context) throws IOException, InterruptedException { context.write(null, value); } } }
public static File getParquetFile( String name, String version, String module, boolean failIfNotExist) throws IOException { File parquetFile = new File("../" + version + "/target/parquet/", getParquetFileName(name, module)); parquetFile.getParentFile().mkdirs(); if (!parquetFile.exists()) { String msg = "File " + parquetFile.getAbsolutePath() + " does not exist"; if (failIfNotExist) { throw new IOException(msg); } LOG.warn(msg); } return parquetFile; }
/** * writes a dictionary page page * * @param dictionaryPage the dictionary page */ public void writeDictionaryPage(DictionaryPage dictionaryPage) throws IOException { state = state.write(); if (DEBUG) LOG.debug( out.getPos() + ": write dictionary page: " + dictionaryPage.getDictionarySize() + " values"); currentChunkDictionaryPageOffset = out.getPos(); int uncompressedSize = dictionaryPage.getUncompressedSize(); int compressedPageSize = (int) dictionaryPage.getBytes().size(); // TODO: fix casts metadataConverter.writeDictionaryPageHeader( uncompressedSize, compressedPageSize, dictionaryPage.getDictionarySize(), dictionaryPage.getEncoding(), out); long headerSize = out.getPos() - currentChunkDictionaryPageOffset; this.uncompressedLength += uncompressedSize + headerSize; this.compressedLength += compressedPageSize + headerSize; if (DEBUG) LOG.debug(out.getPos() + ": write dictionary page content " + compressedPageSize); dictionaryPage.getBytes().writeAllTo(out); currentEncodings.add(dictionaryPage.getEncoding()); }
/** * start a column inside a block * * @param descriptor the column descriptor * @param valueCount the value count in this column * @param statistics the statistics in this column * @param compressionCodecName * @throws IOException */ public void startColumn( ColumnDescriptor descriptor, long valueCount, CompressionCodecName compressionCodecName) throws IOException { state = state.startColumn(); if (DEBUG) LOG.debug(out.getPos() + ": start column: " + descriptor + " count=" + valueCount); currentEncodings = new HashSet<parquet.column.Encoding>(); currentChunkPath = ColumnPath.get(descriptor.getPath()); currentChunkType = descriptor.getType(); currentChunkCodec = compressionCodecName; currentChunkValueCount = valueCount; currentChunkFirstDataPage = out.getPos(); compressedLength = 0; uncompressedLength = 0; // need to know what type of stats to initialize to // better way to do this? currentStatistics = Statistics.getStatsBasedOnType(currentChunkType); }
/** {@inheritDoc} */ @Override public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException { Configuration configuration = taskAttemptContext.getConfiguration(); ParquetInputSplit parquetInputSplit = (ParquetInputSplit) inputSplit; this.requestedSchema = MessageTypeParser.parseMessageType(parquetInputSplit.getRequestedSchema()); this.columnCount = this.requestedSchema.getPaths().size(); this.recordConverter = readSupport.prepareForRead( configuration, parquetInputSplit.getExtraMetadata(), MessageTypeParser.parseMessageType(parquetInputSplit.getSchema()), new ReadSupport.ReadContext(requestedSchema)); Path path = parquetInputSplit.getPath(); List<BlockMetaData> blocks = parquetInputSplit.getBlocks(); List<ColumnDescriptor> columns = requestedSchema.getColumns(); reader = new ParquetFileReader(configuration, path, blocks, columns); for (BlockMetaData block : blocks) { total += block.getRowCount(); } LOG.info("RecordReader initialized will read a total of " + total + " records."); }
public RecordWriter<Void, T> getRecordWriter( Configuration conf, Path file, CompressionCodecName codec) throws IOException, InterruptedException { final WriteSupport<T> writeSupport = getWriteSupport(conf); CodecFactory codecFactory = new CodecFactory(conf); int blockSize = getBlockSize(conf); if (INFO) LOG.info("Parquet block size to " + blockSize); int pageSize = getPageSize(conf); if (INFO) LOG.info("Parquet page size to " + pageSize); int dictionaryPageSize = getDictionaryPageSize(conf); if (INFO) LOG.info("Parquet dictionary page size to " + dictionaryPageSize); boolean enableDictionary = getEnableDictionary(conf); if (INFO) LOG.info("Dictionary is " + (enableDictionary ? "on" : "off")); boolean validating = getValidation(conf); if (INFO) LOG.info("Validation is " + (validating ? "on" : "off")); WriterVersion writerVersion = getWriterVersion(conf); if (INFO) LOG.info("Writer version is: " + writerVersion); WriteContext init = writeSupport.init(conf); ParquetFileWriter w = new ParquetFileWriter(conf, init.getSchema(), file); w.start(); return new ParquetRecordWriter<T>( w, writeSupport, init.getSchema(), init.getExtraMetaData(), blockSize, pageSize, codecFactory.getCompressor(codec, pageSize), dictionaryPageSize, enableDictionary, validating, writerVersion); }
/** Writes data to parquet file using MapReduce job. */ public class WriteUsingMR { private static final Log LOG = Log.getLog(WriteUsingMR.class); Configuration conf = new Configuration(); private static List<Message> inputMessages; Path outputPath; public Configuration getConfiguration() { return conf; } public static class WritingMapper extends Mapper<LongWritable, Text, Void, Message> { public void run(Context context) throws IOException, InterruptedException { if (inputMessages == null || inputMessages.size() == 0) { throw new RuntimeException("No mock data given"); } else { for (Message msg : inputMessages) { context.write(null, msg); LOG.debug("Reading msg from mock writing mapper" + msg); } } } } public Path write(Message... messages) throws Exception { synchronized (WriteUsingMR.class) { outputPath = TestUtils.someTemporaryFilePath(); Path inputPath = TestUtils.someTemporaryFilePath(); FileSystem fileSystem = inputPath.getFileSystem(conf); fileSystem.create(inputPath); inputMessages = Collections.unmodifiableList(Arrays.asList(messages)); final Job job = new Job(conf, "write"); // input not really used TextInputFormat.addInputPath(job, inputPath); job.setInputFormatClass(TextInputFormat.class); job.setMapperClass(WritingMapper.class); job.setNumReduceTasks(0); job.setOutputFormatClass(ProtoParquetOutputFormat.class); ProtoParquetOutputFormat.setOutputPath(job, outputPath); ProtoParquetOutputFormat.setProtobufClass(job, TestUtils.inferRecordsClass(messages)); waitForJob(job); inputMessages = null; return outputPath; } } static void waitForJob(Job job) throws Exception { job.submit(); while (!job.isComplete()) { LOG.debug("waiting for job " + job.getJobName()); sleep(50); } LOG.debug( "status for job " + job.getJobName() + ": " + (job.isSuccessful() ? "SUCCESS" : "FAILURE")); if (!job.isSuccessful()) { throw new RuntimeException("job failed " + job.getJobName()); } } }
/** * one page in a chunk * * @author Julien Le Dem */ public class Page { private static final boolean DEBUG = Log.DEBUG; private static final Log LOG = Log.getLog(Page.class); private static int nextId = 0; private final BytesInput bytes; private final int valueCount; private final int uncompressedSize; private final Encoding encoding; private final int id; /** * @param bytes the bytes for this page * @param valueCount count of values in this page * @param uncompressedSize the uncompressed size of the page * @param encoding the encoding for this page */ public Page(BytesInput bytes, int valueCount, int uncompressedSize, Encoding encoding) { this.bytes = bytes; this.valueCount = valueCount; this.uncompressedSize = uncompressedSize; this.encoding = encoding; this.id = nextId++; if (DEBUG) LOG.debug("new Page #" + id + " : " + bytes.size() + " bytes and " + valueCount + " records"); } /** @return the bytes for the page */ public BytesInput getBytes() { return bytes; } /** @return the number of values in that page */ public int getValueCount() { return valueCount; } /** @return the uncompressed size of the page when the bytes are compressed */ public int getUncompressedSize() { return uncompressedSize; } /** @return the encoding for this page */ public Encoding getEncoding() { return encoding; } @Override public String toString() { return "Page [id: " + id + ", bytes.size=" + bytes.size() + ", valueCount=" + valueCount + ", uncompressedSize=" + uncompressedSize + ", encoding=" + encoding + "]"; } }
/** * Will attempt to encode values using a dictionary and fall back to plain encoding if the * dictionary gets too big * * @author Julien Le Dem */ public class DictionaryValuesWriter extends ValuesWriter { private static final Log LOG = Log.getLog(DictionaryValuesWriter.class); private static final int MAX_DICTIONARY_ENTRIES = 65535 /* 2^16 - 1 */; /** * maximum size in bytes allowed for the dictionary will fail over to plain encoding if reached */ private final int maxDictionaryByteSize; /** contains the values encoded in plain if the dictionary grows too big */ private final PlainValuesWriter plainValuesWriter; /** will become true if the dictionary becomes too big */ private boolean dictionaryTooBig; /** current size in bytes the dictionary will take once serialized */ private int dictionaryByteSize; /** * size in bytes of the dictionary at the end of last dictionary encoded page (in case the current * page falls back to PLAIN) */ private int lastUsedDictionaryByteSize; /** * size in items of the dictionary at the end of last dictionary encoded page (in case the current * page falls back to PLAIN) */ private int lastUsedDictionarySize; /** dictionary */ private Map<Binary, Integer> dict; /** dictionary encoded values */ private IntList out = new IntList(); public DictionaryValuesWriter(int maxDictionaryByteSize, int initialSize) { this.maxDictionaryByteSize = maxDictionaryByteSize; this.plainValuesWriter = new PlainValuesWriter(initialSize); resetDictionary(); } @Override public void writeBytes(Binary v) { if (!dictionaryTooBig) { writeBytesUsingDict(v); if (dictionaryByteSize > maxDictionaryByteSize || dict.size() > MAX_DICTIONARY_ENTRIES) { // if the dictionary reaches the max byte size or the values can not be encoded on two bytes // anymore. if (DEBUG) LOG.debug( "dictionary is now too big, falling back to plain: " + dictionaryByteSize + "B and " + dict.size() + " entries"); dictionaryTooBig = true; if (lastUsedDictionarySize == 0) { // if we never used the dictionary // we free dictionary encoded data dict = null; dictionaryByteSize = 0; out = null; } } } // write also to plain encoding if we need to fall back plainValuesWriter.writeBytes(v); } /** * will add an entry to the dictionary if the value is new * * @param v the value to dictionary encode */ private void writeBytesUsingDict(Binary v) { Integer id = dict.get(v); if (id == null) { id = dict.size(); dict.put(v, id); // length as int (2 bytes) + actual bytes dictionaryByteSize += 2 + v.length(); } out.add(id); } @Override public long getBufferedSize() { // size that will be written to a page // not including the dictionary size return dictionaryTooBig ? plainValuesWriter.getBufferedSize() : out.size() * 4; } @Override public long getAllocatedSize() { // size used in memory return (out == null ? 0 : out.size() * 4) + dictionaryByteSize + plainValuesWriter.getAllocatedSize(); } @Override public BytesInput getBytes() { if (!dictionaryTooBig && dict.size() > 0) { // remember size of dictionary when we last wrote a page lastUsedDictionarySize = dict.size(); lastUsedDictionaryByteSize = dictionaryByteSize; int maxDicId = dict.size() - 1; if (DEBUG) LOG.debug("max dic id " + maxDicId); int bitWidth = BytesUtils.getWidthFromMaxInt(maxDicId); // TODO: what is a good initialCapacity? final RunLengthBitPackingHybridEncoder encoder = new RunLengthBitPackingHybridEncoder(BytesUtils.getWidthFromMaxInt(maxDicId), 64 * 1024); IntIterator iterator = out.iterator(); try { while (iterator.hasNext()) { encoder.writeInt(iterator.next()); } // encodes the bit width byte[] bytesHeader = new byte[] {(byte) bitWidth}; BytesInput rleEncodedBytes = encoder.toBytes(); if (DEBUG) LOG.debug("rle encoded bytes " + rleEncodedBytes.size()); return concat(BytesInput.from(bytesHeader), rleEncodedBytes); } catch (IOException e) { throw new ParquetEncodingException("could not encode the values", e); } } return plainValuesWriter.getBytes(); } @Override public Encoding getEncoding() { if (!dictionaryTooBig && dict.size() > 0) { return PLAIN_DICTIONARY; } return plainValuesWriter.getEncoding(); } @Override public void reset() { if (out != null) { out = new IntList(); } plainValuesWriter.reset(); } @Override public DictionaryPage createDictionaryPage() { if (lastUsedDictionarySize > 0) { // return a dictionary only if we actually used it try { CapacityByteArrayOutputStream dictBuf = new CapacityByteArrayOutputStream(lastUsedDictionaryByteSize); LittleEndianDataOutputStream dictOut = new LittleEndianDataOutputStream(dictBuf); Iterator<Binary> entryIterator = dict.keySet().iterator(); // write only the part of the dict that we used for (int i = 0; i < lastUsedDictionarySize; i++) { Binary entry = entryIterator.next(); dictOut.writeInt(entry.length()); entry.writeTo(dictOut); } return new DictionaryPage( BytesInput.from(dictBuf), lastUsedDictionarySize, PLAIN_DICTIONARY); } catch (IOException e) { throw new ParquetEncodingException("Could not generate dictionary Page", e); } } return plainValuesWriter.createDictionaryPage(); } @Override public void resetDictionary() { lastUsedDictionaryByteSize = 0; lastUsedDictionarySize = 0; dictionaryByteSize = 0; dictionaryTooBig = false; if (dict == null) { dict = new LinkedHashMap<Binary, Integer>(); } else { dict.clear(); } } public int getDictionaryByteSize() { return dictionaryByteSize; } @Override public String memUsageString(String prefix) { return String.format( "%s DictionaryValuesWriter{\n%s\n%s\n%s\n%s}\n", prefix, plainValuesWriter.memUsageString(prefix + " plain:"), prefix + " dict:" + dictionaryByteSize, prefix + " values:" + (out.size() * 4), prefix); } }
/** * OutputFormat to write to a Parquet file * * <p>It requires a {@link WriteSupport} to convert the actual records to the underlying format. It * requires the schema of the incoming records. (provided by the write support) It allows storing * extra metadata in the footer (for example: for schema compatibility purpose when converting from * a different schema language). * * <p>The format configuration settings in the job configuration: * * <pre> * # The block size is the size of a row group being buffered in memory * # this limits the memory usage when writing * # Larger values will improve the IO when reading but consume more memory when writing * parquet.block.size=134217728 # in bytes, default = 128 * 1024 * 1024 * * # The page size is for compression. When reading, each page can be decompressed independently. * # A block is composed of pages. The page is the smallest unit that must be read fully to access a single record. * # If this value is too small, the compression will deteriorate * parquet.page.size=1048576 # in bytes, default = 1 * 1024 * 1024 * * # There is one dictionary page per column per row group when dictionary encoding is used. * # The dictionary page size works like the page size but for dictionary * parquet.dictionary.page.size=1048576 # in bytes, default = 1 * 1024 * 1024 * * # The compression algorithm used to compress pages * parquet.compression=UNCOMPRESSED # one of: UNCOMPRESSED, SNAPPY, GZIP, LZO. Default: UNCOMPRESSED. Supersedes mapred.output.compress* * * # The write support class to convert the records written to the OutputFormat into the events accepted by the record consumer * # Usually provided by a specific ParquetOutputFormat subclass * parquet.write.support.class= # fully qualified name * * # To enable/disable dictionary encoding * parquet.enable.dictionary=true # false to disable dictionary encoding * * # To enable/disable summary metadata aggregation at the end of a MR job * # The default is true (enabled) * parquet.enable.summary-metadata=true # false to disable summary aggregation * </pre> * * If parquet.compression is not set, the following properties are checked (FileOutputFormat * behavior). Note that we explicitely disallow custom Codecs * * <pre> * mapred.output.compress=true * mapred.output.compression.codec=org.apache.hadoop.io.compress.SomeCodec # the codec must be one of Snappy, GZip or LZO * </pre> * * if none of those is set the data is uncompressed. * * @author Julien Le Dem * @param <T> the type of the materialized records */ public class ParquetOutputFormat<T> extends FileOutputFormat<Void, T> { private static final Log LOG = Log.getLog(ParquetOutputFormat.class); public static final String BLOCK_SIZE = "parquet.block.size"; public static final String PAGE_SIZE = "parquet.page.size"; public static final String COMPRESSION = "parquet.compression"; public static final String WRITE_SUPPORT_CLASS = "parquet.write.support.class"; public static final String DICTIONARY_PAGE_SIZE = "parquet.dictionary.page.size"; public static final String ENABLE_DICTIONARY = "parquet.enable.dictionary"; public static final String VALIDATION = "parquet.validation"; public static final String WRITER_VERSION = "parquet.writer.version"; public static final String ENABLE_JOB_SUMMARY = "parquet.enable.summary-metadata"; public static void setWriteSupportClass(Job job, Class<?> writeSupportClass) { getConfiguration(job).set(WRITE_SUPPORT_CLASS, writeSupportClass.getName()); } public static void setWriteSupportClass(JobConf job, Class<?> writeSupportClass) { job.set(WRITE_SUPPORT_CLASS, writeSupportClass.getName()); } public static Class<?> getWriteSupportClass(Configuration configuration) { final String className = configuration.get(WRITE_SUPPORT_CLASS); if (className == null) { return null; } final Class<?> writeSupportClass = ConfigurationUtil.getClassFromConfig( configuration, WRITE_SUPPORT_CLASS, WriteSupport.class); return writeSupportClass; } public static void setBlockSize(Job job, int blockSize) { getConfiguration(job).setInt(BLOCK_SIZE, blockSize); } public static void setPageSize(Job job, int pageSize) { getConfiguration(job).setInt(PAGE_SIZE, pageSize); } public static void setDictionaryPageSize(Job job, int pageSize) { getConfiguration(job).setInt(DICTIONARY_PAGE_SIZE, pageSize); } public static void setCompression(Job job, CompressionCodecName compression) { getConfiguration(job).set(COMPRESSION, compression.name()); } public static void setEnableDictionary(Job job, boolean enableDictionary) { getConfiguration(job).setBoolean(ENABLE_DICTIONARY, enableDictionary); } public static boolean getEnableDictionary(JobContext jobContext) { return getEnableDictionary(getConfiguration(jobContext)); } public static int getBlockSize(JobContext jobContext) { return getBlockSize(getConfiguration(jobContext)); } public static int getPageSize(JobContext jobContext) { return getPageSize(getConfiguration(jobContext)); } public static int getDictionaryPageSize(JobContext jobContext) { return getDictionaryPageSize(getConfiguration(jobContext)); } public static CompressionCodecName getCompression(JobContext jobContext) { return getCompression(getConfiguration(jobContext)); } public static boolean isCompressionSet(JobContext jobContext) { return isCompressionSet(getConfiguration(jobContext)); } public static void setValidation(JobContext jobContext, boolean validating) { setValidation(getConfiguration(jobContext), validating); } public static boolean getValidation(JobContext jobContext) { return getValidation(getConfiguration(jobContext)); } public static boolean getEnableDictionary(Configuration configuration) { return configuration.getBoolean(ENABLE_DICTIONARY, true); } public static int getBlockSize(Configuration configuration) { return configuration.getInt(BLOCK_SIZE, DEFAULT_BLOCK_SIZE); } public static int getPageSize(Configuration configuration) { return configuration.getInt(PAGE_SIZE, DEFAULT_PAGE_SIZE); } public static int getDictionaryPageSize(Configuration configuration) { return configuration.getInt(DICTIONARY_PAGE_SIZE, DEFAULT_PAGE_SIZE); } public static WriterVersion getWriterVersion(Configuration configuration) { String writerVersion = configuration.get(WRITER_VERSION, WriterVersion.PARQUET_1_0.toString()); return WriterVersion.fromString(writerVersion); } public static CompressionCodecName getCompression(Configuration configuration) { return CodecConfig.getParquetCompressionCodec(configuration); } public static boolean isCompressionSet(Configuration configuration) { return CodecConfig.isParquetCompressionSet(configuration); } public static void setValidation(Configuration configuration, boolean validating) { configuration.setBoolean(VALIDATION, validating); } public static boolean getValidation(Configuration configuration) { return configuration.getBoolean(VALIDATION, false); } private CompressionCodecName getCodec(TaskAttemptContext taskAttemptContext) { return CodecConfig.from(taskAttemptContext).getCodec(); } private WriteSupport<T> writeSupport; private ParquetOutputCommitter committer; /** * constructor used when this OutputFormat in wrapped in another one (In Pig for example) * * @param writeSupportClass the class used to convert the incoming records * @param schema the schema of the records * @param extraMetaData extra meta data to be stored in the footer of the file */ public <S extends WriteSupport<T>> ParquetOutputFormat(S writeSupport) { this.writeSupport = writeSupport; } /** * used when directly using the output format and configuring the write support implementation * using parquet.write.support.class */ public <S extends WriteSupport<T>> ParquetOutputFormat() {} /** {@inheritDoc} */ @Override public RecordWriter<Void, T> getRecordWriter(TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException { final Configuration conf = getConfiguration(taskAttemptContext); CompressionCodecName codec = getCodec(taskAttemptContext); String extension = codec.getExtension() + ".parquet"; Path file = getDefaultWorkFile(taskAttemptContext, extension); return getRecordWriter(conf, file, codec); } public RecordWriter<Void, T> getRecordWriter(TaskAttemptContext taskAttemptContext, Path file) throws IOException, InterruptedException { return getRecordWriter( getConfiguration(taskAttemptContext), file, getCodec(taskAttemptContext)); } public RecordWriter<Void, T> getRecordWriter( Configuration conf, Path file, CompressionCodecName codec) throws IOException, InterruptedException { final WriteSupport<T> writeSupport = getWriteSupport(conf); CodecFactory codecFactory = new CodecFactory(conf); int blockSize = getBlockSize(conf); if (INFO) LOG.info("Parquet block size to " + blockSize); int pageSize = getPageSize(conf); if (INFO) LOG.info("Parquet page size to " + pageSize); int dictionaryPageSize = getDictionaryPageSize(conf); if (INFO) LOG.info("Parquet dictionary page size to " + dictionaryPageSize); boolean enableDictionary = getEnableDictionary(conf); if (INFO) LOG.info("Dictionary is " + (enableDictionary ? "on" : "off")); boolean validating = getValidation(conf); if (INFO) LOG.info("Validation is " + (validating ? "on" : "off")); WriterVersion writerVersion = getWriterVersion(conf); if (INFO) LOG.info("Writer version is: " + writerVersion); WriteContext init = writeSupport.init(conf); ParquetFileWriter w = new ParquetFileWriter(conf, init.getSchema(), file); w.start(); return new ParquetRecordWriter<T>( w, writeSupport, init.getSchema(), init.getExtraMetaData(), blockSize, pageSize, codecFactory.getCompressor(codec, pageSize), dictionaryPageSize, enableDictionary, validating, writerVersion); } /** * @param configuration to find the configuration for the write support class * @return the configured write support */ @SuppressWarnings("unchecked") public WriteSupport<T> getWriteSupport(Configuration configuration) { if (writeSupport != null) return writeSupport; Class<?> writeSupportClass = getWriteSupportClass(configuration); try { return (WriteSupport<T>) checkNotNull(writeSupportClass, "writeSupportClass").newInstance(); } catch (InstantiationException e) { throw new BadConfigurationException( "could not instantiate write support class: " + writeSupportClass, e); } catch (IllegalAccessException e) { throw new BadConfigurationException( "could not instantiate write support class: " + writeSupportClass, e); } } @Override public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException { if (committer == null) { Path output = getOutputPath(context); committer = new ParquetOutputCommitter(output, context); } return committer; } }
/** * Reads the records from a block of a Parquet file * * @see ParquetInputFormat * @author Julien Le Dem * @param <T> type of the materialized records */ public class ParquetRecordReader<T> extends RecordReader<Void, T> { private static final Log LOG = Log.getLog(ParquetRecordReader.class); private final ColumnIOFactory columnIOFactory = new ColumnIOFactory(); private MessageType requestedSchema; private int columnCount; private final ReadSupport<T> readSupport; private RecordMaterializer<T> recordConverter; private T currentValue; private long total; private int current = 0; private ParquetFileReader reader; private parquet.io.RecordReader<T> recordReader; private long totalTimeSpentReadingBytes; private long totalTimeSpentProcessingRecords; private long startedAssemblingCurrentBlockAt; private long totalCountLoadedSoFar = 0; /** * @param requestedSchema the requested schema (a subset of the original schema) for record * projection * @param readSupportClass */ public ParquetRecordReader(ReadSupport<T> readSupport) { this.readSupport = readSupport; } private void checkRead() throws IOException { if (current == totalCountLoadedSoFar) { if (current != 0) { long timeAssembling = System.currentTimeMillis() - startedAssemblingCurrentBlockAt; totalTimeSpentProcessingRecords += timeAssembling; LOG.info( "Assembled and processed " + totalCountLoadedSoFar + " records from " + columnCount + " columns in " + totalTimeSpentProcessingRecords + " ms: " + ((float) totalCountLoadedSoFar / totalTimeSpentProcessingRecords) + " rec/ms, " + ((float) totalCountLoadedSoFar * columnCount / totalTimeSpentProcessingRecords) + " cell/ms"); long totalTime = totalTimeSpentProcessingRecords + totalTimeSpentReadingBytes; long percentReading = 100 * totalTimeSpentReadingBytes / totalTime; long percentProcessing = 100 * totalTimeSpentProcessingRecords / totalTime; LOG.info( "time spent so far " + percentReading + "% reading (" + totalTimeSpentReadingBytes + " ms) and " + percentProcessing + "% processing (" + totalTimeSpentProcessingRecords + " ms)"); } LOG.info("at row " + current + ". reading next block"); long t0 = System.currentTimeMillis(); PageReadStore pages = reader.readNextRowGroup(); if (pages == null) { throw new IOException( "expecting more rows but reached last block. Read " + current + " out of " + total); } long timeSpentReading = System.currentTimeMillis() - t0; totalTimeSpentReadingBytes += timeSpentReading; LOG.info( "block read in memory in " + timeSpentReading + " ms. row count = " + pages.getRowCount()); if (Log.DEBUG) LOG.debug("initializing Record assembly with requested schema " + requestedSchema); MessageColumnIO columnIO = columnIOFactory.getColumnIO(requestedSchema); recordReader = columnIO.getRecordReader(pages, recordConverter); startedAssemblingCurrentBlockAt = System.currentTimeMillis(); totalCountLoadedSoFar += pages.getRowCount(); } } /** {@inheritDoc} */ @Override public void close() throws IOException { reader.close(); } /** always returns null */ @Override public Void getCurrentKey() throws IOException, InterruptedException { return null; } /** {@inheritDoc} */ @Override public T getCurrentValue() throws IOException, InterruptedException { return currentValue; } /** {@inheritDoc} */ @Override public float getProgress() throws IOException, InterruptedException { return (float) current / total; } /** {@inheritDoc} */ @Override public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException { Configuration configuration = taskAttemptContext.getConfiguration(); ParquetInputSplit parquetInputSplit = (ParquetInputSplit) inputSplit; this.requestedSchema = MessageTypeParser.parseMessageType(parquetInputSplit.getRequestedSchema()); this.columnCount = this.requestedSchema.getPaths().size(); this.recordConverter = readSupport.prepareForRead( configuration, parquetInputSplit.getExtraMetadata(), MessageTypeParser.parseMessageType(parquetInputSplit.getSchema()), new ReadSupport.ReadContext(requestedSchema)); Path path = parquetInputSplit.getPath(); List<BlockMetaData> blocks = parquetInputSplit.getBlocks(); List<ColumnDescriptor> columns = requestedSchema.getColumns(); reader = new ParquetFileReader(configuration, path, blocks, columns); for (BlockMetaData block : blocks) { total += block.getRowCount(); } LOG.info("RecordReader initialized will read a total of " + total + " records."); } private boolean contains(GroupType group, String[] path, int index) { if (index == path.length) { return false; } if (group.containsField(path[index])) { Type type = group.getType(path[index]); if (type.isPrimitive()) { return index + 1 == path.length; } else { return contains(type.asGroupType(), path, index + 1); } } return false; } /** {@inheritDoc} */ @Override public boolean nextKeyValue() throws IOException, InterruptedException { if (current < total) { checkRead(); currentValue = recordReader.read(); if (DEBUG) LOG.debug("read value: " + currentValue); current++; return true; } return false; } }
public class Utils { private static final Log LOG = Log.getLog(Utils.class); public static void closeQuietly(Closeable res) { try { if (res != null) { res.close(); } } catch (IOException ioe) { LOG.warn("Exception closing reader " + res + ": " + ioe.getMessage()); } } public static File createTestFile(long largerThanMB) throws IOException { File outputFile = new File("target/test/csv/perftest.csv"); if (outputFile.exists()) { return outputFile; } File toCopy = new File("../parquet-testdata/tpch/customer.csv"); FileUtils.copyFile( new File("../parquet-testdata/tpch/customer.schema"), new File("target/test/csv/perftest.schema")); OutputStream output = null; InputStream input = null; try { output = new BufferedOutputStream(new FileOutputStream(outputFile, true)); input = new BufferedInputStream(new FileInputStream(toCopy)); input.mark(Integer.MAX_VALUE); while (outputFile.length() <= largerThanMB * 1024 * 1024) { // appendFile(output, toCopy); IOUtils.copy(input, output); input.reset(); } } finally { closeQuietly(input); closeQuietly(output); } return outputFile; } public static File[] getAllOriginalCSVFiles() { File baseDir = new File("../parquet-testdata/tpch"); final File[] csvFiles = baseDir.listFiles( new FilenameFilter() { public boolean accept(File dir, String name) { return name.endsWith(".csv"); } }); return csvFiles; } public static String[] getAllPreviousVersionDirs() throws IOException { File baseDir = new File(".."); final String currentVersion = new File(".").getCanonicalFile().getName(); final String[] versions = baseDir.list( new FilenameFilter() { public boolean accept(File dir, String name) { return name.startsWith("parquet-compat-") && name.compareTo(currentVersion) <= 0; } }); return versions; } public static File getParquetOutputFile(String name, String module, boolean deleteIfExists) { File outputFile = new File("target/parquet/", getParquetFileName(name, module)); outputFile.getParentFile().mkdirs(); if (deleteIfExists) { outputFile.delete(); } return outputFile; } private static String getParquetFileName(String name, String module) { return name + (module != null ? "." + module : "") + ".parquet"; } public static File getParquetFile( String name, String version, String module, boolean failIfNotExist) throws IOException { File parquetFile = new File("../" + version + "/target/parquet/", getParquetFileName(name, module)); parquetFile.getParentFile().mkdirs(); if (!parquetFile.exists()) { String msg = "File " + parquetFile.getAbsolutePath() + " does not exist"; if (failIfNotExist) { throw new IOException(msg); } LOG.warn(msg); } return parquetFile; } public static String[] getImpalaDirectories() { File baseDir = new File("../parquet-testdata/impala"); final String[] impalaVersions = baseDir.list( new FilenameFilter() { public boolean accept(File dir, String name) { return !name.startsWith("."); } }); return impalaVersions; } public static File getParquetImpalaFile(String name, String impalaVersion) throws IOException { String fileName = name + ".impala.parquet"; File parquetFile = new File("../parquet-testdata/impala/" + impalaVersion, fileName); if (!parquetFile.exists()) { throw new IOException("File " + fileName + " does not exist"); } return parquetFile; } public static String getFileNamePrefix(File file) { return file.getName().substring(0, file.getName().indexOf(".")); } public static File getCsvTestFile(String name, String module, boolean deleteIfExists) { File outputFile = new File("target/test/csv/", name + (module != null ? "." + module : "") + ".csv"); outputFile.getParentFile().mkdirs(); if (deleteIfExists) { outputFile.delete(); } return outputFile; } public static File getParquetTestFile(String name, String module, boolean deleteIfExists) { File outputFile = new File("target/test/parquet/", name + (module != null ? "." + module : "") + ".csv"); outputFile.getParentFile().mkdirs(); if (deleteIfExists) { outputFile.delete(); } return outputFile; } public static void verify(File expectedCsvFile, File outputCsvFile) throws IOException { BufferedReader expected = null; BufferedReader out = null; try { expected = new BufferedReader(new FileReader(expectedCsvFile)); out = new BufferedReader(new FileReader(outputCsvFile)); String lineIn; String lineOut = null; int lineNumber = 0; while ((lineIn = expected.readLine()) != null && (lineOut = out.readLine()) != null) { ++lineNumber; lineOut = lineOut.substring(lineOut.indexOf("\t") + 1); assertEquals("line " + lineNumber, lineIn, lineOut); } assertNull("line " + lineNumber, lineIn); assertNull("line " + lineNumber, out.readLine()); } finally { Utils.closeQuietly(expected); Utils.closeQuietly(out); } } public static void verify(File expectedCsvFile, File outputCsvFile, boolean orderMatters) throws IOException { if (!orderMatters) { // sort the files before diff'ing them expectedCsvFile = sortFile(expectedCsvFile); outputCsvFile = sortFile(outputCsvFile); } verify(expectedCsvFile, outputCsvFile); } private static File sortFile(File inFile) throws IOException { File sortedFile = new File(inFile.getAbsolutePath().concat(".sorted")); BufferedReader reader = new BufferedReader(new FileReader(inFile)); PrintWriter out = new PrintWriter(new FileWriter(sortedFile)); try { String inputLine; List<String> lineList = new ArrayList<String>(); while ((inputLine = reader.readLine()) != null) { lineList.add(inputLine); } Collections.sort(lineList); for (String outputLine : lineList) { out.println(outputLine); } out.flush(); } finally { closeQuietly(reader); closeQuietly(out); } return sortedFile; } }