Uploaded image for project: 'Hive'
  1. Hive
  2. HIVE-20627

Concurrent async queries intermittently fails with LockException and cause memory leak.

    XMLWordPrintableJSON

Details

    Description

      When multiple async queries are executed from same session, it leads to multiple async query execution DAGs share the same Hive object which is set by caller for all threads. In case of loading dynamic partitions, it creates MoveTask which re-creates the Hive object and closes the shared Hive object which causes metastore connection issues for other async execution thread who still access it. This is also seen if ReplDumpTask and ReplLoadTask are part of the DAG.

      Call Stack:

      2018-09-16T04:38:04,280 ERROR [load-dynamic-partitions-7]: metadata.Hive (Hive.java:call(2436)) - Exception when loading partition with parameters partPath=hdfs://mycluster/warehouse/tablespace/managed/hive/tbl_3bcvvdubni/.hive-staging_hive_2018-09-16_04-35-50_708_7776079613819042057-1147/-ext-10000/age=55, table=tbl_3bcvvdubni, partSpec={age=55}, loadFileType=KEEP_EXISTING, listBucketingLevel=0, isAcid=true, hasFollowingStatsTask=true
      org.apache.hadoop.hive.ql.lockmgr.LockException: Error communicating with the metastore
      at org.apache.hadoop.hive.ql.lockmgr.DbTxnManager.getValidWriteIds(DbTxnManager.java:714) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at org.apache.hadoop.hive.ql.io.AcidUtils.getTableValidWriteIdListWithTxnList(AcidUtils.java:1791) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at org.apache.hadoop.hive.ql.io.AcidUtils.getTableSnapshot(AcidUtils.java:1756) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at org.apache.hadoop.hive.ql.io.AcidUtils.getTableSnapshot(AcidUtils.java:1714) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at org.apache.hadoop.hive.ql.metadata.Hive.loadPartition(Hive.java:1976) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at org.apache.hadoop.hive.ql.metadata.Hive$5.call(Hive.java:2415) [hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at org.apache.hadoop.hive.ql.metadata.Hive$5.call(Hive.java:2406) [hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_171]
      at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_171]
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_171]
      at java.lang.Thread.run(Thread.java:748) [?:1.8.0_171]
      Caused by: org.apache.thrift.protocol.TProtocolException: Required field 'validTxnList' is unset! Struct:GetValidWriteIdsRequest(fullTableNames:[default.tbl_3bcvvdubni], validTxnList:null)
      at org.apache.hadoop.hive.metastore.api.GetValidWriteIdsRequest.validate(GetValidWriteIdsRequest.java:396) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$get_valid_write_ids_args.validate(ThriftHiveMetastore.java) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$get_valid_write_ids_args$get_valid_write_ids_argsStandardScheme.write(ThriftHiveMetastore.java) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$get_valid_write_ids_args$get_valid_write_ids_argsStandardScheme.write(ThriftHiveMetastore.java) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$get_valid_write_ids_args.write(ThriftHiveMetastore.java) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at org.apache.thrift.TServiceClient.sendBase(TServiceClient.java:71) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at org.apache.thrift.TServiceClient.sendBase(TServiceClient.java:62) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.send_get_valid_write_ids(ThriftHiveMetastore.java:5443) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.get_valid_write_ids(ThriftHiveMetastore.java:5435) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.getValidWriteIds(HiveMetaStoreClient.java:2589) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at sun.reflect.GeneratedMethodAccessor125.invoke(Unknown Source) ~[?:?]
      at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) ~[?:1.8.0_171]
      at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_171]
      at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:212) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at com.sun.proxy.$Proxy57.getValidWriteIds(Unknown Source) ~[?:?]
      at sun.reflect.GeneratedMethodAccessor125.invoke(Unknown Source) ~[?:?]
      at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) ~[?:1.8.0_171]
      at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_171]
      at org.apache.hadoop.hive.metastore.HiveMetaStoreClient$SynchronizedHandler.invoke(HiveMetaStoreClient.java:2934) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      at com.sun.proxy.$Proxy57.getValidWriteIds(Unknown Source) ~[?:?]
      at org.apache.hadoop.hive.ql.lockmgr.DbTxnManager.getValidWriteIds(DbTxnManager.java:712) ~[hive-exec-3.1.0.3.0.1.0-184.jar:3.1.0.3.0.1.0-184]
      ... 10 more

      Root cause:
      For Async query execution from SQLOperation.runInternal, we set the Thread local Hive object for all the child threads as parentHive (parentSession.getSessionHive())

      @Override
       public void run() {
       PrivilegedExceptionAction<Object> doAsAction = new PrivilegedExceptionAction<Object>() {
       @Override
       public Object run() throws HiveSQLException {
       Hive.set(parentHive); // Setting parentHive for all async operations.
       // TODO: can this result in cross-thread reuse of session state?
       SessionState.setCurrentSessionState(parentSessionState);
       PerfLogger.setPerfLogger(parentPerfLogger);
       LogUtils.registerLoggingContext(queryState.getConf());
       try {
       if (asyncPrepare) {
       prepare(queryState);
       }
       runQuery();
       } catch (HiveSQLException e) {
       // TODO: why do we invent our own error path op top of the one from Future.get?
       setOperationException(e);
       LOG.error("Error running hive query: ", e);
       } finally {
       LogUtils.unregisterLoggingContext();
       }
       return null;
       }
       };
      

      Now, when async execution in progress and if one of the thread re-creates the Hive object, it closes the parentHive object first which impacts other threads using it and hence conf object it refers too gets cleaned up and hence we get null for VALID_TXNS_KEY value.

      private static Hive create(HiveConf c, boolean needsRefresh, Hive db, boolean doRegisterAllFns)
       throws HiveException {
       if (db != null) {
       LOG.debug("Creating new db. db = " + db + ", needsRefresh = " + needsRefresh +
       ", db.isCurrentUserOwner = " + db.isCurrentUserOwner());
       db.close();
       }
       closeCurrent();
       if (c == null) {
       c = createHiveConf();
       }
       c.set("fs.scheme.class", "dfs");
       Hive newdb = new Hive(c, doRegisterAllFns);
       hiveDB.set(newdb);
       return newdb;
       }
      

      Fix:
      We shouldn't clean the old Hive object if it is shared by multiple threads. Shall use a flag to know this.

      Memory leak issue:
      Memory leak is found if one of the threads from Hive.loadDynamicPartitions throw exception. rawStoreMap is used to store rawStore objects which has to be cleaned. In this case, it is populated only in success flow but if there are exceptions, it is not and hence there is a leak.

      futures.add(pool.submit(new Callable<Void>() {
       @Override
       public Void call() throws Exception {
       try {
       // move file would require session details (needCopy() invokes SessionState.get)
       SessionState.setCurrentSessionState(parentSession);
       LOG.info("New loading path = " + partPath + " with partSpec " + fullPartSpec);
      
      // load the partition
       Partition newPartition = loadPartition(partPath, tbl, fullPartSpec, loadFileType,
       true, false, numLB > 0, false, isAcid, hasFollowingStatsTask, writeId, stmtId,
       isInsertOverwrite);
       partitionsMap.put(fullPartSpec, newPartition);
      
      if (inPlaceEligible) {
       synchronized (ps) {
       InPlaceUpdate.rePositionCursor(ps);
       partitionsLoaded.incrementAndGet();
       InPlaceUpdate.reprintLine(ps, "Loaded : " + partitionsLoaded.get() + "/"
       + partsToLoad + " partitions.");
       }
       }
       // Add embedded rawstore, so we can cleanup later to avoid memory leak
       if (getMSC().isLocalMetaStore()) {
       if (!rawStoreMap.containsKey(Thread.currentThread().getId())) {
       rawStoreMap.put(Thread.currentThread().getId(), HiveMetaStore.HMSHandler.getRawStore());
       }
       }
       return null;
       } catch (Exception t) {
       }
      

      Attachments

        1. HIVE-20627.01.patch
          7 kB
          Sankar Hariappan

        Issue Links

          Activity

            People

              sankarh Sankar Hariappan
              sankarh Sankar Hariappan
              Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: