You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/10/12 05:57:14 UTC

[GitHub] [iceberg] Px951213 opened a new issue, #5963: Aliyun-OssFileIO: Premature end of Content-Length delimited message body

Px951213 opened a new issue, #5963:
URL: https://github.com/apache/iceberg/issues/5963

   ### Query engine
   
   Spark 3.1.2 + iceberg 0.14.0
   
   ### Question
   
   ```
   Caused by: org.apache.iceberg.exceptions.RuntimeIOException: org.apache.http.ConnectionClosedException: Premature end of Content-Length delimited message body (expected: 43,440,653; received: 26,194,942)
           at org.apache.iceberg.parquet.ParquetReader$FileIterator.advance(ParquetReader.java:135)
           at org.apache.iceberg.parquet.ParquetReader$FileIterator.next(ParquetReader.java:112)
           at org.apache.iceberg.io.FilterIterator.advance(FilterIterator.java:66)
           at org.apache.iceberg.io.FilterIterator.hasNext(FilterIterator.java:50)
           at org.apache.iceberg.spark.source.BaseDataReader.next(BaseDataReader.java:97)
           at org.apache.spark.sql.execution.datasources.v2.PartitionIterator.hasNext(DataSourceRDD.scala:79)
           at org.apache.spark.sql.execution.datasources.v2.MetricsIterator.hasNext(DataSourceRDD.scala:112)
           at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
           at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
           at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
           at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
           at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:755)
           at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
           at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
           at org.apache.spark.shuffle.rss.SortBasedShuffleWriter.fastWrite0(SortBasedShuffleWriter.java:178)
           at org.apache.spark.shuffle.rss.SortBasedShuffleWriter.write(SortBasedShuffleWriter.java:154)
           at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
           at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
           at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
           at org.apache.spark.scheduler.Task.run(Task.scala:131)
           at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:497)
           at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1439)
           at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:500)
           at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
           at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
           at java.lang.Thread.run(Thread.java:750)
   Caused by: org.apache.http.ConnectionClosedException: Premature end of Content-Length delimited message body (expected: 43,440,653; received: 26,194,942)
           at org.apache.http.impl.io.ContentLengthInputStream.read(ContentLengthInputStream.java:178)
           at org.apache.http.impl.io.ContentLengthInputStream.read(ContentLengthInputStream.java:198)
           at org.apache.http.impl.io.ContentLengthInputStream.close(ContentLengthInputStream.java:101)
           at org.apache.http.impl.execchain.ResponseEntityProxy.streamClosed(ResponseEntityProxy.java:143)
           at org.apache.http.conn.EofSensorInputStream.checkClose(EofSensorInputStream.java:228)
           at org.apache.http.conn.EofSensorInputStream.close(EofSensorInputStream.java:172)
           at java.io.FilterInputStream.close(FilterInputStream.java:181)
           at java.io.FilterInputStream.close(FilterInputStream.java:181)
           at com.aliyun.oss.event.ProgressInputStream.close(ProgressInputStream.java:149)
           at java.io.FilterInputStream.close(FilterInputStream.java:181)
           at org.apache.iceberg.aliyun.oss.OSSInputStream.closeStream(OSSInputStream.java:164)
           at org.apache.iceberg.aliyun.oss.OSSInputStream.openStream(OSSInputStream.java:156)
           at org.apache.iceberg.aliyun.oss.OSSInputStream.positionStream(OSSInputStream.java:152)
           at org.apache.iceberg.aliyun.oss.OSSInputStream.read(OSSInputStream.java:105)
           at org.apache.iceberg.shaded.org.apache.parquet.io.DelegatingSeekableInputStream.readFully(DelegatingSeekableInputStream.java:102)
           at org.apache.iceberg.shaded.org.apache.parquet.io.DelegatingSeekableInputStream.readFullyHeapBuffer(DelegatingSeekableInputStream.java:127)
           at org.apache.iceberg.shaded.org.apache.parquet.io.DelegatingSeekableInputStream.readFully(DelegatingSeekableInputStream.java:91)
           at org.apache.iceberg.shaded.org.apache.parquet.hadoop.ParquetFileReader$ConsecutivePartList.readAll(ParquetFileReader.java:1781)
           at org.apache.iceberg.shaded.org.apache.parquet.hadoop.ParquetFileReader.internalReadRowGroup(ParquetFileReader.java:953)
           at org.apache.iceberg.shaded.org.apache.parquet.hadoop.ParquetFileReader.readNextRowGroup(ParquetFileReader.java:909)
           at org.apache.iceberg.parquet.ParquetReader$FileIterator.advance(ParquetReader.java:133)
   ```
   
   sparkConf : 
   ```
    spark.sql.catalog.spark_catalog                 org.apache.iceberg.spark.SparkSessionCatalog
    spark.sql.catalog.spark_catalog.type            hive
    spark.sql.catalog.spark_catalog.uri              *** 
    spark.sql.catalog.spark_catalog.io-impl           org.apache.iceberg.aliyun.oss.OSSFileIO
    spark.sql.catalog.spark_catalog.oss.endpoint      ***
    spark.sql.catalog.spark_catalog.client.access-key-id  ***
    spark.sql.catalog.spark_catalog.client.access-key-secret  ***
   ```


-- 
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: issues-unsubscribe@iceberg.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] github-actions[bot] closed issue #5963: Aliyun-OssFileIO: Premature end of Content-Length delimited message body

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] closed issue #5963: Aliyun-OssFileIO:  Premature end of Content-Length delimited message body
URL: https://github.com/apache/iceberg/issues/5963


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] github-actions[bot] commented on issue #5963: Aliyun-OssFileIO: Premature end of Content-Length delimited message body

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on issue #5963:
URL: https://github.com/apache/iceberg/issues/5963#issuecomment-1603464509

   This issue has been automatically marked as stale because it has been open for 180 days with no activity. It will be closed in next 14 days if no further activity occurs. To permanently prevent this issue from being considered stale, add the label 'not-stale', but commenting on the issue is preferred when possible.


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] luoyuxia commented on issue #5963: Aliyun-OssFileIO: Premature end of Content-Length delimited message body

Posted by GitBox <gi...@apache.org>.
luoyuxia commented on issue #5963:
URL: https://github.com/apache/iceberg/issues/5963#issuecomment-1306482700

   We also meet the same problem, the 1 minute may be a reason, but I'm pretty sure it not for my case after a long debuging.
   after apply a patch, the exception won't be reproduced, but I can't figure out the reason. Others who meet the same problem may apply the patch and see whether it solve the problem.
   ```
   Index: aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSInputStream.java
   IDEA additional info:
   Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
   <+>UTF-8
   ===================================================================
   diff --git a/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSInputStream.java b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSInputStream.java
   --- a/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSInputStream.java	(revision 7d34482e1e5c763bea403d1cffcf7c69ddb06dc4)
   +++ b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSInputStream.java	(revision b02e73c1e2ba0705d8d7d308ddac157861daa2ba)
   @@ -21,6 +21,7 @@
    
    import com.aliyun.oss.OSS;
    import com.aliyun.oss.model.GetObjectRequest;
   +import com.aliyun.oss.model.OSSObject;
    import java.io.IOException;
    import java.io.InputStream;
    import java.util.Arrays;
   @@ -39,6 +40,7 @@
      private final OSS client;
      private final OSSURI uri;
    
   +  private OSSObject ossObject = null;
      private InputStream stream = null;
      private long pos = 0;
      private long next = 0;
   @@ -128,7 +130,12 @@
    
        GetObjectRequest request = new GetObjectRequest(uri.bucket(), uri.key())
            .withRange(pos, -1);
   -    stream = client.getObject(request).getObjectContent();
   +
   +    ossObject = client.getObject(request);
   +    LOG.debug("OSS GetObject with uri={}, startPos={}, endPos={}, requestId={}", uri, pos, -1,
   +        ossObject.getRequestId());
   +
   +    stream = ossObject.getObjectContent();
      }
    
      private void closeStream() throws IOException {
   Index: aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSOutputStream.java
   IDEA additional info:
   Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP
   <+>UTF-8
   ===================================================================
   diff --git a/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSOutputStream.java b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSOutputStream.java
   --- a/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSOutputStream.java	(revision 7d34482e1e5c763bea403d1cffcf7c69ddb06dc4)
   +++ b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSOutputStream.java	(revision b02e73c1e2ba0705d8d7d308ddac157861daa2ba)
   @@ -27,6 +27,7 @@
    import com.aliyun.oss.model.ObjectMetadata;
    import com.aliyun.oss.model.PartETag;
    import com.aliyun.oss.model.PutObjectRequest;
   +import com.aliyun.oss.model.PutObjectResult;
    import com.aliyun.oss.model.UploadPartRequest;
    import com.aliyun.oss.model.UploadPartResult;
    import java.io.BufferedInputStream;
   @@ -284,7 +285,10 @@
          metadata.setContentLength(contentLength);
    
          PutObjectRequest request = new PutObjectRequest(uri.bucket(), uri.key(), contentStream, metadata);
   -      client.putObject(request);
   +      PutObjectResult result = client.putObject(request);
   +
   +      LOG.debug("OSS PutObjectResult with uri={}, contentLength={}, requestId={}", uri,
   +          contentLength, result.getRequestId());
        } else {
          uploadParts();
          completeMultiPartUpload();
   ```
   


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] github-actions[bot] commented on issue #5963: Aliyun-OssFileIO: Premature end of Content-Length delimited message body

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on issue #5963:
URL: https://github.com/apache/iceberg/issues/5963#issuecomment-1626379388

   This issue has been closed because it has not received any activity in the last 14 days since being marked as 'stale'


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ethan7811 commented on issue #5963: Aliyun-OssFileIO: Premature end of Content-Length delimited message body

Posted by GitBox <gi...@apache.org>.
ethan7811 commented on issue #5963:
URL: https://github.com/apache/iceberg/issues/5963#issuecomment-1308429585

   maybe the root cause is similar to this ...
   https://github.com/apache/hadoop/pull/2692/files


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org