You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2021/11/04 17:00:47 UTC

[GitHub] [accumulo] dlmarion opened a new pull request #2346: Client ThreadPool and Error handling changes

dlmarion opened a new pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346


   Changes include:
   
    1. Don't halt the client VM when an unhandled Error occurs, try to report it back to the client. This required reverting some of the changes from PR #1689. This includes:
   
       a. Modified AccumuloUncaughtExceptionHandler to not halt the VM unless it was running in an Accumulo server process
       b. Be sure to report back to client any exceptions/errors that happen in background threads
   
   2. Consolidated client ThreadPools usage to one location, provide better clean-up of client ThreadPools. This includes:
   
     a. Consolidated client ThreadPools calls to ClientThreadPools class accessible from ClientContext
     b. Be sure that all client ThreadPools use a Cleaner when close() might be missed
   


-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
dlmarion commented on pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#issuecomment-963577965


   I will look at making changes for the comments where I did not respond.


-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#discussion_r745643701



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
##########
@@ -65,9 +67,9 @@
 
   private ScannerImpl.Reporter reporter;
 
-  private static ThreadPoolExecutor readaheadPool = ThreadPools.createThreadPool(0,

Review comment:
       > If you notice here the max threads is Integer.MAX_VALUE. Now that this is not static anymore, what should the max value be?
   
   oh I completely missed the int.max_val.  I saw a 3 and thought that was the max. So maybe there will not be a performance diff.  The new max should probably be lower. The scan code may never submit more than 1 or 2 task to the pool.  Not sure, would need to look at that code.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#discussion_r745580638



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientThreadPools.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.accumulo.core.clientImpl;
+
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+
+public class ClientThreadPools {
+
+  public static class ThreadPoolConfig {
+
+    public static final ThreadPoolConfig EMPTY_CONFIG =
+        new ThreadPoolConfig(Optional.empty(), Optional.empty(), Optional.empty());
+
+    private final Optional<Iterable<Entry<String,String>>> configuration;
+    private final Optional<Integer> numThreads;
+    private final Optional<String> threadName;
+
+    public ThreadPoolConfig(Iterable<Entry<String,String>> configuration) {
+      this(Optional.of(configuration), Optional.empty(), Optional.empty());
+    }
+
+    public ThreadPoolConfig(Iterable<Entry<String,String>> configuration, int numThreads) {
+      this(Optional.of(configuration), Optional.of(numThreads), Optional.empty());
+    }
+
+    public ThreadPoolConfig(Iterable<Entry<String,String>> configuration, int numThreads,
+        String threadName) {
+      this(Optional.of(configuration), Optional.of(numThreads), Optional.of(threadName));
+    }
+
+    private ThreadPoolConfig(Optional<Iterable<Entry<String,String>>> configuration,
+        Optional<Integer> numThreads, Optional<String> threadName) {
+      this.configuration = configuration;
+      this.numThreads = numThreads;
+      this.threadName = threadName;
+    }
+
+    public Optional<Iterable<Entry<String,String>>> getConfiguration() {
+      return configuration;
+    }
+
+    public Optional<Integer> getNumThreads() {
+      return numThreads;
+    }
+
+    public Optional<String> getThreadName() {
+      return threadName;
+    }
+  }
+
+  public static enum ThreadPoolType {
+    /**
+     * ThreadPoolExecutor that runs bulk import tasks
+     */
+    BULK_IMPORT_POOL,
+    /**
+     * ThreadPoolExecutor that runs tasks to contact Compactors to get running compaction
+     * information
+     */
+    ACTIVE_EXTERNAL_COMPACTION_POOL,
+    /**
+     * ThreadPoolExecutor used for fetching data from the TabletServers
+     */
+    SCANNER_READ_AHEAD_POOL,
+    /**
+     * ThreadPoolExecutor used for adding splits to a table
+     */
+    ADD_SPLITS_THREAD_POOL,
+    /**
+     * ThreadPoolExecutor used for fetching data from the TabletServers
+     */
+    BATCH_SCANNER_READ_AHEAD_POOL,
+    /**
+     * ThreadPoolExecutor that runs the tasks of binning mutations
+     */
+    BATCH_WRITER_BINNING_POOL,
+    /**
+     * ThreadPoolExecutor that runs the tasks of sending mutations to TabletServers
+     */
+    BATCH_WRITER_SEND_POOL,
+    /**
+     * ThreadPoolExecutor that runs clean up tasks when close is called on the ConditionalWriter
+     */
+    CONDITIONAL_WRITER_CLEANUP_TASK_POOL,
+    /**
+     * ThreadPoolExecutor responsible for loading bloom filters
+     */
+    BLOOM_FILTER_LAYER_LOADER_POOL
+  }
+
+  public static enum ScheduledThreadPoolType {
+    /**
+     * shared scheduled executor for trivial tasks
+     */
+    SHARED_GENERAL_SCHEDULED_TASK_POOL,
+    /**
+     * ScheduledThreadPoolExecutor that runs tasks for the BatchWriter to meet the users latency
+     * goals.
+     */
+    BATCH_WRITER_LATENCY_TASK_POOL,
+    /**
+     * ScheduledThreadPoolExecutor that periodically runs tasks to handle failed write mutations and
+     * send mutations to TabletServers
+     */
+    CONDITIONAL_WRITER_RETRY_POOL
+  }
+
+  private ScheduledThreadPoolExecutor sharedScheduledThreadPool = null;
+
+  public ThreadPoolExecutor newThreadPool(ThreadPoolType usage, ThreadPoolConfig config) {
+    switch (usage) {
+      case BULK_IMPORT_POOL:
+        Objects.requireNonNull(config.getNumThreads().get(), "Number of threads must be set");

Review comment:
       Resolved in 918b20f




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#discussion_r745572987



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
##########
@@ -513,10 +522,17 @@ private synchronized void updateServerErrors(String server, Exception e) {
     log.error("Server side error on {}", server, e);
   }
 
-  private synchronized void updateUnknownErrors(String msg, Exception t) {
+  private synchronized void updateUnknownErrors(String msg, Throwable t) {
     somethingFailed = true;
     unknownErrors++;
-    this.lastUnknownError = t;
+    // Multiple errors may occur between the time checkForFailures() is called
+    // by the client. Be sure to return an Error if one (or more) occurred.
+    // Set lastUnknownError if it's null, to an Error, or to an Exception if it's not already an
+    // Error
+    if (this.lastUnknownError == null
+        || !(t instanceof Exception && this.lastUnknownError instanceof Error)) {

Review comment:
       I reworked this logic in 918b20f




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#discussion_r746910449



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
##########
@@ -65,9 +67,9 @@
 
   private ScannerImpl.Reporter reporter;
 
-  private static ThreadPoolExecutor readaheadPool = ThreadPools.createThreadPool(0,

Review comment:
       So maybe nothing needs to be done here. The core pool size is already being set to [zero](https://github.com/apache/accumulo/pull/2346/files#diff-4c63ba161b875d96d2612de9bfc7cf93649762e2034d0700ec4fbfe6de0d6ef0R144). 




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#discussion_r745094092



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
##########
@@ -386,15 +386,12 @@ public void run() {
           fatalException = new TableDeletedException(tableId.canonical());
       } catch (SampleNotPresentException e) {
         fatalException = e;
-      } catch (Exception t) {
+      } catch (Throwable t) {
         if (queryThreadPool.isShutdown())
           log.debug("Caught exception, but queryThreadPool is shutdown", t);
         else
           log.warn("Caught exception, but queryThreadPool is not shutdown", t);
         fatalException = t;
-      } catch (Throwable t) {
-        fatalException = t;
-        throw t; // let uncaught exception handler deal with the Error

Review comment:
       The user is not providing their own thread pools in this PR. I will review and see what happens if an exception is thrown from this iterator

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
##########
@@ -625,19 +641,26 @@ public void run() {
 
     private static final int MUTATION_BATCH_SIZE = 1 << 17;
     private final ThreadPoolExecutor sendThreadPool;
+    private final Cleanable sendThreadPoolCleanable;
     private final ThreadPoolExecutor binningThreadPool;
+    private final Cleanable binningThreadPoolCleanable;
     private final Map<String,TabletServerMutations<Mutation>> serversMutations;
     private final Set<String> queued;
     private final Map<TableId,TabletLocator> locators;
 
     public MutationWriter(int numSendThreads) {
       serversMutations = new HashMap<>();
       queued = new HashSet<>();
-      sendThreadPool =
-          ThreadPools.createFixedThreadPool(numSendThreads, this.getClass().getName(), false);
+      sendThreadPool = context.getThreadPools().newThreadPool(ThreadPoolType.BATCH_WRITER_SEND_POOL,
+          new ThreadPoolConfig(context.getConfiguration(), numSendThreads));
+      sendThreadPoolCleanable =
+          CleanerUtil.shutdownThreadPoolExecutor(sendThreadPool, () -> {}, log);

Review comment:
       Take a look to at the changes I made to CleanerUtil below. The empty runnable is a pre-shutdown function. There is one place in the code where that is being used.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
##########
@@ -801,6 +831,8 @@ public Result write(ConditionalMutation mutation) {
   @Override
   public void close() {
     threadPool.shutdownNow();
+    threadPoolCleanable.clean(); // deregister the cleaner, will not call shutdownNow() because
+                                 // closed is now true
     cleanupThreadPool.execute(Threads.createNamedRunnable("ConditionalWriterCleanupTask",

Review comment:
       I could potentially modify it to use the SHARED_GENERAL_SCHEDULED_TASK_POOL which is static. I could submit a fire once task.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
##########
@@ -411,7 +408,7 @@ public void run() {
               e.setTableInfo(getTableInfo());
               log.debug("{}", e.getMessage(), e);
               fatalException = e;
-            } catch (Exception t) {
+            } catch (Throwable t) {

Review comment:
       I went back through the changes I made to the client code in #1840 and un-did them.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
##########
@@ -65,9 +67,9 @@
 
   private ScannerImpl.Reporter reporter;
 
-  private static ThreadPoolExecutor readaheadPool = ThreadPools.createThreadPool(0,

Review comment:
       If you notice here the max threads is Integer.MAX_VALUE. Now that this is not static anymore, what should the max value be? 

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
##########
@@ -124,20 +133,26 @@ public boolean hasNext() {
     throw new NoSuchElementException();
   }
 
+  void closeThriftScanner() {
+    synchronized (scanState) {
+      // this is synchronized so its mutually exclusive with readBatch()
+      try {
+        closed = true;
+        ThriftScanner.close(scanState);
+      } catch (Exception e) {
+        LoggerFactory.getLogger(ScannerIterator.class).debug("Exception when closing scan session",
+            e);
+      }
+    }
+  }
+
   void close() {
     // run actual close operation in the background so this does not block.
     readaheadPool.execute(() -> {
-      synchronized (scanState) {
-        // this is synchronized so its mutually exclusive with readBatch()
-        try {
-          closed = true;
-          ThriftScanner.close(scanState);
-        } catch (Exception e) {
-          LoggerFactory.getLogger(ScannerIterator.class)
-              .debug("Exception when closing scan session", e);
-        }
-      }
+      closeThriftScanner();

Review comment:
       poolCloser is a reference to the shared scheduled executor pool

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
##########
@@ -801,6 +831,8 @@ public Result write(ConditionalMutation mutation) {
   @Override
   public void close() {
     threadPool.shutdownNow();
+    threadPoolCleanable.clean(); // deregister the cleaner, will not call shutdownNow() because
+                                 // closed is now true
     cleanupThreadPool.execute(Threads.createNamedRunnable("ConditionalWriterCleanupTask",

Review comment:
       Addressed in 918b20f. CleanupThreadPool is now a reference to the shared scheduled executor pool.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
##########
@@ -801,6 +831,8 @@ public Result write(ConditionalMutation mutation) {
   @Override
   public void close() {
     threadPool.shutdownNow();
+    threadPoolCleanable.clean(); // deregister the cleaner, will not call shutdownNow() because
+                                 // closed is now true

Review comment:
       See the changes I made to CleanerUtil below. The empty runnable is a preshutdown function, which is not needed here. It still shuts down the threadpool.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
##########
@@ -65,9 +67,9 @@
 
   private ScannerImpl.Reporter reporter;
 
-  private static ThreadPoolExecutor readaheadPool = ThreadPools.createThreadPool(0,

Review comment:
       I was surprised by that also, although the max is rather large.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#discussion_r746836588



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
##########
@@ -65,9 +67,9 @@
 
   private ScannerImpl.Reporter reporter;
 
-  private static ThreadPoolExecutor readaheadPool = ThreadPools.createThreadPool(0,

Review comment:
       Do you think we should reuse the value of `BATCH_SCANNER_NUM_QUERY_THREADS` for this, create a new property `SCANNER_NUM_QUERY_THREADS`, or something else?




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
dlmarion commented on pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#issuecomment-1067883187


   Closing this in favor of #2554 


-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#discussion_r746875462



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
##########
@@ -65,9 +67,9 @@
 
   private ScannerImpl.Reporter reporter;
 
-  private static ThreadPoolExecutor readaheadPool = ThreadPools.createThreadPool(0,

Review comment:
       > Do you think we should reuse the value of BATCH_SCANNER_NUM_QUERY_THREADS for this, create a new property SCANNER_NUM_QUERY_THREADS, or something else?
   
   Looking at [this code](https://github.com/apache/accumulo/blob/bf4d8e4ad5c06e2881a8235ffd0bb293c54289e4/core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java#L143-L146) it seems like the read ahead thread pool on a per scanner basis would only ever use a max one thread.  With that in mind I don't think we need to make the pool configurable, just create a pool with 0 to 1 threads.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#discussion_r746930818



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
##########
@@ -65,9 +67,9 @@
 
   private ScannerImpl.Reporter reporter;
 
-  private static ThreadPoolExecutor readaheadPool = ThreadPools.createThreadPool(0,

Review comment:
       > So maybe nothing needs to be done here. The core pool size is already being set to zero.
   
   Could possibly change the max from max int to 1.  W/ the synchronous queue I think it would throw an exception if more than one task was added to the pool, which if I Am reading the scanner code correctly is not expected.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#discussion_r745028976



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientThreadPools.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.accumulo.core.clientImpl;
+
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+
+public class ClientThreadPools {
+
+  public static class ThreadPoolConfig {
+
+    public static final ThreadPoolConfig EMPTY_CONFIG =
+        new ThreadPoolConfig(Optional.empty(), Optional.empty(), Optional.empty());
+
+    private final Optional<Iterable<Entry<String,String>>> configuration;
+    private final Optional<Integer> numThreads;
+    private final Optional<String> threadName;
+
+    public ThreadPoolConfig(Iterable<Entry<String,String>> configuration) {
+      this(Optional.of(configuration), Optional.empty(), Optional.empty());
+    }
+
+    public ThreadPoolConfig(Iterable<Entry<String,String>> configuration, int numThreads) {
+      this(Optional.of(configuration), Optional.of(numThreads), Optional.empty());
+    }
+
+    public ThreadPoolConfig(Iterable<Entry<String,String>> configuration, int numThreads,
+        String threadName) {
+      this(Optional.of(configuration), Optional.of(numThreads), Optional.of(threadName));
+    }
+
+    private ThreadPoolConfig(Optional<Iterable<Entry<String,String>>> configuration,
+        Optional<Integer> numThreads, Optional<String> threadName) {
+      this.configuration = configuration;
+      this.numThreads = numThreads;
+      this.threadName = threadName;
+    }
+
+    public Optional<Iterable<Entry<String,String>>> getConfiguration() {
+      return configuration;
+    }
+
+    public Optional<Integer> getNumThreads() {
+      return numThreads;
+    }
+
+    public Optional<String> getThreadName() {
+      return threadName;
+    }
+  }
+
+  public static enum ThreadPoolType {
+    /**
+     * ThreadPoolExecutor that runs bulk import tasks
+     */
+    BULK_IMPORT_POOL,
+    /**
+     * ThreadPoolExecutor that runs tasks to contact Compactors to get running compaction
+     * information
+     */
+    ACTIVE_EXTERNAL_COMPACTION_POOL,
+    /**
+     * ThreadPoolExecutor used for fetching data from the TabletServers
+     */
+    SCANNER_READ_AHEAD_POOL,
+    /**
+     * ThreadPoolExecutor used for adding splits to a table
+     */
+    ADD_SPLITS_THREAD_POOL,
+    /**
+     * ThreadPoolExecutor used for fetching data from the TabletServers
+     */
+    BATCH_SCANNER_READ_AHEAD_POOL,
+    /**
+     * ThreadPoolExecutor that runs the tasks of binning mutations
+     */
+    BATCH_WRITER_BINNING_POOL,
+    /**
+     * ThreadPoolExecutor that runs the tasks of sending mutations to TabletServers
+     */
+    BATCH_WRITER_SEND_POOL,
+    /**
+     * ThreadPoolExecutor that runs clean up tasks when close is called on the ConditionalWriter
+     */
+    CONDITIONAL_WRITER_CLEANUP_TASK_POOL,
+    /**
+     * ThreadPoolExecutor responsible for loading bloom filters
+     */
+    BLOOM_FILTER_LAYER_LOADER_POOL
+  }
+
+  public static enum ScheduledThreadPoolType {
+    /**
+     * shared scheduled executor for trivial tasks
+     */
+    SHARED_GENERAL_SCHEDULED_TASK_POOL,
+    /**
+     * ScheduledThreadPoolExecutor that runs tasks for the BatchWriter to meet the users latency
+     * goals.
+     */
+    BATCH_WRITER_LATENCY_TASK_POOL,
+    /**
+     * ScheduledThreadPoolExecutor that periodically runs tasks to handle failed write mutations and
+     * send mutations to TabletServers
+     */
+    CONDITIONAL_WRITER_RETRY_POOL
+  }
+
+  private ScheduledThreadPoolExecutor sharedScheduledThreadPool = null;
+
+  public ThreadPoolExecutor newThreadPool(ThreadPoolType usage, ThreadPoolConfig config) {
+    switch (usage) {
+      case BULK_IMPORT_POOL:
+        Objects.requireNonNull(config.getNumThreads().get(), "Number of threads must be set");
+        return ThreadPools.createFixedThreadPool(config.getNumThreads().get(), "BulkImportThread",
+            false);
+      case ACTIVE_EXTERNAL_COMPACTION_POOL:
+        Objects.requireNonNull(config.getNumThreads().get(), "Number of threads must be set");
+        return ThreadPools.createFixedThreadPool(config.getNumThreads().get(),
+            "getactivecompactions", false);
+      case SCANNER_READ_AHEAD_POOL:
+        return ThreadPools.createThreadPool(0, Integer.MAX_VALUE, 3L, TimeUnit.SECONDS,
+            "Accumulo scanner read ahead thread", new SynchronousQueue<>(), OptionalInt.empty(),
+            false);
+      case ADD_SPLITS_THREAD_POOL:
+        return ThreadPools.createFixedThreadPool(16, "addSplits", false);
+      case BATCH_SCANNER_READ_AHEAD_POOL:
+        Objects.requireNonNull(config.getNumThreads().get(), "Number of threads must be set");
+        Objects.requireNonNull(config.getThreadName().get(), "Name of threads must be set");
+        return ThreadPools.createFixedThreadPool(config.getNumThreads().get(),
+            "batch scanner " + config.getThreadName().get() + "-", false);
+      case BATCH_WRITER_BINNING_POOL:
+        return ThreadPools.createFixedThreadPool(1, "BinMutations", new SynchronousQueue<>(),
+            false);
+      case BATCH_WRITER_SEND_POOL:
+        Objects.requireNonNull(config.getNumThreads().get(), "Number of threads must be set");
+        return ThreadPools.createFixedThreadPool(config.getNumThreads().get(), "MutationWriter",
+            false);
+      case CONDITIONAL_WRITER_CLEANUP_TASK_POOL:
+        return ThreadPools.createFixedThreadPool(1, 3, TimeUnit.SECONDS,
+            "Conditional Writer Cleanup Thread", false);
+      case BLOOM_FILTER_LAYER_LOADER_POOL:
+        Objects.requireNonNull(config.getConfiguration().get(), "Configuration must be set");
+        return ThreadPools.createThreadPool(0,
+            new ConfigurationCopy(config.getConfiguration().get())
+                .getCount(Property.TSERV_BLOOM_LOAD_MAXCONCURRENT),
+            60, TimeUnit.SECONDS, "bloom-loader", false);
+      default:
+        throw new IllegalArgumentException("Unhandled thread pool usage value: " + usage.name());
+    }
+  }
+
+  public ScheduledThreadPoolExecutor newScheduledThreadPool(ScheduledThreadPoolType usage,
+      ThreadPoolConfig config) {
+    switch (usage) {
+      case SHARED_GENERAL_SCHEDULED_TASK_POOL:
+        Objects.requireNonNull(config.getConfiguration().get(), "Configuration must be set");
+        if (sharedScheduledThreadPool == null) {
+          sharedScheduledThreadPool = (ScheduledThreadPoolExecutor) ThreadPools
+              .createExecutorService(new ConfigurationCopy(config.getConfiguration().get()),
+                  Property.GENERAL_SIMPLETIMER_THREADPOOL_SIZE);
+        }
+        return sharedScheduledThreadPool;

Review comment:
       I don't see any sync here, so it seems like multiple instances of this thread pool could be created when this method is called by multiple threads.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
##########
@@ -801,6 +831,8 @@ public Result write(ConditionalMutation mutation) {
   @Override
   public void close() {
     threadPool.shutdownNow();
+    threadPoolCleanable.clean(); // deregister the cleaner, will not call shutdownNow() because
+                                 // closed is now true
     cleanupThreadPool.execute(Threads.createNamedRunnable("ConditionalWriterCleanupTask",

Review comment:
       I this code was written w/ the assumption that there was a static thread pool where we could submit a task that reaches out to tservers and does some optional clean up.  Now that thread pool is not static, using this thread pool seems to complicate things.  We could create a daemon thread to do this `new Thread(new CleanupTask(getActiveSessions())).setDeamon(true).run()` instead of submitting a task to the thread pool.  That way the thread pool could closed. When the runnable finishes I think the thread will just go away.
   
   Also do you know if the cleaner actually shuts the thread pool down?  I wonder if the threads in a thread pool can reference the thread pool object so it always looks referenced from the GC perspective.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
##########
@@ -124,20 +133,26 @@ public boolean hasNext() {
     throw new NoSuchElementException();
   }
 
+  void closeThriftScanner() {
+    synchronized (scanState) {
+      // this is synchronized so its mutually exclusive with readBatch()
+      try {
+        closed = true;
+        ThriftScanner.close(scanState);
+      } catch (Exception e) {
+        LoggerFactory.getLogger(ScannerIterator.class).debug("Exception when closing scan session",
+            e);
+      }
+    }
+  }
+
   void close() {
     // run actual close operation in the background so this does not block.
     readaheadPool.execute(() -> {
-      synchronized (scanState) {
-        // this is synchronized so its mutually exclusive with readBatch()
-        try {
-          closed = true;
-          ThriftScanner.close(scanState);
-        } catch (Exception e) {
-          LoggerFactory.getLogger(ScannerIterator.class)
-              .debug("Exception when closing scan session", e);
-        }
-      }
+      closeThriftScanner();

Review comment:
       Could possibly create a thread to run this task if it would be simpler to shut the thread pool down here.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
##########
@@ -65,9 +67,9 @@
 
   private ScannerImpl.Reporter reporter;
 
-  private static ThreadPoolExecutor readaheadPool = ThreadPools.createThreadPool(0,

Review comment:
       I did not realize this pool was static and shared among all scanners. I wonder if changing this to per-scanner could improve performance for the case of a client process w/ many concurrent scanners.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
##########
@@ -411,7 +408,7 @@ public void run() {
               e.setTableInfo(getTableInfo());
               log.debug("{}", e.getMessage(), e);
               fatalException = e;
-            } catch (Exception t) {
+            } catch (Throwable t) {

Review comment:
       How did you find the places to catch Throwable in the client code?




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#discussion_r745580828



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientThreadPools.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.accumulo.core.clientImpl;
+
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+
+public class ClientThreadPools {
+
+  public static class ThreadPoolConfig {
+
+    public static final ThreadPoolConfig EMPTY_CONFIG =
+        new ThreadPoolConfig(Optional.empty(), Optional.empty(), Optional.empty());
+
+    private final Optional<Iterable<Entry<String,String>>> configuration;
+    private final Optional<Integer> numThreads;
+    private final Optional<String> threadName;
+
+    public ThreadPoolConfig(Iterable<Entry<String,String>> configuration) {
+      this(Optional.of(configuration), Optional.empty(), Optional.empty());
+    }
+
+    public ThreadPoolConfig(Iterable<Entry<String,String>> configuration, int numThreads) {
+      this(Optional.of(configuration), Optional.of(numThreads), Optional.empty());
+    }
+
+    public ThreadPoolConfig(Iterable<Entry<String,String>> configuration, int numThreads,
+        String threadName) {
+      this(Optional.of(configuration), Optional.of(numThreads), Optional.of(threadName));
+    }
+
+    private ThreadPoolConfig(Optional<Iterable<Entry<String,String>>> configuration,
+        Optional<Integer> numThreads, Optional<String> threadName) {
+      this.configuration = configuration;
+      this.numThreads = numThreads;
+      this.threadName = threadName;
+    }
+
+    public Optional<Iterable<Entry<String,String>>> getConfiguration() {
+      return configuration;
+    }
+
+    public Optional<Integer> getNumThreads() {
+      return numThreads;
+    }
+
+    public Optional<String> getThreadName() {
+      return threadName;
+    }
+  }
+
+  public static enum ThreadPoolType {
+    /**
+     * ThreadPoolExecutor that runs bulk import tasks
+     */
+    BULK_IMPORT_POOL,
+    /**
+     * ThreadPoolExecutor that runs tasks to contact Compactors to get running compaction
+     * information
+     */
+    ACTIVE_EXTERNAL_COMPACTION_POOL,
+    /**
+     * ThreadPoolExecutor used for fetching data from the TabletServers
+     */
+    SCANNER_READ_AHEAD_POOL,
+    /**
+     * ThreadPoolExecutor used for adding splits to a table
+     */
+    ADD_SPLITS_THREAD_POOL,
+    /**
+     * ThreadPoolExecutor used for fetching data from the TabletServers
+     */
+    BATCH_SCANNER_READ_AHEAD_POOL,
+    /**
+     * ThreadPoolExecutor that runs the tasks of binning mutations
+     */
+    BATCH_WRITER_BINNING_POOL,
+    /**
+     * ThreadPoolExecutor that runs the tasks of sending mutations to TabletServers
+     */
+    BATCH_WRITER_SEND_POOL,
+    /**
+     * ThreadPoolExecutor that runs clean up tasks when close is called on the ConditionalWriter
+     */
+    CONDITIONAL_WRITER_CLEANUP_TASK_POOL,
+    /**
+     * ThreadPoolExecutor responsible for loading bloom filters
+     */
+    BLOOM_FILTER_LAYER_LOADER_POOL
+  }
+
+  public static enum ScheduledThreadPoolType {
+    /**
+     * shared scheduled executor for trivial tasks
+     */
+    SHARED_GENERAL_SCHEDULED_TASK_POOL,
+    /**
+     * ScheduledThreadPoolExecutor that runs tasks for the BatchWriter to meet the users latency
+     * goals.
+     */
+    BATCH_WRITER_LATENCY_TASK_POOL,
+    /**
+     * ScheduledThreadPoolExecutor that periodically runs tasks to handle failed write mutations and
+     * send mutations to TabletServers
+     */
+    CONDITIONAL_WRITER_RETRY_POOL
+  }
+
+  private ScheduledThreadPoolExecutor sharedScheduledThreadPool = null;
+
+  public ThreadPoolExecutor newThreadPool(ThreadPoolType usage, ThreadPoolConfig config) {
+    switch (usage) {
+      case BULK_IMPORT_POOL:
+        Objects.requireNonNull(config.getNumThreads().get(), "Number of threads must be set");
+        return ThreadPools.createFixedThreadPool(config.getNumThreads().get(), "BulkImportThread",
+            false);
+      case ACTIVE_EXTERNAL_COMPACTION_POOL:
+        Objects.requireNonNull(config.getNumThreads().get(), "Number of threads must be set");
+        return ThreadPools.createFixedThreadPool(config.getNumThreads().get(),
+            "getactivecompactions", false);
+      case SCANNER_READ_AHEAD_POOL:
+        return ThreadPools.createThreadPool(0, Integer.MAX_VALUE, 3L, TimeUnit.SECONDS,
+            "Accumulo scanner read ahead thread", new SynchronousQueue<>(), OptionalInt.empty(),
+            false);
+      case ADD_SPLITS_THREAD_POOL:
+        return ThreadPools.createFixedThreadPool(16, "addSplits", false);
+      case BATCH_SCANNER_READ_AHEAD_POOL:
+        Objects.requireNonNull(config.getNumThreads().get(), "Number of threads must be set");
+        Objects.requireNonNull(config.getThreadName().get(), "Name of threads must be set");
+        return ThreadPools.createFixedThreadPool(config.getNumThreads().get(),
+            "batch scanner " + config.getThreadName().get() + "-", false);
+      case BATCH_WRITER_BINNING_POOL:
+        return ThreadPools.createFixedThreadPool(1, "BinMutations", new SynchronousQueue<>(),
+            false);
+      case BATCH_WRITER_SEND_POOL:
+        Objects.requireNonNull(config.getNumThreads().get(), "Number of threads must be set");
+        return ThreadPools.createFixedThreadPool(config.getNumThreads().get(), "MutationWriter",
+            false);
+      case CONDITIONAL_WRITER_CLEANUP_TASK_POOL:
+        return ThreadPools.createFixedThreadPool(1, 3, TimeUnit.SECONDS,
+            "Conditional Writer Cleanup Thread", false);
+      case BLOOM_FILTER_LAYER_LOADER_POOL:
+        Objects.requireNonNull(config.getConfiguration().get(), "Configuration must be set");
+        return ThreadPools.createThreadPool(0,
+            new ConfigurationCopy(config.getConfiguration().get())
+                .getCount(Property.TSERV_BLOOM_LOAD_MAXCONCURRENT),
+            60, TimeUnit.SECONDS, "bloom-loader", false);
+      default:
+        throw new IllegalArgumentException("Unhandled thread pool usage value: " + usage.name());
+    }
+  }
+
+  public ScheduledThreadPoolExecutor newScheduledThreadPool(ScheduledThreadPoolType usage,
+      ThreadPoolConfig config) {
+    switch (usage) {
+      case SHARED_GENERAL_SCHEDULED_TASK_POOL:
+        Objects.requireNonNull(config.getConfiguration().get(), "Configuration must be set");
+        if (sharedScheduledThreadPool == null) {
+          sharedScheduledThreadPool = (ScheduledThreadPoolExecutor) ThreadPools
+              .createExecutorService(new ConfigurationCopy(config.getConfiguration().get()),
+                  Property.GENERAL_SIMPLETIMER_THREADPOOL_SIZE);
+        }
+        return sharedScheduledThreadPool;

Review comment:
       Resolved in 918b20f




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#discussion_r745573371



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
##########
@@ -386,15 +386,12 @@ public void run() {
           fatalException = new TableDeletedException(tableId.canonical());
       } catch (SampleNotPresentException e) {
         fatalException = e;
-      } catch (Exception t) {
+      } catch (Throwable t) {
         if (queryThreadPool.isShutdown())
           log.debug("Caught exception, but queryThreadPool is shutdown", t);
         else
           log.warn("Caught exception, but queryThreadPool is not shutdown", t);
         fatalException = t;
-      } catch (Throwable t) {
-        fatalException = t;
-        throw t; // let uncaught exception handler deal with the Error

Review comment:
       I resurrected the `throw t` line in 918b20f




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#discussion_r746878178



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
##########
@@ -65,9 +67,9 @@
 
   private ScannerImpl.Reporter reporter;
 
-  private static ThreadPoolExecutor readaheadPool = ThreadPools.createThreadPool(0,

Review comment:
       The user has control over wether read ahead will happen w/ [this scanner API](https://github.com/apache/accumulo/blob/bf4d8e4ad5c06e2881a8235ffd0bb293c54289e4/core/src/main/java/org/apache/accumulo/core/client/Scanner.java#L93).  If that is set really high, then read ahead would never happen.  So maybe it nice if the thread pool can go to zero threads instead of always having 1 thread.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion closed pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
dlmarion closed pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346


   


-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#discussion_r745572727



##########
File path: core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java
##########
@@ -34,7 +35,11 @@
 
   @Before
   public void setup() {
-    context = EasyMock.createMock(ClientContext.class);
+    context = EasyMock.createStrictMock(ClientContext.class);
+    AccumuloConfiguration conf = EasyMock.createStrictMock(AccumuloConfiguration.class);
+    EasyMock.expect(context.getConfiguration()).andReturn(conf).anyTimes();
+    EasyMock.expect(context.getThreadPools()).andReturn(new ClientThreadPools()).anyTimes();
+    EasyMock.replay(context);

Review comment:
       Resolved in 918b20f




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#discussion_r746930818



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
##########
@@ -65,9 +67,9 @@
 
   private ScannerImpl.Reporter reporter;
 
-  private static ThreadPoolExecutor readaheadPool = ThreadPools.createThreadPool(0,

Review comment:
       > So maybe nothing needs to be done here. The core pool size is already being set to zero.
   
   Could possibly change the max from max int to 1.  W/ the synchronous queue I think it would throw an exception if more than one task was added to the pool (w/ a max of 1), which if I Am reading the scanner code correctly is not expected.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#discussion_r745082245



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientThreadPools.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.accumulo.core.clientImpl;
+
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+
+public class ClientThreadPools {
+
+  public static class ThreadPoolConfig {

Review comment:
       Yeah, sorry for the confusion. I closed the prior pull request  #2340 where I was trying to allow the user to supply their own thread pools and uncaught exception handlers. It appeared based on the conversation in that PR that it was not going to be possible. This is a brand new PR which is only trying to do [this](https://github.com/apache/accumulo/pull/2346#issue-1044991496)




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
dlmarion commented on pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#issuecomment-963577504


   I will look at making changes for the comments where I did not respond.


-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#discussion_r745580520



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientThreadPools.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.accumulo.core.clientImpl;
+
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+
+public class ClientThreadPools {
+
+  public static class ThreadPoolConfig {
+
+    public static final ThreadPoolConfig EMPTY_CONFIG =
+        new ThreadPoolConfig(Optional.empty(), Optional.empty(), Optional.empty());
+
+    private final Optional<Iterable<Entry<String,String>>> configuration;

Review comment:
       Resolved in 918b20f




-- 
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: notifications-unsubscribe@accumulo.apache.org

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



[GitHub] [accumulo] ctubbsii commented on a change in pull request #2346: Client ThreadPool and Error handling changes

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #2346:
URL: https://github.com/apache/accumulo/pull/2346#discussion_r745000366



##########
File path: core/src/test/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderTest.java
##########
@@ -34,7 +35,11 @@
 
   @Before
   public void setup() {
-    context = EasyMock.createMock(ClientContext.class);
+    context = EasyMock.createStrictMock(ClientContext.class);
+    AccumuloConfiguration conf = EasyMock.createStrictMock(AccumuloConfiguration.class);
+    EasyMock.expect(context.getConfiguration()).andReturn(conf).anyTimes();
+    EasyMock.expect(context.getThreadPools()).andReturn(new ClientThreadPools()).anyTimes();
+    EasyMock.replay(context);

Review comment:
       EasyMock methods are good candidates for using static imports, just because they make the tests slightly more readable without losing anything.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
##########
@@ -716,6 +718,9 @@ public void close() {
       thriftTransportPool.shutdown();
     }
     singletonReservation.close();
+    if (threadPools != null) {
+      threadPools.close();
+    }

Review comment:
       `threadPools` is `final` and set explicitly to a new object in the constructor, so it can never be null.
   
   ```suggestion
       threadPools.close();
   ```

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
##########
@@ -703,7 +726,7 @@ void queueMutations(final MutationSet mutationsToSend) {
             log.trace("{} - binning {} mutations", Thread.currentThread().getName(),
                 mutationsToSend.size());
             addMutations(mutationsToSend);
-          } catch (Exception e) {
+          } catch (Throwable e) {
             updateUnknownErrors("Error processing mutation set", e);

Review comment:
       Catching Throwable here seems counterproductive to the goal of this... which I understand is to give control to the calling code that provides the thread pool when an error occurs. We can't do that if we're swallowing the errors instead of letting the user specify their own uncaught exception handler on their thread pool.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ConditionalWriterImpl.java
##########
@@ -801,6 +831,8 @@ public Result write(ConditionalMutation mutation) {
   @Override
   public void close() {
     threadPool.shutdownNow();
+    threadPoolCleanable.clean(); // deregister the cleaner, will not call shutdownNow() because
+                                 // closed is now true

Review comment:
       The comment does not appear to be correct. The cleaner will not call shutdownNow because the closed is now true, it will not call shutdownNow because it was passed a NOOP runnable. It's not passed any kind of Closeable to be able to detect whether closed is true now or not. It's doing nothing solely because it was only given a NOOP runnable.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
##########
@@ -513,10 +522,17 @@ private synchronized void updateServerErrors(String server, Exception e) {
     log.error("Server side error on {}", server, e);
   }
 
-  private synchronized void updateUnknownErrors(String msg, Exception t) {
+  private synchronized void updateUnknownErrors(String msg, Throwable t) {
     somethingFailed = true;
     unknownErrors++;
-    this.lastUnknownError = t;
+    // Multiple errors may occur between the time checkForFailures() is called
+    // by the client. Be sure to return an Error if one (or more) occurred.
+    // Set lastUnknownError if it's null, to an Error, or to an Exception if it's not already an
+    // Error
+    if (this.lastUnknownError == null
+        || !(t instanceof Exception && this.lastUnknownError instanceof Error)) {

Review comment:
       Is this trying to store the most severe of the problems? (Error preferred, then Exception, then null)?
   It's probably still a good idea to keep the old problems using `addSuppressed`.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientThreadPools.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.accumulo.core.clientImpl;
+
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+
+public class ClientThreadPools {
+
+  public static class ThreadPoolConfig {

Review comment:
       I like this class being consolidated, for the most part, but it's not clear how having this consolidated here allows users to be able to have greater control over the thread pool lifecycles/error handling, which is what I thought was part of the goal. I don't see it exposed in the public API anywhere.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
##########
@@ -625,19 +641,26 @@ public void run() {
 
     private static final int MUTATION_BATCH_SIZE = 1 << 17;
     private final ThreadPoolExecutor sendThreadPool;
+    private final Cleanable sendThreadPoolCleanable;
     private final ThreadPoolExecutor binningThreadPool;
+    private final Cleanable binningThreadPoolCleanable;
     private final Map<String,TabletServerMutations<Mutation>> serversMutations;
     private final Set<String> queued;
     private final Map<TableId,TabletLocator> locators;
 
     public MutationWriter(int numSendThreads) {
       serversMutations = new HashMap<>();
       queued = new HashSet<>();
-      sendThreadPool =
-          ThreadPools.createFixedThreadPool(numSendThreads, this.getClass().getName(), false);
+      sendThreadPool = context.getThreadPools().newThreadPool(ThreadPoolType.BATCH_WRITER_SEND_POOL,
+          new ThreadPoolConfig(context.getConfiguration(), numSendThreads));
+      sendThreadPoolCleanable =
+          CleanerUtil.shutdownThreadPoolExecutor(sendThreadPool, () -> {}, log);

Review comment:
       This seems to register a cleaner that does nothing to attempt to shutdown the thread pool. It seems odd to have a cleaner that doesn't do any cleaning, and just does a NOOP (`() -> {}`)

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientThreadPools.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.accumulo.core.clientImpl;
+
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+
+public class ClientThreadPools {
+
+  public static class ThreadPoolConfig {
+
+    public static final ThreadPoolConfig EMPTY_CONFIG =
+        new ThreadPoolConfig(Optional.empty(), Optional.empty(), Optional.empty());
+
+    private final Optional<Iterable<Entry<String,String>>> configuration;
+    private final Optional<Integer> numThreads;
+    private final Optional<String> threadName;
+
+    public ThreadPoolConfig(Iterable<Entry<String,String>> configuration) {
+      this(Optional.of(configuration), Optional.empty(), Optional.empty());
+    }
+
+    public ThreadPoolConfig(Iterable<Entry<String,String>> configuration, int numThreads) {
+      this(Optional.of(configuration), Optional.of(numThreads), Optional.empty());
+    }
+
+    public ThreadPoolConfig(Iterable<Entry<String,String>> configuration, int numThreads,
+        String threadName) {
+      this(Optional.of(configuration), Optional.of(numThreads), Optional.of(threadName));
+    }
+
+    private ThreadPoolConfig(Optional<Iterable<Entry<String,String>>> configuration,
+        Optional<Integer> numThreads, Optional<String> threadName) {
+      this.configuration = configuration;
+      this.numThreads = numThreads;
+      this.threadName = threadName;
+    }
+
+    public Optional<Iterable<Entry<String,String>>> getConfiguration() {
+      return configuration;
+    }
+
+    public Optional<Integer> getNumThreads() {
+      return numThreads;
+    }
+
+    public Optional<String> getThreadName() {
+      return threadName;
+    }
+  }
+
+  public static enum ThreadPoolType {
+    /**
+     * ThreadPoolExecutor that runs bulk import tasks
+     */
+    BULK_IMPORT_POOL,
+    /**
+     * ThreadPoolExecutor that runs tasks to contact Compactors to get running compaction
+     * information
+     */
+    ACTIVE_EXTERNAL_COMPACTION_POOL,
+    /**
+     * ThreadPoolExecutor used for fetching data from the TabletServers
+     */
+    SCANNER_READ_AHEAD_POOL,
+    /**
+     * ThreadPoolExecutor used for adding splits to a table
+     */
+    ADD_SPLITS_THREAD_POOL,
+    /**
+     * ThreadPoolExecutor used for fetching data from the TabletServers
+     */
+    BATCH_SCANNER_READ_AHEAD_POOL,
+    /**
+     * ThreadPoolExecutor that runs the tasks of binning mutations
+     */
+    BATCH_WRITER_BINNING_POOL,
+    /**
+     * ThreadPoolExecutor that runs the tasks of sending mutations to TabletServers
+     */
+    BATCH_WRITER_SEND_POOL,
+    /**
+     * ThreadPoolExecutor that runs clean up tasks when close is called on the ConditionalWriter
+     */
+    CONDITIONAL_WRITER_CLEANUP_TASK_POOL,
+    /**
+     * ThreadPoolExecutor responsible for loading bloom filters
+     */
+    BLOOM_FILTER_LAYER_LOADER_POOL
+  }
+
+  public static enum ScheduledThreadPoolType {
+    /**
+     * shared scheduled executor for trivial tasks
+     */
+    SHARED_GENERAL_SCHEDULED_TASK_POOL,
+    /**
+     * ScheduledThreadPoolExecutor that runs tasks for the BatchWriter to meet the users latency
+     * goals.
+     */
+    BATCH_WRITER_LATENCY_TASK_POOL,
+    /**
+     * ScheduledThreadPoolExecutor that periodically runs tasks to handle failed write mutations and
+     * send mutations to TabletServers
+     */
+    CONDITIONAL_WRITER_RETRY_POOL
+  }
+
+  private ScheduledThreadPoolExecutor sharedScheduledThreadPool = null;
+
+  public ThreadPoolExecutor newThreadPool(ThreadPoolType usage, ThreadPoolConfig config) {
+    switch (usage) {
+      case BULK_IMPORT_POOL:
+        Objects.requireNonNull(config.getNumThreads().get(), "Number of threads must be set");

Review comment:
       Some of these lines might be shorter and read more easily if you statically import requireNonNull

##########
File path: core/src/main/java/org/apache/accumulo/core/util/threads/AccumuloUncaughtExceptionHandler.java
##########
@@ -44,7 +46,10 @@ public void uncaughtException(Thread t, Throwable e) {
         // If e == OutOfMemoryError, then it's probably that another Error might be
         // thrown when trying to print to System.err.
       } finally {
-        Runtime.getRuntime().halt(-1);
+        Mode m = SingletonManager.getMode();
+        if (m != null && m.equals(Mode.SERVER)) {

Review comment:
       The mode should never be null. Also, you should use `==` for comparing enums, not `.equals`
   
   ```suggestion
           if (SingletonManager.getMode() == Mode.SERVER) {
   ```

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java
##########
@@ -386,15 +386,12 @@ public void run() {
           fatalException = new TableDeletedException(tableId.canonical());
       } catch (SampleNotPresentException e) {
         fatalException = e;
-      } catch (Exception t) {
+      } catch (Throwable t) {
         if (queryThreadPool.isShutdown())
           log.debug("Caught exception, but queryThreadPool is shutdown", t);
         else
           log.warn("Caught exception, but queryThreadPool is not shutdown", t);
         fatalException = t;
-      } catch (Throwable t) {
-        fatalException = t;
-        throw t; // let uncaught exception handler deal with the Error

Review comment:
       I like the simplicity of this earlier solution. If the user is providing their own thread pools, why wouldn't they be able to provide their own uncaught exception handler to receive these and handle them on their own, if they wish?

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java
##########
@@ -124,20 +133,26 @@ public boolean hasNext() {
     throw new NoSuchElementException();
   }
 
+  void closeThriftScanner() {
+    synchronized (scanState) {
+      // this is synchronized so its mutually exclusive with readBatch()
+      try {
+        closed = true;
+        ThriftScanner.close(scanState);
+      } catch (Exception e) {
+        LoggerFactory.getLogger(ScannerIterator.class).debug("Exception when closing scan session",
+            e);
+      }
+    }
+  }
+
   void close() {
     // run actual close operation in the background so this does not block.
     readaheadPool.execute(() -> {
-      synchronized (scanState) {
-        // this is synchronized so its mutually exclusive with readBatch()
-        try {
-          closed = true;
-          ThriftScanner.close(scanState);
-        } catch (Exception e) {
-          LoggerFactory.getLogger(ScannerIterator.class)
-              .debug("Exception when closing scan session", e);
-        }
-      }
+      closeThriftScanner();
     });
+    readaheadPoolCleanable.clean();
+    this.poolCloser.execute(() -> readaheadPool.shutdownNow());

Review comment:
       Why is a separate thread pool being used to shut down this thread pool instead of the cleaner doing that?

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/ClientThreadPools.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.accumulo.core.clientImpl;
+
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+
+public class ClientThreadPools {
+
+  public static class ThreadPoolConfig {
+
+    public static final ThreadPoolConfig EMPTY_CONFIG =
+        new ThreadPoolConfig(Optional.empty(), Optional.empty(), Optional.empty());
+
+    private final Optional<Iterable<Entry<String,String>>> configuration;

Review comment:
       Because this is an internal class only now, it might be more readable to just use the internal class, AccumuloConfiguration instead of the Iterable interface type.




-- 
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: notifications-unsubscribe@accumulo.apache.org

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