Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id A2F7C200C5E for ; Sat, 8 Apr 2017 02:08:23 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id A1C24160BA5; Sat, 8 Apr 2017 00:08:23 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id F24D9160B97 for ; Sat, 8 Apr 2017 02:08:22 +0200 (CEST) Received: (qmail 42846 invoked by uid 500); 8 Apr 2017 00:08:22 -0000 Mailing-List: contact commits-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@accumulo.apache.org Delivered-To: mailing list commits@accumulo.apache.org Received: (qmail 42834 invoked by uid 99); 8 Apr 2017 00:08:22 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 08 Apr 2017 00:08:22 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 1A2B7E00C5; Sat, 8 Apr 2017 00:08:22 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: kturner@apache.org To: commits@accumulo.apache.org Date: Sat, 08 Apr 2017 00:08:22 -0000 Message-Id: <455aba6f27084ff7b6a0463f646a49f4@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [1/2] accumulo git commit: ACCUMULO-4619 fix split hanging on Error archived-at: Sat, 08 Apr 2017 00:08:23 -0000 Repository: accumulo Updated Branches: refs/heads/1.8 8d6b7299e -> ba3dea165 ACCUMULO-4619 fix split hanging on Error Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/8c0f03ac Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/8c0f03ac Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/8c0f03ac Branch: refs/heads/1.8 Commit: 8c0f03ac2d8e06b220e2882d54914f2034625e15 Parents: 4ba5faf Author: Keith Turner Authored: Fri Apr 7 18:43:55 2017 -0400 Committer: Keith Turner Committed: Fri Apr 7 18:43:55 2017 -0400 ---------------------------------------------------------------------- .../core/client/impl/TableOperationsImpl.java | 39 +++++++++++--------- 1 file changed, 21 insertions(+), 18 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/8c0f03ac/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java index 1a7a38c..42afb9d 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java @@ -318,9 +318,9 @@ public class TableOperationsImpl extends TableOperationsHelper { private String tableId; private ExecutorService executor; private CountDownLatch latch; - private AtomicReference exception; + private AtomicReference exception; - SplitEnv(String tableName, String tableId, ExecutorService executor, CountDownLatch latch, AtomicReference exception) { + SplitEnv(String tableName, String tableId, ExecutorService executor, CountDownLatch latch, AtomicReference exception) { this.tableName = tableName; this.tableId = tableId; this.executor = executor; @@ -359,11 +359,11 @@ public class TableOperationsImpl extends TableOperationsHelper { addSplits(env.tableName, new TreeSet<>(splits.subList(mid, mid + 1)), env.tableId); env.latch.countDown(); - env.executor.submit(new SplitTask(env, splits.subList(0, mid))); - env.executor.submit(new SplitTask(env, splits.subList(mid + 1, splits.size()))); + env.executor.execute(new SplitTask(env, splits.subList(0, mid))); + env.executor.execute(new SplitTask(env, splits.subList(mid + 1, splits.size()))); - } catch (Exception e) { - env.exception.compareAndSet(null, e); + } catch (Throwable t) { + env.exception.compareAndSet(null, t); } } @@ -379,26 +379,29 @@ public class TableOperationsImpl extends TableOperationsHelper { Collections.sort(splits); CountDownLatch latch = new CountDownLatch(splits.size()); - AtomicReference exception = new AtomicReference<>(null); + AtomicReference exception = new AtomicReference<>(null); ExecutorService executor = Executors.newFixedThreadPool(16, new NamingThreadFactory("addSplits")); try { - executor.submit(new SplitTask(new SplitEnv(tableName, tableId, executor, latch, exception), splits)); + executor.execute(new SplitTask(new SplitEnv(tableName, tableId, executor, latch, exception), splits)); while (!latch.await(100, TimeUnit.MILLISECONDS)) { if (exception.get() != null) { executor.shutdownNow(); - Exception excep = exception.get(); - if (excep instanceof TableNotFoundException) - throw (TableNotFoundException) excep; - else if (excep instanceof AccumuloException) - throw (AccumuloException) excep; - else if (excep instanceof AccumuloSecurityException) - throw (AccumuloSecurityException) excep; - else if (excep instanceof RuntimeException) - throw (RuntimeException) excep; + Throwable excep = exception.get(); + // Below all exceptions are wrapped and rethrown. This is done so that the user knows what code path got them here. If the wrapping was not done, the + // user would only have the stack trace for the background thread. + if (excep instanceof TableNotFoundException) { + TableNotFoundException tnfe = (TableNotFoundException) excep; + throw new TableNotFoundException(tableId, tableName, "Table not found by background thread", tnfe); + } else if (excep instanceof AccumuloSecurityException) { + // base == background accumulo security exception + AccumuloSecurityException base = (AccumuloSecurityException) excep; + throw new AccumuloSecurityException(base.getUser(), base.asThriftException().getCode(), base.getTableInfo(), excep); + } else if (excep instanceof Error) + throw new Error(excep); else - throw new RuntimeException(excep); + throw new AccumuloException(excep); } } } catch (InterruptedException e) {