protected PageHeader readPageHeader() throws IOException { return Util.readPageHeader(this); }
private DictionaryPage readDictionaryPage(PageHeader pageHeader, int uncompressedPageSize, int compressedPageSize) { DictionaryPageHeader dicHeader = pageHeader.getDictionary_page_header(); return new DictionaryPage( getSlice(compressedPageSize), uncompressedPageSize, dicHeader.getNum_values(), getParquetEncoding(Encoding.valueOf(dicHeader.getEncoding().name()))); }
private static Optional<DictionaryPage> readDictionaryPage(byte[] data, CompressionCodecName codecName) { try { ByteArrayInputStream inputStream = new ByteArrayInputStream(data); PageHeader pageHeader = Util.readPageHeader(inputStream); if (pageHeader.type != PageType.DICTIONARY_PAGE) { return Optional.empty(); } Slice compressedData = wrappedBuffer(data, data.length - inputStream.available(), pageHeader.getCompressed_page_size()); DictionaryPageHeader dicHeader = pageHeader.getDictionary_page_header(); ParquetEncoding encoding = getParquetEncoding(Encoding.valueOf(dicHeader.getEncoding().name())); int dictionarySize = dicHeader.getNum_values(); return Optional.of(new DictionaryPage(decompress(codecName, compressedData, pageHeader.getUncompressed_page_size()), dictionarySize, encoding)); } catch (IOException ignored) { return Optional.empty(); } }
private long readDataPageV2(PageHeader pageHeader, int uncompressedPageSize, int compressedPageSize, List<DataPage> pages) { DataPageHeaderV2 dataHeaderV2 = pageHeader.getData_page_header_v2(); int dataSize = compressedPageSize - dataHeaderV2.getRepetition_levels_byte_length() - dataHeaderV2.getDefinition_levels_byte_length(); pages.add(new DataPageV2( dataHeaderV2.getNum_rows(), dataHeaderV2.getNum_nulls(), dataHeaderV2.getNum_values(), getSlice(dataHeaderV2.getRepetition_levels_byte_length()), getSlice(dataHeaderV2.getDefinition_levels_byte_length()), getParquetEncoding(Encoding.valueOf(dataHeaderV2.getEncoding().name())), getSlice(dataSize), uncompressedPageSize, MetadataReader.readStats( dataHeaderV2.getStatistics(), descriptor.getColumnDescriptor().getType()), dataHeaderV2.isIs_compressed())); return dataHeaderV2.getNum_values(); } }
metadataIndex); inputStream.seek(metadataIndex); FileMetaData fileMetaData = readFileMetaData(inputStream); List<SchemaElement> schema = fileMetaData.getSchema(); validateParquet(!schema.isEmpty(), "Empty Parquet schema in file: %s", file); List<RowGroup> rowGroups = fileMetaData.getRow_groups(); if (rowGroups != null) { for (RowGroup rowGroup : rowGroups) { BlockMetaData blockMetaData = new BlockMetaData(); blockMetaData.setRowCount(rowGroup.getNum_rows()); blockMetaData.setTotalByteSize(rowGroup.getTotal_byte_size()); List<ColumnChunk> columns = rowGroup.getColumns(); validateParquet(!columns.isEmpty(), "No columns in row group: %s", rowGroup); String filePath = columns.get(0).getFile_path(); for (ColumnChunk columnChunk : columns) { validateParquet( (filePath == null && columnChunk.getFile_path() == null) || (filePath != null && filePath.equals(columnChunk.getFile_path())), "all column chunks of the same row group must be in the same file"); ColumnMetaData metaData = columnChunk.meta_data; List<KeyValue> keyValueList = fileMetaData.getKey_value_metadata(); if (keyValueList != null) { for (KeyValue keyValue : keyValueList) { return new ParquetMetadata(new parquet.hadoop.metadata.FileMetaData(messageType, keyValueMetaData, fileMetaData.getCreated_by()), blocks);
private long readDataPageV1(PageHeader pageHeader, int uncompressedPageSize, int compressedPageSize, List<DataPage> pages) { DataPageHeader dataHeaderV1 = pageHeader.getData_page_header(); pages.add(new DataPageV1( getSlice(compressedPageSize), dataHeaderV1.getNum_values(), uncompressedPageSize, MetadataReader.readStats( dataHeaderV1.getStatistics(), descriptor.getColumnDescriptor().getType()), getParquetEncoding(Encoding.valueOf(dataHeaderV1.getRepetition_level_encoding().name())), getParquetEncoding(Encoding.valueOf(dataHeaderV1.getDefinition_level_encoding().name())), getParquetEncoding(Encoding.valueOf(dataHeaderV1.getEncoding().name())))); return dataHeaderV1.getNum_values(); }
Types.Builder<?, ?> typeBuilder; if (element.type == null) { typeBuilder = builder.group(Repetition.valueOf(element.repetition_type.name())); readTypeSchema((Types.GroupBuilder<?>) typeBuilder, schemaIterator, element.num_children); Types.PrimitiveBuilder<?> primitiveBuilder = builder.primitive(getTypeName(element.type), Repetition.valueOf(element.repetition_type.name())); if (element.isSetType_length()) { primitiveBuilder.length(element.type_length); if (element.isSetPrecision()) { primitiveBuilder.precision(element.precision); if (element.isSetScale()) { primitiveBuilder.scale(element.scale); if (element.isSetConverted_type()) { typeBuilder.as(getOriginalType(element.converted_type)); if (element.isSetField_id()) { typeBuilder.id(element.field_id);
public static parquet.column.statistics.Statistics<?> readStats(Statistics statistics, PrimitiveTypeName type) { parquet.column.statistics.Statistics<?> stats = parquet.column.statistics.Statistics.getStatsBasedOnType(type); if (statistics != null) { if (statistics.isSetMax() && statistics.isSetMin()) { stats.setMinMaxFromBytes(statistics.min.array(), statistics.max.array()); } stats.setNumNulls(statistics.null_count); } return stats; }
public PageReader readAllPages() throws IOException { List<DataPage> pages = new ArrayList<>(); DictionaryPage dictionaryPage = null; long valueCount = 0; while (valueCount < descriptor.getColumnChunkMetaData().getValueCount()) { PageHeader pageHeader = readPageHeader(); int uncompressedPageSize = pageHeader.getUncompressed_page_size(); int compressedPageSize = pageHeader.getCompressed_page_size(); switch (pageHeader.type) { case DICTIONARY_PAGE: if (dictionaryPage != null) { throw new ParquetCorruptionException("%s has more than one dictionary page in column chunk", descriptor.getColumnDescriptor()); } dictionaryPage = readDictionaryPage(pageHeader, uncompressedPageSize, compressedPageSize); break; case DATA_PAGE: valueCount += readDataPageV1(pageHeader, uncompressedPageSize, compressedPageSize, pages); break; case DATA_PAGE_V2: valueCount += readDataPageV2(pageHeader, uncompressedPageSize, compressedPageSize, pages); break; default: skip(compressedPageSize); break; } } return new PageReader(descriptor.getColumnChunkMetaData().getCodec(), pages, dictionaryPage); }
static long getOffset(ColumnChunk columnChunk) { ColumnMetaData md = columnChunk.getMeta_data(); long offset = md.getData_page_offset(); if (md.isSetDictionary_page_offset() && offset > md.getDictionary_page_offset()) { offset = md.getDictionary_page_offset(); } return offset; }
public static PageHeader readPageHeader(InputStream from) throws IOException { return read(from, new PageHeader()); }
private static MessageType readParquetSchema(List<SchemaElement> schema) { Iterator<SchemaElement> schemaIterator = schema.iterator(); SchemaElement rootSchema = schemaIterator.next(); Types.MessageTypeBuilder builder = Types.buildMessage(); readTypeSchema(builder, schemaIterator, rootSchema.getNum_children()); return builder.named(rootSchema.name); }
private void addKeyValue(FileMetaData fileMetaData, String key, String value) { KeyValue keyValue = new KeyValue(key); keyValue.value = value; fileMetaData.addToKey_value_metadata(keyValue); }
public static FileMetaData readFileMetaData(InputStream from) throws IOException { return read(from, new FileMetaData()); } /**
public PageHeader( PageType type, int uncompressed_page_size, int compressed_page_size) { this(); this.type = type; this.uncompressed_page_size = uncompressed_page_size; setUncompressed_page_sizeIsSet(true); this.compressed_page_size = compressed_page_size; setCompressed_page_sizeIsSet(true); }
private static Set<parquet.column.Encoding> readEncodings(List<Encoding> encodings) { Set<parquet.column.Encoding> columnEncodings = new HashSet<>(); for (Encoding encoding : encodings) { columnEncodings.add(parquet.column.Encoding.valueOf(encoding.name())); } return Collections.unmodifiableSet(columnEncodings); }
/** * If true, indicates this column is sorted in descending order. * */ public SortingColumn setDescending(boolean descending) { this.descending = descending; setDescendingIsSet(true); return this; }
/** * Nested fields. Since thrift does not support nested fields, * the nesting is flattened to a single list by a depth-first traversal. * The children count is used to construct the nested relationship. * This field is not set when the element is a primitive type */ public SchemaElement setNum_children(int num_children) { this.num_children = num_children; setNum_childrenIsSet(true); return this; }
/** * When the original schema supports field ids, this will save the * original field id in the parquet schema */ public SchemaElement setField_id(int field_id) { this.field_id = field_id; setField_idIsSet(true); return this; }
@Override public void clear() { this.file_path = null; setFile_offsetIsSet(false); this.file_offset = 0; this.meta_data = null; }