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

Naveen Gangam commented on HIVE-12250:
--------------------------------------

According to 
https://hbase.apache.org/0.94/apidocs/org/apache/hadoop/hbase/client/HTable.html
 , each new instance of HTable that uses a new instance of the Configuration 
object will create a new ZK connection. In the HiveHBaseStorageHandler, 
HiveHBaseTableInputFormat and HiveHBaseTableOutputFormat, a new instance of 
HTable is created each time. 

{code}
@Override
  public void setConf(Configuration conf) {
    jobConf = conf;
    hbaseConf = HBaseConfiguration.create(conf); // this clones the object
  }
{code}

and in the preCreateTable
{code}
...
      // ensure the table is online
      htable = new HTable(hbaseConf, tableDesc.getName());
...
{code}

We cannot share the HiveConf instances because they are session specific. I 
dont think we could change this code.

There are other potential causes in TableInputFormat 
{code}
    setHTable(new HTable(HBaseConfiguration.create(jobConf), 
Bytes.toBytes(hbaseTableName)));
    String hbaseColumnsMapping = jobConf.get(HBaseSerDe.HBASE_COLUMNS_MAPPING);
    boolean doColumnRegexMatching = 
jobConf.getBoolean(HBaseSerDe.HBASE_COLUMNS_REGEX_MATCHING, true);

    if (hbaseColumnsMapping == null) {
      //// Naveen we never close the connections associated with the HTable we 
instantiated above.
      throw new IOException(HBaseSerDe.HBASE_COLUMNS_MAPPING + " required for 
HBase Table.");
    }

    ColumnMappings columnMappings = null;
    try {
      columnMappings = HBaseSerDe.parseColumnsMapping(hbaseColumnsMapping, 
doColumnRegexMatching);
    } catch (SerDeException e) {
      //// Naveen we never close the connections associated with the HTable we 
instantiated a few lines above.
      throw new IOException(e);
    }
...
    InputSplit [] results = new InputSplit[splits.size()];
    for (int i = 0; i < splits.size(); i++) {
      results[i] = new HBaseSplit((TableSplit) splits.get(i), tablePaths[0]);
    }
    return results;
    /// Naveen Method end without cleaning up the underlying connections.
  }


> Zookeeper connection leaks in Hive's HBaseHandler.
> --------------------------------------------------
>
>                 Key: HIVE-12250
>                 URL: https://issues.apache.org/jira/browse/HIVE-12250
>             Project: Hive
>          Issue Type: Bug
>          Components: HiveServer2
>    Affects Versions: 1.1.0
>            Reporter: Naveen Gangam
>            Assignee: Naveen Gangam
>
> HiveServer2 performance regresses severely due to what appears to be a leak 
> in the ZooKeeper connections. lsof output on the HS2 process shows about 8000 
> TCP connections to the ZK ensemble nodes.
> grep TCP lsof-hive-node11 | grep node11 | grep -E "node03|node04|node05" | wc 
> -l
>     7866 
> grep TCP lsof-hive-node11 | grep node11 | grep -E "node03" | wc -l
>     2615
> grep TCP lsof-hive-node11 | grep node11 | grep -E "node04" | wc -l
>     2622
> grep TCP lsof-hive-node11 | grep node11 | grep -E "node05" | wc -l
>     2629
> node11 - HMS node
> node03, node04 and node05 are the hosts for zookeeper ensemble.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to