diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java index b39236ef8b9a..6b57edd8d929 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java @@ -68,6 +68,7 @@ public class Worker extends RemoteCompactorThread implements MetaStoreThread { private String workerName; private final CompactorFactory compactorFactory; + private CompactionInfo ci; public Worker() { compactorFactory = CompactorFactory.getInstance(); @@ -99,15 +100,16 @@ public void run() { try { launchedJob = singleRun.get(timeout, TimeUnit.MILLISECONDS); } catch (TimeoutException te) { - LOG.info("Timeout during executing compaction", te); + LOG.error("Timeout during executing compaction", te); // Cancel the job, and recreate the Executor as well, so we can be sure that we have an available thread // even if we can not interrupt the task somehow. (Trade possible resource hogging for compactor stability) singleRun.cancel(true); + markFailed(ci, "Timeout during executing compaction"); executor.shutdownNow(); executor = getTimeoutHandlingExecutor(); err = true; } catch (ExecutionException e) { - LOG.info("Exception during executing compaction", e); + LOG.error("Exception during executing compaction", e); err = true; } catch (InterruptedException ie) { Thread.currentThread().interrupt(); @@ -182,7 +184,6 @@ protected Boolean findNextCompactionAndExecute(boolean collectGenericStats, bool // so wrap it in a big catch Throwable statement. PerfLogger perfLogger = SessionState.getPerfLogger(false); String workerMetric = null; - CompactionInfo ci = null; Table table = null; CompactionService compactionService = null; boolean compactionResult = false;