You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hive.apache.org by "Peter Vary (JIRA)" <ji...@apache.org> on 2018/10/01 12:55:00 UTC

[jira] [Commented] (HIVE-20627) Concurrent async queries intermittently fails with LockException and cause memory leak.

    [ https://issues.apache.org/jira/browse/HIVE-20627?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16633957#comment-16633957 ] 

Peter Vary commented on HIVE-20627:
-----------------------------------

[~sankarh]: Very nice description! Thanks for it and for the patch too!
Reading it I think there could be a problem the other way around too:
* Do you know what will happen with Hive objects used by the async threads if the master threads get another asynchronous query which requires HiveMetaStoreClient reinitialization, like {{set hive.metastore.client.socket.timeout=500ms}}?

My guess is in this case the {{Hive.isCompatible}} returns false and the original Hive instance is closed and a new one is created on the master thread. This might cause the same problem for the async threads.
Sadly I did not have time to test this out, but probably you already have better understanding of this part of the code, and can confirm or refute my theory.

Thanks,
Peter

> Concurrent async queries intermittently fails with LockException and cause memory leak.
> ---------------------------------------------------------------------------------------
>
>                 Key: HIVE-20627
>                 URL: https://issues.apache.org/jira/browse/HIVE-20627
>             Project: Hive
>          Issue Type: Bug
>          Components: HiveServer2, Transactions
>    Affects Versions: 4.0.0
>            Reporter: Sankar Hariappan
>            Assignee: Sankar Hariappan
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 4.0.0
>
>         Attachments: HIVE-20627.01.patch
>
>
> 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:*
> {code:java}
> 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{code}
> *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())
> {code:java}
> @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;
>  }
>  };
> {code}
> 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.
> {code:java}
> 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;
>  }
> {code}
> *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.
> {code:java}
> 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) {
>  }
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)