This is an automated email from the ASF dual-hosted git repository.

gangwu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-java.git


The following commit(s) were added to refs/heads/master by this push:
     new fafd9b033 GH-2986: Fails the file writing when footer size exceeds int 
max value (#2987)
fafd9b033 is described below

commit fafd9b0339ff069b53f38269a8a9271b4e6c068a
Author: Xianyang Liu <[email protected]>
AuthorDate: Thu Aug 29 10:12:11 2024 +0800

    GH-2986: Fails the file writing when footer size exceeds int max value 
(#2987)
---
 .../parquet/ParquetSizeOverflowException.java      | 34 +++++++++++++++++
 .../apache/parquet/hadoop/ParquetFileWriter.java   | 43 ++++++++++++----------
 2 files changed, 58 insertions(+), 19 deletions(-)

diff --git 
a/parquet-common/src/main/java/org/apache/parquet/ParquetSizeOverflowException.java
 
b/parquet-common/src/main/java/org/apache/parquet/ParquetSizeOverflowException.java
new file mode 100644
index 000000000..946390991
--- /dev/null
+++ 
b/parquet-common/src/main/java/org/apache/parquet/ParquetSizeOverflowException.java
@@ -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 {
+  private static final long serialVersionUID = 1L;
+
+  public ParquetSizeOverflowException() {
+    super();
+  }
+
+  public ParquetSizeOverflowException(String message) {
+    super(message);
+  }
+}
diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java
index 814b98c50..f0a912f59 100644
--- 
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java
+++ 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.conf.Configuration;
 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;
@@ -703,7 +704,7 @@ public class ParquetFileWriter implements AutoCloseable {
     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;
@@ -879,7 +880,7 @@ public class ParquetFileWriter implements AutoCloseable {
         pageHeaderAAD,
         sizeStatistics);
     offsetIndexBuilder.add(
-        (int) (out.getPos() - beforeHeader),
+        toIntWithCheck(out.getPos() - beforeHeader, "page"),
         rowCount,
         sizeStatistics != null ? 
sizeStatistics.getUnencodedByteArrayDataBytes() : Optional.empty());
   }
@@ -979,7 +980,7 @@ public class ParquetFileWriter implements AutoCloseable {
       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);
@@ -1146,12 +1147,14 @@ public class ParquetFileWriter implements AutoCloseable 
{
       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) {
@@ -1209,7 +1212,7 @@ public class ParquetFileWriter implements AutoCloseable {
     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());
   }
@@ -1626,8 +1629,8 @@ public class ParquetFileWriter implements AutoCloseable {
     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);
       }
@@ -1707,15 +1710,16 @@ public class ParquetFileWriter implements AutoCloseable 
{
         }
         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;
   }
@@ -1787,7 +1791,8 @@ public class ParquetFileWriter implements AutoCloseable {
             out,
             offsetIndexEncryptor,
             offsetIndexAAD);
-        column.setOffsetIndexReference(new IndexReference(offset, (int) 
(out.getPos() - offset)));
+        column.setOffsetIndexReference(
+            new IndexReference(offset, toIntWithCheck(out.getPos() - offset, 
"page")));
       }
     }
   }
@@ -1852,7 +1857,7 @@ public class ParquetFileWriter implements AutoCloseable {
         }
         out.write(serializedBitset);
 
-        int length = (int) (out.getPos() - offset);
+        int length = Math.toIntExact(out.getPos() - offset);
         column.setBloomFilterLength(length);
       }
     }
@@ -1872,7 +1877,7 @@ public class ParquetFileWriter implements AutoCloseable {
           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;
     }
@@ -1910,7 +1915,7 @@ public class ParquetFileWriter implements AutoCloseable {
       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;
     }
@@ -1920,7 +1925,7 @@ public class ParquetFileWriter implements AutoCloseable {
     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);

Reply via email to