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/12/03 18:12:27 UTC

[GitHub] [iceberg] islamismailov opened a new pull request, #6353: Make sure S3 stream opened by ReadConf ctor is closed

islamismailov opened a new pull request, #6353:
URL: https://github.com/apache/iceberg/pull/6353

   What the title says. This is what currently is seen: 
   
   ```2022-12-03 10:25:11.154  WARN 5408 --- [      Finalizer] o.a.i.a.s.S3InputStream                  : Unclosed input stream created by:
           org.apache.iceberg.aws.s3.S3InputStream.<init>(S3InputStream.java:60)
           org.apache.iceberg.aws.s3.S3InputFile.newStream(S3InputFile.java:48)
           org.apache.iceberg.parquet.ParquetIO$ParquetInputFile.newStream(ParquetIO.java:183)
           org.apache.iceberg.bdp.shaded.org.apache.parquet.hadoop.ParquetFileReader.<init>(ParquetFileReader.java:802)
           org.apache.iceberg.bdp.shaded.org.apache.parquet.hadoop.ParquetFileReader.open(ParquetFileReader.java:667)
           org.apache.iceberg.parquet.ReadConf.newReader(ReadConf.java:218)
           org.apache.iceberg.parquet.ReadConf.<init>(ReadConf.java:74)
           org.apache.iceberg.parquet.ParquetReader.init(ParquetReader.java:66)
           org.apache.iceberg.parquet.ParquetReader.iterator(ParquetReader.java:77)
           org.apache.iceberg.data.TableScanIterable$ScanIterator.hasNext(TableScanIterable.java:200)```


-- 
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] islamismailov commented on pull request #6353: Make sure S3 stream opened by ReadConf ctor is closed

Posted by GitBox <gi...@apache.org>.
islamismailov commented on PR #6353:
URL: https://github.com/apache/iceberg/pull/6353#issuecomment-1340077268

   @nastra  - done. can you have another look?


-- 
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] islamismailov commented on a diff in pull request #6353: Make sure S3 stream opened by ReadConf ctor is closed

Posted by GitBox <gi...@apache.org>.
islamismailov commented on code in PR #6353:
URL: https://github.com/apache/iceberg/pull/6353#discussion_r1054828310


##########
parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java:
##########
@@ -79,9 +83,11 @@ private ReadConf<T> init() {
               nameMapping,
               reuseContainers,
               caseSensitive,
-              null);
-      this.conf = readConf.copy();
-      return readConf;
+              null)) {
+        this.conf = readConf.copy();

Review Comment:
   The temporary reader that doesn't get closed is basically used to read file schema.



-- 
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] islamismailov commented on a diff in pull request #6353: Make sure S3 stream opened by ReadConf ctor is closed

Posted by GitBox <gi...@apache.org>.
islamismailov commented on code in PR #6353:
URL: https://github.com/apache/iceberg/pull/6353#discussion_r1042460459


##########
parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java:
##########
@@ -79,9 +83,11 @@ private ReadConf<T> init() {
               nameMapping,
               reuseContainers,
               caseSensitive,
-              null);
-      this.conf = readConf.copy();
-      return readConf;
+              null)) {
+        this.conf = readConf.copy();
+      } catch (IOException e) {
+        LOG.warn("Failed to close ReadConf", e);

Review Comment:
   I think it's kind of an anti-pattern. `close` normally throws IOException and it's up to the "user" of the class on how to handle that



-- 
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] islamismailov commented on pull request #6353: Make sure S3 stream opened by ReadConf ctor is closed

Posted by GitBox <gi...@apache.org>.
islamismailov commented on PR #6353:
URL: https://github.com/apache/iceberg/pull/6353#issuecomment-1377912350

   friendly ping - can i get an update on this?


-- 
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] islamismailov closed pull request #6353: Make sure S3 stream opened by ReadConf ctor is closed

Posted by "islamismailov (via GitHub)" <gi...@apache.org>.
islamismailov closed pull request #6353: Make sure S3 stream opened by ReadConf ctor is closed
URL: https://github.com/apache/iceberg/pull/6353


-- 
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] nastra commented on a diff in pull request #6353: Make sure S3 stream opened by ReadConf ctor is closed

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6353:
URL: https://github.com/apache/iceberg/pull/6353#discussion_r1041833505


##########
parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java:
##########
@@ -79,9 +83,11 @@ private ReadConf<T> init() {
               nameMapping,
               reuseContainers,
               caseSensitive,
-              null);
-      this.conf = readConf.copy();
-      return readConf;
+              null)) {
+        this.conf = readConf.copy();
+      } catch (IOException e) {
+        LOG.warn("Failed to close ReadConf", e);

Review Comment:
   I think it would make more sense to catch+log this inside `ReadConf.close()` so that `ReadConf.close()` doesn't throw anything



-- 
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] nastra commented on a diff in pull request #6353: Make sure S3 stream opened by ReadConf ctor is closed

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6353:
URL: https://github.com/apache/iceberg/pull/6353#discussion_r1042500106


##########
parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java:
##########
@@ -68,7 +72,7 @@ public ParquetReader(
 
   private ReadConf<T> init() {
     if (conf == null) {
-      ReadConf<T> readConf =
+      try (ReadConf<T> readConf =

Review Comment:
   ```suggestion
         try (ReadConf<T> readConf =
             new ReadConf<>(
                 input,
                 options,
                 expectedSchema,
                 filter,
                 readerFunc,
                 null,
                 nameMapping,
                 reuseContainers,
                 caseSensitive,
                 null)) {
           this.conf = readConf.copy();
         }
   ```



##########
parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java:
##########
@@ -79,9 +83,11 @@ private ReadConf<T> init() {
               nameMapping,
               reuseContainers,
               caseSensitive,
-              null);
-      this.conf = readConf.copy();
-      return readConf;
+              null)) {
+        this.conf = readConf.copy();
+      } catch (IOException e) {
+        LOG.warn("Failed to close ReadConf", e);

Review Comment:
   The contract of `AutoCloseable` is slightly different. It says
   
   > While this interface method is declared to throw Exception, implementers are strongly encouraged to declare concrete implementations of the close method to throw more specific exceptions, or to throw no exception at all if the close operation cannot fail.
   
   So we don't have to throw any exception in `ReadConf.close()`. We can add the try-catch + logging to `ReadConf.close()` directly (because I'm not sure there's value to propagate that exception in this particular case). See the suggestions I've made. I think this is cleaner, because then you don't have to use a `try-with-resources` + `catch` to mention that the underlying `ParquetFileReader` failed to be closed.



##########
parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java:
##########
@@ -257,4 +257,11 @@ private List<Map<ColumnPath, ColumnChunkMetaData>> getColumnChunkMetadataForRowG
     }
     return listBuilder.build();
   }
+
+  @Override

Review Comment:
   ```suggestion
     @Override
     public void close() {
       if (reader != null) {
         try {
           reader.close();
         } catch (IOException e) {
           LOG.warn("Failed to close ReadConf", e);
         }
       }
     }
   ```



-- 
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] islamismailov commented on pull request #6353: Make sure S3 stream opened by ReadConf ctor is closed

Posted by GitBox <gi...@apache.org>.
islamismailov commented on PR #6353:
URL: https://github.com/apache/iceberg/pull/6353#issuecomment-1362265920

   please see my example in the sub-thread above


-- 
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] islamismailov commented on pull request #6353: Make sure S3 stream opened by ReadConf ctor is closed

Posted by GitBox <gi...@apache.org>.
islamismailov commented on PR #6353:
URL: https://github.com/apache/iceberg/pull/6353#issuecomment-1353743389

   i will try to update this PR with the feedback provided


-- 
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] rdblue commented on a diff in pull request #6353: Make sure S3 stream opened by ReadConf ctor is closed

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6353:
URL: https://github.com/apache/iceberg/pull/6353#discussion_r1067212201


##########
parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java:
##########
@@ -79,9 +83,11 @@ private ReadConf<T> init() {
               nameMapping,
               reuseContainers,
               caseSensitive,
-              null);
-      this.conf = readConf.copy();
-      return readConf;
+              null)) {
+        this.conf = readConf.copy();

Review Comment:
   @islamismailov, the first call to init, the `ReadConf` is copied and stored in the `ParquetReader` for future use. But the `ReadConf` that was just created is the one that is returned. That way the first init call uses the same stream for the `FileIterator` and doesn't reopen one.



-- 
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] islamismailov commented on a diff in pull request #6353: Make sure S3 stream opened by ReadConf ctor is closed

Posted by GitBox <gi...@apache.org>.
islamismailov commented on code in PR #6353:
URL: https://github.com/apache/iceberg/pull/6353#discussion_r1054796838


##########
parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java:
##########
@@ -79,9 +83,11 @@ private ReadConf<T> init() {
               nameMapping,
               reuseContainers,
               caseSensitive,
-              null);
-      this.conf = readConf.copy();
-      return readConf;
+              null)) {
+        this.conf = readConf.copy();

Review Comment:
   You call `this.conf = readConf.copy();`. `copy()` calls a private constructor that does not copy the reader itself: 
   
   ```
   private ReadConf(ReadConf<T> toCopy) {
       this.reader = null;
   ```
   
   so the reader is created to just fill all the other fields and then immediately discarded without getting closed.
   
   then, inside the iterator you call `conf.reader()` which now creates a reader because it's null:
   
   ```
     ParquetFileReader reader() {
       if (reader != null) {
         reader.setRequestedSchema(projection);
         return reader;
       }
   ```
   
   i suppose the other way to "fix" this would be to copy the reader as part of the `copy` but the fact that was done looked very deliberate, perhaps you have a context why.



-- 
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] rdblue commented on a diff in pull request #6353: Make sure S3 stream opened by ReadConf ctor is closed

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6353:
URL: https://github.com/apache/iceberg/pull/6353#discussion_r1051688387


##########
parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java:
##########
@@ -79,9 +83,11 @@ private ReadConf<T> init() {
               nameMapping,
               reuseContainers,
               caseSensitive,
-              null);
-      this.conf = readConf.copy();
-      return readConf;
+              null)) {
+        this.conf = readConf.copy();

Review Comment:
   The reason why `ReadConf` is not closed here was because we wanted to reuse the reader rather than opening a new one. This change will cause normal file reads to add an additional GET request round trip to open the file again.
   
   What is the case where this wasn't being closed? I think that the originally logic should be safe because the `init` method is only called if a `FileIterator` is created, and that calls `ReadConf.reader()`. So either a new reader is opened, or the existing reader is reused and will be closed by the iterator.



-- 
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] nastra commented on a diff in pull request #6353: Make sure S3 stream opened by ReadConf ctor is closed

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #6353:
URL: https://github.com/apache/iceberg/pull/6353#discussion_r1039765677


##########
parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java:
##########
@@ -46,7 +47,7 @@
  *
  * @param <T> type of value to read
  */
-class ReadConf<T> {
+class ReadConf<T> implements Closeable {

Review Comment:
   what about making this implement `AutoCloseable`?



-- 
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