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/06/07 19:27:42 UTC

[GitHub] [iceberg] bryanck opened a new pull request, #4991: Improved shutdown of token refresh executor during REST catalog close

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

   This PR improves the shutdown of the refresh executor in the REST catalog client. During catalog close, all pending scheduled tasks will also be canceled in addition to shutting down the executor. It also moves the HTTP client close until after the executor shutdown to help prevent spurious errors in the scheduled tasks during shutdown.
   
   Finally, the max refresh window was bumped from 1 min to 5 min to give a little bit more leeway for connection timeouts and such, to prevent the token from expiring before calling the refresh endpoint.
   


-- 
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 diff in pull request #4991: Improved shutdown of token refresh executor during REST catalog close

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -83,7 +84,7 @@
 
 public class RESTSessionCatalog extends BaseSessionCatalog implements Configurable<Configuration>, Closeable {
   private static final Logger LOG = LoggerFactory.getLogger(RESTSessionCatalog.class);
-  private static final long MAX_REFRESH_WINDOW_MILLIS = 60_000; // 1 minute
+  private static final long MAX_REFRESH_WINDOW_MILLIS = 300_000; // 5 minutes

Review Comment:
   👍 



##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -361,15 +362,25 @@ private void scheduleTokenRefresh(
 
   @Override
   public void close() throws IOException {
+    shutdownRefreshExecutor();
+
     if (client != null) {
       client.close();
     }
+  }
 
+  private void shutdownRefreshExecutor() {
     if (refreshExecutor != null) {
       ScheduledExecutorService service = refreshExecutor;
       this.refreshExecutor = null;
 
-      service.shutdown();
+      List<Runnable> tasks = service.shutdownNow();
+      tasks.forEach(task -> {
+        if (task instanceof Future) {
+          ((Future<?>) task).cancel(true);
+        }
+      });

Review Comment:
   What I saw in the j[avadocs for `ExecutorService`](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/ExecutorService.html) used the following code, which first calls `shutdown`, then `awaitTermination`, then if that is unsuccessful, calls `shutdownNow`, to still allow tasks some time to finish.
   
   Do we want to consider that instead?
   
   ```java
    void shutdownAndAwaitTermination(ExecutorService pool) {
      pool.shutdown(); // Disable new tasks from being submitted
      try {
        // Wait a while for existing tasks to terminate
        if (!pool.awaitTermination(60, TimeUnit.SECONDS)) {
          pool.shutdownNow(); // Cancel currently executing tasks
          // Wait a while for tasks to respond to being cancelled
          if (!pool.awaitTermination(60, TimeUnit.SECONDS))
              System.err.println("Pool did not terminate");
        }
      } catch (InterruptedException ie) {
        // (Re-)Cancel if current thread also interrupted
        pool.shutdownNow();
        // Preserve interrupt status
        Thread.currentThread().interrupt();
      }
    }
   ```
   
   I admittedly don't have a super strong opinion either way, but it is in the official javadocs and doesn't involve explicit cancelling of futures.
   
   Otherwise, I'm not sure if the `awaitTermination` afterwards is necesssary.



-- 
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 #4991: Improved shutdown of token refresh executor during REST catalog close

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

   Thanks, @bryanck!


-- 
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] bryanck commented on a diff in pull request #4991: Improved shutdown of token refresh executor during REST catalog close

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


##########
core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java:
##########
@@ -361,15 +362,25 @@ private void scheduleTokenRefresh(
 
   @Override
   public void close() throws IOException {
+    shutdownRefreshExecutor();
+
     if (client != null) {
       client.close();
     }
+  }
 
+  private void shutdownRefreshExecutor() {
     if (refreshExecutor != null) {
       ScheduledExecutorService service = refreshExecutor;
       this.refreshExecutor = null;
 
-      service.shutdown();
+      List<Runnable> tasks = service.shutdownNow();
+      tasks.forEach(task -> {
+        if (task instanceof Future) {
+          ((Future<?>) task).cancel(true);
+        }
+      });

Review Comment:
   If we did that, we'd almost always await termination, as the next scheduled task will be pending, so we don't want to wait for it to run (it could be scheduled several minutes or more in the future). If a refresh task is actually running, letting it finish doesn't really add benefit as the response won't be used for anything once the catalog is closed.



-- 
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 #4991: Improved shutdown of token refresh executor during REST catalog close

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


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