Skip to content

Commit fafd9b0

Browse files
authored
apacheGH-2986: Fails the file writing when footer size exceeds int max value (apache#2987)
1 parent 3b5fb4b commit fafd9b0

File tree

2 files changed

+58
-19
lines changed

2 files changed

+58
-19
lines changed
Lines changed: 34 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,34 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
package org.apache.parquet;
20+
21+
/**
22+
* RuntimeException occurs when size overflow.
23+
*/
24+
public class ParquetSizeOverflowException extends ParquetRuntimeException {
25+
private static final long serialVersionUID = 1L;
26+
27+
public ParquetSizeOverflowException() {
28+
super();
29+
}
30+
31+
public ParquetSizeOverflowException(String message) {
32+
super(message);
33+
}
34+
}

parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java

Lines changed: 24 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,7 @@
4141
import org.apache.hadoop.fs.FSDataInputStream;
4242
import org.apache.hadoop.fs.FileSystem;
4343
import org.apache.hadoop.fs.Path;
44+
import org.apache.parquet.ParquetSizeOverflowException;
4445
import org.apache.parquet.Preconditions;
4546
import org.apache.parquet.Version;
4647
import org.apache.parquet.bytes.ByteBufferAllocator;
@@ -703,7 +704,7 @@ public void writeDataPage(
703704
columnIndexBuilder = ColumnIndexBuilder.getNoOpBuilder();
704705
long beforeHeader = out.getPos();
705706
LOG.debug("{}: write data page: {} values", beforeHeader, valueCount);
706-
int compressedPageSize = (int) bytes.size();
707+
int compressedPageSize = toIntWithCheck(bytes.size(), "page");
707708
metadataConverter.writeDataPageV1Header(
708709
uncompressedPageSize, compressedPageSize, valueCount, rlEncoding, dlEncoding, valuesEncoding, out);
709710
long headerSize = out.getPos() - beforeHeader;
@@ -879,7 +880,7 @@ public void writeDataPage(
879880
pageHeaderAAD,
880881
sizeStatistics);
881882
offsetIndexBuilder.add(
882-
(int) (out.getPos() - beforeHeader),
883+
toIntWithCheck(out.getPos() - beforeHeader, "page"),
883884
rowCount,
884885
sizeStatistics != null ? sizeStatistics.getUnencodedByteArrayDataBytes() : Optional.empty());
885886
}
@@ -979,7 +980,7 @@ public void writeDataPage(
979980
currentChunkFirstDataPage = beforeHeader;
980981
}
981982
LOG.debug("{}: write data page: {} values", beforeHeader, valueCount);
982-
int compressedPageSize = (int) bytes.size();
983+
int compressedPageSize = toIntWithCheck(bytes.size(), "page");
983984
if (pageWriteChecksumEnabled) {
984985
crc.reset();
985986
crcUpdate(bytes);
@@ -1146,12 +1147,14 @@ public void writeDataPageV2(
11461147
SizeStatistics sizeStatistics)
11471148
throws IOException {
11481149
state = state.write();
1149-
int rlByteLength = toIntWithCheck(repetitionLevels.size());
1150-
int dlByteLength = toIntWithCheck(definitionLevels.size());
1150+
int rlByteLength = toIntWithCheck(repetitionLevels.size(), "page repetition levels");
1151+
int dlByteLength = toIntWithCheck(definitionLevels.size(), "page definition levels");
11511152

1152-
int compressedSize = toIntWithCheck(compressedData.size() + repetitionLevels.size() + definitionLevels.size());
1153+
int compressedSize =
1154+
toIntWithCheck(compressedData.size() + repetitionLevels.size() + definitionLevels.size(), "page");
11531155

1154-
int uncompressedSize = toIntWithCheck(uncompressedDataSize + repetitionLevels.size() + definitionLevels.size());
1156+
int uncompressedSize =
1157+
toIntWithCheck(uncompressedDataSize + repetitionLevels.size() + definitionLevels.size(), "page");
11551158

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

12111214
offsetIndexBuilder.add(
1212-
(int) (out.getPos() - beforeHeader),
1215+
toIntWithCheck(out.getPos() - beforeHeader, "page"),
12131216
rowCount,
12141217
sizeStatistics != null ? sizeStatistics.getUnencodedByteArrayDataBytes() : Optional.empty());
12151218
}
@@ -1626,8 +1629,8 @@ private static void copy(SeekableInputStream from, PositionOutputStream to, long
16261629
long bytesCopied = 0;
16271630
byte[] buffer = COPY_BUFFER.get();
16281631
while (bytesCopied < length) {
1629-
long bytesLeft = length - bytesCopied;
1630-
int bytesRead = from.read(buffer, 0, (buffer.length < bytesLeft ? buffer.length : (int) bytesLeft));
1632+
int bytesLeft = Math.toIntExact(length - bytesCopied);
1633+
int bytesRead = from.read(buffer, 0, (Math.min(buffer.length, bytesLeft)));
16311634
if (bytesRead < 0) {
16321635
throw new IllegalArgumentException("Unexpected end of input file at " + start + bytesCopied);
16331636
}
@@ -1707,15 +1710,16 @@ private static void serializeColumnIndexes(
17071710
}
17081711
long offset = out.getPos();
17091712
Util.writeColumnIndex(columnIndex, out, columnIndexEncryptor, columnIndexAAD);
1710-
column.setColumnIndexReference(new IndexReference(offset, (int) (out.getPos() - offset)));
1713+
column.setColumnIndexReference(
1714+
new IndexReference(offset, toIntWithCheck(out.getPos() - offset, "page")));
17111715
}
17121716
}
17131717
}
17141718

1715-
private int toIntWithCheck(long size) {
1719+
private static int toIntWithCheck(long size, String obj) {
17161720
if ((int) size != size) {
1717-
throw new ParquetEncodingException(
1718-
"Cannot write page larger than " + Integer.MAX_VALUE + " bytes: " + size);
1721+
throw new ParquetSizeOverflowException(
1722+
String.format("Cannot write %s larger than %s bytes: %s", obj, Integer.MAX_VALUE, size));
17191723
}
17201724
return (int) size;
17211725
}
@@ -1787,7 +1791,8 @@ private static void serializeOffsetIndexes(
17871791
out,
17881792
offsetIndexEncryptor,
17891793
offsetIndexAAD);
1790-
column.setOffsetIndexReference(new IndexReference(offset, (int) (out.getPos() - offset)));
1794+
column.setOffsetIndexReference(
1795+
new IndexReference(offset, toIntWithCheck(out.getPos() - offset, "page")));
17911796
}
17921797
}
17931798
}
@@ -1852,7 +1857,7 @@ private static void serializeBloomFilters(
18521857
}
18531858
out.write(serializedBitset);
18541859

1855-
int length = (int) (out.getPos() - offset);
1860+
int length = Math.toIntExact(out.getPos() - offset);
18561861
column.setBloomFilterLength(length);
18571862
}
18581863
}
@@ -1872,7 +1877,7 @@ private static void serializeFooter(
18721877
metadataConverter.toParquetMetadata(CURRENT_VERSION, footer);
18731878
writeFileMetaData(parquetMetadata, out);
18741879
LOG.debug("{}: footer length = {}", out.getPos(), (out.getPos() - footerIndex));
1875-
BytesUtils.writeIntLittleEndian(out, (int) (out.getPos() - footerIndex));
1880+
BytesUtils.writeIntLittleEndian(out, toIntWithCheck(out.getPos() - footerIndex, "footer"));
18761881
out.write(MAGIC);
18771882
return;
18781883
}
@@ -1910,7 +1915,7 @@ private static void serializeFooter(
19101915
out.write(serializedFooter);
19111916
out.write(signature);
19121917
LOG.debug("{}: footer and signature length = {}", out.getPos(), (out.getPos() - footerIndex));
1913-
BytesUtils.writeIntLittleEndian(out, (int) (out.getPos() - footerIndex));
1918+
BytesUtils.writeIntLittleEndian(out, toIntWithCheck(out.getPos() - footerIndex, "page"));
19141919
out.write(MAGIC);
19151920
return;
19161921
}
@@ -1920,7 +1925,7 @@ private static void serializeFooter(
19201925
writeFileCryptoMetaData(fileEncryptor.getFileCryptoMetaData(), out);
19211926
byte[] footerAAD = AesCipher.createFooterAAD(fileEncryptor.getFileAAD());
19221927
writeFileMetaData(parquetMetadata, out, fileEncryptor.getFooterEncryptor(), footerAAD);
1923-
int combinedMetaDataLength = (int) (out.getPos() - cryptoFooterIndex);
1928+
int combinedMetaDataLength = toIntWithCheck(out.getPos() - cryptoFooterIndex, "page");
19241929
LOG.debug("{}: crypto metadata and footer length = {}", out.getPos(), combinedMetaDataLength);
19251930
BytesUtils.writeIntLittleEndian(out, combinedMetaDataLength);
19261931
out.write(EFMAGIC);

0 commit comments

Comments
 (0)