Copilot commented on code in PR #6470:
URL: https://github.com/apache/paimon/pull/6470#discussion_r2459503639


##########
paimon-common/src/main/java/org/apache/paimon/fs/MultiPartUploadTwoPhaseOutputStream.java:
##########
@@ -133,9 +145,11 @@ private void uploadPart() throws IOException {
         }
 
         File tempFile = null;
+        int partNumber = uploadedParts.size() + 1;
         try {
             byte[] data = buffer.toByteArray();
             tempFile = Files.createTempFile("multi-part-" + UUID.randomUUID(), 
".tmp").toFile();
+            tempFile.deleteOnExit();

Review Comment:
   The deleteOnExit() call registers the file with the JVM shutdown hook, which 
can cause memory leaks in long-running applications as these references 
accumulate and are never cleared until shutdown. Consider removing this line 
since the file is already explicitly deleted in the finally block at line 167.
   ```suggestion
   
   ```



##########
paimon-filesystems/paimon-jindo/src/main/java/org/apache/paimon/jindo/JindoMultiPartUpload.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.paimon.jindo;
+
+import org.apache.paimon.fs.MultiPartUploadStore;
+
+import com.aliyun.jindodata.api.spec.protos.JdoMpuUploadPartReply;
+import com.aliyun.jindodata.api.spec.protos.JdoObjectPart;
+import com.aliyun.jindodata.api.spec.protos.JdoObjectPartList;
+import com.aliyun.jindodata.common.JindoHadoopSystem;
+import com.aliyun.jindodata.store.JindoMpuStore;
+import org.apache.hadoop.fs.Path;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.List;
+
+/** Provides the multipart upload by Jindo. */
+public class JindoMultiPartUpload implements 
MultiPartUploadStore<JdoObjectPart, String> {
+
+    private final JindoHadoopSystem fs;
+    private final JindoMpuStore mpuStore;
+
+    public JindoMultiPartUpload(JindoHadoopSystem fs, Path filePath) {
+        this.fs = fs;
+        this.mpuStore = fs.getMpuStore(filePath);
+    }
+
+    @Override
+    public Path workingDirectory() {
+        return fs.getWorkingDirectory();
+    }
+
+    @Override
+    public String startMultiPartUpload(String objectName) throws IOException {
+        return mpuStore.initMultiPartUpload(new Path(objectName));
+    }
+
+    @Override
+    public String completeMultipartUpload(
+            String objectName,
+            String uploadId,
+            List<JdoObjectPart> partETags,
+            long numBytesInParts) {
+        try {
+            JdoObjectPartList partList =
+                    new 
com.aliyun.jindodata.api.spec.protos.JdoObjectPartList();
+            partList.setParts(partETags.toArray(new JdoObjectPart[0]));
+            mpuStore.commitMultiPartUpload(new Path(objectName), uploadId, 
partList);
+            return uploadId;
+        } catch (Exception e) {
+            throw new RuntimeException("Failed to complete multipart upload 
for: " + objectName, e);
+        }
+    }
+
+    @Override
+    public JdoObjectPart uploadPart(
+            String objectName, String uploadId, int partNumber, File file, 
long byteLength)
+            throws IOException {
+        try {
+            ByteBuffer buffer;
+            try (FileInputStream fis = new FileInputStream(file);
+                    FileChannel channel = fis.getChannel()) {

Review Comment:
   Potential integer overflow when casting byteLength from long to int. If 
byteLength exceeds Integer.MAX_VALUE, this will cause an incorrect buffer 
allocation or negative array size exception.
   ```suggestion
                       FileChannel channel = fis.getChannel()) {
                   if (byteLength > Integer.MAX_VALUE) {
                       throw new IllegalArgumentException("byteLength (" + 
byteLength + ") exceeds maximum supported buffer size (" + Integer.MAX_VALUE + 
")");
                   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to