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(); } }
/** * 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; }
/** {@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); }