Skip to content
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

GH-2986: Fails the file writing when footer size exceeds int max value #2987

Merged
merged 3 commits into from
Aug 29, 2024
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
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* 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;

/**
* RuntimeException occurs when size overflow.
*/
public class ParquetSizeOverflowException extends ParquetRuntimeException {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

put it into the common module since it is a common class.

private static final long serialVersionUID = 1L;

public ParquetSizeOverflowException() {
super();
}

public ParquetSizeOverflowException(String message) {
super(message);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.ParquetSizeOverflowException;
import org.apache.parquet.Preconditions;
import org.apache.parquet.Version;
import org.apache.parquet.bytes.ByteBufferAllocator;
Expand Down Expand Up @@ -703,7 +704,7 @@ public void writeDataPage(
columnIndexBuilder = ColumnIndexBuilder.getNoOpBuilder();
long beforeHeader = out.getPos();
LOG.debug("{}: write data page: {} values", beforeHeader, valueCount);
int compressedPageSize = (int) bytes.size();
int compressedPageSize = toIntWithCheck(bytes.size(), "page");
metadataConverter.writeDataPageV1Header(
uncompressedPageSize, compressedPageSize, valueCount, rlEncoding, dlEncoding, valuesEncoding, out);
long headerSize = out.getPos() - beforeHeader;
Expand Down Expand Up @@ -879,7 +880,7 @@ public void writeDataPage(
pageHeaderAAD,
sizeStatistics);
offsetIndexBuilder.add(
(int) (out.getPos() - beforeHeader),
toIntWithCheck(out.getPos() - beforeHeader, "page"),
rowCount,
sizeStatistics != null ? sizeStatistics.getUnencodedByteArrayDataBytes() : Optional.empty());
}
Expand Down Expand Up @@ -979,7 +980,7 @@ public void writeDataPage(
currentChunkFirstDataPage = beforeHeader;
}
LOG.debug("{}: write data page: {} values", beforeHeader, valueCount);
int compressedPageSize = (int) bytes.size();
int compressedPageSize = toIntWithCheck(bytes.size(), "page");
if (pageWriteChecksumEnabled) {
crc.reset();
crcUpdate(bytes);
Expand Down Expand Up @@ -1146,12 +1147,14 @@ public void writeDataPageV2(
SizeStatistics sizeStatistics)
throws IOException {
state = state.write();
int rlByteLength = toIntWithCheck(repetitionLevels.size());
int dlByteLength = toIntWithCheck(definitionLevels.size());
int rlByteLength = toIntWithCheck(repetitionLevels.size(), "page repetition levels");
int dlByteLength = toIntWithCheck(definitionLevels.size(), "page definition levels");

int compressedSize = toIntWithCheck(compressedData.size() + repetitionLevels.size() + definitionLevels.size());
int compressedSize =
toIntWithCheck(compressedData.size() + repetitionLevels.size() + definitionLevels.size(), "page");

int uncompressedSize = toIntWithCheck(uncompressedDataSize + repetitionLevels.size() + definitionLevels.size());
int uncompressedSize =
toIntWithCheck(uncompressedDataSize + repetitionLevels.size() + definitionLevels.size(), "page");

long beforeHeader = out.getPos();
if (currentChunkFirstDataPage < 0) {
Expand Down Expand Up @@ -1209,7 +1212,7 @@ public void writeDataPageV2(
BytesInput.concat(repetitionLevels, definitionLevels, compressedData).writeAllTo(out);

offsetIndexBuilder.add(
(int) (out.getPos() - beforeHeader),
toIntWithCheck(out.getPos() - beforeHeader, "page"),
rowCount,
sizeStatistics != null ? sizeStatistics.getUnencodedByteArrayDataBytes() : Optional.empty());
}
Expand Down Expand Up @@ -1626,8 +1629,8 @@ private static void copy(SeekableInputStream from, PositionOutputStream to, long
long bytesCopied = 0;
byte[] buffer = COPY_BUFFER.get();
while (bytesCopied < length) {
long bytesLeft = length - bytesCopied;
int bytesRead = from.read(buffer, 0, (buffer.length < bytesLeft ? buffer.length : (int) bytesLeft));
int bytesLeft = Math.toIntExact(length - bytesCopied);
int bytesRead = from.read(buffer, 0, (Math.min(buffer.length, bytesLeft)));
if (bytesRead < 0) {
throw new IllegalArgumentException("Unexpected end of input file at " + start + bytesCopied);
}
Expand Down Expand Up @@ -1707,15 +1710,16 @@ private static void serializeColumnIndexes(
}
long offset = out.getPos();
Util.writeColumnIndex(columnIndex, out, columnIndexEncryptor, columnIndexAAD);
column.setColumnIndexReference(new IndexReference(offset, (int) (out.getPos() - offset)));
column.setColumnIndexReference(
new IndexReference(offset, toIntWithCheck(out.getPos() - offset, "page")));
}
}
}

private int toIntWithCheck(long size) {
private static int toIntWithCheck(long size, String obj) {
if ((int) size != size) {
throw new ParquetEncodingException(
"Cannot write page larger than " + Integer.MAX_VALUE + " bytes: " + size);
throw new ParquetSizeOverflowException(
String.format("Cannot write %s larger than %s bytes: %s", obj, Integer.MAX_VALUE, size));
}
return (int) size;
}
Expand Down Expand Up @@ -1787,7 +1791,8 @@ private static void serializeOffsetIndexes(
out,
offsetIndexEncryptor,
offsetIndexAAD);
column.setOffsetIndexReference(new IndexReference(offset, (int) (out.getPos() - offset)));
column.setOffsetIndexReference(
new IndexReference(offset, toIntWithCheck(out.getPos() - offset, "page")));
}
}
}
Expand Down Expand Up @@ -1852,7 +1857,7 @@ private static void serializeBloomFilters(
}
out.write(serializedBitset);

int length = (int) (out.getPos() - offset);
int length = Math.toIntExact(out.getPos() - offset);
column.setBloomFilterLength(length);
}
}
Expand All @@ -1872,7 +1877,7 @@ private static void serializeFooter(
metadataConverter.toParquetMetadata(CURRENT_VERSION, footer);
writeFileMetaData(parquetMetadata, out);
LOG.debug("{}: footer length = {}", out.getPos(), (out.getPos() - footerIndex));
BytesUtils.writeIntLittleEndian(out, (int) (out.getPos() - footerIndex));
BytesUtils.writeIntLittleEndian(out, toIntWithCheck(out.getPos() - footerIndex, "footer"));
out.write(MAGIC);
return;
}
Expand Down Expand Up @@ -1910,7 +1915,7 @@ private static void serializeFooter(
out.write(serializedFooter);
out.write(signature);
LOG.debug("{}: footer and signature length = {}", out.getPos(), (out.getPos() - footerIndex));
BytesUtils.writeIntLittleEndian(out, (int) (out.getPos() - footerIndex));
BytesUtils.writeIntLittleEndian(out, toIntWithCheck(out.getPos() - footerIndex, "page"));
out.write(MAGIC);
return;
}
Expand All @@ -1920,7 +1925,7 @@ private static void serializeFooter(
writeFileCryptoMetaData(fileEncryptor.getFileCryptoMetaData(), out);
byte[] footerAAD = AesCipher.createFooterAAD(fileEncryptor.getFileAAD());
writeFileMetaData(parquetMetadata, out, fileEncryptor.getFooterEncryptor(), footerAAD);
int combinedMetaDataLength = (int) (out.getPos() - cryptoFooterIndex);
int combinedMetaDataLength = toIntWithCheck(out.getPos() - cryptoFooterIndex, "page");
LOG.debug("{}: crypto metadata and footer length = {}", out.getPos(), combinedMetaDataLength);
BytesUtils.writeIntLittleEndian(out, combinedMetaDataLength);
out.write(EFMAGIC);
Expand Down