Uploaded image for project: 'Kylin'
  1. Kylin
  2. KYLIN-4263

Inapproprate exception handling causes job stuck on running status

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Minor
    • Resolution: Fixed
    • v2.6.4
    • v2.6.5, v3.1.0
    • Job Engine
    • None
    • Kylin-2.6.4-release, HBase-1.1.5

    Description

          Recently, I notice there are several jobs stuck on running status at step 3: "Extract Fact Table Distinct Columns".
      After a deep dive into the kylin.log, I found that the RetriesExhaustedException was raised when interactive with HBase.
      When the sub-task - MapReduceExecutable.doWork is invoked, the method "isDiscarded" (org/apache/kylin/engine/mr/common/MapReduceExecutable.java:151) will check the status of the task which will trigger RetriesExhaustedException. 

      2019-11-15 03:03:00,827 ERROR [Scheduler 1760924857 Job 758cd282-4b8a-8c07-2a37-d42186b74fba-174] dao.ExecutableDao:400 : error get job output id:758cd282-4b8a-8c07-2a37-d42186b74fba-02
      org.apache.hadoop.hbase.client.RetriesExhaustedException: Failed after attempts=1, exceptions:
      Fri Nov 15 03:03:00 GMT+08:00 2019, RpcRetryingCaller{globalStartTime=1573758175808, pause=100, retries=1}, java.io.IOException: Call to HOSTNAME failed on local exception: org.apache.hadoop.hbase.ipc.CallTimeoutException: Call id=143230, waitTime=5001, operationTimeout=5000 expired.
      at org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithRetries(RpcRetryingCaller.java:157)
       at org.apache.hadoop.hbase.client.HTable.get(HTable.java:865)
       at org.apache.hadoop.hbase.client.HTable.get(HTable.java:831)
       at org.apache.kylin.storage.hbase.HBaseResourceStore.internalGetFromHTable(HBaseResourceStore.java:472)
       at org.apache.kylin.storage.hbase.HBaseResourceStore.getFromHTable(HBaseResourceStore.java:450)
       at org.apache.kylin.storage.hbase.HBaseResourceStore.getResourceImpl(HBaseResourceStore.java:273)
       at org.apache.kylin.common.persistence.ResourceStore.lambda$getResourceWithRetry$0(ResourceStore.java:289)
       at org.apache.kylin.common.persistence.ExponentialBackoffRetry.doWithRetry(ExponentialBackoffRetry.java:52)
       at org.apache.kylin.common.persistence.ResourceStore.getResourceWithRetry(ResourceStore.java:289)
       at org.apache.kylin.common.persistence.ResourceStore.getResource(ResourceStore.java:266)
       at org.apache.kylin.common.persistence.ResourceStore.getResource(ResourceStore.java:257)
       at org.apache.kylin.job.dao.ExecutableDao.readJobOutputResource(ExecutableDao.java:248)
       at org.apache.kylin.job.dao.ExecutableDao.getJobOutput(ExecutableDao.java:392)
       at org.apache.kylin.job.execution.ExecutableManager.getOutput(ExecutableManager.java:169)
       at org.apache.kylin.job.execution.AbstractExecutable.getOutput(AbstractExecutable.java:373)
       at org.apache.kylin.job.execution.AbstractExecutable.isDiscarded(AbstractExecutable.java:499)
       at org.apache.kylin.engine.mr.common.MapReduceExecutable.doWork(MapReduceExecutable.java:151)
       at org.apache.kylin.job.execution.AbstractExecutable.execute(AbstractExecutable.java:167)
       at org.apache.kylin.job.execution.DefaultChainedExecutable.doWork(DefaultChainedExecutable.java:71)
       at org.apache.kylin.job.execution.AbstractExecutable.execute(AbstractExecutable.java:167)
       at org.apache.kylin.job.impl.threadpool.DefaultScheduler$JobRunner.run(DefaultScheduler.java:114)
       at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
       at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
       at java.lang.Thread.run(Thread.java:745)
      

           Ideally, the exception will be caught by "catch (Exception e) (org/apache/kylin/engine/mr/common/MapReduceExecutable.java:197)". Indeed, the exception was caught, but the "logger.error("error execute " + this.toString(), e)org/apache/kylin/engine/mr/common/MapReduceExecutable.java:198" raises a new exception.
      Let's take a look inside the "toString" method. 

      public String toString() {
         return Objects.toStringHelper(this).add("id", getId()).add("name", getName()).add("state", getStatus()).toString();
      }

          The method "getStatus" is invoked which will cause RuntimeException again (cause HBase timeout again). And then, the "RuntimeException" is caught by "catch (Exception e) org/apache/kylin/job/execution/AbstractExecutable.java:189" and raises a new exception - "ExecuteException". Let's OMIT one step AbstractExecutable.execute, it's too hard to describe cause the call stack is too deep. The "org.apache.kylin.job.execution.AbstractExecutable#execute" is invoked by "org.apache.kylin.job.execution.DefaultChainedExecutable#doWork", but there's no exception handling inside the "doWork" method.

      at org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithRetries(RpcRetryingCaller.java:157)
       at org.apache.hadoop.hbase.client.HTable.get(HTable.java:865)
       at org.apache.hadoop.hbase.client.HTable.get(HTable.java:831)
       at org.apache.kylin.storage.hbase.HBaseResourceStore.internalGetFromHTable(HBaseResourceStore.java:472)
       at org.apache.kylin.storage.hbase.HBaseResourceStore.getFromHTable(HBaseResourceStore.java:450)
       at org.apache.kylin.storage.hbase.HBaseResourceStore.getResourceImpl(HBaseResourceStore.java:273)
       at org.apache.kylin.common.persistence.ResourceStore.lambda$getResourceWithRetry$0(ResourceStore.java:289)
       at org.apache.kylin.common.persistence.ExponentialBackoffRetry.doWithRetry(ExponentialBackoffRetry.java:52)
       at org.apache.kylin.common.persistence.ResourceStore.getResourceWithRetry(ResourceStore.java:289)
       at org.apache.kylin.common.persistence.ResourceStore.getResource(ResourceStore.java:266)
       at org.apache.kylin.common.persistence.ResourceStore.getResource(ResourceStore.java:257)
       at org.apache.kylin.job.dao.ExecutableDao.readJobOutputResource(ExecutableDao.java:248)
       at org.apache.kylin.job.dao.ExecutableDao.getJobOutput(ExecutableDao.java:392)
       at org.apache.kylin.job.execution.ExecutableManager.getOutput(ExecutableManager.java:169)
       at org.apache.kylin.job.execution.AbstractExecutable.getStatus(AbstractExecutable.java:267)
       at org.apache.kylin.job.execution.AbstractExecutable.toString(AbstractExecutable.java:527)
       at org.apache.kylin.engine.mr.common.MapReduceExecutable.doWork(MapReduceExecutable.java:198)
       at org.apache.kylin.job.execution.AbstractExecutable.execute(AbstractExecutable.java:167)
       at org.apache.kylin.job.execution.DefaultChainedExecutable.doWork(DefaultChainedExecutable.java:71)
       at org.apache.kylin.job.execution.AbstractExecutable.execute(AbstractExecutable.java:167)
       at org.apache.kylin.job.impl.threadpool.DefaultScheduler$JobRunner.run(DefaultScheduler.java:114)
       at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
       at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
       at java.lang.Thread.run(Thread.java:745)

          Finally, the "ExecuteException" is passed to "org.apache.kylin.job.impl.threadpool.DefaultScheduler.JobRunner#run". The "ExecuteException" is caught by "catch (ExecuteException e)
      " and prints out error log.

      2019-11-15 03:03:04,667 ERROR [pool-9-thread-10] threadpool.DefaultScheduler:116 : ExecuteException job:52d3840d-02c7-f5cc-cb34-43ce3015d191
      org.apache.kylin.job.exception.ExecuteException: java.lang.RuntimeException: org.apache.kylin.job.exception.PersistentException: org.apache.hadoop.hbase.client.RetriesExhaustedException: Failed after attempts=1, exceptions:
      Fri Nov 15 03:03:04 GMT+08:00 2019, RpcRetryingCaller{globalStartTime=1573758184665, pause=100, retries=1}, org.apache.hadoop.hbase.CallQueueTooBigException: Call queue is full on /0:0:0:0:0:0:0:0:16020, too many items queued ?
      at org.apache.kylin.job.execution.AbstractExecutable.execute(AbstractExecutable.java:189)
       at org.apache.kylin.job.impl.threadpool.DefaultScheduler$JobRunner.run(DefaultScheduler.java:114)
       at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
       at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
       at java.lang.Thread.run(Thread.java:745)
      

           And then the "MapReduceExecutable" sub-task is removed by "context.removeRunningJob(executable)". However the task is removed, BUT its status still RUNNING. In this situation, the status of the task is inconsistent.

      2019-11-15 03:03:04,669 INFO [FetcherRunner 2113735343-72] threadpool.DefaultFetcherRunner:85 : Job Fetcher: 3 should running, 3 actual running, 0 stopped, 0 ready, 367 already succeed, 9 error, 5 discarded, 1 others

           I think the is a BUG, caused by inapproprate exception handling. For a quick fix, we should add a try-catch statement in the toString method, because throws an exception in toString method is bad behavior. I just created a patch for v2.6.4-release branch. It should works.

      Attachments

        Activity

          People

            pwrliang Liang Geng
            pwrliang Liang Geng
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: