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 2021/07/28 02:16:24 UTC

[GitHub] [iceberg] yyanyy opened a new pull request #2878: API: extend CloseableGroup and close resources when closing aws catalogs

yyanyy opened a new pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878


   - Noticed that we have not closed all resources that we could when we close Glue/DynamoDB based catalog
   - In order to close all resources, `S3FileIO` may need to be updated too since it holds a reference to S3 client. 
      - I believe only `S3FileIO` would actually need it for now but made `FileIO` to be closeable since I think other cloud services may also need it once they are supported
   - Extend `CloseableGroup` to be configurable and support close failure suppression, leave default suppression behavior to false for backward compatibility, and added tests
   - Didn't update other catalog other than aws ones since they usually only need to close one resource, that overriding `close` should be sufficient


-- 
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] kbendick commented on a change in pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r704940786



##########
File path: aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java
##########
@@ -105,4 +107,14 @@ public void initialize(Map<String, String> properties) {
     this.awsClientFactory = AwsClientFactories.from(properties);
     this.s3 = awsClientFactory::s3;
   }
+
+  @Override
+  public void close() {
+    // handles concurrent calls to close()
+    if (isResourceClosed.compareAndSet(false, true)) {
+      if (client != null) {
+        client.close();
+      }

Review comment:
       If this is checking for null (plus the compareAndSet), should it set it to null as well or is the AtomicBoolean sufficient? I imagine it's sufficient but thought to ask.




-- 
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] kbendick commented on a change in pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r685817756



##########
File path: aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java
##########
@@ -105,4 +105,9 @@ public void initialize(Map<String, String> properties) {
     this.awsClientFactory = AwsClientFactories.from(properties);
     this.s3 = awsClientFactory::s3;
   }
+
+  @Override
+  public void close() {
+    client().close();

Review comment:
       Since we'e not 100% sure if `close` will be called more than once, should we set `client` to `null` or add an `AtomicBoolean closed` that will then handle the idempotency issue?




-- 
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] yyanyy commented on pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
yyanyy commented on pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#issuecomment-900663109


   Sorry for the late response and thank you for all the feedback @rymurr @kbendick @rdblue ! 
   
   The use case we have of closing a catalog/FileIO is to handle multi-tenant scenario with Java API interactions to the library, as when a catalog is initiated, all the clients and configurations are for this specific user, and when a new user arrives with a new set of credential and configuration, we need to initiate a new catalog object. Thus we would like to close the catalog and underlying clients to avoid resource leaks if possible. But I agree that when the credentials are static throughout the lifecycle of the application (which is majority of the cases especially when the interaction occurs through engines like Spark), explicitly calling `close()` in either places is not required. 
   
   The reason for changing catalogs to `AutoCloseable` is because unfortunately the AWS SDK clients extends AutoCloseable which throws `Exception` in method signature, and we need to wrap it with `IOException` if the catalog extends `Closeable` even if we handle idempotency ourselves, when we use a util class to close multiple resources. But since Iceberg uses `Closeable` consistently, I'm happy to do the wrapping. 
   
   I should be able to update the PR today. Thanks again for reviewing! 


-- 
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 change in pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r694093815



##########
File path: aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java
##########
@@ -80,6 +81,7 @@
   private AwsProperties awsProperties;
   private FileIO fileIO;
   private LockManager lockManager;
+  private CloseGroupHelper closeGroupHelper;

Review comment:
       Could this implement `CloseableGroup` instead of using a helper? That does basically the same thing only you can add closeables at any time.




-- 
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 change in pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r694287623



##########
File path: api/src/main/java/org/apache/iceberg/io/CloseGroupHelper.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.iceberg.io;
+
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.stream.Collectors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class acts as a helper for handling the closure of multiple resource.
+ * To use it. register resources to be closed in constructor, and call the corresponding close method when needed.
+ * <p>
+ * It can take both closeable and autocloseable objects, and handle closeable as autocloseable and guarantee close
+ * idempotency by ensuring that each resource will be closed once even with concurrent close calls. It will also
+ * wrap checked exceptions into IOException to follow Closeable method signature.
+ */
+public class CloseGroupHelper {
+  private static final Logger LOG = LoggerFactory.getLogger(CloseGroupHelper.class);
+
+  private final Deque<AutoCloseable> closeables;
+
+  /**
+   * Register closeables and/or autocloseables to be managed by this class.
+   */
+  public CloseGroupHelper(AutoCloseable... autoCloseables) {
+    this.closeables = Arrays.stream(autoCloseables)
+        .filter(Objects::nonNull)
+        .collect(Collectors.toCollection(ConcurrentLinkedDeque::new));
+  }
+
+  /**
+   * Close all the registered resources. Close method of each resource will only be called once.
+   * Checked exception from AutoCloseable will be wrapped to IOException. Users can also configure to
+   * suppress the failure during closing to ensure the close method of all resources to be called.
+   *
+   * @param suppressCloseFailure true if user wants to suppress close failures
+   * @throws IOException exception thrown if close encounters errors and suppress is false
+   */
+  public void closeAsCloseable(boolean suppressCloseFailure) throws IOException {
+    while (!closeables.isEmpty()) {
+      AutoCloseable toClose = closeables.pollFirst();
+      if (toClose != null) {
+        try {
+          toClose.close();
+        } catch (Exception e) {
+          if (suppressCloseFailure) {
+            LOG.error("Exception suppressed when attempting to close resources", e);
+          } else if (e instanceof IOException) {
+            throw (IOException) e;
+          } else if (e instanceof RuntimeException) {
+            throw (RuntimeException) e;
+          } else {
+            throw new IOException("Exception occurs when closing AutoCloseable", e);

Review comment:
       I don't think it is correct to throw `IOException`. If you're wrapping an exception, then I'd suggest using `RuntimeException`.




-- 
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] rymurr commented on a change in pull request #2878: API: extend CloseableGroup and close resources when closing aws catalogs

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r678973697



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
##########
@@ -25,13 +25,14 @@
 import org.apache.iceberg.exceptions.AlreadyExistsException;
 import org.apache.iceberg.exceptions.CommitFailedException;
 import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.CloseableGroup;
 import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public abstract class BaseMetastoreCatalog implements Catalog {
+public abstract class BaseMetastoreCatalog extends CloseableGroup implements Catalog {

Review comment:
       I don't think this should be implemented by inheritance. `BaseCatalog` provides utilities to help orchestrate a `Catalog` and does not deal with lifetimes. I don't think its the responsibility of the abstract class to deal w/ lifetimes. This is a concern of the individual catalogs imho. It shoud be relatively clean to instaitate a closeablegroup in the constructor of a catalog and close it in the catalogs close method. 
   
   We then are reducing the side effects for other catalog implementers and not adding unneccessary contracts for catalog implementers.




-- 
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] yyanyy commented on a change in pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r701510115



##########
File path: aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java
##########
@@ -80,6 +81,7 @@
   private AwsProperties awsProperties;
   private FileIO fileIO;
   private LockManager lockManager;
+  private CloseGroupHelper closeGroupHelper;

Review comment:
       Thank you! I have merged the new class into the existing `CloseableGroup` and make it no longer abstract. Please let me know if this was not your intended suggestion! 




-- 
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 change in pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r705761976



##########
File path: aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java
##########
@@ -105,4 +107,14 @@ public void initialize(Map<String, String> properties) {
     this.awsClientFactory = AwsClientFactories.from(properties);
     this.s3 = awsClientFactory::s3;
   }
+
+  @Override
+  public void close() {
+    // handles concurrent calls to close()
+    if (isResourceClosed.compareAndSet(false, true)) {
+      if (client != null) {
+        client.close();
+      }

Review comment:
       I think this is a good idea, but minor.




-- 
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 change in pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r694288158



##########
File path: api/src/main/java/org/apache/iceberg/io/CloseGroupHelper.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.iceberg.io;
+
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.stream.Collectors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class acts as a helper for handling the closure of multiple resource.
+ * To use it. register resources to be closed in constructor, and call the corresponding close method when needed.
+ * <p>
+ * It can take both closeable and autocloseable objects, and handle closeable as autocloseable and guarantee close
+ * idempotency by ensuring that each resource will be closed once even with concurrent close calls. It will also
+ * wrap checked exceptions into IOException to follow Closeable method signature.
+ */
+public class CloseGroupHelper {
+  private static final Logger LOG = LoggerFactory.getLogger(CloseGroupHelper.class);
+
+  private final Deque<AutoCloseable> closeables;
+
+  /**
+   * Register closeables and/or autocloseables to be managed by this class.
+   */
+  public CloseGroupHelper(AutoCloseable... autoCloseables) {
+    this.closeables = Arrays.stream(autoCloseables)
+        .filter(Objects::nonNull)
+        .collect(Collectors.toCollection(ConcurrentLinkedDeque::new));
+  }
+
+  /**
+   * Close all the registered resources. Close method of each resource will only be called once.
+   * Checked exception from AutoCloseable will be wrapped to IOException. Users can also configure to
+   * suppress the failure during closing to ensure the close method of all resources to be called.
+   *
+   * @param suppressCloseFailure true if user wants to suppress close failures
+   * @throws IOException exception thrown if close encounters errors and suppress is false
+   */
+  public void closeAsCloseable(boolean suppressCloseFailure) throws IOException {
+    while (!closeables.isEmpty()) {
+      AutoCloseable toClose = closeables.pollFirst();
+      if (toClose != null) {
+        try {
+          toClose.close();
+        } catch (Exception e) {
+          if (suppressCloseFailure) {
+            LOG.error("Exception suppressed when attempting to close resources", e);
+          } else if (e instanceof IOException) {
+            throw (IOException) e;
+          } else if (e instanceof RuntimeException) {
+            throw (RuntimeException) e;
+          } else {
+            throw new IOException("Exception occurs when closing AutoCloseable", e);

Review comment:
       This could just use `ExceptionUtil.castAndThrow(e, IOException.class);` after checking the `suppressCloseFailure` flag.




-- 
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] yyanyy commented on pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
yyanyy commented on pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#issuecomment-900663109


   Sorry for the late response and thank you for all the feedback @rymurr @kbendick @rdblue ! 
   
   The use case we have of closing a catalog/FileIO is to handle multi-tenant scenario with Java API interactions to the library, as when a catalog is initiated, all the clients and configurations are for this specific user, and when a new user arrives with a new set of credential and configuration, we need to initiate a new catalog object. Thus we would like to close the catalog and underlying clients to avoid resource leaks if possible. But I agree that when the credentials are static throughout the lifecycle of the application (which is majority of the cases especially when the interaction occurs through engines like Spark), explicitly calling `close()` in either places is not required. 
   
   The reason for changing catalogs to `AutoCloseable` is because unfortunately the AWS SDK clients extends AutoCloseable which throws `Exception` in method signature, and we need to wrap it with `IOException` if the catalog extends `Closeable` even if we handle idempotency ourselves, when we use a util class to close multiple resources. But since Iceberg uses `Closeable` consistently, I'm happy to do the wrapping. 
   
   I should be able to update the PR today. Thanks again for reviewing! 


-- 
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] yyanyy commented on a change in pull request #2878: API: extend CloseableGroup and close resources when closing aws catalogs

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r678535373



##########
File path: api/src/main/java/org/apache/iceberg/io/CloseableGroup.java
##########
@@ -23,20 +23,47 @@
 import java.io.IOException;
 import java.util.Deque;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public abstract class CloseableGroup implements Closeable {
-  private final Deque<Closeable> closeables = Lists.newLinkedList();
+  private static final Logger LOG = LoggerFactory.getLogger(CloseableGroup.class);
 
-  protected void addCloseable(Closeable closeable) {
+  private final Deque<AutoCloseable> closeables = Lists.newLinkedList();
+  private boolean suppressCloseFailure = false;
+
+  protected void addCloseable(AutoCloseable closeable) {
     closeables.add(closeable);
   }
 
+  protected void setSuppressCloseFailure(boolean shouldSuppress) {
+    this.suppressCloseFailure = shouldSuppress;
+  }
+
   @Override
   public void close() throws IOException {
     while (!closeables.isEmpty()) {
-      Closeable toClose = closeables.removeFirst();
+      AutoCloseable toClose = closeables.removeFirst();
       if (toClose != null) {
-        toClose.close();
+        if (suppressCloseFailure) {

Review comment:
       Thanks for the quick review! 
   
   This is more of a helper utility function to me and on opt-in basis. In the case we were looking at (aws catalogs), if one of the resource fails to be closed, we don't really plan to handle it further, and we want to make sure an earlier close failure shouldn't block other resources from getting closed. So we either do something like 
   ```
   try {
     resource1.close();
   } catch (Exception e) {
     log.error("Error closing resource1, suppressing");
   }
   
   try {
     resource2.close();
   } catch (Exception e) {
     log.error("Error closing resource2, suppressing");
   }
   ```
   or create a utility class to make the code simpler; I noticed this class when I was trying to create the utility class, and decided to reuse this class directly (the current solution). 




-- 
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] rymurr commented on a change in pull request #2878: API: extend CloseableGroup and close resources when closing aws catalogs

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r678092705



##########
File path: api/src/main/java/org/apache/iceberg/io/CloseableGroup.java
##########
@@ -23,20 +23,47 @@
 import java.io.IOException;
 import java.util.Deque;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public abstract class CloseableGroup implements Closeable {
-  private final Deque<Closeable> closeables = Lists.newLinkedList();
+  private static final Logger LOG = LoggerFactory.getLogger(CloseableGroup.class);
 
-  protected void addCloseable(Closeable closeable) {
+  private final Deque<AutoCloseable> closeables = Lists.newLinkedList();
+  private boolean suppressCloseFailure = false;
+
+  protected void addCloseable(AutoCloseable closeable) {
     closeables.add(closeable);
   }
 
+  protected void setSuppressCloseFailure(boolean shouldSuppress) {
+    this.suppressCloseFailure = shouldSuppress;
+  }
+
   @Override
   public void close() throws IOException {

Review comment:
       Is the close method idempotent? What happens if it is called twice?

##########
File path: api/src/main/java/org/apache/iceberg/io/CloseableGroup.java
##########
@@ -23,20 +23,47 @@
 import java.io.IOException;
 import java.util.Deque;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public abstract class CloseableGroup implements Closeable {
-  private final Deque<Closeable> closeables = Lists.newLinkedList();
+  private static final Logger LOG = LoggerFactory.getLogger(CloseableGroup.class);
 
-  protected void addCloseable(Closeable closeable) {
+  private final Deque<AutoCloseable> closeables = Lists.newLinkedList();
+  private boolean suppressCloseFailure = false;
+
+  protected void addCloseable(AutoCloseable closeable) {
     closeables.add(closeable);
   }
 
+  protected void setSuppressCloseFailure(boolean shouldSuppress) {
+    this.suppressCloseFailure = shouldSuppress;
+  }
+
   @Override
   public void close() throws IOException {
     while (!closeables.isEmpty()) {
-      Closeable toClose = closeables.removeFirst();
+      AutoCloseable toClose = closeables.removeFirst();
       if (toClose != null) {
-        toClose.close();
+        if (suppressCloseFailure) {

Review comment:
       Personally I think the caller should be handling errors. Why suppress this at the source?

##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
##########
@@ -25,13 +25,14 @@
 import org.apache.iceberg.exceptions.AlreadyExistsException;
 import org.apache.iceberg.exceptions.CommitFailedException;
 import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.CloseableGroup;
 import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public abstract class BaseMetastoreCatalog implements Catalog {
+public abstract class BaseMetastoreCatalog extends CloseableGroup implements Catalog {

Review comment:
       I think closeable should be handled by the concrete impl of a catalog and not by the base class, I don't think all catalogs must implement closeable by design. This should be done via composition rather than inheritance. Additionally, I don't know what the effect of this change is on other catalogs so this feels a bit on the risky side.




-- 
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] kbendick commented on a change in pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r704940786



##########
File path: aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java
##########
@@ -105,4 +107,14 @@ public void initialize(Map<String, String> properties) {
     this.awsClientFactory = AwsClientFactories.from(properties);
     this.s3 = awsClientFactory::s3;
   }
+
+  @Override
+  public void close() {
+    // handles concurrent calls to close()
+    if (isResourceClosed.compareAndSet(false, true)) {
+      if (client != null) {
+        client.close();
+      }

Review comment:
       If this is checking for null (plus the compareAndSet), should it set it to null as well or is the AtomicBoolean sufficient?




-- 
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] rymurr commented on a change in pull request #2878: API: extend CloseableGroup and close resources when closing aws catalogs

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r678968259



##########
File path: api/src/main/java/org/apache/iceberg/io/CloseableGroup.java
##########
@@ -23,20 +23,47 @@
 import java.io.IOException;
 import java.util.Deque;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public abstract class CloseableGroup implements Closeable {
-  private final Deque<Closeable> closeables = Lists.newLinkedList();
+  private static final Logger LOG = LoggerFactory.getLogger(CloseableGroup.class);
 
-  protected void addCloseable(Closeable closeable) {
+  private final Deque<AutoCloseable> closeables = Lists.newLinkedList();
+  private boolean suppressCloseFailure = false;
+
+  protected void addCloseable(AutoCloseable closeable) {
     closeables.add(closeable);
   }
 
+  protected void setSuppressCloseFailure(boolean shouldSuppress) {
+    this.suppressCloseFailure = shouldSuppress;
+  }
+
   @Override
   public void close() throws IOException {

Review comment:
       `CloseableGroup` is `Closeable` so it has to be idempotent. This is especially true if a `CloseableGroup` contains `AutoClosables`. Its not about existing functionality or expected use cases its about following the contract of `Closeable`




-- 
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 pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#issuecomment-896124921


   I agree that the catalog should be `Closeable` and not `AutoCloseable`. We use `Closeable` in Iceberg rather than `AutoCloseable` and I don't see much benefit in making the APIs more confusing by introducing `AutoCloseable` too.
   
   I'm also not clear on the lifecycle of catalogs. Can you give me more context on when you'd close a catalog?


-- 
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] yyanyy commented on a change in pull request #2878: API: extend CloseableGroup and close resources when closing aws catalogs

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r678588138



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
##########
@@ -25,13 +25,14 @@
 import org.apache.iceberg.exceptions.AlreadyExistsException;
 import org.apache.iceberg.exceptions.CommitFailedException;
 import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.CloseableGroup;
 import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public abstract class BaseMetastoreCatalog implements Catalog {
+public abstract class BaseMetastoreCatalog extends CloseableGroup implements Catalog {

Review comment:
       Currently for all six catalog implementations, only `HiveCatalog` does not implement Closeable/AutoCloseable, so I was thinking it may be fine to extend it directly from the base class, and as we are not letting the interface `Catalog` to implement closeable so I don't consider it as part of the interface contract/design. I agree with composition over inheritance, and the main reason for the current approach was mainly because I noticed this existing `CloseableGroup` that does a very similar job as the goal I tried to achieve here. In terms of the impact to other catalogs, all other non-hive and non-aws catalogs implement `close()` explicitly which would override the logic of `close()` in `CloseableGroup`, so I think there wouldn't be impact to the current behavior; also `CloseableGroup` requires registration of closeable resources in order to work, so even if a catalog does not implement `close()` I don't think it would result in behavior change either. 
   
   I think the only thing that may be impacted by this is the nessie catalog - currently nessie catalog extends `AutoCloseable` that in throws `Exception` in `close()` signature, but now as we extends `CloseableGroup` in base, I believe it will be extending `Closeable` (which extends `AutoCloseable`) which would throw `IOException` in `close()` (this is the only difference in terms of method signature between closeable and auto-closeable). However in the meanwhile, nessie catalog [doesn't throw any checked exception](https://github.com/apache/iceberg/blob/master/nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java#L129) in `close()` at all, so I believe the code behavior wouldn't change either. But please let me know if you think we should continue to let nessie catalog to implement `AutoCloseable`, and in this case I can create a separate utility class and only use them for aws related catalogs. 




-- 
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] yyanyy commented on a change in pull request #2878: API: extend CloseableGroup and close resources when closing aws catalogs

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r678535373



##########
File path: api/src/main/java/org/apache/iceberg/io/CloseableGroup.java
##########
@@ -23,20 +23,47 @@
 import java.io.IOException;
 import java.util.Deque;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public abstract class CloseableGroup implements Closeable {
-  private final Deque<Closeable> closeables = Lists.newLinkedList();
+  private static final Logger LOG = LoggerFactory.getLogger(CloseableGroup.class);
 
-  protected void addCloseable(Closeable closeable) {
+  private final Deque<AutoCloseable> closeables = Lists.newLinkedList();
+  private boolean suppressCloseFailure = false;
+
+  protected void addCloseable(AutoCloseable closeable) {
     closeables.add(closeable);
   }
 
+  protected void setSuppressCloseFailure(boolean shouldSuppress) {
+    this.suppressCloseFailure = shouldSuppress;
+  }
+
   @Override
   public void close() throws IOException {
     while (!closeables.isEmpty()) {
-      Closeable toClose = closeables.removeFirst();
+      AutoCloseable toClose = closeables.removeFirst();
       if (toClose != null) {
-        toClose.close();
+        if (suppressCloseFailure) {

Review comment:
       Thanks for the quick review! 
   
   This is more of a helper utility function to me and on opt-in basis. In the case we were looking at (aws catalogs), if one of the resource fails to be closed, we don't really plan to handle it further, and we want to make sure an earlier close failure shouldn't block other resources from getting closed. So we either do something like 
   ```
   try {
     resource1.close();
   } catch (Exception e) {
     log.error("Error closing resource1, suppressing");
   }
   
   try {
     resource2.close();
   } catch (Exception e) {
     log.error("Error closing resource2, suppressing");
   }
   ```
   or reuse this class to make the code simpler (the current solution). 




-- 
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] rymurr commented on a change in pull request #2878: API: extend CloseableGroup and close resources when closing aws catalogs

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r678966742



##########
File path: api/src/main/java/org/apache/iceberg/io/CloseableGroup.java
##########
@@ -23,20 +23,47 @@
 import java.io.IOException;
 import java.util.Deque;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public abstract class CloseableGroup implements Closeable {
-  private final Deque<Closeable> closeables = Lists.newLinkedList();
+  private static final Logger LOG = LoggerFactory.getLogger(CloseableGroup.class);
 
-  protected void addCloseable(Closeable closeable) {
+  private final Deque<AutoCloseable> closeables = Lists.newLinkedList();
+  private boolean suppressCloseFailure = false;
+
+  protected void addCloseable(AutoCloseable closeable) {
     closeables.add(closeable);
   }
 
+  protected void setSuppressCloseFailure(boolean shouldSuppress) {
+    this.suppressCloseFailure = shouldSuppress;
+  }
+
   @Override
   public void close() throws IOException {
     while (!closeables.isEmpty()) {
-      Closeable toClose = closeables.removeFirst();
+      AutoCloseable toClose = closeables.removeFirst();
       if (toClose != null) {
-        toClose.close();
+        if (suppressCloseFailure) {

Review comment:
       That makes sense. I think it should be a static utility rather than part of an abstract class.




-- 
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] yyanyy commented on a change in pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r690758046



##########
File path: api/src/main/java/org/apache/iceberg/io/FileIO.java
##########
@@ -66,4 +66,10 @@ default void deleteFile(OutputFile file) {
    */
   default void initialize(Map<String, String> properties) {
   }
+
+  /**
+   * Close File IO and release underlying resources.
+   */
+  default void close() {

Review comment:
       Thanks for the feedback! I will update the comment but for the default impl, I mentioned this a little bit in the other response I had, the reason for me to add a default implementation here was because I think this use case of having to call `close` is rare and for the majority of the use case it is not needed, and because of this, to me breaking compatibility of custom FileIO implementation by adding a non-default method in the interface might not be worth it. Please let me know if you have more feedback!




-- 
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] yyanyy commented on a change in pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r701510519



##########
File path: api/src/main/java/org/apache/iceberg/io/CloseGroupHelper.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.iceberg.io;
+
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.stream.Collectors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class acts as a helper for handling the closure of multiple resource.
+ * To use it. register resources to be closed in constructor, and call the corresponding close method when needed.
+ * <p>
+ * It can take both closeable and autocloseable objects, and handle closeable as autocloseable and guarantee close
+ * idempotency by ensuring that each resource will be closed once even with concurrent close calls. It will also
+ * wrap checked exceptions into IOException to follow Closeable method signature.
+ */
+public class CloseGroupHelper {
+  private static final Logger LOG = LoggerFactory.getLogger(CloseGroupHelper.class);
+
+  private final Deque<AutoCloseable> closeables;
+
+  /**
+   * Register closeables and/or autocloseables to be managed by this class.
+   */
+  public CloseGroupHelper(AutoCloseable... autoCloseables) {
+    this.closeables = Arrays.stream(autoCloseables)
+        .filter(Objects::nonNull)
+        .collect(Collectors.toCollection(ConcurrentLinkedDeque::new));
+  }
+
+  /**
+   * Close all the registered resources. Close method of each resource will only be called once.
+   * Checked exception from AutoCloseable will be wrapped to IOException. Users can also configure to
+   * suppress the failure during closing to ensure the close method of all resources to be called.
+   *
+   * @param suppressCloseFailure true if user wants to suppress close failures
+   * @throws IOException exception thrown if close encounters errors and suppress is false
+   */
+  public void closeAsCloseable(boolean suppressCloseFailure) throws IOException {
+    while (!closeables.isEmpty()) {
+      AutoCloseable toClose = closeables.pollFirst();
+      if (toClose != null) {
+        try {
+          toClose.close();
+        } catch (Exception e) {
+          if (suppressCloseFailure) {
+            LOG.error("Exception suppressed when attempting to close resources", e);
+          } else if (e instanceof IOException) {
+            throw (IOException) e;
+          } else if (e instanceof RuntimeException) {
+            throw (RuntimeException) e;
+          } else {
+            throw new IOException("Exception occurs when closing AutoCloseable", e);

Review comment:
       Thanks for the info! I originally put it `IOException` to match the class signature contract, but I guess either way works. `ExceptionUtil` was originally in core only and was not public so I moved them to api and marked the method public as I think there shouldn't be downside doing so, please let me know if you have comment 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] yyanyy commented on a change in pull request #2878: API: extend CloseableGroup and close resources when closing aws catalogs

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r679442940



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
##########
@@ -25,13 +25,14 @@
 import org.apache.iceberg.exceptions.AlreadyExistsException;
 import org.apache.iceberg.exceptions.CommitFailedException;
 import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.CloseableGroup;
 import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public abstract class BaseMetastoreCatalog implements Catalog {
+public abstract class BaseMetastoreCatalog extends CloseableGroup implements Catalog {

Review comment:
       You are right, I think I tried too hard to fit the catalogs into the existing `CloseableGroup` usage pattern, I'll refactor the code to use the closeable group as a utility class so that we can adopt those best practices and minimize changes to other catalogs. Thanks for the feedback! 




-- 
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 pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#issuecomment-916491764


   Looks great, thanks for fixing this, @yyanyy!


-- 
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 change in pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r694287044



##########
File path: aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java
##########
@@ -80,6 +81,7 @@
   private AwsProperties awsProperties;
   private FileIO fileIO;
   private LockManager lockManager;
+  private CloseGroupHelper closeGroupHelper;

Review comment:
       Yeah, I like the idea to use `CloseableGroup` and merge the helper there. There is no reason for that class to be abstract -- that just prevents it from being used for composition rather than inheritance. We can also use `ExceptionUtil.castAndThrow` to fix up the exception handling in it and update `addCloseable` to accept `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


[GitHub] [iceberg] rdblue commented on a change in pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r694283430



##########
File path: aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java
##########
@@ -80,6 +81,7 @@
   private AwsProperties awsProperties;
   private FileIO fileIO;
   private LockManager lockManager;
+  private CloseGroupHelper closeGroupHelper;

Review comment:
       Oh, I see. You can't use `CloseableGroup` unless the base class extends it. That makes sense.




-- 
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] yyanyy commented on a change in pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r690758046



##########
File path: api/src/main/java/org/apache/iceberg/io/FileIO.java
##########
@@ -66,4 +66,10 @@ default void deleteFile(OutputFile file) {
    */
   default void initialize(Map<String, String> properties) {
   }
+
+  /**
+   * Close File IO and release underlying resources.
+   */
+  default void close() {

Review comment:
       Thanks for the feedback! I will update the comment but for the default impl, I mentioned this a little bit in the other response I had, the reason for me to add a default implementation here was because I think this use case of having to call `close` is rare and for the majority of the use case it is not needed, and because of this, to me breaking compatibility of custom FileIO implementation by adding a non-default method in the interface might not be worth it. Please let me know if you have more feedback!




-- 
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] yyanyy commented on a change in pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r694184807



##########
File path: aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java
##########
@@ -80,6 +81,7 @@
   private AwsProperties awsProperties;
   private FileIO fileIO;
   private LockManager lockManager;
+  private CloseGroupHelper closeGroupHelper;

Review comment:
       Just to ensure I understand the suggestion, do you mean to make this class implement `CloseableGroup` or move the logic from `CloseGroupHelper` to `CloseableGroup` and continue to declare a `private CloseableGroup closeableGroup = new CloseableGroup(){}` here? If the former, Ryan M and I had a discussion around this in https://github.com/apache/iceberg/pull/2878#discussion_r678098958 but I should be able to do the latter. Thanks! 




-- 
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] yyanyy commented on a change in pull request #2878: API: extend CloseableGroup and close resources when closing aws catalogs

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r679442940



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
##########
@@ -25,13 +25,14 @@
 import org.apache.iceberg.exceptions.AlreadyExistsException;
 import org.apache.iceberg.exceptions.CommitFailedException;
 import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.io.CloseableGroup;
 import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public abstract class BaseMetastoreCatalog implements Catalog {
+public abstract class BaseMetastoreCatalog extends CloseableGroup implements Catalog {

Review comment:
       You are right, I think I tried too hard to fit the catalogs into the existing `CloseableGroup` usage pattern, I'll refactor the code to make closeable group a utility class so that we can adopt those best practices and minimize changes to other catalogs. Thanks for the feedback! 




-- 
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] rymurr commented on a change in pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r685835452



##########
File path: api/src/main/java/org/apache/iceberg/io/CloseGroupUtil.java
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.iceberg.io;
+
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CloseGroupUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(CloseGroupUtil.class);
+
+  private CloseGroupUtil() {
+  }
+
+  public static void closeAutoCloseables(boolean suppressCloseFailure, AutoCloseable... closeables) throws Exception {

Review comment:
       A javadoc would be useful here to clarify the supression and its effect.

##########
File path: api/src/main/java/org/apache/iceberg/io/FileIO.java
##########
@@ -66,4 +66,10 @@ default void deleteFile(OutputFile file) {
    */
   default void initialize(Map<String, String> properties) {
   }
+
+  /**
+   * Close File IO and release underlying resources.
+   */
+  default void close() {

Review comment:
       I agree w. @kbendick. I don't think we should extend `AutoCloseable` *and* add a default impl.
   
   I think there is a good argument to have `FileIO` implement `Closeable` (rather than `Autocloseable` as its fairly explicitly related to IO), what do you think @rdblue? But I would make `FileIO` implementations implement `close` rather than add a default here, otherwise its not clear to implementers that they may have to deal w/ closing their resources.




-- 
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] kbendick commented on a change in pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r685816190



##########
File path: api/src/main/java/org/apache/iceberg/io/FileIO.java
##########
@@ -66,4 +66,10 @@ default void deleteFile(OutputFile file) {
    */
   default void initialize(Map<String, String> properties) {
   }
+
+  /**
+   * Close File IO and release underlying resources.
+   */
+  default void close() {

Review comment:
       Nit: Possibly add some documentation here about whether or not `close` needs to be called? Currently it's somewhat confusing (as the underlying issue is somewhat confusing), but end user's are exposed this API and encouraged / allowed to write their own FileIO per the docs, so I feel that the requirements should be clearly laid out.




-- 
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] yyanyy commented on a change in pull request #2878: API: extend CloseableGroup and close resources when closing aws catalogs

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878#discussion_r678536728



##########
File path: api/src/main/java/org/apache/iceberg/io/CloseableGroup.java
##########
@@ -23,20 +23,47 @@
 import java.io.IOException;
 import java.util.Deque;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public abstract class CloseableGroup implements Closeable {
-  private final Deque<Closeable> closeables = Lists.newLinkedList();
+  private static final Logger LOG = LoggerFactory.getLogger(CloseableGroup.class);
 
-  protected void addCloseable(Closeable closeable) {
+  private final Deque<AutoCloseable> closeables = Lists.newLinkedList();
+  private boolean suppressCloseFailure = false;
+
+  protected void addCloseable(AutoCloseable closeable) {
     closeables.add(closeable);
   }
 
+  protected void setSuppressCloseFailure(boolean shouldSuppress) {
+    this.suppressCloseFailure = shouldSuppress;
+  }
+
   @Override
   public void close() throws IOException {

Review comment:
       Per API contract of `Closeable` and `AutoCloseable`, `AutoCloseable`'s close does not require to be idempotent, but `Closeable` would be. Since we are modifying this class to also accept `AutoCloseable` too, if the class extending it registers an `AutoCloseable` resource then it may not be idempotent. I personally think this would be low risk since `close()` call is something the caller can control, and usually I don't think there's a use case where the `close` would be explicitly called twice, and even so and problem occurs, if the caller is not interested they can suppress the exceptions. Are you aware of any such use case that would have issue with 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] rdblue merged pull request #2878: API: allow closing resources in aws catalogs and S3FileIO

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #2878:
URL: https://github.com/apache/iceberg/pull/2878


   


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