nastra commented on code in PR #7513:
URL: https://github.com/apache/iceberg/pull/7513#discussion_r1183546289
##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java:
##########
@@ -393,4 +399,19 @@ public void close() {
}
}
}
+
+ @SuppressWarnings("checkstyle:NoFinalizer")
+ @Override
+ protected void finalize() throws Throwable {
+ super.finalize();
+ if (!isResourceClosed.get()) {
+ if (null == createStack) {
+ close();
+ } else {
+ String trace =
+ Joiner.on("\n\t").join(Arrays.copyOfRange(createStack, 1,
createStack.length));
+ LOG.warn("Unclosed S3FileIO instance created by:\n\t{}", trace);
Review Comment:
Not sure if we want to log this as a `WARN` here, because this is then being
propagated to the Spark shell. During some local testing I saw the below stack
trace when running `select * from ns.tbl.all_manifests`
```
org.apache.iceberg.aws.s3.S3FileIO.<init>(S3FileIO.java:100)
jdk.internal.reflect.GeneratedConstructorAccessor103.newInstance(Unknown Source)
java.base/jdk.internal.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
java.base/java.lang.reflect.Constructor.newInstance(Constructor.java:490)
org.apache.iceberg.common.DynConstructors$Ctor.newInstanceChecked(DynConstructors.java:55)
org.apache.iceberg.common.DynConstructors$Ctor.newInstance(DynConstructors.java:68)
org.apache.iceberg.CatalogUtil.loadFileIO(CatalogUtil.java:317)
org.apache.iceberg.io.ResolvingFileIO.io(ResolvingFileIO.java:137)
org.apache.iceberg.io.ResolvingFileIO.newInputFile(ResolvingFileIO.java:61)
org.apache.iceberg.AllManifestsTable$ManifestListReadTask.file(AllManifestsTable.java:212)
org.apache.iceberg.AllManifestsTable$ManifestListReadTask.file(AllManifestsTable.java:151)
org.apache.iceberg.spark.source.RowDataReader.open(RowDataReader.java:84)
org.apache.iceberg.spark.source.RowDataReader.open(RowDataReader.java:43)
org.apache.iceberg.spark.source.BaseReader.next(BaseReader.java:141)
org.apache.spark.sql.execution.datasources.v2.PartitionIterator.hasNext(DataSourceRDD.scala:119)
org.apache.spark.sql.execution.datasources.v2.MetricsIterator.hasNext(DataSourceRDD.scala:156)
org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.$anonfun$hasNext$1(DataSourceRDD.scala:63)
org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.$anonfun$hasNext$1$adapted(DataSourceRDD.scala:63)
scala.Option.exists(Option.scala:376)
org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.hasNext(DataSourceRDD.scala:63)
org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.advanceToNextIter(DataSourceRDD.scala:97)
org.apache.spark.sql.execution.datasources.v2.DataSourceRDD$$anon$1.hasNext(DataSourceRDD.scala:63)
org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
Source)
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:760)
org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:364)
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:890)
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:890)
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
org.apache.spark.scheduler.Task.run(Task.scala:136)
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548)
org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504)
org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551)
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
java.base/java.lang.Thread.run(Thread.java:829)
```
--
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]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]