hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From bijieshan <bijies...@huawei.com>
Subject Error occured while RegionServer report to Master "we are up" should get master address again
Date Tue, 14 Jun 2011 11:08:34 GMT
I happened to fall across a problem, after some further analysis, I found the problem(The logs
was attached at the end of the email)

Consider the following scenario which is similar with my problem :

1. Due to some unclear reason,  the report to master got error. And retrying several times,
but also failed.
2. During this time , the standby master becomes the active one. So the endless loop is still
running, and it won't success, for the master address has updated, but it didn't know. And
won't know again.

   while (!stopped && (masterAddress = getMaster()) == null) {
      sleeper.sleep();
      LOG.warn("Unable to get master for initialization");
    }

    MapWritable result = null;
    long lastMsg = 0;
    while (!stopped) {
      try {
        this.requestCount.set(0);
        lastMsg = System.currentTimeMillis();
        ZKUtil.setAddressAndWatch(zooKeeper,
          ZKUtil.joinZNode(zooKeeper.rsZNode, ZKUtil.getNodeName(serverInfo)),
          this.serverInfo.getServerAddress());
        this.serverInfo.setLoad(buildServerLoad());
        LOG.info("Telling master at " + masterAddress + " that we are up");
        result = this.hbaseMaster.regionServerStartup(this.serverInfo,
            EnvironmentEdgeManager.currentTimeMillis());
        break;
      } catch (RemoteException e) {
        IOException ioe = e.unwrapRemoteException();
        if (ioe instanceof ClockOutOfSyncException) {
          LOG.fatal("Master rejected startup because clock is out of sync",
              ioe);
          // Re-throw IOE will cause RS to abort
          throw ioe;
        } else {
          LOG.warn("remote error telling master we are up", e);
        }
      } catch (IOException e) {
        LOG.warn("error telling master we are up", e);
      } catch (KeeperException e) {
        LOG.warn("error putting up ephemeral node in zookeeper", e);
      }
      sleeper.sleep(lastMsg);
    }


Here's the logs:

2011-06-13 11:25:12,236 WARN org.apache.hadoop.hbase.regionserver.HRegionServer: error telling
master we are up
java.net.ConnectException: Connection refused
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:574)
	at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:207)
	at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:419)
	at org.apache.hadoop.hbase.ipc.HBaseClient$Connection.setupIOstreams(HBaseClient.java:328)
	at org.apache.hadoop.hbase.ipc.HBaseClient.getConnection(HBaseClient.java:883)
	at org.apache.hadoop.hbase.ipc.HBaseClient.call(HBaseClient.java:750)
	at org.apache.hadoop.hbase.ipc.HBaseRPC$Invoker.invoke(HBaseRPC.java:257)
	at $Proxy5.regionServerStartup(Unknown Source)
	at org.apache.hadoop.hbase.regionserver.HRegionServer.reportForDuty(HRegionServer.java:1511)
	at org.apache.hadoop.hbase.regionserver.HRegionServer.tryReportForDuty(HRegionServer.java:1479)
	at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:571)
	at java.lang.Thread.run(Thread.java:662)
2011-06-13 11:25:15,231 INFO org.apache.hadoop.hbase.regionserver.HRegionServer: Telling master
at 157-5-111-22:20000 that we are up
2011-06-13 11:25:15,232 WARN org.apache.hadoop.hbase.regionserver.HRegionServer: error telling
master we are up
java.net.ConnectException: Connection refused
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:574)
	at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:207)
	at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:419)
	at org.apache.hadoop.hbase.ipc.HBaseClient$Connection.setupIOstreams(HBaseClient.java:328)
	at org.apache.hadoop.hbase.ipc.HBaseClient.getConnection(HBaseClient.java:883)
	at org.apache.hadoop.hbase.ipc.HBaseClient.call(HBaseClient.java:750)
	at org.apache.hadoop.hbase.ipc.HBaseRPC$Invoker.invoke(HBaseRPC.java:257)
	at $Proxy5.regionServerStartup(Unknown Source)
	at org.apache.hadoop.hbase.regionserver.HRegionServer.reportForDuty(HRegionServer.java:1511)
	at org.apache.hadoop.hbase.regionserver.HRegionServer.tryReportForDuty(HRegionServer.java:1479)
	at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:571)
	at java.lang.Thread.run(Thread.java:662)
2011-06-13 11:25:18,225 INFO org.apache.hadoop.hbase.regionserver.HRegionServer: Telling master
at 157-5-111-22:20000 that we are up

So I think, while the error orrured, we should re-get the master address. This problem could
be solved.

Thanks!

Jieshan Bean.



Mime
View raw message