Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions parquet-cli/src/main/java/org/apache/parquet/cli/Main.java
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.parquet.cli.commands.ConvertCommand;
import org.apache.parquet.cli.commands.ParquetMetadataCommand;
import org.apache.parquet.cli.commands.SchemaCommand;
import org.apache.parquet.cli.commands.ShowColumnIndexCommand;
import org.apache.parquet.cli.commands.ShowDictionaryCommand;
import org.apache.parquet.cli.commands.ShowPagesCommand;
import org.apache.parquet.cli.commands.ToAvroCommand;
Expand Down Expand Up @@ -87,6 +88,7 @@ public class Main extends Configured implements Tool {
jc.addCommand("to-avro", new ToAvroCommand(console));
jc.addCommand("cat", new CatCommand(console, 0));
jc.addCommand("head", new CatCommand(console, 10));
jc.addCommand("column-index", new ShowColumnIndexCommand(console));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,157 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.parquet.cli.commands;

import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;

import org.apache.parquet.cli.BaseCommand;
import org.apache.parquet.hadoop.ParquetFileReader;
import org.apache.parquet.hadoop.metadata.BlockMetaData;
import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
import org.apache.parquet.hadoop.util.HadoopInputFile;
import org.apache.parquet.internal.column.columnindex.ColumnIndex;
import org.apache.parquet.internal.column.columnindex.OffsetIndex;
import org.apache.parquet.io.InputFile;
import org.slf4j.Logger;

import com.beust.jcommander.Parameter;
import com.beust.jcommander.Parameters;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;

/**
* parquet-cli command to print column and offset indexes.
*/
@Parameters(commandDescription = "Prints the column and offset indexes of a Parquet file")
public class ShowColumnIndexCommand extends BaseCommand {
public ShowColumnIndexCommand(Logger console) {
super(console);
}

@Parameter(description = "<parquet path>")
List<String> files;

@Parameter(names = { "-c", "--column" }, description = "Shows the column/offset indexes for the given column only")
List<String> ColumnPaths;

@Parameter(names = { "-r",
"--row-group" }, description = "Shows the column/offset indexes for the given row-groups only; "
+ "row-groups are referenced by their indexes from 0")
List<String> rowGroupIndexes;

@Parameter(names = { "-i", "--column-index" }, description = "Shows the column indexes; "
+ "active by default unless -o is used")
boolean showColumnIndex;

@Parameter(names = { "-o", "--offset-index" }, description = "Shows the offset indexes; "
+ "active by default unless -i is used")
boolean showOffsetIndex;

@Override
public List<String> getExamples() {
return Lists.newArrayList(
"# Show only column indexes for column 'col' from a Parquet file",
"-c col -i sample.parquet");
}

@Override
public int run() throws IOException {
Preconditions.checkArgument(files != null && files.size() >= 1,
"A Parquet file is required.");
Preconditions.checkArgument(files.size() == 1,
"Cannot process multiple Parquet files.");

InputFile in = HadoopInputFile.fromPath(qualifiedPath(files.get(0)), getConf());
if (!showColumnIndex && !showOffsetIndex) {
showColumnIndex = true;
showOffsetIndex = true;
}

Set<String> rowGroupIndexSet = new HashSet<>();
if (rowGroupIndexes != null) {
rowGroupIndexSet.addAll(rowGroupIndexes);
}

try (ParquetFileReader reader = ParquetFileReader.open(in)) {
boolean firstBlock = true;
int rowGroupIndex = 0;
for (BlockMetaData block : reader.getFooter().getBlocks()) {
if (!rowGroupIndexSet.isEmpty() && !rowGroupIndexSet.contains(Integer.toString(rowGroupIndex))) {
++rowGroupIndex;
continue;
}
if (!firstBlock) {
console.info("");
}
firstBlock = false;
console.info("row-group {}:", rowGroupIndex);
for (ColumnChunkMetaData column : getColumns(block)) {
String path = column.getPath().toDotString();
if (showColumnIndex) {
console.info("column index for column {}:", path);
ColumnIndex columnIndex = reader.readColumnIndex(column);
if (columnIndex == null) {
console.info("NONE");
} else {
console.info(columnIndex.toString());
}
}
if (showOffsetIndex) {
console.info("offset index for column {}:", path);
OffsetIndex offsetIndex = reader.readOffsetIndex(column);
if (offsetIndex == null) {
console.info("NONE");
} else {
console.info(offsetIndex.toString());
}
}
}
++rowGroupIndex;
}
}
return 0;
}

private List<ColumnChunkMetaData> getColumns(BlockMetaData block) {
List<ColumnChunkMetaData> columns = block.getColumns();
if (ColumnPaths == null || ColumnPaths.isEmpty()) {
return columns;
}
Map<String, ColumnChunkMetaData> pathMap = new HashMap<>();
for (ColumnChunkMetaData column : columns) {
pathMap.put(column.getPath().toDotString(), column);
}

List<ColumnChunkMetaData> filtered = new ArrayList<>();
for (String path : ColumnPaths) {
ColumnChunkMetaData column = pathMap.get(path);
if (column != null) {
filtered.add(column);
}
}
return filtered;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,10 @@ public interface ColumnReader {

/**
* @return the totalCount of values to be consumed
* @deprecated will be removed in 2.0.0; Total values might not be able to be counted before reading the values (e.g.
* in case of column index based filtering)
*/
@Deprecated
long getTotalValueCount();

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ public class ParquetProperties {
public static final boolean DEFAULT_ESTIMATE_ROW_COUNT_FOR_PAGE_SIZE_CHECK = true;
public static final int DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK = 100;
public static final int DEFAULT_MAXIMUM_RECORD_COUNT_FOR_CHECK = 10000;
public static final int DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH = 64;

public static final ValuesWriterFactory DEFAULT_VALUES_WRITER_FACTORY = new DefaultValuesWriterFactory();

Expand Down Expand Up @@ -83,10 +84,11 @@ public static WriterVersion fromString(String name) {
private final boolean estimateNextSizeCheck;
private final ByteBufferAllocator allocator;
private final ValuesWriterFactory valuesWriterFactory;
private final int columnIndexTruncateLength;

private ParquetProperties(WriterVersion writerVersion, int pageSize, int dictPageSize, boolean enableDict, int minRowCountForPageSizeCheck,
int maxRowCountForPageSizeCheck, boolean estimateNextSizeCheck, ByteBufferAllocator allocator,
ValuesWriterFactory writerFactory) {
ValuesWriterFactory writerFactory, int columnIndexMinMaxTruncateLength) {
this.pageSizeThreshold = pageSize;
this.initialSlabSize = CapacityByteArrayOutputStream
.initialSlabSizeHeuristic(MIN_SLAB_SIZE, pageSizeThreshold, 10);
Expand All @@ -99,6 +101,7 @@ private ParquetProperties(WriterVersion writerVersion, int pageSize, int dictPag
this.allocator = allocator;

this.valuesWriterFactory = writerFactory;
this.columnIndexTruncateLength = columnIndexMinMaxTruncateLength;
}

public ValuesWriter newRepetitionLevelWriter(ColumnDescriptor path) {
Expand Down Expand Up @@ -163,7 +166,7 @@ public ColumnWriteStore newColumnWriteStore(MessageType schema,
PageWriteStore pageStore) {
switch (writerVersion) {
case PARQUET_1_0:
return new ColumnWriteStoreV1(pageStore, this);
return new ColumnWriteStoreV1(schema, pageStore, this);
case PARQUET_2_0:
return new ColumnWriteStoreV2(schema, pageStore, this);
default:
Expand All @@ -183,6 +186,10 @@ public ValuesWriterFactory getValuesWriterFactory() {
return valuesWriterFactory;
}

public int getColumnIndexTruncateLength() {
return columnIndexTruncateLength;
}

public boolean estimateNextSizeCheck() {
return estimateNextSizeCheck;
}
Expand All @@ -205,6 +212,7 @@ public static class Builder {
private boolean estimateNextSizeCheck = DEFAULT_ESTIMATE_ROW_COUNT_FOR_PAGE_SIZE_CHECK;
private ByteBufferAllocator allocator = new HeapByteBufferAllocator();
private ValuesWriterFactory valuesWriterFactory = DEFAULT_VALUES_WRITER_FACTORY;
private int columnIndexTruncateLength = DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;

private Builder() {
}
Expand Down Expand Up @@ -299,11 +307,17 @@ public Builder withValuesWriterFactory(ValuesWriterFactory factory) {
return this;
}

public Builder withColumnIndexTruncateLength(int length) {
Preconditions.checkArgument(length > 0, "Invalid column index min/max truncate length (negative) : %s", length);
this.columnIndexTruncateLength = length;
return this;
}

public ParquetProperties build() {
ParquetProperties properties =
new ParquetProperties(writerVersion, pageSize, dictPageSize,
enableDict, minRowCountForPageSizeCheck, maxRowCountForPageSizeCheck,
estimateNextSizeCheck, allocator, valuesWriterFactory);
estimateNextSizeCheck, allocator, valuesWriterFactory, columnIndexTruncateLength);
// we pass a constructed but uninitialized factory to ParquetProperties above as currently
// creation of ValuesWriters is invoked from within ParquetProperties. In the future
// we'd like to decouple that and won't need to pass an object to properties and then pass the
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,9 @@
*/
package org.apache.parquet.column.impl;

import java.util.Optional;
import java.util.PrimitiveIterator;

import org.apache.parquet.VersionParser;
import org.apache.parquet.VersionParser.ParsedVersion;
import org.apache.parquet.VersionParser.VersionParseException;
Expand Down Expand Up @@ -72,7 +75,14 @@ public ColumnReadStoreImpl(PageReadStore pageReadStore,

@Override
public ColumnReader getColumnReader(ColumnDescriptor path) {
return newMemColumnReader(path, pageReadStore.getPageReader(path));
PrimitiveConverter converter = getPrimitiveConverter(path);
PageReader pageReader = pageReadStore.getPageReader(path);
Optional<PrimitiveIterator.OfLong> rowIndexes = pageReadStore.getRowIndexes();
if (rowIndexes.isPresent()) {
return new SynchronizingColumnReader(path, pageReader, converter, writerVersion, rowIndexes.get());
} else {
return new ColumnReaderImpl(path, pageReader, converter, writerVersion);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why doesn't this use newMemColumnReader? Since there are only two uses of that function, I think that either both of them should be inlined like this, or both should continue calling it.

Copy link
Contributor Author

@gszadovszky gszadovszky Oct 1, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

newMemColumnReader is used by ParquetFileWriter.merge(List<InputFile>, BytesCompressor, String, long) introduced in PARQUET-1381. The implementation logic is different in the two methods.
getColumnReader(ColumnDescriptor) uses the internal PageReadStore instance to get the PageReader and the row indexes (to create the synchronizing reader if required). In the other hand newMemColumnReader gets the PageReader as a parameter and the internal PageReadStore is not used (no way of creating a synchronizing reader).
Because of these differences the two logic cannot be merged.

}
}

public ColumnReaderImpl newMemColumnReader(ColumnDescriptor path, PageReader pageReader) {
Expand Down
Loading