public ParquetRecordReader( FragmentContext fragmentContext, long batchSize, long numRecordsToRead, String path, int rowGroupIndex, FileSystem fs, CodecFactory codecFactory, ParquetMetadata footer, List<SchemaPath> columns, ParquetReaderUtility.DateCorruptionStatus dateCorruptionStatus) throws ExecutionSetupException { this.hadoopPath = new Path(path); this.fileSystem = fs; this.codecFactory = codecFactory; this.rowGroupIndex = rowGroupIndex; this.batchSize = batchSize; this.footer = footer; this.dateCorruptionStatus = dateCorruptionStatus; this.fragmentContext = fragmentContext; // Callers can pass -1 if they want to read all rows. if (numRecordsToRead == NUM_RECORDS_TO_READ_NOT_SPECIFIED) { this.numRecordsToRead = footer.getBlocks().get(rowGroupIndex).getRowCount(); } else { assert (numRecordsToRead >= 0); this.numRecordsToRead = Math.min(numRecordsToRead, footer.getBlocks().get(rowGroupIndex).getRowCount()); } setColumns(columns); }
public ParquetRecordReader( FragmentContext fragmentContext, String path, int rowGroupIndex, FileSystem fs, CodecFactory codecFactory, ParquetMetadata footer, List<SchemaPath> columns, ParquetReaderUtility.DateCorruptionStatus dateCorruptionStatus) throws ExecutionSetupException { this( fragmentContext, DEFAULT_BATCH_LENGTH_IN_BITS, footer.getBlocks().get(rowGroupIndex).getRowCount(), path, rowGroupIndex, fs, codecFactory, footer, columns, dateCorruptionStatus); }
@Override public int next() { resetBatch(); long recordsToRead = 0; try { ColumnReader<?> firstColumnStatus; if (columnStatuses.size() > 0) { firstColumnStatus = columnStatuses.iterator().next(); } else { if (varLengthReader.columns.size() > 0) { firstColumnStatus = varLengthReader.columns.iterator().next(); } else { firstColumnStatus = null; } } // No columns found in the file were selected, simply return a full batch of null records for // each column requested if (firstColumnStatus == null) { if (mockRecordsRead == footer.getBlocks().get(rowGroupIndex).getRowCount()) { updateStats(); return 0; } recordsToRead = Math.min( DEFAULT_RECORDS_TO_READ_IF_NOT_FIXED_WIDTH, footer.getBlocks().get(rowGroupIndex).getRowCount() - mockRecordsRead); // Pick the minimum of recordsToRead calculated above and numRecordsToRead (based on // rowCount and limit). recordsToRead = Math.min(recordsToRead, numRecordsToRead); for (final ValueVector vv : nullFilledVectors) { vv.getMutator().setValueCount((int) recordsToRead); } mockRecordsRead += recordsToRead; totalRecordsRead += recordsToRead; numRecordsToRead -= recordsToRead; updateStats(); return (int) recordsToRead; } if (allFieldsFixedLength) { recordsToRead = Math.min( recordsPerBatch, firstColumnStatus.columnChunkMetaData.getValueCount() - firstColumnStatus.totalValuesRead); } else { recordsToRead = DEFAULT_RECORDS_TO_READ_IF_NOT_FIXED_WIDTH; } // Pick the minimum of recordsToRead calculated above and numRecordsToRead (based on rowCount // and limit) recordsToRead = Math.min(recordsToRead, numRecordsToRead); if (allFieldsFixedLength) { readAllFixedFields(recordsToRead); } else { // variable length columns long fixedRecordsToRead = varLengthReader.readFields(recordsToRead, firstColumnStatus); readAllFixedFields(fixedRecordsToRead); } // if we have requested columns that were not found in the file fill their vectors with null // (by simply setting the value counts inside of them, as they start null filled) if (nullFilledVectors != null) { for (final ValueVector vv : nullFilledVectors) { vv.getMutator().setValueCount(firstColumnStatus.getRecordsReadInCurrentPass()); } } // logger.debug("So far read {} records out of row group({}) in file '{}'", // totalRecordsRead, rowGroupIndex, hadoopPath.toUri().getPath()); totalRecordsRead += firstColumnStatus.getRecordsReadInCurrentPass(); numRecordsToRead -= firstColumnStatus.getRecordsReadInCurrentPass(); updateStats(); return firstColumnStatus.getRecordsReadInCurrentPass(); } catch (Exception e) { handleAndRaise( "\nHadoop path: " + hadoopPath.toUri().getPath() + "\nTotal records read: " + totalRecordsRead + "\nMock records read: " + mockRecordsRead + "\nRecords to read: " + recordsToRead + "\nRow group index: " + rowGroupIndex + "\nRecords in row group: " + footer.getBlocks().get(rowGroupIndex).getRowCount(), e); } // this is never reached return 0; }
@Override public void setup(OperatorContext operatorContext, OutputMutator output) throws ExecutionSetupException { this.operatorContext = operatorContext; if (!isStarQuery()) { columnsFound = new boolean[getColumns().size()]; nullFilledVectors = new ArrayList<>(); } columnStatuses = new ArrayList<>(); // totalRecords = footer.getBlocks().get(rowGroupIndex).getRowCount(); List<ColumnDescriptor> columns = footer.getFileMetaData().getSchema().getColumns(); allFieldsFixedLength = true; ColumnDescriptor column; ColumnChunkMetaData columnChunkMetaData; int columnsToScan = 0; mockRecordsRead = 0; MaterializedField field; // ParquetMetadataConverter metaConverter = new ParquetMetadataConverter(); FileMetaData fileMetaData; logger.debug( "Reading row group({}) with {} records in file {}.", rowGroupIndex, footer.getBlocks().get(rowGroupIndex).getRowCount(), hadoopPath.toUri().getPath()); totalRecordsRead = 0; // TODO - figure out how to deal with this better once we add nested reading, note also look // where this map is used below // store a map from column name to converted types if they are non-null Map<String, SchemaElement> schemaElements = ParquetReaderUtility.getColNameToSchemaElementMapping(footer); // loop to add up the length of the fixed width columns and build the schema for (int i = 0; i < columns.size(); ++i) { column = columns.get(i); SchemaElement se = schemaElements.get(column.getPath()[0]); MajorType mt = ParquetToDrillTypeConverter.toMajorType( column.getType(), se.getType_length(), getDataMode(column), se, fragmentContext.getOptions()); field = MaterializedField.create(toFieldName(column.getPath()), mt); if (!fieldSelected(field)) { continue; } columnsToScan++; int dataTypeLength = getDataTypeLength(column, se); if (dataTypeLength == -1) { allFieldsFixedLength = false; } else { bitWidthAllFixedFields += dataTypeLength; } } // rowGroupOffset = // footer.getBlocks().get(rowGroupIndex).getColumns().get(0).getFirstDataPageOffset(); if (columnsToScan != 0 && allFieldsFixedLength) { recordsPerBatch = (int) Math.min( Math.min( batchSize / bitWidthAllFixedFields, footer.getBlocks().get(0).getColumns().get(0).getValueCount()), 65535); } else { recordsPerBatch = DEFAULT_RECORDS_TO_READ_IF_NOT_FIXED_WIDTH; } try { ValueVector vector; SchemaElement schemaElement; final ArrayList<VarLengthColumn<? extends ValueVector>> varLengthColumns = new ArrayList<>(); // initialize all of the column read status objects boolean fieldFixedLength; // the column chunk meta-data is not guaranteed to be in the same order as the columns in the // schema // a map is constructed for fast access to the correct columnChunkMetadata to correspond // to an element in the schema Map<String, Integer> columnChunkMetadataPositionsInList = new HashMap<>(); BlockMetaData rowGroupMetadata = footer.getBlocks().get(rowGroupIndex); int colChunkIndex = 0; for (ColumnChunkMetaData colChunk : rowGroupMetadata.getColumns()) { columnChunkMetadataPositionsInList.put( Arrays.toString(colChunk.getPath().toArray()), colChunkIndex); colChunkIndex++; } for (int i = 0; i < columns.size(); ++i) { column = columns.get(i); columnChunkMetaData = rowGroupMetadata .getColumns() .get(columnChunkMetadataPositionsInList.get(Arrays.toString(column.getPath()))); schemaElement = schemaElements.get(column.getPath()[0]); MajorType type = ParquetToDrillTypeConverter.toMajorType( column.getType(), schemaElement.getType_length(), getDataMode(column), schemaElement, fragmentContext.getOptions()); field = MaterializedField.create(toFieldName(column.getPath()), type); // the field was not requested to be read if (!fieldSelected(field)) { continue; } fieldFixedLength = column.getType() != PrimitiveType.PrimitiveTypeName.BINARY; vector = output.addField( field, (Class<? extends ValueVector>) TypeHelper.getValueVectorClass(type.getMinorType(), type.getMode())); if (column.getType() != PrimitiveType.PrimitiveTypeName.BINARY) { if (column.getMaxRepetitionLevel() > 0) { final RepeatedValueVector repeatedVector = RepeatedValueVector.class.cast(vector); ColumnReader<?> dataReader = ColumnReaderFactory.createFixedColumnReader( this, fieldFixedLength, column, columnChunkMetaData, recordsPerBatch, repeatedVector.getDataVector(), schemaElement); varLengthColumns.add( new FixedWidthRepeatedReader( this, dataReader, getTypeLengthInBits(column.getType()), -1, column, columnChunkMetaData, false, repeatedVector, schemaElement)); } else { columnStatuses.add( ColumnReaderFactory.createFixedColumnReader( this, fieldFixedLength, column, columnChunkMetaData, recordsPerBatch, vector, schemaElement)); } } else { // create a reader and add it to the appropriate list varLengthColumns.add( ColumnReaderFactory.getReader( this, -1, column, columnChunkMetaData, false, vector, schemaElement)); } } varLengthReader = new VarLenBinaryReader(this, varLengthColumns); if (!isStarQuery()) { List<SchemaPath> projectedColumns = Lists.newArrayList(getColumns()); SchemaPath col; for (int i = 0; i < columnsFound.length; i++) { col = projectedColumns.get(i); assert col != null; if (!columnsFound[i] && !col.equals(STAR_COLUMN)) { nullFilledVectors.add( (NullableIntVector) output.addField( MaterializedField.create( col.getAsUnescapedPath(), Types.optional(TypeProtos.MinorType.INT)), (Class<? extends ValueVector>) TypeHelper.getValueVectorClass( TypeProtos.MinorType.INT, DataMode.OPTIONAL))); } } } } catch (Exception e) { handleAndRaise("Failure in setting up reader", e); } }