You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hbase.apache.org by Gaojinchao <ga...@huawei.com> on 2011/05/19 03:48:10 UTC

about HBaseAdmin

The api HBaseAdmin has modified, So we should add some introduce:

If new a instance , it needs delete connection.

eg:
HBaseAdmin hba = new HBaseAdmin(conf);
.......

HConnectionManager.deleteConnection(hba.getConfiguration(), false);



public HBaseAdmin (Configuration c)

  throws MasterNotRunningException, ZooKeeperConnectionException {

    this.conf = HBaseConfiguration.create(c);                           // new conf, so It will create a new connection

    this.connection = HConnectionManager.getConnection(this.conf);

    this.pause = this.conf.getLong("hbase.client.pause", 1000);

    this.numRetries = this.conf.getInt("hbase.client.retries.number", 10);

    this.retryLongerMultiplier = this.conf.getInt("hbase.client.retries.longer.multiplier", 10);

    this.connection.getMaster();

  }



In my cluster.

New HBaseAdmin instance will create a new connection for zk.( we share HBaseConfiguration for multithread, It is ok for 0.90.2)

But, In 0.90.3 throw exception:

checkHtableState happen an exception. begin reconnect. exception info:org.apache.hadoop.hbase.ZooKeeperConnectionException: HBase is able to connect to ZooKeeper but the connection closes immediately. This could be a sign that the server has too many connections (30 is the default). Consider inspecting your ZK server logs for that error and then make sure you are reusing HBaseConfiguration as often as you can. See HTable's javadoc for more information




Re: about HBaseAdmin

Posted by Gaojinchao <ga...@huawei.com>.
In my case:
Hbase cluster and HDFS cluster share the Zk cluster(so , It is case "only when you have
30 connections" or is lower than 2000)

It seems this code has some issue:
  this.conf = HBaseConfiguration.create(c); // new a instance in 0.90.3 verison

In function getConnection, It creates a new HConnectionImplementation
So it is closed when HBaseAdmin finish. It looks better.

public static HConnection getConnection(Configuration conf)
  throws ZooKeeperConnectionException {
    HConnectionImplementation connection;
    synchronized (HBASE_INSTANCES) {
      connection = HBASE_INSTANCES.get(conf);    // It uses object hashcode. 
      if (connection == null) {
        connection = new HConnectionImplementation(conf);
        HBASE_INSTANCES.put(conf, connection);
      }
    }
    return connection;
  }


-----邮件原件-----
发件人: saint.ack@gmail.com [mailto:saint.ack@gmail.com] 代表 Stack
发送时间: 2011年5月19日 12:23
收件人: dev@hbase.apache.org
抄送: Chenjian
主题: Re: about HBaseAdmin

Gao:

So, each time you create an HBaseAdmin with a shared Configuration,
you get the below exception?  Or not every time but only when you have
30 connections?  What do you think changed things in 0.90.3?  Was it "
  HBASE-3734  HBaseAdmin creates new configurations in
getCatalogTracker"

Thanks,
St.Ack



On Wed, May 18, 2011 at 6:48 PM, Gaojinchao <ga...@huawei.com> wrote:
> The api HBaseAdmin has modified, So we should add some introduce:
>
> If new a instance , it needs delete connection.
>
> eg:
> HBaseAdmin hba = new HBaseAdmin(conf);
> .......
>
> HConnectionManager.deleteConnection(hba.getConfiguration(), false);
>
>
>
> public HBaseAdmin (Configuration c)
>
>  throws MasterNotRunningException, ZooKeeperConnectionException {
>
>    this.conf = HBaseConfiguration.create(c);                           // new conf, so It will create a new connection
>
>    this.connection = HConnectionManager.getConnection(this.conf);
>
>    this.pause = this.conf.getLong("hbase.client.pause", 1000);
>
>    this.numRetries = this.conf.getInt("hbase.client.retries.number", 10);
>
>    this.retryLongerMultiplier = this.conf.getInt("hbase.client.retries.longer.multiplier", 10);
>
>    this.connection.getMaster();
>
>  }
>
>
>
> In my cluster.
>
> New HBaseAdmin instance will create a new connection for zk.( we share HBaseConfiguration for multithread, It is ok for 0.90.2)
>
> But, In 0.90.3 throw exception:
>
> checkHtableState happen an exception. begin reconnect. exception info:org.apache.hadoop.hbase.ZooKeeperConnectionException: HBase is able to connect to ZooKeeper but the connection closes immediately. This could be a sign that the server has too many connections (30 is the default). Consider inspecting your ZK server logs for that error and then make sure you are reusing HBaseConfiguration as often as you can. See HTable's javadoc for more information
>
>
>
>

Re: about HBaseAdmin

Posted by Stack <st...@duboce.net>.
Gao:

So, each time you create an HBaseAdmin with a shared Configuration,
you get the below exception?  Or not every time but only when you have
30 connections?  What do you think changed things in 0.90.3?  Was it "
  HBASE-3734  HBaseAdmin creates new configurations in
getCatalogTracker"

Thanks,
St.Ack



On Wed, May 18, 2011 at 6:48 PM, Gaojinchao <ga...@huawei.com> wrote:
> The api HBaseAdmin has modified, So we should add some introduce:
>
> If new a instance , it needs delete connection.
>
> eg:
> HBaseAdmin hba = new HBaseAdmin(conf);
> .......
>
> HConnectionManager.deleteConnection(hba.getConfiguration(), false);
>
>
>
> public HBaseAdmin (Configuration c)
>
>  throws MasterNotRunningException, ZooKeeperConnectionException {
>
>    this.conf = HBaseConfiguration.create(c);                           // new conf, so It will create a new connection
>
>    this.connection = HConnectionManager.getConnection(this.conf);
>
>    this.pause = this.conf.getLong("hbase.client.pause", 1000);
>
>    this.numRetries = this.conf.getInt("hbase.client.retries.number", 10);
>
>    this.retryLongerMultiplier = this.conf.getInt("hbase.client.retries.longer.multiplier", 10);
>
>    this.connection.getMaster();
>
>  }
>
>
>
> In my cluster.
>
> New HBaseAdmin instance will create a new connection for zk.( we share HBaseConfiguration for multithread, It is ok for 0.90.2)
>
> But, In 0.90.3 throw exception:
>
> checkHtableState happen an exception. begin reconnect. exception info:org.apache.hadoop.hbase.ZooKeeperConnectionException: HBase is able to connect to ZooKeeper but the connection closes immediately. This could be a sign that the server has too many connections (30 is the default). Consider inspecting your ZK server logs for that error and then make sure you are reusing HBaseConfiguration as often as you can. See HTable's javadoc for more information
>
>
>
>