-
Notifications
You must be signed in to change notification settings - Fork 1.5k
PARQUET-2078: Failed to read parquet file after writing with the same … #925
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
Changes from all commits
0a0a00c
12f39c3
0328fb7
0e01c90
3a2757d
cd13468
347858d
aefe136
00e737e
c1802ec
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,29 @@ | ||
| /* | ||
| * 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.io; | ||
|
|
||
| import org.apache.parquet.ParquetRuntimeException; | ||
|
|
||
| public class InvalidFileOffsetException extends ParquetRuntimeException { | ||
| private static final long serialVersionUID = 1L; | ||
|
|
||
| public InvalidFileOffsetException(String message) { | ||
| super(message); | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -45,6 +45,7 @@ | |
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.parquet.CorruptStatistics; | ||
| import org.apache.parquet.ParquetReadOptions; | ||
| import org.apache.parquet.Preconditions; | ||
| import org.apache.parquet.bytes.ByteBufferInputStream; | ||
| import org.apache.parquet.column.ParquetProperties; | ||
| import org.apache.parquet.column.statistics.BinaryStatistics; | ||
|
|
@@ -119,10 +120,10 @@ | |
| import org.apache.parquet.internal.column.columnindex.ColumnIndexBuilder; | ||
| import org.apache.parquet.internal.column.columnindex.OffsetIndexBuilder; | ||
| import org.apache.parquet.internal.hadoop.metadata.IndexReference; | ||
| import org.apache.parquet.io.InvalidFileOffsetException; | ||
| import org.apache.parquet.io.ParquetDecodingException; | ||
| import org.apache.parquet.io.api.Binary; | ||
| import org.apache.parquet.schema.ColumnOrder.ColumnOrderName; | ||
| import org.apache.parquet.schema.LogicalTypeAnnotation.LogicalTypeAnnotationVisitor; | ||
| import org.apache.parquet.schema.LogicalTypeAnnotation.UUIDLogicalTypeAnnotation; | ||
| import org.apache.parquet.schema.GroupType; | ||
| import org.apache.parquet.schema.MessageType; | ||
|
|
@@ -201,8 +202,22 @@ public FileMetaData toParquetMetadata(int currentVersion, ParquetMetadata parque | |
| List<BlockMetaData> blocks = parquetMetadata.getBlocks(); | ||
| List<RowGroup> rowGroups = new ArrayList<RowGroup>(); | ||
| long numRows = 0; | ||
| long preBlockStartPos = 0; | ||
| long preBlockCompressedSize = 0; | ||
| for (BlockMetaData block : blocks) { | ||
| numRows += block.getRowCount(); | ||
| long blockStartPos = block.getStartingPos(); | ||
| // first block | ||
| if (blockStartPos == 4) { | ||
| preBlockStartPos = 0; | ||
| preBlockCompressedSize = 0; | ||
| } | ||
| if (preBlockStartPos != 0) { | ||
| Preconditions.checkState(blockStartPos >= preBlockStartPos + preBlockCompressedSize, | ||
| "Invalid block starting position:" + blockStartPos); | ||
| } | ||
| preBlockStartPos = blockStartPos; | ||
| preBlockCompressedSize = block.getCompressedSize(); | ||
| addRowGroup(parquetMetadata, rowGroups, block, fileEncryptor); | ||
| } | ||
| FileMetaData fileMetaData = new FileMetaData( | ||
|
|
@@ -1226,14 +1241,36 @@ public ParquetMetadata readParquetMetadata(InputStream from) throws IOException | |
| static FileMetaData filterFileMetaDataByMidpoint(FileMetaData metaData, RangeMetadataFilter filter) { | ||
| List<RowGroup> rowGroups = metaData.getRow_groups(); | ||
| List<RowGroup> newRowGroups = new ArrayList<RowGroup>(); | ||
| long preStartIndex = 0; | ||
| long preCompressedSize = 0; | ||
| boolean firstColumnWithMetadata = true; | ||
| if (rowGroups != null && rowGroups.size() > 0) { | ||
| firstColumnWithMetadata = rowGroups.get(0).getColumns().get(0).isSetMeta_data(); | ||
| } | ||
| for (RowGroup rowGroup : rowGroups) { | ||
| long totalSize = 0; | ||
| long startIndex; | ||
| ColumnChunk columnChunk = rowGroup.getColumns().get(0); | ||
| if (firstColumnWithMetadata) { | ||
| startIndex = getOffset(columnChunk); | ||
| } else { | ||
| assert rowGroup.isSetFile_offset(); | ||
| assert rowGroup.isSetTotal_compressed_size(); | ||
|
|
||
| if (rowGroup.isSetFile_offset()) { | ||
| //the file_offset of first block always holds the truth, while other blocks don't : | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think this is no longer true with the issue we found in parquet-cpp.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @sunchao Thanks for your information.May be the first rowgroup check is enough for this situation(firstFileOffset==4)? I will submit a new commit. |
||
| //see PARQUET-2078 for details | ||
| startIndex = rowGroup.getFile_offset(); | ||
| } else { | ||
| startIndex = getOffset(rowGroup.getColumns().get(0)); | ||
| if (invalidFileOffset(startIndex, preStartIndex, preCompressedSize)) { | ||
| //first row group's offset is always 4 | ||
| if (preStartIndex == 0) { | ||
| startIndex = 4; | ||
| } else { | ||
| // use minStartIndex(imprecise in case of padding, but good enough for filtering) | ||
| startIndex = preStartIndex + preCompressedSize; | ||
| } | ||
| } | ||
| preStartIndex = startIndex; | ||
| preCompressedSize = rowGroup.getTotal_compressed_size(); | ||
| } | ||
|
|
||
| if (rowGroup.isSetTotal_compressed_size()) { | ||
|
|
@@ -1254,16 +1291,59 @@ static FileMetaData filterFileMetaDataByMidpoint(FileMetaData metaData, RangeMet | |
| return metaData; | ||
| } | ||
|
|
||
| private static boolean invalidFileOffset(long startIndex, long preStartIndex, long preCompressedSize) { | ||
| boolean invalid = false; | ||
| assert preStartIndex <= startIndex; | ||
| // checking the first rowGroup | ||
| if (preStartIndex == 0 && startIndex != 4) { | ||
| invalid = true; | ||
| return invalid; | ||
| } | ||
|
|
||
| //calculate start index for other blocks | ||
| long minStartIndex = preStartIndex + preCompressedSize; | ||
| if (startIndex < minStartIndex) { | ||
| // a bad offset detected, try first column's offset | ||
| // can not use minStartIndex in case of padding | ||
| invalid = true; | ||
| } | ||
|
|
||
| return invalid; | ||
| } | ||
|
|
||
| // Visible for testing | ||
| static FileMetaData filterFileMetaDataByStart(FileMetaData metaData, OffsetMetadataFilter filter) { | ||
| List<RowGroup> rowGroups = metaData.getRow_groups(); | ||
| List<RowGroup> newRowGroups = new ArrayList<RowGroup>(); | ||
| long preStartIndex = 0; | ||
| long preCompressedSize = 0; | ||
| boolean firstColumnWithMetadata = true; | ||
| if (rowGroups != null && rowGroups.size() > 0) { | ||
| firstColumnWithMetadata = rowGroups.get(0).getColumns().get(0).isSetMeta_data(); | ||
| } | ||
| for (RowGroup rowGroup : rowGroups) { | ||
| long startIndex; | ||
| if (rowGroup.isSetFile_offset()) { | ||
| startIndex = rowGroup.getFile_offset(); | ||
| ColumnChunk columnChunk = rowGroup.getColumns().get(0); | ||
| if (firstColumnWithMetadata) { | ||
| startIndex = getOffset(columnChunk); | ||
| } else { | ||
| startIndex = getOffset(rowGroup.getColumns().get(0)); | ||
| assert rowGroup.isSetFile_offset(); | ||
| assert rowGroup.isSetTotal_compressed_size(); | ||
|
|
||
| //the file_offset of first block always holds the truth, while other blocks don't : | ||
| //see PARQUET-2078 for details | ||
| startIndex = rowGroup.getFile_offset(); | ||
| if (invalidFileOffset(startIndex, preStartIndex, preCompressedSize)) { | ||
| //first row group's offset is always 4 | ||
| if (preStartIndex == 0) { | ||
| startIndex = 4; | ||
| } else { | ||
| throw new InvalidFileOffsetException("corrupted RowGroup.file_offset found, " + | ||
| "please use file range instead of block offset for split."); | ||
| } | ||
| } | ||
| preStartIndex = startIndex; | ||
| preCompressedSize = rowGroup.getTotal_compressed_size(); | ||
| } | ||
|
|
||
| if (filter.contains(startIndex)) { | ||
|
|
||
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 this check is necessary, doesn't the first block always start at 4? Or this addresses a file merging usecase?
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.
To address _common_metadata file case, which will merge multiple file footers into a single meta file.