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

Failed to read big resource /dict/xxxx at "Build Dimension Dictionary" Step

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • v2.6.0
    • v3.0.0-beta, v2.6.4
    • Metadata
    • None

    Description

      At the version of Kylin 2.6.0, kylin team has introduce an important refactor of Kylin's Metadata Store, which add a lot of enhancement such as upload/download metadata concurrently, store metadata with JDBC etc. Please refer to https://issues.apache.org/jira/browse/KYLIN-3671 for detail.

      When kylin want to save a big resource(such as dict or snapshot) into metadata store, it won't store it into metadata store(HBase or RDBMS) directly. Instead, kylin will first save it into HDFS(Step 1), and then write a empty byte array as marker into metadata store(Step 2) . If first action succeed and second action failed, a rollback method will be called to revert modification for HDFS files. We could regard it as a complete and atomic transaction.

      Here is part of the source code added in KYLIN-3671. Check it at https://github.com/apache/kylin/blob/8737bc1f555a2789a67462c8f8420b6ab3be97ce/core-common/src/main/java/org/apache/kylin/common/persistence/PushdownResourceStore.java#L58 .

      final void putBigResource(String resPath, ContentWriter content, long newTS) throws IOException {
          // pushdown the big resource to DFS file
          RollbackablePushdown pushdown = writePushdown(resPath, content); // Step 1: write big resource into HDFS
      
          try {
              // write a marker in resource store, to indicate the resource is now available
              logger.debug("Writing marker for big resource {}", resPath);
              putResourceWithRetry(resPath, ContentWriter.create(BytesUtil.EMPTY_BYTE_ARRAY), newTS); // Step 2: write marker into HBase/RDBMS
      
          } catch (Throwable ex) {
              pushdown.rollback();
              throw ex;
          } finally {
              pushdown.close();
          }
      }
      

      But in some case, both step 1 and step 2 succeed but an exception still throwed in step 2, the rollback won't clear marker written in Step 2, which break the atomicity of this put action, thus cause the FileNotFoundException when Kylin want to read that dict later.

      Here is part of reporter's kylin.log of incomplete rollback action.

       2019-08-29 05:13:51,237 INFO  [Scheduler 169045403 Job ca4a4a08-54e2-b922-70bb-2aa2bf58709f-492] dict.DictionaryManager:388 : Saving dictionary at /dict/KYLIN_VIEW.USER_SECRET_TABLE/COUNTRY/66292068-e8eb-975a-3e44-b56c933c14cc.dict
      2019-08-29 05:13:51,238 DEBUG [Scheduler 169045403 Job ca4a4a08-54e2-b922-70bb-2aa2bf58709f-492] persistence.HDFSResourceStore:98 : Writing pushdown file /kylin/kylin_metadata/resources/dict/KYLIN_VIEW.USER_SECRET_TABLE/COUNTRY/66292068-e8eb-975a-3e44-b56c933c14cc.dict.temp.-1798610090
      2019-08-29 05:13:51,256 DEBUG [Scheduler 169045403 Job ca4a4a08-54e2-b922-70bb-2aa2bf58709f-492] persistence.HDFSResourceStore:117 : Move /kylin/kylin_metadata/resources/dict/KYLIN_VIEW.USER_SECRET_TABLE/COUNTRY/66292068-e8eb-975a-3e44-b56c933c14cc.dict.temp.-1798610090 to /kylin/kylin_metadata/resources/dict/KYLIN_VIEW.USER_SECRET_TABLE/COUNTRY/66292068-e8eb-975a-3e44-b56c933c14cc.dict
      2019-08-29 05:13:51,258 DEBUG [Scheduler 169045403 Job ca4a4a08-54e2-b922-70bb-2aa2bf58709f-492] persistence.HDFSResourceStore:65 : Writing marker for big resource /dict/KYLIN_VIEW.USER_SECRET_TABLE/COUNTRY/66292068-e8eb-975a-3e44-b56c933c14cc.dict
      
      2019-08-29 05:13:56,263 WARN  [hconnection-0x56f3258e-shared--pool10944-t54867] client.AsyncProcess:1263 : #10545, table=kylin_metadata, attempt=1/1 failed=1ops, last exception: java.io.IOException: Call to tx-dn41.data/10.14.243.51:60020 failed on local exception: org.apache.hadoop.hbase.ipc.CallTimeoutException: Call id=2662317, waitTime=5001, operationTimeout=5000 expired. on tx-dn41.data,60020,1565943919204, tracking started Thu Aug 29 05:13:51 GMT+08:00 2019; not retrying 1 - final failure
      2019-08-29 05:13:56,266 ERROR [Scheduler 169045403 Job ca4a4a08-54e2-b922-70bb-2aa2bf58709f-492] persistence.HDFSResourceStore:134 : Rollback /kylin/kylin_metadata/resources/dict/KYLIN_VIEW.USER_SECRET_TABLE/COUNTRY/66292068-e8eb-975a-3e44-b56c933c14cc.dict from <empty>
      2019-08-29 05:13:56,274 ERROR [Scheduler 169045403 Job ca4a4a08-54e2-b922-70bb-2aa2bf58709f-492] common.HadoopShellExecutable:65 : error execute HadoopShellExecutable{id=ca4a4a08-54e2-b922-70bb-2aa2bf58709f-03, name=Build Dimension Dictionary, state=RUNNING}
      2019-08-29 05:13:56,274 INFO  [Scheduler 169045403 Job ca4a4a08-54e2-b922-70bb-2aa2bf58709f-492] execution.AbstractExecutable:162 : Retry 1
      

      Here is part of reporter's kylin.log of reading a non-exist dict in HDFS in "Build Dimension Dictionary" Step.

      2019-08-29 14:54:59,602 INFO  [Scheduler 343338459 Job af4b847d-afa6-3729-4c19-03a5db08447b-498] steps.CreateDictionaryJob:110 : DictionaryProvider read dict from file: hdfs://CDH-cluster-main/kylin/kylin_metadata/kylin-af4b847d-afa6-3729-4c19-03a5db08447b/209_new_device/fact_distinct_columns/USER_SECRET_TABLE.COUNTRY/COUNTRY.rldict-r-00004
      2019-08-29 14:54:59,602 DEBUG [Scheduler 343338459 Job af4b847d-afa6-3729-4c19-03a5db08447b-498] cli.DictionaryGeneratorCLI:73 : Dict for 'COUNTRY' has already been built, save it
      2019-08-29 14:54:59,720 ERROR [Scheduler 343338459 Job af4b847d-afa6-3729-4c19-03a5db08447b-498] persistence.ResourceStore:233 : Error reading resource /dict/KYLIN_VIEW.USER_SECRET_TABLE/COUNTRY/66292068-e8eb-975a-3e44-b56c933c14cc.dict
      java.io.IOException: Failed to read big resource /dict/KYLIN_VIEW.USER_SECRET_TABLE/COUNTRY/66292068-e8eb-975a-3e44-b56c933c14cc.dict
             at org.apache.kylin.common.persistence.PushdownResourceStore.openPushdown(PushdownResourceStore.java:176)
             at org.apache.kylin.storage.hbase.HBaseResourceStore.getInputStream(HBaseResourceStore.java:256)
             at org.apache.kylin.storage.hbase.HBaseResourceStore.rawResource(HBaseResourceStore.java:226)
             at org.apache.kylin.storage.hbase.HBaseResourceStore.access$000(HBaseResourceStore.java:64)
             at org.apache.kylin.storage.hbase.HBaseResourceStore$1.visit(HBaseResourceStore.java:159)
             at org.apache.kylin.storage.hbase.HBaseResourceStore.visitFolder(HBaseResourceStore.java:204)
             at org.apache.kylin.storage.hbase.HBaseResourceStore.visitFolderImpl(HBaseResourceStore.java:152)
             at org.apache.kylin.common.persistence.ResourceStore.visitFolderInner(ResourceStore.java:689)
             at org.apache.kylin.common.persistence.ResourceStore.visitFolderAndContent(ResourceStore.java:675)
             at org.apache.kylin.common.persistence.ResourceStore$2.call(ResourceStore.java:224)
             at org.apache.kylin.common.persistence.ResourceStore$2.call(ResourceStore.java:220)
             at org.apache.kylin.common.persistence.ExponentialBackoffRetry.doWithRetry(ExponentialBackoffRetry.java:52)
             at org.apache.kylin.common.persistence.ResourceStore.getAllResources(ResourceStore.java:220)
             at org.apache.kylin.common.persistence.ResourceStore.getAllResources(ResourceStore.java:209)
             at org.apache.kylin.dict.DictionaryManager.checkDupByInfo(DictionaryManager.java:334)
             at org.apache.kylin.dict.DictionaryManager.saveDictionary(DictionaryManager.java:314)
             at org.apache.kylin.cube.CubeManager$DictionaryAssist.saveDictionary(CubeManager.java:1127)
             at org.apache.kylin.cube.CubeManager.saveDictionary(CubeManager.java:1089)
             at org.apache.kylin.cube.cli.DictionaryGeneratorCLI.processSegment(DictionaryGeneratorCLI.java:74)
             at org.apache.kylin.cube.cli.DictionaryGeneratorCLI.processSegment(DictionaryGeneratorCLI.java:55)
             at org.apache.kylin.engine.mr.steps.CreateDictionaryJob.run(CreateDictionaryJob.java:73)
             at org.apache.kylin.engine.mr.MRUtil.runMRJob(MRUtil.java:93)
             at org.apache.kylin.engine.mr.common.HadoopShellExecutable.doWork(HadoopShellExecutable.java:63)
             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)
      Caused by: java.io.FileNotFoundException: /kylin/kylin_metadata/resources/dict/KYLIN_VIEW.USER_SECRET_TABLE/COUNTRY/66292068-e8eb-975a-3e44-b56c933c14cc.dict  (FS: DFS[DFSClient[clientName=DFSClient_NONMAPREDUCE_-784809092_27, ugi=kylin (auth:SIMPLE)]])
             at org.apache.kylin.common.persistence.PushdownResourceStore.openPushdown(PushdownResourceStore.java:173)
             ... 29 more
      

      This often happen in Build Step 4: Build Dimension Dictionary. And this incomplete metadata entry will cause same failure(FileNotFoundException) of ALL following cube rebuild job.

      As far as I can see, my workaround should be delete that marker. Since this is a broken metadata entry, deletion won't make damage. After the deletion, following rebuilt job will succeed.

      This is some related report mail :
      1. http://apache-kylin.74782.x6.nabble.com/How-to-repair-the-cube-that-it-lost-someone-dictionary-td12989.html
      2. http://mail-archives.apache.org/mod_mbox/kylin-user/201908.mbox/%3c4bcca64e.4af8.16cdb473a62.Coremail.itzhangqiang@163.com%3e

      Attachments

        Issue Links

          Activity

            People

              hit_lacus Xiaoxiang Yu
              hit_lacus Xiaoxiang Yu
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: