-
Notifications
You must be signed in to change notification settings - Fork 1.5k
PARQUET-1201: Column indexes #527
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
14 commits
Select commit
Hold shift + click to select a range
aa571d7
PARQUET-1211: Column indexes: read/write API (#456)
gszadovszky 6165a0c
PARQUET-1212: Column indexes: Show indexes in tools (#479)
gszadovszky 1001994
PARQUET-1213: Column indexes: Limit index size (#480)
gszadovszky dc645db
PARQUET-1214: Column indexes: Truncate min/max values (#481)
gszadovszky 43ac3e1
PARQUET-1364: Invalid row indexes for pages starting with nulls (#507)
gszadovszky d8e78eb
PARQUET-1310: Column indexes: Filtering (#509)
gszadovszky 1f95eca
PARQUET-1386: Fix issues of NaN and +-0.0 in case of float/double col…
gszadovszky 55d791c
PARQUET-1389: Improve value skipping at page synchronization (#514)
gszadovszky 85e699c
Merge branch 'master' into column-indexes
gszadovszky c215f1f
PARQUET-1381: Fix missing endRecord after merging columnIndex
gszadovszky e551893
PARQUET-1201: Fix review findings
gszadovszky 1206c60
PARQUET-1201: Delete class NotInPageFilteringModeException as it is n…
gszadovszky 5b55d87
PARQUET-1201: Complete removing NotInPageFilteringModeException
gszadovszky 6781c8d
Merge branch 'master' into column-indexes
gszadovszky File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
157 changes: 157 additions & 0 deletions
157
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowColumnIndexCommand.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| 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; | ||
| } | ||
|
|
||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
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.Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
newMemColumnReaderis used byParquetFileWriter.merge(List<InputFile>, BytesCompressor, String, long)introduced in PARQUET-1381. The implementation logic is different in the two methods.getColumnReader(ColumnDescriptor)uses the internalPageReadStoreinstance to get thePageReaderand the row indexes (to create the synchronizing reader if required). In the other handnewMemColumnReadergets thePageReaderas a parameter and the internalPageReadStoreis not used (no way of creating a synchronizing reader).Because of these differences the two logic cannot be merged.