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/08 19:54:52 UTC

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

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