Return-Path: X-Original-To: apmail-drill-commits-archive@www.apache.org Delivered-To: apmail-drill-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 10897186F2 for ; Wed, 2 Dec 2015 23:25:36 +0000 (UTC) Received: (qmail 6893 invoked by uid 500); 2 Dec 2015 23:25:32 -0000 Delivered-To: apmail-drill-commits-archive@drill.apache.org Received: (qmail 6859 invoked by uid 500); 2 Dec 2015 23:25:32 -0000 Mailing-List: contact commits-help@drill.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: commits@drill.apache.org Delivered-To: mailing list commits@drill.apache.org Received: (qmail 6850 invoked by uid 99); 2 Dec 2015 23:25:32 -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; Wed, 02 Dec 2015 23:25:32 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id B6E66E67D2; Wed, 2 Dec 2015 23:25:32 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: venki@apache.org To: commits@drill.apache.org Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: drill git commit: DRILL-4124: Make all uses of AutoCloseables use addSuppressed exceptions to avoid noise in logs Date: Wed, 2 Dec 2015 23:25:32 +0000 (UTC) Repository: drill Updated Branches: refs/heads/master 8f56250ae -> 46c47a24f DRILL-4124: Make all uses of AutoCloseables use addSuppressed exceptions to avoid noise in logs This closes #281 Project: http://git-wip-us.apache.org/repos/asf/drill/repo Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/46c47a24 Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/46c47a24 Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/46c47a24 Branch: refs/heads/master Commit: 46c47a24f67e53146521d50062180c4aa902f687 Parents: 8f56250 Author: Julien Le Dem Authored: Mon Nov 23 16:25:52 2015 -0800 Committer: vkorukanti Committed: Wed Dec 2 14:49:52 2015 -0800 ---------------------------------------------------------------------- .../org/apache/drill/common/AutoCloseables.java | 63 +++++++++++++------- .../hive/HiveDrillNativeScanBatchCreator.java | 11 +--- .../drill/exec/store/jdbc/JdbcRecordReader.java | 6 +- .../drill/exec/physical/impl/ImplCreator.java | 6 +- .../physical/impl/xsort/ExternalSortBatch.java | 35 ++--------- .../org/apache/drill/exec/server/Drillbit.java | 18 +++--- .../exec/server/TestDrillbitResilience.java | 7 ++- 7 files changed, 69 insertions(+), 77 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/drill/blob/46c47a24/common/src/main/java/org/apache/drill/common/AutoCloseables.java ---------------------------------------------------------------------- diff --git a/common/src/main/java/org/apache/drill/common/AutoCloseables.java b/common/src/main/java/org/apache/drill/common/AutoCloseables.java index 3c4aa23..43cbbf5 100644 --- a/common/src/main/java/org/apache/drill/common/AutoCloseables.java +++ b/common/src/main/java/org/apache/drill/common/AutoCloseables.java @@ -17,41 +17,64 @@ */ package org.apache.drill.common; -import org.slf4j.Logger; +import java.util.Arrays; +import java.util.Collection; /** * Utilities for AutoCloseable classes. */ public class AutoCloseables { + + public static AutoCloseable all(final Collection autoCloseables) { + return new AutoCloseable() { + @Override + public void close() throws Exception { + AutoCloseables.close(autoCloseables); + } + }; + } + /** - * Close an {@link AutoCloseable}, catching and logging any exceptions at - * INFO level. - * - *

This can be dangerous if there is any possibility of recovery. See - * the - * notes regarding the deprecation of Guava's - * {@link com.google.common.io.Closeables#closeQuietly}. - * - * @param ac the AutoCloseable to close - * @param logger the logger to use to record the exception if there was one + * Closes all autoCloseables if not null and suppresses exceptions by adding them to t + * @param t the throwable to add suppressed exception to + * @param autoCloseables the closeables to close */ - public static void close(final AutoCloseable ac, final Logger logger) { - if (ac == null) { - return; - } + public static void close(Throwable t, AutoCloseable... autoCloseables) { + close(t, Arrays.asList(autoCloseables)); + } + /** + * Closes all autoCloseables if not null and suppresses exceptions by adding them to t + * @param t the throwable to add suppressed exception to + * @param autoCloseables the closeables to close + */ + public static void close(Throwable t, Collection autoCloseables) { try { - ac.close(); - } catch(Exception e) { - logger.warn("Failure on close(): {}", e); + close(autoCloseables); + } catch (Exception e) { + t.addSuppressed(e); } } - public static void close(AutoCloseable[] ac) throws Exception { + /** + * Closes all autoCloseables if not null and suppresses subsequent exceptions if more than one + * @param autoCloseables the closeables to close + */ + public static void close(AutoCloseable... autoCloseables) throws Exception { + close(Arrays.asList(autoCloseables)); + } + + /** + * Closes all autoCloseables if not null and suppresses subsequent exceptions if more than one + * @param autoCloseables the closeables to close + */ + public static void close(Collection ac) throws Exception { Exception topLevelException = null; for (AutoCloseable closeable : ac) { try { - closeable.close(); + if (closeable != null) { + closeable.close(); + } } catch (Exception e) { if (topLevelException == null) { topLevelException = e; http://git-wip-us.apache.org/repos/asf/drill/blob/46c47a24/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java ---------------------------------------------------------------------- diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java index 1e6ad99..1a76136 100644 --- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java +++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java @@ -98,7 +98,6 @@ public class HiveDrillNativeScanBatchCreator implements BatchCreator readers = Lists.newArrayList(); final Configuration conf = getConf(hiveConfigOverride); @@ -139,15 +138,9 @@ public class HiveDrillNativeScanBatchCreator implements BatchCreator { http://git-wip-us.apache.org/repos/asf/drill/blob/46c47a24/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java index e7ee84d..8103030 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java @@ -90,13 +90,9 @@ public class ImplCreator { return rootExec; } catch(Exception e) { - e.printStackTrace(); + AutoCloseables.close(e, creator.getOperators()); context.fail(e); - for(final CloseableRecordBatch crb : creator.getOperators()) { - AutoCloseables.close(crb, logger); - } } - return null; } http://git-wip-us.apache.org/repos/asf/drill/blob/46c47a24/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java index a99a986..1ac82bf 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java @@ -567,11 +567,9 @@ public class ExternalSortBatch extends AbstractRecordBatch { String outputFile = Joiner.on("/").join(dirs.next(), fileName, spillCount++); stats.setLongStat(Metric.SPILL_COUNT, spillCount); - BatchGroup newGroup = new BatchGroup(c1, fs, outputFile, oContext); - boolean threw = true; // true if an exception is thrown in the try block below - logger.info("Merging and spilling to {}", outputFile); - try { + try (AutoCloseable a = AutoCloseables.all(batchGroupList)) { + logger.info("Merging and spilling to {}", outputFile); while ((count = copier.next(targetRecordCount)) > 0) { outputContainer.buildSchema(BatchSchema.SelectionVectorMode.NONE); outputContainer.setRecordCount(count); @@ -580,18 +578,14 @@ public class ExternalSortBatch extends AbstractRecordBatch { } injector.injectChecked(context.getExecutionControls(), INTERRUPTION_WHILE_SPILLING, IOException.class); newGroup.closeOutputStream(); - threw = false; // this should always be the last statement of this try block to make sure we cleanup properly - } catch (IOException e) { + } catch (Exception e) { + // we only need to cleanup newGroup if spill failed + AutoCloseables.close(e, newGroup); throw UserException.resourceError(e) .message("External Sort encountered an error while spilling to disk") .build(logger); } finally { hyperBatch.clear(); - cleanAfterMergeAndSpill(batchGroupList, threw); - if (threw) { - // we only need to cleanup newGroup if spill failed - AutoCloseables.close(newGroup, logger); - } } takeOwnership(c1); // transfer ownership from copier allocator to external sort allocator long bufSize = getBufferSize(c1); @@ -601,25 +595,6 @@ public class ExternalSortBatch extends AbstractRecordBatch { return newGroup; } - /** - * Make sure we cleanup properly after merge and spill.
If there was any error during the spilling, - * we cleanup the resources silently, otherwise we throw any exception we hit during the cleanup - * - * @param batchGroups spilled batch groups - * @param silently true to log any exception that happens during cleanup, false to throw it - */ - private void cleanAfterMergeAndSpill(final List batchGroups, boolean silently) { - try { - AutoCloseables.close(batchGroups.toArray(new BatchGroup[batchGroups.size()])); - } catch (Exception e) { - if (silently) { - logger.warn("Error while cleaning up after merge and spill", e); - } else { - throw new RuntimeException("Error while cleaning up after merge and spill", e); - } - } - } - private void takeOwnership(VectorAccessible batch) { for (VectorWrapper w : batch) { DrillBuf[] bufs = w.getValueVector().getBuffers(false); http://git-wip-us.apache.org/repos/asf/drill/blob/46c47a24/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java index 9267003..feebbb2 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java @@ -245,13 +245,17 @@ public class Drillbit implements AutoCloseable { Thread.currentThread().interrupt(); } - // TODO these should use a DeferredException - AutoCloseables.close(webServer, logger); - AutoCloseables.close(engine, logger); - AutoCloseables.close(storeProvider, logger); - AutoCloseables.close(coord, logger); - AutoCloseables.close(manager, logger); - AutoCloseables.close(context, logger); + try { + AutoCloseables.close( + webServer, + engine, + storeProvider, + coord, + manager, + context); + } catch(Exception e) { + logger.warn("Failure on close()", e); + } logger.info("Shutdown completed ({} ms).", w.elapsed(TimeUnit.MILLISECONDS)); isClosed = true; http://git-wip-us.apache.org/repos/asf/drill/blob/46c47a24/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java index ae3c97f..4200073 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestDrillbitResilience.java @@ -34,7 +34,6 @@ import org.apache.commons.math3.util.Pair; import org.apache.drill.BaseTestQuery; import org.apache.drill.QueryTestUtil; import org.apache.drill.SingleRowListener; -import org.apache.drill.common.AutoCloseables; import org.apache.drill.common.concurrent.ExtendedLatch; import org.apache.drill.common.DrillAutoCloseables; import org.apache.drill.common.config.DrillConfig; @@ -211,7 +210,11 @@ public class TestDrillbitResilience extends DrillTest { stopAllDrillbits(); if (remoteServiceSet != null) { - AutoCloseables.close(remoteServiceSet, logger); + try { + remoteServiceSet.close(); + } catch (Exception e) { + logger.warn("Failure on close()", e); + } remoteServiceSet = null; }