You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@hbase.apache.org by Dejan Menges <de...@gmail.com> on 2015/03/16 14:40:51 UTC

Strange issue when DataNode goes down

Hi All,

We have a strange issue with HBase performance (overall cluster
performance) in case one of datanodes in the cluster unexpectedly goes
down.

So scenario is like follows:
- Cluster works fine, it's stable.
- One DataNode unexpectedly goes down (PSU issue, network issue, anything)
- Whole HBase cluster goes down (performance becomes so bad that we have to
restart all RegionServers to get it back to life).

Most funny and latest issue that happened was that we added new node to the
cluster (having 8 x 4T SATA disks) and we left just DataNode running on it
to give it couple of days to get some data. At some point in time, due to
hardware issue, server rebooted (twice during three hours) in moment when
it had maybe 5% of data it would have in a couple of days. Nothing else
beside DataNode was running, and once it went down, it affected literary
everything, and restarting RegionServers in the end fixed it.

We are using HBase 0.98.0 with Hadoop 2.4.0

Re: Strange issue when DataNode goes down

Posted by Sean Busbey <bu...@cloudera.com>.
Can you post some redacted log files from the period after the data node
failed, up to the restart?

-- 
Sean
On Mar 16, 2015 8:41 AM, "Dejan Menges" <de...@gmail.com> wrote:

> Hi All,
>
> We have a strange issue with HBase performance (overall cluster
> performance) in case one of datanodes in the cluster unexpectedly goes
> down.
>
> So scenario is like follows:
> - Cluster works fine, it's stable.
> - One DataNode unexpectedly goes down (PSU issue, network issue, anything)
> - Whole HBase cluster goes down (performance becomes so bad that we have to
> restart all RegionServers to get it back to life).
>
> Most funny and latest issue that happened was that we added new node to the
> cluster (having 8 x 4T SATA disks) and we left just DataNode running on it
> to give it couple of days to get some data. At some point in time, due to
> hardware issue, server rebooted (twice during three hours) in moment when
> it had maybe 5% of data it would have in a couple of days. Nothing else
> beside DataNode was running, and once it went down, it affected literary
> everything, and restarting RegionServers in the end fixed it.
>
> We are using HBase 0.98.0 with Hadoop 2.4.0
>

Re: Strange issue when DataNode goes down

Posted by Dejan Menges <de...@gmail.com>.
Sorry, forgot to paste the log part:

2015-03-23 08:53:44,381 WARN org.apache.hadoop.hbase.util.FSHDFSUtils:
Cannot recoverLease after trying for 900000ms
(hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!;
attempt=40 on
file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intentionally_get_down_by_getting_network_down},60020,1426862900506-splitting/{node_i_intentionally_get_down_by_getting_network_down}%2C60020%2C1426862900506.1427096924508
after 908210ms

2015-03-23 08:53:44,540 INFO
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: Finishing writing
output logs and closing down.

2015-03-23 08:53:44,540 INFO
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: Waiting for split
writer threads to finish

2015-03-23 08:53:44,541 INFO
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: Split writers
finished

2015-03-23 08:53:44,541 INFO
org.apache.hadoop.hbase.regionserver.wal.HLogSplitter: Processed 0 edits
across 0 regions; log
file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intentionally_get_down_by_getting_network_down},60020,1426862900506-splitting/{node_i_intentionally_get_down_by_getting_network_down}%2C60020%2C1426862900506.1427096924508
is corrupted = false progress failed = false

2015-03-23 08:53:44,612 INFO
org.apache.hadoop.hbase.regionserver.handler.HLogSplitterHandler:
successfully transitioned task
/hbase/splitWAL/WALs%2F{node_i_intentionally_get_down_by_getting_network_down}%2C60020%2C1426862900506-splitting%2F{node_i_intentionally_get_down_by_getting_network_down}%252C60020%252C1426862900506.1427096924508
to final state DONE
{node_taking_over_regions_from_failed_node},60020,1426859445623

2015-03-23 08:53:44,612 INFO
org.apache.hadoop.hbase.regionserver.handler.HLogSplitterHandler: worker
{node_taking_over_regions_from_failed_node},60020,1426859445623 done with
task
/hbase/splitWAL/WALs%2F{node_i_intentionally_get_down_by_getting_network_down}%2C60020%2C1426862900506-splitting%2F{node_i_intentionally_get_down_by_getting_network_down}%252C60020%252C1426862900506.1427096924508
in 908486ms


So it's also here mentioning that there can be possible data loss (of
course, node died) but in this case it took 15 minutes (default one).

On Mon, Mar 23, 2015 at 3:05 PM Dejan Menges <de...@gmail.com> wrote:

> Will take a look.
>
> Actually, if node is down (someone unplugged network cable, it just died,
> whatever) what's chance it's going to become live so write can continue? On
> the other side, HBase is not starting recovery trying to contact node which
> is not there anymore, and even elected as dead on Namenode side (another
> thing I didn't understood quite good).
>
> So what I was expecting is that as soon as Namenode decided node is dead,
> that it would be enough for RegionServer to stop trying to recover from the
> dead node, but it wasn't the case. Also, this whole MTTR article in HBase
> book doesn't work at all with this parameter set to it's default value (15
> minutes).
>
> So I'm still struggling to figure out what's drawback exactly on this?
>
> On Mon, Mar 23, 2015 at 2:50 PM Nicolas Liochon <nk...@gmail.com> wrote:
>
>> Thanks for the explanation. There is an issue if you modify this setting
>> however.
>> hbase tries to recover the lease (i.e. be sure that nobody is writing)
>> If you change hbase.lease.recovery.timeout hbase will start the recovery
>> (i.e. start to read) even if it's not sure that nobody's writing. That
>> means there is a dataloss risk.
>> Basically, you must not see this warning: WARN
>> org.apache.hadoop.hbase.util.FSHDFSUtils: Cannot recoverLease after
>> trying
>> for[]
>>
>> The recoverLease must succeed. The fact that it does not after X tries is
>> strange.
>> There may be a mistmatch between the hbase parameters and the hdfs ones.
>> You may need to have a look at the comments in FSHDFSUtils.java
>>
>>
>>
>>
>> On Mon, Mar 23, 2015 at 2:15 PM, Dejan Menges <de...@gmail.com>
>> wrote:
>>
>> > I found the issue and fixed it, and will try to explain here what was
>> > exactly in our case in case someone else finds this interesting too.
>> >
>> > So initially, we had (couple of times) some network and hardware issues
>> in
>> > our datacenters. When one server would die (literary die, we had some
>> issue
>> > with PSUs) we saw issues with overall cluster performance on HBase
>> side. As
>> > cluster is quite big and live, it was also quite hard to figure out
>> exact
>> > root cause and how to fix the stuff we wanted to fix.
>> >
>> > So I set up another cluster, four nodes (with DataNode and RegionServer)
>> > and two other nodes with HMaster and Namenode in HA, using same stuff we
>> > use on production. We pumped some data into it, and I was able to
>> reproduce
>> > same issue last week on it. What I tried to do is to cut one server
>> (shut
>> > down it's interface) when all is good with cluster, when we have load,
>> and
>> > see what's going to happen.
>> >
>> > On Friday, after Nicolas mentioned, I started taking a look in HBase
>> logs
>> > on the node which was mentioned in HMaster log as the one taking over
>> > regions for the dead server. Basically what I was able to observe was 15
>> > minutes time (+- couple of seconds, what was also interesting, and will
>> got
>> > later to that) between HMaster figures out that one of it's
>> RegionServers
>> > is dead, and the time one of the mentioned nodes starts taking over
>> those
>> > regions and they start appearing in HMaster's Web UI.
>> >
>> > I then set up everything like mentioned here
>> > http://hbase.apache.org/book.html#mttr - but still had exactly the same
>> > issues. Went over (again and again) all currently configured stuff, but
>> > still had the same issue.
>> >
>> > Then I started looking into HDFS. Opened NameNode UI, saw all is good,
>> took
>> > one node down, was also looking RegionServer logs in the same time -
>> and I
>> > also see that it took ~15 minutes for Namenode to elect dead node as
>> dead.
>> > Somehow in the same moment regions started getting back to life. I
>> remember
>> > in some older versions dfs timeout checks and check retries. Looked into
>> > defaults for our Hadoop version -
>> >
>> > http://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/had
>> oop-hdfs/hdfs-default.xml
>> > - and saw there that there's no recheck parameter anymore. Strange, as
>> on
>> > StackOverflow I found post from month ago, for newer version than we use
>> > (we use 2.4.1, guy was using 2.6 - dfs.namenode.heartbeat.recheck
>> -interval)
>> > I set it to 10 seconds as he mentioned, having checks every three
>> seconds
>> > (default) and got DataNode elected as dead in ~45 seconds, as he
>> mentioned.
>> > Not sure why this parameter is not documented, but obviously it works.
>> >
>> > Then figured out it still didn't fix our HBase failover issue. I was
>> > looking into HBase book again and again, and then saw this part:
>> >
>> > "How much time we allow elapse between calls to recover lease. Should be
>> > larger than the dfs timeout."
>> >
>> > This was description for hbase.lease.recovery.dfs.timeout. Wasn't sure
>> from
>> > the comment what of all timeouts that's possible to set in Hadoop/HBase
>> and
>> > that has something to do with DFS is this all about. But picked
>> > hbase.lease.recovery.timeout, and that was the catch.
>> >
>> > Initially, by default, hbase.lease.recovery.timeout is set to 15
>> minutes.
>> > Not sure why, and wasn't able to find yet why, but getting this down to
>> one
>> > minute (what's more than OK for us) I was able to get rid of our issue.
>> Not
>> > also sure why this is not mentioned in MTTR section in HBase book, as
>> > obviously MTTR doesn't work at all with this default timeout, at least
>> it
>> > doesn't work the way we expected it to work.
>> >
>> > So thanks again for everyone being spammed with this, and specially
>> thanks
>> > to Nicolas pointing me to the right direction.
>> >
>> > On Mon, Mar 23, 2015 at 1:37 PM Nicolas Liochon <nk...@gmail.com>
>> wrote:
>> >
>> > > the attachments are rejected by the mailing list, can you put then on
>> > > pastebin?
>> > >
>> > > stale is mandatory (so it's good), but the issue here is just before.
>> The
>> > > region server needs to read the file. In order to be sure that there
>> is
>> > no
>> > > data loss, it needs to "recover the lease", that means ensuring that
>> > nobody
>> > > is writing the file. The regionserver calls the namenode to do this
>> > > recoverLease. So there should be some info in the namenode logs. You
>> have
>> > > HDFS-4721 on your hdfs? The hbase book details (more or less...) this
>> > > recoverLease stuff.
>> > >
>> > >
>> > > On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <
>> dejan.menges@gmail.com>
>> > > wrote:
>> > >
>> > > > And also, just checked - dfs.namenode.avoid.read.stale.datanode and
>> > > > dfs.namenode.avoid.write.stale.datanode
>> > > > are both true, and dfs.namenode.stale.datanode.interval is set to
>> > > default
>> > > > 30000.
>> > > >
>> > > > On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <
>> dejan.menges@gmail.com>
>> > > > wrote:
>> > > >
>> > > > > Hi Nicolas,
>> > > > >
>> > > > > Please find log attached.
>> > > > >
>> > > > > As I see it now more clearly, it was trying to recover HDFS WALs
>> from
>> > > > node
>> > > > > that's dead:
>> > > > >
>> > > > > 2015-03-23 08:53:44,381 WARN
>> > org.apache.hadoop.hbase.util.FSHDFSUtils:
>> > > > > Cannot recoverLease after trying for 900000ms
>> > > > > (hbase.lease.recovery.timeout); continuing, but may be
>> DATALOSS!!!;
>> > > > > attempt=40 on
>> > > > >
>> > > > file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intent
>> > > ionally_get_down_by_getting_network_down},60020,1426862900
>> > > 506-splitting/{node_i_intentionally_get_down_by_gett
>> > > ing_network_down}%2C60020%2C1426862900506.1427096924508
>> > > > > after 908210ms
>> > > > >
>> > > > > And as you can see from the log, it tried 40 times, what took it
>> > > exactly
>> > > > > 15 minutes.
>> > > > >
>> > > > > There's probably some parameter to tune to cut it of from 40
>> times /
>> > 15
>> > > > > minutes to something more useful, as for 15 minutes we don't have
>> our
>> > > > > regions available, and HDFS have however replication factor 3.
>> > > > >
>> > > > > Googling, if I figure out what's this I will post it here. Will
>> also
>> > > > > appreciate if someone knows how to cut this down.
>> > > > >
>> > > > > Thanks,
>> > > > >
>> > > > > Dejan
>> > > > >
>> > > > > On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <
>> nkeywal@gmail.com>
>> > > > wrote:
>> > > > >
>> > > > >> The split is done by the region servers (the master
>> coordinates). Is
>> > > > there
>> > > > >> some interesting stuff in their logs?
>> > > > >>
>> > > > >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <
>> > dejan.menges@gmail.com
>> > > >
>> > > > >> wrote:
>> > > > >>
>> > > > >> > With client issue was that it was retrying connecting to the
>> same
>> > > > region
>> > > > >> > servers even when they were reassigned. Lowering it down
>> helped in
>> > > > this
>> > > > >> > specific use case, but yes, we still want servers to reallocate
>> > > > quickly.
>> > > > >> >
>> > > > >> > What got me here:
>> > > > >> >
>> > > > >> > http://hbase.apache.org/book.html#mttr
>> > > > >> >
>> > > > >> > I basically set configuration exactly the same way as it's
>> > explained
>> > > > >> here.
>> > > > >> > *zookeeper.session.timeout* is (and was before) 60000 (one
>> > minute).
>> > > > >> >
>> > > > >> > So basically what happens is: - simulating network issues we
>> had
>> > > > >> recently).
>> > > > >> > - After short time I see in HBase that my RegionServer is dead,
>> > and
>> > > as
>> > > > >> > total number of regions I see previous total minus number of
>> > regions
>> > > > >> that
>> > > > >> > were hosted on the node hosting RegionServer that just
>> > > 'disappeared'.
>> > > > >> > - In this point I want my clus
>> > > > >> >
>> > > > >> > - I have test cluster consisting of four nodes, every node
>> being
>> > > > >> DataNode
>> > > > >> > and RegionServer.
>> > > > >> > - I simulate network partition on one (connect to it through
>> > console
>> > > > and
>> > > > >> > take network interface downter to recover as soon as possible,
>> to
>> > > > start
>> > > > >> > serving missing regions.
>> > > > >> > - First thing I see in HMaster logs are:
>> > > > >> >
>> > > > >> > 2015-03-20 14:17:26,015 INFO
>> > > > >> > org.apache.hadoop.hbase.zookeeper.RegionServerTracker:
>> > RegionServer
>> > > > >> > ephemeral node deleted, processing expiration
>> > > > >> > [{name_of_node_I_took_down},60020,1426860403261]
>> > > > >> >
>> > > > >> > 2015-03-20 14:17:26,067 INFO
>> > > > >> > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler:
>> > > > Splitting
>> > > > >> > logs for {name_of_node_I_took_down},60020,1426860403261 before
>> > > > >> assignment.
>> > > > >> >
>> > > > >> > 2015-03-20 14:17:26,105 INFO
>> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: dead splitlog
>> > > workers
>> > > > [
>> > > > >> > {name_of_node_I_took_down},60020,1426860403261]
>> > > > >> >
>> > > > >> > 2015-03-20 14:17:26,107 INFO
>> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: started
>> splitting
>> > 1
>> > > > >> logs in
>> > > > >> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_I_
>> > > took_down}
>> > > > >> > ,60020,1426860403261-splitting]
>> > > > >> >
>> > > > >> > 2015-03-20 14:17:26,150 INFO
>> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
>> > > > >> > /hbase/splitWAL/WALs%2F
>> > > > >> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting
>> %2F
>> > > > >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.14268
>> > > 60404905
>> > > > >> > acquired by {fqdn_of_another_live_hnode},60020,1426859445623
>> > > > >> >
>> > > > >> > 2015-03-20 14:17:26,182 INFO
>> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks =
>> 1
>> > > > >> unassigned
>> > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>> > > > >> >
>> > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
>> > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
>> > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
>> > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
>> > > in_progress
>> > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
>> > error =
>> > > > 0}
>> > > > >> >
>> > > > >> > 2015-03-20 14:17:31,183 INFO
>> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks =
>> 1
>> > > > >> unassigned
>> > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>> > > > >> >
>> > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
>> > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
>> > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
>> > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
>> > > in_progress
>> > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
>> > error =
>> > > > 0}
>> > > > >> >
>> > > > >> > 2015-03-20 14:17:36,184 INFO
>> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks =
>> 1
>> > > > >> unassigned
>> > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>> > > > >> >
>> > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
>> > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
>> > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
>> > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
>> > > in_progress
>> > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
>> > error =
>> > > > 0}
>> > > > >> >
>> > > > >> > 2015-03-20 14:17:42,185 INFO
>> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks =
>> 1
>> > > > >> unassigned
>> > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>> > > > >> >
>> > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
>> > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
>> > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
>> > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
>> > > in_progress
>> > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
>> > error =
>> > > > 0}
>> > > > >> >
>> > > > >> > 2015-03-20 14:17:48,184 INFO
>> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks =
>> 1
>> > > > >> unassigned
>> > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>> > > > >> >
>> > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
>> > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
>> > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
>> > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
>> > > in_progress
>> > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
>> > error =
>> > > > 0}
>> > > > >> > In the meantime, In hbase...out log I got this:
>> > > > >> >
>> > > > >> > ==> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out <==
>> > > > >> >
>> > > > >> > java.io.IOException: Call to
>> > > > >> > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_dow
>> > > n}:60020
>> > > > >> > failed on local exception:
>> > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$CallTimeoutException:
>> Call
>> > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
>> > > > >> >
>> > > > >> > at org.apache.hadoop.hbase.ipc.RpcClient.wrapException(
>> RpcClien
>> > > > >> t.java:1532)
>> > > > >> >
>> > > > >> > at org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:
>> 1502)
>> > > > >> >
>> > > > >> > at
>> > > > >> >
>> > > > >> > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(Rpc
>> > > > >> Client.java:1684)
>> > > > >> >
>> > > > >> > at
>> > > > >> >
>> > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$BlockingRpcChannelImpl
>> > > ementati
>> > > > >> on.callBlockingMethod(RpcClient.java:1737)
>> > > > >> >
>> > > > >> > at
>> > > > >> >
>> > > > >> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$
>> > > AdminService$
>> > > > >> BlockingStub.getRegionInfo(AdminProtos.java:20806)
>> > > > >> >
>> > > > >> > at
>> > > > >> >
>> > > > >> > org.apache.hadoop.hbase.client.HBaseAdmin.getCompactionState
>> > > > >> (HBaseAdmin.java:2524)
>> > > > >> >
>> > > > >> > at
>> > > > >> >
>> > > > >> > org.apache.hadoop.hbase.generated.master.table_jsp._jspServi
>> > > > >> ce(table_jsp.java:167)
>> > > > >> >
>> > > > >> > at org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.
>> > > java:98)
>> > > > >> >
>> > > > >> > at javax.servlet.http.HttpServlet.service(
>> HttpServlet.java:770)
>> > > > >> >
>> > > > >> > at org.mortbay.jetty.servlet.ServletHolder.handle(
>> ServletHolder
>> > > > >> .java:511)
>> > > > >> >
>> > > > >> > at
>> > > > >> >
>> > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
>> > > > >> r(ServletHandler.java:1221)
>> > > > >> >
>> > > > >> > at
>> > > > >> >
>> > > > >> > org.apache.hadoop.http.lib.StaticUserWebFilter$StaticUserFil
>> > > > >> ter.doFilter(StaticUserWebFilter.java:109)
>> > > > >> >
>> > > > >> > at
>> > > > >> >
>> > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
>> > > > >> r(ServletHandler.java:1212)
>> > > > >> >
>> > > > >> > at
>> > > > >> >
>> > > > >> > org.apache.hadoop.http.HttpServer$QuotingInputFilter.doFilte
>> > > > >> r(HttpServer.java:1081)
>> > > > >> >
>> > > > >> > at
>> > > > >> >
>> > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
>> > > > >> r(ServletHandler.java:1212)
>> > > > >> >
>> > > > >> > at
>> > > > org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.
>> java:45)
>> > > > >> >
>> > > > >> > at
>> > > > >> >
>> > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
>> > > > >> r(ServletHandler.java:1212)
>> > > > >> >
>> > > > >> > at org.mortbay.jetty.servlet.ServletHandler.handle(
>> ServletHandl
>> > > > >> er.java:399)
>> > > > >> >
>> > > > >> > at
>> > > > >> > org.mortbay.jetty.security.SecurityHandler.handle(SecurityHa
>> > > > >> ndler.java:216)
>> > > > >> >
>> > > > >> > at org.mortbay.jetty.servlet.SessionHandler.handle(
>> SessionHandl
>> > > > >> er.java:182)
>> > > > >> >
>> > > > >> > at org.mortbay.jetty.handler.ContextHandler.handle(
>> ContextHandl
>> > > > >> er.java:766)
>> > > > >> >
>> > > > >> > at org.mortbay.jetty.webapp.WebAppContext.handle(
>> WebAppContext.
>> > > > >> java:450)
>> > > > >> >
>> > > > >> > at
>> > > > >> >
>> > > > >> > org.mortbay.jetty.handler.ContextHandlerCollection.handle(Co
>> > > > >> ntextHandlerCollection.java:230)
>> > > > >> >
>> > > > >> > at org.mortbay.jetty.handler.HandlerWrapper.handle(
>> HandlerWrapp
>> > > > >> er.java:152)
>> > > > >> >
>> > > > >> > at org.mortbay.jetty.Server.handle(Server.java:326)
>> > > > >> >
>> > > > >> > at org.mortbay.jetty.HttpConnection.handleRequest(
>> HttpConnectio
>> > > > >> n.java:542)
>> > > > >> >
>> > > > >> > at
>> > > > >> >
>> > > > >> > org.mortbay.jetty.HttpConnection$RequestHandler.headerComple
>> > > > >> te(HttpConnection.java:928)
>> > > > >> >
>> > > > >> > at org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
>> > > > >> >
>> > > > >> > at
>> > org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
>> > > > >> >
>> > > > >> > at
>> > org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
>> > > > >> >
>> > > > >> > at
>> > > > >> >
>> > > > >> > org.mortbay.io.nio.SelectChannelEndPoint.run(SelectChannelEn
>> > > > >> dPoint.java:410)
>> > > > >> >
>> > > > >> > at
>> > > > >> >
>> > > > >> > org.mortbay.thread.QueuedThreadPool$PoolThread.run(
>> > > > >> QueuedThreadPool.java:582)
>> > > > >> >
>> > > > >> > Caused by: org.apache.hadoop.hbase.ipc.RpcClient$
>> > > CallTimeoutException:
>> > > > >> Call
>> > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
>> > > > >> >
>> > > > >> > at
>> > > > >> >
>> > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.cleanupCall
>> > > > >> s(RpcClient.java:1234)
>> > > > >> >
>> > > > >> > at
>> > > > >> >
>> > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.readRespons
>> > > > >> e(RpcClient.java:1171)
>> > > > >> >
>> > > > >> > at org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(
>> RpcClie
>> > > > >> nt.java:751)
>> > > > >> > Beside this same issue, please note that first message was at
>> > > > 2015-03-20
>> > > > >> > 14:17:26,015. And then (we got to the point when it started
>> > > > transition):
>> > > > >> >
>> > > > >> > 2015-03-20 14:32:35,059 INFO
>> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
>> > > > >> > /hbase/splitWAL/WALs%2F
>> > > > >> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting
>> %2F
>> > > > >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.14268
>> > > 60404905
>> > > > >> > entered state: DONE {fqdn_of_new_live_node},60020,
>> 1426859445623
>> > > > >> >
>> > > > >> > 2015-03-20 14:32:35,109 INFO
>> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: Done splitting
>> > > > >> > /hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>> > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
>> down}
>> > > > >> > %252C60020%252C1426860403261.1426860404905
>> > > > >> >
>> > > > >> > 2015-03-20 14:32:35,190 INFO
>> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: finished
>> splitting
>> > > > >> (more
>> > > > >> > than or equal to) 9 bytes in 1 log files in
>> > > > >> >
>> > > > >> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_of_
>> > > > >> node_I_took_down},60020,1426860403261-splitting]
>> > > > >> > in 909083ms
>> > > > >> >
>> > > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
>> > > > >> .RegionStates:
>> > > > >> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c state=OPEN,
>> > > > >> > ts=1426860639088,
>> > > > server={name_of_node_I_took_down},60020,1426860403261}
>> > > > >> to
>> > > > >> > {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE,
>> ts=1426861955191,
>> > > > >> server=
>> > > > >> > {name_of_node_I_took_down},60020,1426860403261}
>> > > > >> >
>> > > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
>> > > > >> .RegionStates:
>> > > > >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
>> > > > >> {name_of_node_I_took_down}
>> > > > >> > ,60020,1426860403261
>> > > > >> >
>> > > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
>> > > > >> .RegionStates:
>> > > > >> > Transitioned {25ab6e7b42e36ddaa723d71be5954543 state=OPEN,
>> > > > >> > ts=1426860641783,
>> > > > server={name_of_node_I_took_down},60020,1426860403261}
>> > > > >> to
>> > > > >> > {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE,
>> ts=1426861955191,
>> > > > >> server=
>> > > > >> > {name_of_node_I_took_down},60020,1426860403261}
>> > > > >> >
>> > > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
>> > > > >> .RegionStates:
>> > > > >> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
>> > > > >> {name_of_node_I_took_down}
>> > > > >> > ,60020,1426860403261
>> > > > >> > At this point, note that it finished SplitLogManager task at
>> > > 14:32:35
>> > > > >> and
>> > > > >> > started transitioning just after that. So this is 15 minutes
>> that
>> > > I'm
>> > > > >> > talking about.
>> > > > >> >
>> > > > >> > What am I missing?
>> > > > >> >
>> > > > >> >
>> > > > >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <
>> > nkeywal@gmail.com>
>> > > > >> wrote:
>> > > > >> >
>> > > > >> > > You've changed the value of hbase.zookeeper.timeout to 15
>> > > minutes? A
>> > > > >> very
>> > > > >> > > reasonable target is 1 minute before relocating the regions.
>> > > That's
>> > > > >> the
>> > > > >> > > default iirc. You can push it to 20s, but then
>> > > gc-stopping-the-world
>> > > > >> > > becomes more of an issue. 15 minutes is really a lot. The
>> hdfs
>> > > stale
>> > > > >> mode
>> > > > >> > > must always be used, with a lower timeout than the hbase one.
>> > > > >> > >
>> > > > >> > > Client side there should be nothing to do (excepted for
>> advanced
>> > > > >> stuff);
>> > > > >> > at
>> > > > >> > > each retry the client checks the location of the regions. If
>> you
>> > > > lower
>> > > > >> > the
>> > > > >> > > number of retry the client will fail sooner, but usually you
>> > don't
>> > > > >> want
>> > > > >> > the
>> > > > >> > > client to fail, you want the servers to reallocate quickly.
>> > > > >> > >
>> > > > >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <
>> > > > dejan.menges@gmail.com
>> > > > >> >
>> > > > >> > > wrote:
>> > > > >> > >
>> > > > >> > > > Hi,
>> > > > >> > > >
>> > > > >> > > > Sorry for little bit late update, but managed to narrow it
>> > > little
>> > > > >> bit
>> > > > >> > > down.
>> > > > >> > > >
>> > > > >> > > > We didn't update yet, as we are using Hortonworks
>> distribution
>> > > > right
>> > > > >> > now,
>> > > > >> > > > and even if we update we will get 0.98.4. However, looks
>> that
>> > > > issue
>> > > > >> > here
>> > > > >> > > > was in our use case and configuration (still looking into
>> it).
>> > > > >> > > >
>> > > > >> > > > Basically, initially I saw that when one server goes down,
>> we
>> > > > start
>> > > > >> > > having
>> > > > >> > > > performance issues in general, but it managed to be on our
>> > > client
>> > > > >> side,
>> > > > >> > > due
>> > > > >> > > > to caching, and clients were trying to reconnect to nodes
>> that
>> > > > were
>> > > > >> > > offline
>> > > > >> > > > and later trying to get regions from those nodes too. This
>> is
>> > > > >> basically
>> > > > >> > > why
>> > > > >> > > > on server side I didn't manage to see anything in logs that
>> > > would
>> > > > >> be at
>> > > > >> > > > least little bit interesting and point me into desired
>> > > direction.
>> > > > >> > > >
>> > > > >> > > > Another question that popped up to me is - in case server
>> is
>> > > down
>> > > > >> (and
>> > > > >> > > with
>> > > > >> > > > it DataNode and HRegionServer it was hosting) - what's
>> optimal
>> > > > time
>> > > > >> to
>> > > > >> > > set
>> > > > >> > > > for HMaster to consider server dead reassign regions
>> somewhere
>> > > > >> else, as
>> > > > >> > > > this is another performance bottleneck we hit during
>> inability
>> > > to
>> > > > >> > access
>> > > > >> > > > regions? In our case it's configured to 15 minutes, and
>> simple
>> > > > logic
>> > > > >> > > tells
>> > > > >> > > > me if you want it earlier then configure lower number of
>> > > retries,
>> > > > >> but
>> > > > >> > > issue
>> > > > >> > > > is as always in details, so not sure if anyone knows some
>> > better
>> > > > >> math
>> > > > >> > for
>> > > > >> > > > this?
>> > > > >> > > >
>> > > > >> > > > And last question - is it possible to manually force HBase
>> to
>> > > > >> reassign
>> > > > >> > > > regions? In this case, while HMaster is retrying to contact
>> > node
>> > > > >> that's
>> > > > >> > > > dead, it's impossible to force it using 'balancer' command.
>> > > > >> > > >
>> > > > >> > > > Thanks a lot!
>> > > > >> > > >
>> > > > >> > > > Dejan
>> > > > >> > > >
>> > > > >> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <
>> > > > >> dejan.menges@gmail.com>
>> > > > >> > > > wrote:
>> > > > >> > > >
>> > > > >> > > > > Hi,
>> > > > >> > > > >
>> > > > >> > > > > To be very honest - there's no particular reason why we
>> > stick
>> > > to
>> > > > >> this
>> > > > >> > > > one,
>> > > > >> > > > > beside just lack of time currently to go through upgrade
>> > > > process,
>> > > > >> but
>> > > > >> > > > looks
>> > > > >> > > > > to me that's going to be next step.
>> > > > >> > > > >
>> > > > >> > > > > Had a crazy day, didn't have time to go through all logs
>> > > again,
>> > > > >> plus
>> > > > >> > > one
>> > > > >> > > > > of the machines (last one where we had this issue) is
>> fully
>> > > > >> > > reprovisioned
>> > > > >> > > > > yesterday so I don't have logs from there anymore.
>> > > > >> > > > >
>> > > > >> > > > > Beside upgrading,  what I will talk about today, can you
>> > just
>> > > > >> point
>> > > > >> > me
>> > > > >> > > to
>> > > > >> > > > > the specific RPC issue in 0.98.0? Thing is that we have
>> some
>> > > > >> strange
>> > > > >> > > > > moments with RPC in this case, and just want to see if
>> > that's
>> > > > the
>> > > > >> > same
>> > > > >> > > > > thing (and we were even suspecting to RPC).
>> > > > >> > > > >
>> > > > >> > > > > Thanks a lot!
>> > > > >> > > > > Dejan
>> > > > >> > > > >
>> > > > >> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew Purtell <
>> > > > >> apurtell@apache.org
>> > > > >> > >
>> > > > >> > > > > wrote:
>> > > > >> > > > >
>> > > > >> > > > >> Is there a particular reason why you are using HBase
>> > 0.98.0?
>> > > > The
>> > > > >> > > latest
>> > > > >> > > > >> 0.98 release is 0.98.11. There's a known performance
>> issue
>> > > with
>> > > > >> > 0.98.0
>> > > > >> > > > >> pertaining to RPC that was fixed in later releases, you
>> > > should
>> > > > >> move
>> > > > >> > up
>> > > > >> > > > >> from
>> > > > >> > > > >> 0.98.0. In addition hundreds of improvements and bug
>> fixes
>> > > have
>> > > > >> gone
>> > > > >> > > > into
>> > > > >> > > > >> the ten releases since 0.98.0.
>> > > > >> > > > >>
>> > > > >> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <
>> > > > >> > dejan.menges@gmail.com
>> > > > >> > > >
>> > > > >> > > > >> wrote:
>> > > > >> > > > >>
>> > > > >> > > > >> > Hi All,
>> > > > >> > > > >> >
>> > > > >> > > > >> > We have a strange issue with HBase performance
>> (overall
>> > > > cluster
>> > > > >> > > > >> > performance) in case one of datanodes in the cluster
>> > > > >> unexpectedly
>> > > > >> > > goes
>> > > > >> > > > >> > down.
>> > > > >> > > > >> >
>> > > > >> > > > >> > So scenario is like follows:
>> > > > >> > > > >> > - Cluster works fine, it's stable.
>> > > > >> > > > >> > - One DataNode unexpectedly goes down (PSU issue,
>> network
>> > > > >> issue,
>> > > > >> > > > >> anything)
>> > > > >> > > > >> > - Whole HBase cluster goes down (performance becomes
>> so
>> > bad
>> > > > >> that
>> > > > >> > we
>> > > > >> > > > >> have to
>> > > > >> > > > >> > restart all RegionServers to get it back to life).
>> > > > >> > > > >> >
>> > > > >> > > > >> > Most funny and latest issue that happened was that we
>> > added
>> > > > new
>> > > > >> > node
>> > > > >> > > > to
>> > > > >> > > > >> the
>> > > > >> > > > >> > cluster (having 8 x 4T SATA disks) and we left just
>> > > DataNode
>> > > > >> > running
>> > > > >> > > > on
>> > > > >> > > > >> it
>> > > > >> > > > >> > to give it couple of days to get some data. At some
>> point
>> > > in
>> > > > >> time,
>> > > > >> > > due
>> > > > >> > > > >> to
>> > > > >> > > > >> > hardware issue, server rebooted (twice during three
>> > hours)
>> > > in
>> > > > >> > moment
>> > > > >> > > > >> when
>> > > > >> > > > >> > it had maybe 5% of data it would have in a couple of
>> > days.
>> > > > >> Nothing
>> > > > >> > > > else
>> > > > >> > > > >> > beside DataNode was running, and once it went down, it
>> > > > affected
>> > > > >> > > > literary
>> > > > >> > > > >> > everything, and restarting RegionServers in the end
>> fixed
>> > > it.
>> > > > >> > > > >> >
>> > > > >> > > > >> > We are using HBase 0.98.0 with Hadoop 2.4.0
>> > > > >> > > > >> >
>> > > > >> > > > >>
>> > > > >> > > > >>
>> > > > >> > > > >>
>> > > > >> > > > >> --
>> > > > >> > > > >> Best regards,
>> > > > >> > > > >>
>> > > > >> > > > >>    - Andy
>> > > > >> > > > >>
>> > > > >> > > > >> Problems worthy of attack prove their worth by hitting
>> > back.
>> > > -
>> > > > >> Piet
>> > > > >> > > Hein
>> > > > >> > > > >> (via Tom White)
>> > > > >> > > > >>
>> > > > >> > > > >
>> > > > >> > > > >
>> > > > >> > > >
>> > > > >> > >
>> > > > >> >
>> > > > >>
>> > > > >
>> > > >
>> > >
>> >
>>
>

Re: Strange issue when DataNode goes down

Posted by Dejan Menges <de...@gmail.com>.
Will do some deeper testing on this to try to narrow it down and will
update then here for sure.

On Mon, Mar 23, 2015 at 5:42 PM Nicolas Liochon <nk...@gmail.com> wrote:

> Ok, so hopefully there are some info in the namenode & datanode logs.
>
> On Mon, Mar 23, 2015 at 5:32 PM, Dejan Menges <de...@gmail.com>
> wrote:
>
> > ...and I also got sure that it's applied with hdfs getconf -confKey...
> >
> > On Mon, Mar 23, 2015 at 5:31 PM Dejan Menges <de...@gmail.com>
> > wrote:
> >
> > > It was true all the time, together with dfs.namenode.avoid.read.stale.
> > > datanode.
> > >
> > > On Mon, Mar 23, 2015 at 5:29 PM Nicolas Liochon <nk...@gmail.com>
> > wrote:
> > >
> > >> Actually, double checking the final patch in HDFS-4721, the stale mode
> > is
> > >> taken in account. Bryan is right, it's worth checking the namenodes
> > >> config.
> > >> Especially, dfs.namenode.avoid.write.stale.datanode must be set to
> true
> > >> on
> > >> the namenode.
> > >>
> > >> On Mon, Mar 23, 2015 at 5:08 PM, Nicolas Liochon <nk...@gmail.com>
> > >> wrote:
> > >>
> > >> > stale should not help for recoverLease: it helps for reads, but it's
> > the
> > >> > step after lease recovery.
> > >> > It's not needed in recoverLease because the recoverLease in hdfs
> just
> > >> > sorts the datanode by the heartbeat time, so, usually the stale
> > datanode
> > >> > will be the last one of the list.
> > >> >
> > >> >
> > >> >
> > >> >
> > >> > On Mon, Mar 23, 2015 at 4:38 PM, Bryan Beaudreault <
> > >> > bbeaudreault@hubspot.com> wrote:
> > >> >
> > >> >> @Nicholas, I see, thanks.  I'll keep the settings at default.  So
> > >> really
> > >> >> if
> > >> >> everything else is configured properly you should never reach the
> > lease
> > >> >> recovery timeout in any failure scenarios.  It seems that the
> > staleness
> > >> >> check would be the thing that prevents this, correct?  I'm
> surprised
> > it
> > >> >> didn't help Dejan.
> > >> >>
> > >> >> On Mon, Mar 23, 2015 at 11:20 AM, Nicolas Liochon <
> nkeywal@gmail.com
> > >
> > >> >> wrote:
> > >> >>
> > >> >> > @bryan: yes, you can change hbase.lease.recovery.timeout if you
> > >> changed
> > >> >> he
> > >> >> > hdfs settings. But this setting is really for desperate cases.
> The
> > >> >> recover
> > >> >> > Lease should have succeeded before. As well, if you depend on
> > >> >> > hbase.lease.recovery.timeout, it means that you're wasting
> recovery
> > >> >> time:
> > >> >> > the lease should be recovered in a few seconds.
> > >> >> >
> > >> >> > On Mon, Mar 23, 2015 at 3:59 PM, Dejan Menges <
> > >> dejan.menges@gmail.com>
> > >> >> > wrote:
> > >> >> >
> > >> >> > > Interesting discussion I also found, gives me some more light
> on
> > >> what
> > >> >> > > Nicolas is talking about -
> > >> >> > https://issues.apache.org/jira/browse/HDFS-3703
> > >> >> > >
> > >> >> > > On Mon, Mar 23, 2015 at 3:53 PM Bryan Beaudreault <
> > >> >> > > bbeaudreault@hubspot.com>
> > >> >> > > wrote:
> > >> >> > >
> > >> >> > > > So it is safe to set hbase.lease.recovery.timeout lower if
> you
> > >> also
> > >> >> > > > set heartbeat.recheck.interval lower (lowering that 10.5 min
> > dead
> > >> >> node
> > >> >> > > > timer)?  Or is it recommended to not touch either of those?
> > >> >> > > >
> > >> >> > > > Reading the above with interest, thanks for digging in here
> > guys.
> > >> >> > > >
> > >> >> > > > On Mon, Mar 23, 2015 at 10:13 AM, Nicolas Liochon <
> > >> >> nkeywal@gmail.com>
> > >> >> > > > wrote:
> > >> >> > > >
> > >> >> > > > > If the node is actually down it's fine. But the node may
> not
> > be
> > >> >> that
> > >> >> > > down
> > >> >> > > > > (CAP theorem here); and then it's looking for trouble.
> > >> >> > > > > HDFS, by default declare a node as dead after 10:30. 15
> > minutes
> > >> >> is an
> > >> >> > > > extra
> > >> >> > > > > security. It seems your hdfs settings are different (or
> there
> > >> is a
> > >> >> > > > bug...).
> > >> >> > > > > There should be some info in the hdfs logs.
> > >> >> > > > >
> > >> >> > > > > On Mon, Mar 23, 2015 at 3:05 PM, Dejan Menges <
> > >> >> > dejan.menges@gmail.com>
> > >> >> > > > > wrote:
> > >> >> > > > >
> > >> >> > > > > > Will take a look.
> > >> >> > > > > >
> > >> >> > > > > > Actually, if node is down (someone unplugged network
> cable,
> > >> it
> > >> >> just
> > >> >> > > > died,
> > >> >> > > > > > whatever) what's chance it's going to become live so
> write
> > >> can
> > >> >> > > > continue?
> > >> >> > > > > On
> > >> >> > > > > > the other side, HBase is not starting recovery trying to
> > >> contact
> > >> >> > node
> > >> >> > > > > which
> > >> >> > > > > > is not there anymore, and even elected as dead on
> Namenode
> > >> side
> > >> >> > > > (another
> > >> >> > > > > > thing I didn't understood quite good).
> > >> >> > > > > >
> > >> >> > > > > > So what I was expecting is that as soon as Namenode
> decided
> > >> >> node is
> > >> >> > > > dead,
> > >> >> > > > > > that it would be enough for RegionServer to stop trying
> to
> > >> >> recover
> > >> >> > > from
> > >> >> > > > > the
> > >> >> > > > > > dead node, but it wasn't the case. Also, this whole MTTR
> > >> >> article in
> > >> >> > > > HBase
> > >> >> > > > > > book doesn't work at all with this parameter set to it's
> > >> default
> > >> >> > > value
> > >> >> > > > > (15
> > >> >> > > > > > minutes).
> > >> >> > > > > >
> > >> >> > > > > > So I'm still struggling to figure out what's drawback
> > >> exactly on
> > >> >> > > this?
> > >> >> > > > > >
> > >> >> > > > > > On Mon, Mar 23, 2015 at 2:50 PM Nicolas Liochon <
> > >> >> nkeywal@gmail.com
> > >> >> > >
> > >> >> > > > > wrote:
> > >> >> > > > > >
> > >> >> > > > > > > Thanks for the explanation. There is an issue if you
> > modify
> > >> >> this
> > >> >> > > > > setting
> > >> >> > > > > > > however.
> > >> >> > > > > > > hbase tries to recover the lease (i.e. be sure that
> > nobody
> > >> is
> > >> >> > > > writing)
> > >> >> > > > > > > If you change hbase.lease.recovery.timeout hbase will
> > start
> > >> >> the
> > >> >> > > > > recovery
> > >> >> > > > > > > (i.e. start to read) even if it's not sure that
> nobody's
> > >> >> writing.
> > >> >> > > > That
> > >> >> > > > > > > means there is a dataloss risk.
> > >> >> > > > > > > Basically, you must not see this warning: WARN
> > >> >> > > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils: Cannot
> > >> recoverLease
> > >> >> > after
> > >> >> > > > > > trying
> > >> >> > > > > > > for[]
> > >> >> > > > > > >
> > >> >> > > > > > > The recoverLease must succeed. The fact that it does
> not
> > >> >> after X
> > >> >> > > > tries
> > >> >> > > > > is
> > >> >> > > > > > > strange.
> > >> >> > > > > > > There may be a mistmatch between the hbase parameters
> and
> > >> the
> > >> >> > hdfs
> > >> >> > > > > ones.
> > >> >> > > > > > > You may need to have a look at the comments in
> > >> >> FSHDFSUtils.java
> > >> >> > > > > > >
> > >> >> > > > > > >
> > >> >> > > > > > >
> > >> >> > > > > > >
> > >> >> > > > > > > On Mon, Mar 23, 2015 at 2:15 PM, Dejan Menges <
> > >> >> > > > dejan.menges@gmail.com>
> > >> >> > > > > > > wrote:
> > >> >> > > > > > >
> > >> >> > > > > > > > I found the issue and fixed it, and will try to
> explain
> > >> here
> > >> >> > what
> > >> >> > > > was
> > >> >> > > > > > > > exactly in our case in case someone else finds this
> > >> >> interesting
> > >> >> > > > too.
> > >> >> > > > > > > >
> > >> >> > > > > > > > So initially, we had (couple of times) some network
> and
> > >> >> > hardware
> > >> >> > > > > issues
> > >> >> > > > > > > in
> > >> >> > > > > > > > our datacenters. When one server would die (literary
> > >> die, we
> > >> >> > had
> > >> >> > > > some
> > >> >> > > > > > > issue
> > >> >> > > > > > > > with PSUs) we saw issues with overall cluster
> > >> performance on
> > >> >> > > HBase
> > >> >> > > > > > side.
> > >> >> > > > > > > As
> > >> >> > > > > > > > cluster is quite big and live, it was also quite hard
> > to
> > >> >> figure
> > >> >> > > out
> > >> >> > > > > > exact
> > >> >> > > > > > > > root cause and how to fix the stuff we wanted to fix.
> > >> >> > > > > > > >
> > >> >> > > > > > > > So I set up another cluster, four nodes (with
> DataNode
> > >> and
> > >> >> > > > > > RegionServer)
> > >> >> > > > > > > > and two other nodes with HMaster and Namenode in HA,
> > >> using
> > >> >> same
> > >> >> > > > stuff
> > >> >> > > > > > we
> > >> >> > > > > > > > use on production. We pumped some data into it, and I
> > was
> > >> >> able
> > >> >> > to
> > >> >> > > > > > > reproduce
> > >> >> > > > > > > > same issue last week on it. What I tried to do is to
> > cut
> > >> one
> > >> >> > > server
> > >> >> > > > > > (shut
> > >> >> > > > > > > > down it's interface) when all is good with cluster,
> > when
> > >> we
> > >> >> > have
> > >> >> > > > > load,
> > >> >> > > > > > > and
> > >> >> > > > > > > > see what's going to happen.
> > >> >> > > > > > > >
> > >> >> > > > > > > > On Friday, after Nicolas mentioned, I started taking
> a
> > >> look
> > >> >> in
> > >> >> > > > HBase
> > >> >> > > > > > logs
> > >> >> > > > > > > > on the node which was mentioned in HMaster log as the
> > one
> > >> >> > taking
> > >> >> > > > over
> > >> >> > > > > > > > regions for the dead server. Basically what I was
> able
> > to
> > >> >> > observe
> > >> >> > > > was
> > >> >> > > > > > 15
> > >> >> > > > > > > > minutes time (+- couple of seconds, what was also
> > >> >> interesting,
> > >> >> > > and
> > >> >> > > > > will
> > >> >> > > > > > > got
> > >> >> > > > > > > > later to that) between HMaster figures out that one
> of
> > >> it's
> > >> >> > > > > > RegionServers
> > >> >> > > > > > > > is dead, and the time one of the mentioned nodes
> starts
> > >> >> taking
> > >> >> > > over
> > >> >> > > > > > those
> > >> >> > > > > > > > regions and they start appearing in HMaster's Web UI.
> > >> >> > > > > > > >
> > >> >> > > > > > > > I then set up everything like mentioned here
> > >> >> > > > > > > > http://hbase.apache.org/book.html#mttr - but still
> had
> > >> >> exactly
> > >> >> > > the
> > >> >> > > > > > same
> > >> >> > > > > > > > issues. Went over (again and again) all currently
> > >> configured
> > >> >> > > stuff,
> > >> >> > > > > but
> > >> >> > > > > > > > still had the same issue.
> > >> >> > > > > > > >
> > >> >> > > > > > > > Then I started looking into HDFS. Opened NameNode UI,
> > saw
> > >> >> all
> > >> >> > is
> > >> >> > > > > good,
> > >> >> > > > > > > took
> > >> >> > > > > > > > one node down, was also looking RegionServer logs in
> > the
> > >> >> same
> > >> >> > > time
> > >> >> > > > -
> > >> >> > > > > > and
> > >> >> > > > > > > I
> > >> >> > > > > > > > also see that it took ~15 minutes for Namenode to
> elect
> > >> dead
> > >> >> > node
> > >> >> > > > as
> > >> >> > > > > > > dead.
> > >> >> > > > > > > > Somehow in the same moment regions started getting
> back
> > >> to
> > >> >> > life.
> > >> >> > > I
> > >> >> > > > > > > remember
> > >> >> > > > > > > > in some older versions dfs timeout checks and check
> > >> retries.
> > >> >> > > Looked
> > >> >> > > > > > into
> > >> >> > > > > > > > defaults for our Hadoop version -
> > >> >> > > > > > > >
> > >> >> > > > > > > > http://hadoop.apache.org/docs/
> > >> r2.4.1/hadoop-project-dist/
> > >> >> > > > > > > hadoop-hdfs/hdfs-default.xml
> > >> >> > > > > > > > - and saw there that there's no recheck parameter
> > >> anymore.
> > >> >> > > Strange,
> > >> >> > > > > as
> > >> >> > > > > > on
> > >> >> > > > > > > > StackOverflow I found post from month ago, for newer
> > >> version
> > >> >> > than
> > >> >> > > > we
> > >> >> > > > > > use
> > >> >> > > > > > > > (we use 2.4.1, guy was using 2.6 -
> > >> dfs.namenode.heartbeat.
> > >> >> > > > > > > recheck-interval)
> > >> >> > > > > > > > I set it to 10 seconds as he mentioned, having checks
> > >> every
> > >> >> > three
> > >> >> > > > > > seconds
> > >> >> > > > > > > > (default) and got DataNode elected as dead in ~45
> > >> seconds,
> > >> >> as
> > >> >> > he
> > >> >> > > > > > > mentioned.
> > >> >> > > > > > > > Not sure why this parameter is not documented, but
> > >> >> obviously it
> > >> >> > > > > works.
> > >> >> > > > > > > >
> > >> >> > > > > > > > Then figured out it still didn't fix our HBase
> failover
> > >> >> issue.
> > >> >> > I
> > >> >> > > > was
> > >> >> > > > > > > > looking into HBase book again and again, and then saw
> > >> this
> > >> >> > part:
> > >> >> > > > > > > >
> > >> >> > > > > > > > "How much time we allow elapse between calls to
> recover
> > >> >> lease.
> > >> >> > > > Should
> > >> >> > > > > > be
> > >> >> > > > > > > > larger than the dfs timeout."
> > >> >> > > > > > > >
> > >> >> > > > > > > > This was description for
> hbase.lease.recovery.dfs.timeo
> > >> ut.
> > >> >> > Wasn't
> > >> >> > > > > sure
> > >> >> > > > > > > from
> > >> >> > > > > > > > the comment what of all timeouts that's possible to
> set
> > >> in
> > >> >> > > > > Hadoop/HBase
> > >> >> > > > > > > and
> > >> >> > > > > > > > that has something to do with DFS is this all about.
> > But
> > >> >> picked
> > >> >> > > > > > > > hbase.lease.recovery.timeout, and that was the catch.
> > >> >> > > > > > > >
> > >> >> > > > > > > > Initially, by default, hbase.lease.recovery.timeout
> is
> > >> set
> > >> >> to
> > >> >> > 15
> > >> >> > > > > > minutes.
> > >> >> > > > > > > > Not sure why, and wasn't able to find yet why, but
> > >> getting
> > >> >> this
> > >> >> > > > down
> > >> >> > > > > to
> > >> >> > > > > > > one
> > >> >> > > > > > > > minute (what's more than OK for us) I was able to get
> > >> rid of
> > >> >> > our
> > >> >> > > > > issue.
> > >> >> > > > > > > Not
> > >> >> > > > > > > > also sure why this is not mentioned in MTTR section
> in
> > >> HBase
> > >> >> > > book,
> > >> >> > > > as
> > >> >> > > > > > > > obviously MTTR doesn't work at all with this default
> > >> >> timeout,
> > >> >> > at
> > >> >> > > > > least
> > >> >> > > > > > it
> > >> >> > > > > > > > doesn't work the way we expected it to work.
> > >> >> > > > > > > >
> > >> >> > > > > > > > So thanks again for everyone being spammed with this,
> > and
> > >> >> > > specially
> > >> >> > > > > > > thanks
> > >> >> > > > > > > > to Nicolas pointing me to the right direction.
> > >> >> > > > > > > >
> > >> >> > > > > > > > On Mon, Mar 23, 2015 at 1:37 PM Nicolas Liochon <
> > >> >> > > nkeywal@gmail.com
> > >> >> > > > >
> > >> >> > > > > > > wrote:
> > >> >> > > > > > > >
> > >> >> > > > > > > > > the attachments are rejected by the mailing list,
> can
> > >> you
> > >> >> put
> > >> >> > > > then
> > >> >> > > > > on
> > >> >> > > > > > > > > pastebin?
> > >> >> > > > > > > > >
> > >> >> > > > > > > > > stale is mandatory (so it's good), but the issue
> here
> > >> is
> > >> >> just
> > >> >> > > > > before.
> > >> >> > > > > > > The
> > >> >> > > > > > > > > region server needs to read the file. In order to
> be
> > >> sure
> > >> >> > that
> > >> >> > > > > there
> > >> >> > > > > > is
> > >> >> > > > > > > > no
> > >> >> > > > > > > > > data loss, it needs to "recover the lease", that
> > means
> > >> >> > ensuring
> > >> >> > > > > that
> > >> >> > > > > > > > nobody
> > >> >> > > > > > > > > is writing the file. The regionserver calls the
> > >> namenode
> > >> >> to
> > >> >> > do
> > >> >> > > > this
> > >> >> > > > > > > > > recoverLease. So there should be some info in the
> > >> namenode
> > >> >> > > logs.
> > >> >> > > > > You
> > >> >> > > > > > > have
> > >> >> > > > > > > > > HDFS-4721 on your hdfs? The hbase book details
> (more
> > or
> > >> >> > > less...)
> > >> >> > > > > this
> > >> >> > > > > > > > > recoverLease stuff.
> > >> >> > > > > > > > >
> > >> >> > > > > > > > >
> > >> >> > > > > > > > > On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <
> > >> >> > > > > > dejan.menges@gmail.com
> > >> >> > > > > > > >
> > >> >> > > > > > > > > wrote:
> > >> >> > > > > > > > >
> > >> >> > > > > > > > > > And also, just checked -
> > >> dfs.namenode.avoid.read.stale.
> > >> >> > > > datanode
> > >> >> > > > > and
> > >> >> > > > > > > > > > dfs.namenode.avoid.write.stale.datanode
> > >> >> > > > > > > > > > are both true, and
> dfs.namenode.stale.datanode.in
> > >> terval
> > >> >> is
> > >> >> > > > set to
> > >> >> > > > > > > > > default
> > >> >> > > > > > > > > > 30000.
> > >> >> > > > > > > > > >
> > >> >> > > > > > > > > > On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <
> > >> >> > > > > > > dejan.menges@gmail.com>
> > >> >> > > > > > > > > > wrote:
> > >> >> > > > > > > > > >
> > >> >> > > > > > > > > > > Hi Nicolas,
> > >> >> > > > > > > > > > >
> > >> >> > > > > > > > > > > Please find log attached.
> > >> >> > > > > > > > > > >
> > >> >> > > > > > > > > > > As I see it now more clearly, it was trying to
> > >> recover
> > >> >> > HDFS
> > >> >> > > > > WALs
> > >> >> > > > > > > from
> > >> >> > > > > > > > > > node
> > >> >> > > > > > > > > > > that's dead:
> > >> >> > > > > > > > > > >
> > >> >> > > > > > > > > > > 2015-03-23 08:53:44,381 WARN
> > >> >> > > > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils:
> > >> >> > > > > > > > > > > Cannot recoverLease after trying for 900000ms
> > >> >> > > > > > > > > > > (hbase.lease.recovery.timeout); continuing,
> but
> > >> may be
> > >> >> > > > > > > DATALOSS!!!;
> > >> >> > > > > > > > > > > attempt=40 on
> > >> >> > > > > > > > > > >
> > >> >> > > > > > > > > >
> > >> >> > file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intent
> > >> >> > > > > > > > > ionally_get_down_by_getting_ne
> > >> twork_down},60020,1426862900
> > >> >> > > > > > > > > 506-splitting/{node_i_
> intentionally_get_down_by_gett
> > >> >> > > > > > > > > ing_network_down}%2C60020%2C14
> > >> 26862900506.1427096924508
> > >> >> > > > > > > > > > > after 908210ms
> > >> >> > > > > > > > > > >
> > >> >> > > > > > > > > > > And as you can see from the log, it tried 40
> > times,
> > >> >> what
> > >> >> > > took
> > >> >> > > > > it
> > >> >> > > > > > > > > exactly
> > >> >> > > > > > > > > > > 15 minutes.
> > >> >> > > > > > > > > > >
> > >> >> > > > > > > > > > > There's probably some parameter to tune to cut
> it
> > >> of
> > >> >> from
> > >> >> > > 40
> > >> >> > > > > > times
> > >> >> > > > > > > /
> > >> >> > > > > > > > 15
> > >> >> > > > > > > > > > > minutes to something more useful, as for 15
> > >> minutes we
> > >> >> > > don't
> > >> >> > > > > have
> > >> >> > > > > > > our
> > >> >> > > > > > > > > > > regions available, and HDFS have however
> > >> replication
> > >> >> > factor
> > >> >> > > > 3.
> > >> >> > > > > > > > > > >
> > >> >> > > > > > > > > > > Googling, if I figure out what's this I will
> post
> > >> it
> > >> >> > here.
> > >> >> > > > Will
> > >> >> > > > > > > also
> > >> >> > > > > > > > > > > appreciate if someone knows how to cut this
> down.
> > >> >> > > > > > > > > > >
> > >> >> > > > > > > > > > > Thanks,
> > >> >> > > > > > > > > > >
> > >> >> > > > > > > > > > > Dejan
> > >> >> > > > > > > > > > >
> > >> >> > > > > > > > > > > On Fri, Mar 20, 2015 at 3:49 PM Nicolas
> Liochon <
> > >> >> > > > > > nkeywal@gmail.com
> > >> >> > > > > > > >
> > >> >> > > > > > > > > > wrote:
> > >> >> > > > > > > > > > >
> > >> >> > > > > > > > > > >> The split is done by the region servers (the
> > >> master
> > >> >> > > > > > coordinates).
> > >> >> > > > > > > Is
> > >> >> > > > > > > > > > there
> > >> >> > > > > > > > > > >> some interesting stuff in their logs?
> > >> >> > > > > > > > > > >>
> > >> >> > > > > > > > > > >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges
> <
> > >> >> > > > > > > > dejan.menges@gmail.com
> > >> >> > > > > > > > > >
> > >> >> > > > > > > > > > >> wrote:
> > >> >> > > > > > > > > > >>
> > >> >> > > > > > > > > > >> > With client issue was that it was retrying
> > >> >> connecting
> > >> >> > to
> > >> >> > > > the
> > >> >> > > > > > > same
> > >> >> > > > > > > > > > region
> > >> >> > > > > > > > > > >> > servers even when they were reassigned.
> > >> Lowering it
> > >> >> > down
> > >> >> > > > > > helped
> > >> >> > > > > > > in
> > >> >> > > > > > > > > > this
> > >> >> > > > > > > > > > >> > specific use case, but yes, we still want
> > >> servers
> > >> >> to
> > >> >> > > > > > reallocate
> > >> >> > > > > > > > > > quickly.
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > What got me here:
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > http://hbase.apache.org/book.html#mttr
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > I basically set configuration exactly the
> same
> > >> way
> > >> >> as
> > >> >> > > it's
> > >> >> > > > > > > > explained
> > >> >> > > > > > > > > > >> here.
> > >> >> > > > > > > > > > >> > *zookeeper.session.timeout* is (and was
> > before)
> > >> >> 60000
> > >> >> > > (one
> > >> >> > > > > > > > minute).
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > So basically what happens is: - simulating
> > >> network
> > >> >> > > issues
> > >> >> > > > we
> > >> >> > > > > > had
> > >> >> > > > > > > > > > >> recently).
> > >> >> > > > > > > > > > >> > - After short time I see in HBase that my
> > >> >> RegionServer
> > >> >> > > is
> > >> >> > > > > > dead,
> > >> >> > > > > > > > and
> > >> >> > > > > > > > > as
> > >> >> > > > > > > > > > >> > total number of regions I see previous total
> > >> minus
> > >> >> > > number
> > >> >> > > > of
> > >> >> > > > > > > > regions
> > >> >> > > > > > > > > > >> that
> > >> >> > > > > > > > > > >> > were hosted on the node hosting RegionServer
> > >> that
> > >> >> just
> > >> >> > > > > > > > > 'disappeared'.
> > >> >> > > > > > > > > > >> > - In this point I want my clus
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > - I have test cluster consisting of four
> > nodes,
> > >> >> every
> > >> >> > > node
> > >> >> > > > > > being
> > >> >> > > > > > > > > > >> DataNode
> > >> >> > > > > > > > > > >> > and RegionServer.
> > >> >> > > > > > > > > > >> > - I simulate network partition on one
> (connect
> > >> to
> > >> >> it
> > >> >> > > > through
> > >> >> > > > > > > > console
> > >> >> > > > > > > > > > and
> > >> >> > > > > > > > > > >> > take network interface downter to recover as
> > >> soon
> > >> >> as
> > >> >> > > > > possible,
> > >> >> > > > > > > to
> > >> >> > > > > > > > > > start
> > >> >> > > > > > > > > > >> > serving missing regions.
> > >> >> > > > > > > > > > >> > - First thing I see in HMaster logs are:
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > 2015-03-20 14:17:26,015 INFO
> > >> >> > > > > > > > > > >> >
> > >> >> org.apache.hadoop.hbase.zookeeper.RegionServerTracker:
> > >> >> > > > > > > > RegionServer
> > >> >> > > > > > > > > > >> > ephemeral node deleted, processing
> expiration
> > >> >> > > > > > > > > > >> > [{name_of_node_I_took_down},60
> > >> 020,1426860403261]
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > 2015-03-20 14:17:26,067 INFO
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > org.apache.hadoop.hbase.master.handler.
> ServerShutdownHandler:
> > >> >> > > > > > > > > > Splitting
> > >> >> > > > > > > > > > >> > logs for
> > >> >> > {name_of_node_I_took_down},60020,1426860403261
> > >> >> > > > > before
> > >> >> > > > > > > > > > >> assignment.
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > 2015-03-20 14:17:26,105 INFO
> > >> >> > > > > > > > > > >> >
> > org.apache.hadoop.hbase.master.SplitLogManager:
> > >> >> dead
> > >> >> > > > > splitlog
> > >> >> > > > > > > > > workers
> > >> >> > > > > > > > > > [
> > >> >> > > > > > > > > > >> >
> > {name_of_node_I_took_down},60020,1426860403261]
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > 2015-03-20 14:17:26,107 INFO
> > >> >> > > > > > > > > > >> >
> > org.apache.hadoop.hbase.master.SplitLogManager:
> > >> >> > started
> > >> >> > > > > > > splitting
> > >> >> > > > > > > > 1
> > >> >> > > > > > > > > > >> logs in
> > >> >> > > > > > > > > > >> >
> > >> >> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_
> > >> >> > > > I_
> > >> >> > > > > > > > > took_down}
> > >> >> > > > > > > > > > >> > ,60020,1426860403261-splitting]
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > 2015-03-20 14:17:26,150 INFO
> > >> >> > > > > > > > > > >> >
> > org.apache.hadoop.hbase.master.SplitLogManager:
> > >> >> task
> > >> >> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > >
> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting
> > >> %2F
> > >> >> > > > > > > > > > >> >
> > >> >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
> > >> >> > > > 14268
> > >> >> > > > > > > > > 60404905
> > >> >> > > > > > > > > > >> > acquired by {fqdn_of_another_live_hnode},
> > >> >> > > > 60020,1426859445623
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > 2015-03-20 14:17:26,182 INFO
> > >> >> > > > > > > > > > >> >
> > org.apache.hadoop.hbase.master.SplitLogManager:
> > >> >> total
> > >> >> > > > tasks
> > >> >> > > > > =
> > >> >> > > > > > 1
> > >> >> > > > > > > > > > >> unassigned
> > >> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > >> >> > > > 2F{name_of_node_I_took_down}
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> >
> > >> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > >> >> > > > took_
> > >> >> > > > > > > > > > >>
> > >> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > >> >> > > > > > > > > > >> > = 1426861046182 last_version = 2
> > >> cur_worker_name =
> > >> >> > > > > > > > > > >> >
> > {fqdn_of_another_live_node},60020,1426859445623
> > >> >> > status =
> > >> >> > > > > > > > > in_progress
> > >> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch =
> > installed
> > >> = 1
> > >> >> > done
> > >> >> > > > = 0
> > >> >> > > > > > > > error =
> > >> >> > > > > > > > > > 0}
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > 2015-03-20 14:17:31,183 INFO
> > >> >> > > > > > > > > > >> >
> > org.apache.hadoop.hbase.master.SplitLogManager:
> > >> >> total
> > >> >> > > > tasks
> > >> >> > > > > =
> > >> >> > > > > > 1
> > >> >> > > > > > > > > > >> unassigned
> > >> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > >> >> > > > 2F{name_of_node_I_took_down}
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> >
> > >> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > >> >> > > > took_
> > >> >> > > > > > > > > > >>
> > >> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > >> >> > > > > > > > > > >> > = 1426861046182 last_version = 2
> > >> cur_worker_name =
> > >> >> > > > > > > > > > >> >
> > {fqdn_of_another_live_node},60020,1426859445623
> > >> >> > status =
> > >> >> > > > > > > > > in_progress
> > >> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch =
> > installed
> > >> = 1
> > >> >> > done
> > >> >> > > > = 0
> > >> >> > > > > > > > error =
> > >> >> > > > > > > > > > 0}
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > 2015-03-20 14:17:36,184 INFO
> > >> >> > > > > > > > > > >> >
> > org.apache.hadoop.hbase.master.SplitLogManager:
> > >> >> total
> > >> >> > > > tasks
> > >> >> > > > > =
> > >> >> > > > > > 1
> > >> >> > > > > > > > > > >> unassigned
> > >> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > >> >> > > > 2F{name_of_node_I_took_down}
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> >
> > >> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > >> >> > > > took_
> > >> >> > > > > > > > > > >>
> > >> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > >> >> > > > > > > > > > >> > = 1426861046182 last_version = 2
> > >> cur_worker_name =
> > >> >> > > > > > > > > > >> >
> > {fqdn_of_another_live_node},60020,1426859445623
> > >> >> > status =
> > >> >> > > > > > > > > in_progress
> > >> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch =
> > installed
> > >> = 1
> > >> >> > done
> > >> >> > > > = 0
> > >> >> > > > > > > > error =
> > >> >> > > > > > > > > > 0}
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > 2015-03-20 14:17:42,185 INFO
> > >> >> > > > > > > > > > >> >
> > org.apache.hadoop.hbase.master.SplitLogManager:
> > >> >> total
> > >> >> > > > tasks
> > >> >> > > > > =
> > >> >> > > > > > 1
> > >> >> > > > > > > > > > >> unassigned
> > >> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > >> >> > > > 2F{name_of_node_I_took_down}
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> >
> > >> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > >> >> > > > took_
> > >> >> > > > > > > > > > >>
> > >> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > >> >> > > > > > > > > > >> > = 1426861046182 last_version = 2
> > >> cur_worker_name =
> > >> >> > > > > > > > > > >> >
> > {fqdn_of_another_live_node},60020,1426859445623
> > >> >> > status =
> > >> >> > > > > > > > > in_progress
> > >> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch =
> > installed
> > >> = 1
> > >> >> > done
> > >> >> > > > = 0
> > >> >> > > > > > > > error =
> > >> >> > > > > > > > > > 0}
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > 2015-03-20 14:17:48,184 INFO
> > >> >> > > > > > > > > > >> >
> > org.apache.hadoop.hbase.master.SplitLogManager:
> > >> >> total
> > >> >> > > > tasks
> > >> >> > > > > =
> > >> >> > > > > > 1
> > >> >> > > > > > > > > > >> unassigned
> > >> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > >> >> > > > 2F{name_of_node_I_took_down}
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> >
> > >> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > >> >> > > > took_
> > >> >> > > > > > > > > > >>
> > >> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > >> >> > > > > > > > > > >> > = 1426861046182 last_version = 2
> > >> cur_worker_name =
> > >> >> > > > > > > > > > >> >
> > {fqdn_of_another_live_node},60020,1426859445623
> > >> >> > status =
> > >> >> > > > > > > > > in_progress
> > >> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch =
> > installed
> > >> = 1
> > >> >> > done
> > >> >> > > > = 0
> > >> >> > > > > > > > error =
> > >> >> > > > > > > > > > 0}
> > >> >> > > > > > > > > > >> > In the meantime, In hbase...out log I got
> > this:
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > ==>
> > >> >> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out
> > >> >> > <==
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > java.io.IOException: Call to
> > >> >> > > > > > > > > > >> >
> > >> >> > > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_
> > >> >> > > > dow
> > >> >> > > > > > > > > n}:60020
> > >> >> > > > > > > > > > >> > failed on local exception:
> > >> >> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> > >> >> > > > CallTimeoutException:
> > >> >> > > > > > > Call
> > >> >> > > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > >
> > org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
> > >> >> > > > > > > > > > >> t.java:1532)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:
> > >> >> > > > > > > 1502)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> >
> > >> >> > > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(
> > >> >> > > > Rpc
> > >> >> > > > > > > > > > >> Client.java:1684)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> > >> >> > > > BlockingRpcChannelImpl
> > >> >> > > > > > > > > ementati
> > >> >> > > > > > > > > > >> on.callBlockingMethod(RpcClient.java:1737)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> >
> > >> >> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$
> > >> >> > > > > > > > > AdminService$
> > >> >> > > > > > > > > > >> BlockingStub.getRegionInfo(Adm
> > >> inProtos.java:20806)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > org.apache.hadoop.hbase.client.HBaseAdmin.
> > >> >> > > > getCompactionState
> > >> >> > > > > > > > > > >> (HBaseAdmin.java:2524)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> >
> > >> >> org.apache.hadoop.hbase.generated.master.table_jsp._
> > >> >> > > > jspServi
> > >> >> > > > > > > > > > >> ce(table_jsp.java:167)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.
> > >> >> > > > > > > > > java:98)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > >
> > javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > >
> > org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
> > >> >> > > > > > > > > > >> .java:511)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> >
> > >> >> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> > >> >> > > > doFilte
> > >> >> > > > > > > > > > >> r(ServletHandler.java:1221)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> >
> > org.apache.hadoop.http.lib.StaticUserWebFilter$
> > >> >> > > > StaticUserFil
> > >> >> > > > > > > > > > >> ter.doFilter(StaticUserWebFilter.java:109)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> >
> > >> >> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> > >> >> > > > doFilte
> > >> >> > > > > > > > > > >> r(ServletHandler.java:1212)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> >
> > >> >> org.apache.hadoop.http.HttpServer$QuotingInputFilter.
> > >> >> > > > doFilte
> > >> >> > > > > > > > > > >> r(HttpServer.java:1081)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> >
> > >> >> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> > >> >> > > > doFilte
> > >> >> > > > > > > > > > >> r(ServletHandler.java:1212)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > >
> > >> >> > > > > >
> > >> >> >
> > org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> >
> > >> >> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> > >> >> > > > doFilte
> > >> >> > > > > > > > > > >> r(ServletHandler.java:1212)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > >
> > org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
> > >> >> > > > > > > > > > >> er.java:399)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> > org.mortbay.jetty.security.Sec
> > >> urityHandler.handle(
> > >> >> > > > SecurityHa
> > >> >> > > > > > > > > > >> ndler.java:216)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > >
> > org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
> > >> >> > > > > > > > > > >> er.java:182)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > >
> > org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
> > >> >> > > > > > > > > > >> er.java:766)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > >
> > org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
> > >> >> > > > > > > > > > >> java:450)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > org.mortbay.jetty.handler.Cont
> > >> extHandlerCollection.
> > >> >> > > > handle(Co
> > >> >> > > > > > > > > > >> ntextHandlerCollection.java:230)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > >
> > org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
> > >> >> > > > > > > > > > >> er.java:152)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at org.mortbay.jetty.Server.handl
> > >> e(Server.java:326)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > >
> > org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
> > >> >> > > > > > > > > > >> n.java:542)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > org.mortbay.jetty.HttpConnecti
> > >> on$RequestHandler.
> > >> >> > > > headerComple
> > >> >> > > > > > > > > > >> te(HttpConnection.java:928)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > org.mortbay.jetty.HttpParser.
> parseNext(HttpParser.java:549)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > >
> > >> >> > org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > >
> > >> >> > org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > org.mortbay.io.nio.
> SelectChannelEndPoint.run(
> > >> >> > > > SelectChannelEn
> > >> >> > > > > > > > > > >> dPoint.java:410)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> >
> > org.mortbay.thread.QueuedThreadPool$PoolThread.
> > >> run(
> > >> >> > > > > > > > > > >> QueuedThreadPool.java:582)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > Caused by: org.apache.hadoop.hbase.ipc.Rp
> > >> cClient$
> > >> >> > > > > > > > > CallTimeoutException:
> > >> >> > > > > > > > > > >> Call
> > >> >> > > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.Rp
> > >> cClient$Connection.
> > >> >> > > > cleanupCall
> > >> >> > > > > > > > > > >> s(RpcClient.java:1234)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.Rp
> > >> cClient$Connection.
> > >> >> > > > readRespons
> > >> >> > > > > > > > > > >> e(RpcClient.java:1171)
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > >
> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
> > >> >> > > > > > > > > > >> nt.java:751)
> > >> >> > > > > > > > > > >> > Beside this same issue, please note that
> first
> > >> >> message
> > >> >> > > was
> > >> >> > > > > at
> > >> >> > > > > > > > > > 2015-03-20
> > >> >> > > > > > > > > > >> > 14:17:26,015. And then (we got to the point
> > >> when it
> > >> >> > > > started
> > >> >> > > > > > > > > > transition):
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > 2015-03-20 14:32:35,059 INFO
> > >> >> > > > > > > > > > >> >
> > org.apache.hadoop.hbase.master.SplitLogManager:
> > >> >> task
> > >> >> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > >
> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting
> > >> %2F
> > >> >> > > > > > > > > > >> >
> > >> >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
> > >> >> > > > 14268
> > >> >> > > > > > > > > 60404905
> > >> >> > > > > > > > > > >> > entered state: DONE
> > >> >> > > > > > {fqdn_of_new_live_node},60020,1426859445623
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > 2015-03-20 14:32:35,109 INFO
> > >> >> > > > > > > > > > >> >
> > org.apache.hadoop.hbase.master.SplitLogManager:
> > >> >> Done
> > >> >> > > > > splitting
> > >> >> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F{name_o
> > >> f_node_I_took_down}
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > >
> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_do
> > >> wn}
> > >> >> > > > > > > > > > >> > %252C60020%252C1426860403261.1426860404905
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > 2015-03-20 14:32:35,190 INFO
> > >> >> > > > > > > > > > >> >
> > org.apache.hadoop.hbase.master.SplitLogManager:
> > >> >> > finished
> > >> >> > > > > > > splitting
> > >> >> > > > > > > > > > >> (more
> > >> >> > > > > > > > > > >> > than or equal to) 9 bytes in 1 log files in
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> >
> > >> >> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_
> > >> >> > > > of_
> > >> >> > > > > > > > > > >> node_I_took_down},60020,
> 1426860403261-splitting]
> > >> >> > > > > > > > > > >> > in 909083ms
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > >> >> > > > org.apache.hadoop.hbase.master
> > >> >> > > > > > > > > > >> .RegionStates:
> > >> >> > > > > > > > > > >> > Transitioned {
> 0e7cc87a4ef6c47a779186f5bf79a01c
> > >> >> > > > state=OPEN,
> > >> >> > > > > > > > > > >> > ts=1426860639088,
> > >> >> > > > > > > > > > server={name_of_node_I_took_do
> > >> wn},60020,1426860403261}
> > >> >> > > > > > > > > > >> to
> > >> >> > > > > > > > > > >> > {0e7cc87a4ef6c47a779186f5bf79a01c
> > >> state=OFFLINE,
> > >> >> > > > > > > ts=1426861955191,
> > >> >> > > > > > > > > > >> server=
> > >> >> > > > > > > > > > >> >
> > {name_of_node_I_took_down},60020,1426860403261}
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > >> >> > > > org.apache.hadoop.hbase.master
> > >> >> > > > > > > > > > >> .RegionStates:
> > >> >> > > > > > > > > > >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c
> from
> > >> >> > > > > > > > > > >> {name_of_node_I_took_down}
> > >> >> > > > > > > > > > >> > ,60020,1426860403261
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > >> >> > > > org.apache.hadoop.hbase.master
> > >> >> > > > > > > > > > >> .RegionStates:
> > >> >> > > > > > > > > > >> > Transitioned {
> 25ab6e7b42e36ddaa723d71be5954543
> > >> >> > > > state=OPEN,
> > >> >> > > > > > > > > > >> > ts=1426860641783,
> > >> >> > > > > > > > > > server={name_of_node_I_took_do
> > >> wn},60020,1426860403261}
> > >> >> > > > > > > > > > >> to
> > >> >> > > > > > > > > > >> > {25ab6e7b42e36ddaa723d71be5954543
> > >> state=OFFLINE,
> > >> >> > > > > > > ts=1426861955191,
> > >> >> > > > > > > > > > >> server=
> > >> >> > > > > > > > > > >> >
> > {name_of_node_I_took_down},60020,1426860403261}
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > >> >> > > > org.apache.hadoop.hbase.master
> > >> >> > > > > > > > > > >> .RegionStates:
> > >> >> > > > > > > > > > >> > Offlined 25ab6e7b42e36ddaa723d71be5954543
> from
> > >> >> > > > > > > > > > >> {name_of_node_I_took_down}
> > >> >> > > > > > > > > > >> > ,60020,1426860403261
> > >> >> > > > > > > > > > >> > At this point, note that it finished
> > >> >> SplitLogManager
> > >> >> > > task
> > >> >> > > > at
> > >> >> > > > > > > > > 14:32:35
> > >> >> > > > > > > > > > >> and
> > >> >> > > > > > > > > > >> > started transitioning just after that. So
> this
> > >> is
> > >> >> 15
> > >> >> > > > minutes
> > >> >> > > > > > > that
> > >> >> > > > > > > > > I'm
> > >> >> > > > > > > > > > >> > talking about.
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > What am I missing?
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas
> > Liochon
> > >> <
> > >> >> > > > > > > > nkeywal@gmail.com>
> > >> >> > > > > > > > > > >> wrote:
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > > You've changed the value of
> > >> >> hbase.zookeeper.timeout
> > >> >> > to
> > >> >> > > > 15
> > >> >> > > > > > > > > minutes? A
> > >> >> > > > > > > > > > >> very
> > >> >> > > > > > > > > > >> > > reasonable target is 1 minute before
> > >> relocating
> > >> >> the
> > >> >> > > > > regions.
> > >> >> > > > > > > > > That's
> > >> >> > > > > > > > > > >> the
> > >> >> > > > > > > > > > >> > > default iirc. You can push it to 20s, but
> > then
> > >> >> > > > > > > > > gc-stopping-the-world
> > >> >> > > > > > > > > > >> > > becomes more of an issue. 15 minutes is
> > >> really a
> > >> >> > lot.
> > >> >> > > > The
> > >> >> > > > > > hdfs
> > >> >> > > > > > > > > stale
> > >> >> > > > > > > > > > >> mode
> > >> >> > > > > > > > > > >> > > must always be used, with a lower timeout
> > than
> > >> >> the
> > >> >> > > hbase
> > >> >> > > > > > one.
> > >> >> > > > > > > > > > >> > >
> > >> >> > > > > > > > > > >> > > Client side there should be nothing to do
> > >> >> (excepted
> > >> >> > > for
> > >> >> > > > > > > advanced
> > >> >> > > > > > > > > > >> stuff);
> > >> >> > > > > > > > > > >> > at
> > >> >> > > > > > > > > > >> > > each retry the client checks the location
> of
> > >> the
> > >> >> > > > regions.
> > >> >> > > > > If
> > >> >> > > > > > > you
> > >> >> > > > > > > > > > lower
> > >> >> > > > > > > > > > >> > the
> > >> >> > > > > > > > > > >> > > number of retry the client will fail
> sooner,
> > >> but
> > >> >> > > usually
> > >> >> > > > > you
> > >> >> > > > > > > > don't
> > >> >> > > > > > > > > > >> want
> > >> >> > > > > > > > > > >> > the
> > >> >> > > > > > > > > > >> > > client to fail, you want the servers to
> > >> >> reallocate
> > >> >> > > > > quickly.
> > >> >> > > > > > > > > > >> > >
> > >> >> > > > > > > > > > >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan
> > Menges
> > >> <
> > >> >> > > > > > > > > > dejan.menges@gmail.com
> > >> >> > > > > > > > > > >> >
> > >> >> > > > > > > > > > >> > > wrote:
> > >> >> > > > > > > > > > >> > >
> > >> >> > > > > > > > > > >> > > > Hi,
> > >> >> > > > > > > > > > >> > > >
> > >> >> > > > > > > > > > >> > > > Sorry for little bit late update, but
> > >> managed
> > >> >> to
> > >> >> > > > narrow
> > >> >> > > > > it
> > >> >> > > > > > > > > little
> > >> >> > > > > > > > > > >> bit
> > >> >> > > > > > > > > > >> > > down.
> > >> >> > > > > > > > > > >> > > >
> > >> >> > > > > > > > > > >> > > > We didn't update yet, as we are using
> > >> >> Hortonworks
> > >> >> > > > > > > distribution
> > >> >> > > > > > > > > > right
> > >> >> > > > > > > > > > >> > now,
> > >> >> > > > > > > > > > >> > > > and even if we update we will get
> 0.98.4.
> > >> >> However,
> > >> >> > > > looks
> > >> >> > > > > > > that
> > >> >> > > > > > > > > > issue
> > >> >> > > > > > > > > > >> > here
> > >> >> > > > > > > > > > >> > > > was in our use case and configuration
> > (still
> > >> >> > looking
> > >> >> > > > > into
> > >> >> > > > > > > it).
> > >> >> > > > > > > > > > >> > > >
> > >> >> > > > > > > > > > >> > > > Basically, initially I saw that when one
> > >> server
> > >> >> > goes
> > >> >> > > > > down,
> > >> >> > > > > > > we
> > >> >> > > > > > > > > > start
> > >> >> > > > > > > > > > >> > > having
> > >> >> > > > > > > > > > >> > > > performance issues in general, but it
> > >> managed
> > >> >> to
> > >> >> > be
> > >> >> > > on
> > >> >> > > > > our
> > >> >> > > > > > > > > client
> > >> >> > > > > > > > > > >> side,
> > >> >> > > > > > > > > > >> > > due
> > >> >> > > > > > > > > > >> > > > to caching, and clients were trying to
> > >> >> reconnect
> > >> >> > to
> > >> >> > > > > nodes
> > >> >> > > > > > > that
> > >> >> > > > > > > > > > were
> > >> >> > > > > > > > > > >> > > offline
> > >> >> > > > > > > > > > >> > > > and later trying to get regions from
> those
> > >> >> nodes
> > >> >> > > too.
> > >> >> > > > > This
> > >> >> > > > > > > is
> > >> >> > > > > > > > > > >> basically
> > >> >> > > > > > > > > > >> > > why
> > >> >> > > > > > > > > > >> > > > on server side I didn't manage to see
> > >> anything
> > >> >> in
> > >> >> > > logs
> > >> >> > > > > > that
> > >> >> > > > > > > > > would
> > >> >> > > > > > > > > > >> be at
> > >> >> > > > > > > > > > >> > > > least little bit interesting and point
> me
> > >> into
> > >> >> > > desired
> > >> >> > > > > > > > > direction.
> > >> >> > > > > > > > > > >> > > >
> > >> >> > > > > > > > > > >> > > > Another question that popped up to me
> is -
> > >> in
> > >> >> case
> > >> >> > > > > server
> > >> >> > > > > > is
> > >> >> > > > > > > > > down
> > >> >> > > > > > > > > > >> (and
> > >> >> > > > > > > > > > >> > > with
> > >> >> > > > > > > > > > >> > > > it DataNode and HRegionServer it was
> > >> hosting) -
> > >> >> > > what's
> > >> >> > > > > > > optimal
> > >> >> > > > > > > > > > time
> > >> >> > > > > > > > > > >> to
> > >> >> > > > > > > > > > >> > > set
> > >> >> > > > > > > > > > >> > > > for HMaster to consider server dead
> > reassign
> > >> >> > regions
> > >> >> > > > > > > somewhere
> > >> >> > > > > > > > > > >> else, as
> > >> >> > > > > > > > > > >> > > > this is another performance bottleneck
> we
> > >> hit
> > >> >> > during
> > >> >> > > > > > > inability
> > >> >> > > > > > > > > to
> > >> >> > > > > > > > > > >> > access
> > >> >> > > > > > > > > > >> > > > regions? In our case it's configured to
> 15
> > >> >> > minutes,
> > >> >> > > > and
> > >> >> > > > > > > simple
> > >> >> > > > > > > > > > logic
> > >> >> > > > > > > > > > >> > > tells
> > >> >> > > > > > > > > > >> > > > me if you want it earlier then configure
> > >> lower
> > >> >> > > number
> > >> >> > > > of
> > >> >> > > > > > > > > retries,
> > >> >> > > > > > > > > > >> but
> > >> >> > > > > > > > > > >> > > issue
> > >> >> > > > > > > > > > >> > > > is as always in details, so not sure if
> > >> anyone
> > >> >> > knows
> > >> >> > > > > some
> > >> >> > > > > > > > better
> > >> >> > > > > > > > > > >> math
> > >> >> > > > > > > > > > >> > for
> > >> >> > > > > > > > > > >> > > > this?
> > >> >> > > > > > > > > > >> > > >
> > >> >> > > > > > > > > > >> > > > And last question - is it possible to
> > >> manually
> > >> >> > force
> > >> >> > > > > HBase
> > >> >> > > > > > > to
> > >> >> > > > > > > > > > >> reassign
> > >> >> > > > > > > > > > >> > > > regions? In this case, while HMaster is
> > >> >> retrying
> > >> >> > to
> > >> >> > > > > > contact
> > >> >> > > > > > > > node
> > >> >> > > > > > > > > > >> that's
> > >> >> > > > > > > > > > >> > > > dead, it's impossible to force it using
> > >> >> 'balancer'
> > >> >> > > > > > command.
> > >> >> > > > > > > > > > >> > > >
> > >> >> > > > > > > > > > >> > > > Thanks a lot!
> > >> >> > > > > > > > > > >> > > >
> > >> >> > > > > > > > > > >> > > > Dejan
> > >> >> > > > > > > > > > >> > > >
> > >> >>

Re: Strange issue when DataNode goes down

Posted by Nicolas Liochon <nk...@gmail.com>.
Ok, so hopefully there are some info in the namenode & datanode logs.

On Mon, Mar 23, 2015 at 5:32 PM, Dejan Menges <de...@gmail.com>
wrote:

> ...and I also got sure that it's applied with hdfs getconf -confKey...
>
> On Mon, Mar 23, 2015 at 5:31 PM Dejan Menges <de...@gmail.com>
> wrote:
>
> > It was true all the time, together with dfs.namenode.avoid.read.stale.
> > datanode.
> >
> > On Mon, Mar 23, 2015 at 5:29 PM Nicolas Liochon <nk...@gmail.com>
> wrote:
> >
> >> Actually, double checking the final patch in HDFS-4721, the stale mode
> is
> >> taken in account. Bryan is right, it's worth checking the namenodes
> >> config.
> >> Especially, dfs.namenode.avoid.write.stale.datanode must be set to true
> >> on
> >> the namenode.
> >>
> >> On Mon, Mar 23, 2015 at 5:08 PM, Nicolas Liochon <nk...@gmail.com>
> >> wrote:
> >>
> >> > stale should not help for recoverLease: it helps for reads, but it's
> the
> >> > step after lease recovery.
> >> > It's not needed in recoverLease because the recoverLease in hdfs just
> >> > sorts the datanode by the heartbeat time, so, usually the stale
> datanode
> >> > will be the last one of the list.
> >> >
> >> >
> >> >
> >> >
> >> > On Mon, Mar 23, 2015 at 4:38 PM, Bryan Beaudreault <
> >> > bbeaudreault@hubspot.com> wrote:
> >> >
> >> >> @Nicholas, I see, thanks.  I'll keep the settings at default.  So
> >> really
> >> >> if
> >> >> everything else is configured properly you should never reach the
> lease
> >> >> recovery timeout in any failure scenarios.  It seems that the
> staleness
> >> >> check would be the thing that prevents this, correct?  I'm surprised
> it
> >> >> didn't help Dejan.
> >> >>
> >> >> On Mon, Mar 23, 2015 at 11:20 AM, Nicolas Liochon <nkeywal@gmail.com
> >
> >> >> wrote:
> >> >>
> >> >> > @bryan: yes, you can change hbase.lease.recovery.timeout if you
> >> changed
> >> >> he
> >> >> > hdfs settings. But this setting is really for desperate cases. The
> >> >> recover
> >> >> > Lease should have succeeded before. As well, if you depend on
> >> >> > hbase.lease.recovery.timeout, it means that you're wasting recovery
> >> >> time:
> >> >> > the lease should be recovered in a few seconds.
> >> >> >
> >> >> > On Mon, Mar 23, 2015 at 3:59 PM, Dejan Menges <
> >> dejan.menges@gmail.com>
> >> >> > wrote:
> >> >> >
> >> >> > > Interesting discussion I also found, gives me some more light on
> >> what
> >> >> > > Nicolas is talking about -
> >> >> > https://issues.apache.org/jira/browse/HDFS-3703
> >> >> > >
> >> >> > > On Mon, Mar 23, 2015 at 3:53 PM Bryan Beaudreault <
> >> >> > > bbeaudreault@hubspot.com>
> >> >> > > wrote:
> >> >> > >
> >> >> > > > So it is safe to set hbase.lease.recovery.timeout lower if you
> >> also
> >> >> > > > set heartbeat.recheck.interval lower (lowering that 10.5 min
> dead
> >> >> node
> >> >> > > > timer)?  Or is it recommended to not touch either of those?
> >> >> > > >
> >> >> > > > Reading the above with interest, thanks for digging in here
> guys.
> >> >> > > >
> >> >> > > > On Mon, Mar 23, 2015 at 10:13 AM, Nicolas Liochon <
> >> >> nkeywal@gmail.com>
> >> >> > > > wrote:
> >> >> > > >
> >> >> > > > > If the node is actually down it's fine. But the node may not
> be
> >> >> that
> >> >> > > down
> >> >> > > > > (CAP theorem here); and then it's looking for trouble.
> >> >> > > > > HDFS, by default declare a node as dead after 10:30. 15
> minutes
> >> >> is an
> >> >> > > > extra
> >> >> > > > > security. It seems your hdfs settings are different (or there
> >> is a
> >> >> > > > bug...).
> >> >> > > > > There should be some info in the hdfs logs.
> >> >> > > > >
> >> >> > > > > On Mon, Mar 23, 2015 at 3:05 PM, Dejan Menges <
> >> >> > dejan.menges@gmail.com>
> >> >> > > > > wrote:
> >> >> > > > >
> >> >> > > > > > Will take a look.
> >> >> > > > > >
> >> >> > > > > > Actually, if node is down (someone unplugged network cable,
> >> it
> >> >> just
> >> >> > > > died,
> >> >> > > > > > whatever) what's chance it's going to become live so write
> >> can
> >> >> > > > continue?
> >> >> > > > > On
> >> >> > > > > > the other side, HBase is not starting recovery trying to
> >> contact
> >> >> > node
> >> >> > > > > which
> >> >> > > > > > is not there anymore, and even elected as dead on Namenode
> >> side
> >> >> > > > (another
> >> >> > > > > > thing I didn't understood quite good).
> >> >> > > > > >
> >> >> > > > > > So what I was expecting is that as soon as Namenode decided
> >> >> node is
> >> >> > > > dead,
> >> >> > > > > > that it would be enough for RegionServer to stop trying to
> >> >> recover
> >> >> > > from
> >> >> > > > > the
> >> >> > > > > > dead node, but it wasn't the case. Also, this whole MTTR
> >> >> article in
> >> >> > > > HBase
> >> >> > > > > > book doesn't work at all with this parameter set to it's
> >> default
> >> >> > > value
> >> >> > > > > (15
> >> >> > > > > > minutes).
> >> >> > > > > >
> >> >> > > > > > So I'm still struggling to figure out what's drawback
> >> exactly on
> >> >> > > this?
> >> >> > > > > >
> >> >> > > > > > On Mon, Mar 23, 2015 at 2:50 PM Nicolas Liochon <
> >> >> nkeywal@gmail.com
> >> >> > >
> >> >> > > > > wrote:
> >> >> > > > > >
> >> >> > > > > > > Thanks for the explanation. There is an issue if you
> modify
> >> >> this
> >> >> > > > > setting
> >> >> > > > > > > however.
> >> >> > > > > > > hbase tries to recover the lease (i.e. be sure that
> nobody
> >> is
> >> >> > > > writing)
> >> >> > > > > > > If you change hbase.lease.recovery.timeout hbase will
> start
> >> >> the
> >> >> > > > > recovery
> >> >> > > > > > > (i.e. start to read) even if it's not sure that nobody's
> >> >> writing.
> >> >> > > > That
> >> >> > > > > > > means there is a dataloss risk.
> >> >> > > > > > > Basically, you must not see this warning: WARN
> >> >> > > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils: Cannot
> >> recoverLease
> >> >> > after
> >> >> > > > > > trying
> >> >> > > > > > > for[]
> >> >> > > > > > >
> >> >> > > > > > > The recoverLease must succeed. The fact that it does not
> >> >> after X
> >> >> > > > tries
> >> >> > > > > is
> >> >> > > > > > > strange.
> >> >> > > > > > > There may be a mistmatch between the hbase parameters and
> >> the
> >> >> > hdfs
> >> >> > > > > ones.
> >> >> > > > > > > You may need to have a look at the comments in
> >> >> FSHDFSUtils.java
> >> >> > > > > > >
> >> >> > > > > > >
> >> >> > > > > > >
> >> >> > > > > > >
> >> >> > > > > > > On Mon, Mar 23, 2015 at 2:15 PM, Dejan Menges <
> >> >> > > > dejan.menges@gmail.com>
> >> >> > > > > > > wrote:
> >> >> > > > > > >
> >> >> > > > > > > > I found the issue and fixed it, and will try to explain
> >> here
> >> >> > what
> >> >> > > > was
> >> >> > > > > > > > exactly in our case in case someone else finds this
> >> >> interesting
> >> >> > > > too.
> >> >> > > > > > > >
> >> >> > > > > > > > So initially, we had (couple of times) some network and
> >> >> > hardware
> >> >> > > > > issues
> >> >> > > > > > > in
> >> >> > > > > > > > our datacenters. When one server would die (literary
> >> die, we
> >> >> > had
> >> >> > > > some
> >> >> > > > > > > issue
> >> >> > > > > > > > with PSUs) we saw issues with overall cluster
> >> performance on
> >> >> > > HBase
> >> >> > > > > > side.
> >> >> > > > > > > As
> >> >> > > > > > > > cluster is quite big and live, it was also quite hard
> to
> >> >> figure
> >> >> > > out
> >> >> > > > > > exact
> >> >> > > > > > > > root cause and how to fix the stuff we wanted to fix.
> >> >> > > > > > > >
> >> >> > > > > > > > So I set up another cluster, four nodes (with DataNode
> >> and
> >> >> > > > > > RegionServer)
> >> >> > > > > > > > and two other nodes with HMaster and Namenode in HA,
> >> using
> >> >> same
> >> >> > > > stuff
> >> >> > > > > > we
> >> >> > > > > > > > use on production. We pumped some data into it, and I
> was
> >> >> able
> >> >> > to
> >> >> > > > > > > reproduce
> >> >> > > > > > > > same issue last week on it. What I tried to do is to
> cut
> >> one
> >> >> > > server
> >> >> > > > > > (shut
> >> >> > > > > > > > down it's interface) when all is good with cluster,
> when
> >> we
> >> >> > have
> >> >> > > > > load,
> >> >> > > > > > > and
> >> >> > > > > > > > see what's going to happen.
> >> >> > > > > > > >
> >> >> > > > > > > > On Friday, after Nicolas mentioned, I started taking a
> >> look
> >> >> in
> >> >> > > > HBase
> >> >> > > > > > logs
> >> >> > > > > > > > on the node which was mentioned in HMaster log as the
> one
> >> >> > taking
> >> >> > > > over
> >> >> > > > > > > > regions for the dead server. Basically what I was able
> to
> >> >> > observe
> >> >> > > > was
> >> >> > > > > > 15
> >> >> > > > > > > > minutes time (+- couple of seconds, what was also
> >> >> interesting,
> >> >> > > and
> >> >> > > > > will
> >> >> > > > > > > got
> >> >> > > > > > > > later to that) between HMaster figures out that one of
> >> it's
> >> >> > > > > > RegionServers
> >> >> > > > > > > > is dead, and the time one of the mentioned nodes starts
> >> >> taking
> >> >> > > over
> >> >> > > > > > those
> >> >> > > > > > > > regions and they start appearing in HMaster's Web UI.
> >> >> > > > > > > >
> >> >> > > > > > > > I then set up everything like mentioned here
> >> >> > > > > > > > http://hbase.apache.org/book.html#mttr - but still had
> >> >> exactly
> >> >> > > the
> >> >> > > > > > same
> >> >> > > > > > > > issues. Went over (again and again) all currently
> >> configured
> >> >> > > stuff,
> >> >> > > > > but
> >> >> > > > > > > > still had the same issue.
> >> >> > > > > > > >
> >> >> > > > > > > > Then I started looking into HDFS. Opened NameNode UI,
> saw
> >> >> all
> >> >> > is
> >> >> > > > > good,
> >> >> > > > > > > took
> >> >> > > > > > > > one node down, was also looking RegionServer logs in
> the
> >> >> same
> >> >> > > time
> >> >> > > > -
> >> >> > > > > > and
> >> >> > > > > > > I
> >> >> > > > > > > > also see that it took ~15 minutes for Namenode to elect
> >> dead
> >> >> > node
> >> >> > > > as
> >> >> > > > > > > dead.
> >> >> > > > > > > > Somehow in the same moment regions started getting back
> >> to
> >> >> > life.
> >> >> > > I
> >> >> > > > > > > remember
> >> >> > > > > > > > in some older versions dfs timeout checks and check
> >> retries.
> >> >> > > Looked
> >> >> > > > > > into
> >> >> > > > > > > > defaults for our Hadoop version -
> >> >> > > > > > > >
> >> >> > > > > > > > http://hadoop.apache.org/docs/
> >> r2.4.1/hadoop-project-dist/
> >> >> > > > > > > hadoop-hdfs/hdfs-default.xml
> >> >> > > > > > > > - and saw there that there's no recheck parameter
> >> anymore.
> >> >> > > Strange,
> >> >> > > > > as
> >> >> > > > > > on
> >> >> > > > > > > > StackOverflow I found post from month ago, for newer
> >> version
> >> >> > than
> >> >> > > > we
> >> >> > > > > > use
> >> >> > > > > > > > (we use 2.4.1, guy was using 2.6 -
> >> dfs.namenode.heartbeat.
> >> >> > > > > > > recheck-interval)
> >> >> > > > > > > > I set it to 10 seconds as he mentioned, having checks
> >> every
> >> >> > three
> >> >> > > > > > seconds
> >> >> > > > > > > > (default) and got DataNode elected as dead in ~45
> >> seconds,
> >> >> as
> >> >> > he
> >> >> > > > > > > mentioned.
> >> >> > > > > > > > Not sure why this parameter is not documented, but
> >> >> obviously it
> >> >> > > > > works.
> >> >> > > > > > > >
> >> >> > > > > > > > Then figured out it still didn't fix our HBase failover
> >> >> issue.
> >> >> > I
> >> >> > > > was
> >> >> > > > > > > > looking into HBase book again and again, and then saw
> >> this
> >> >> > part:
> >> >> > > > > > > >
> >> >> > > > > > > > "How much time we allow elapse between calls to recover
> >> >> lease.
> >> >> > > > Should
> >> >> > > > > > be
> >> >> > > > > > > > larger than the dfs timeout."
> >> >> > > > > > > >
> >> >> > > > > > > > This was description for hbase.lease.recovery.dfs.timeo
> >> ut.
> >> >> > Wasn't
> >> >> > > > > sure
> >> >> > > > > > > from
> >> >> > > > > > > > the comment what of all timeouts that's possible to set
> >> in
> >> >> > > > > Hadoop/HBase
> >> >> > > > > > > and
> >> >> > > > > > > > that has something to do with DFS is this all about.
> But
> >> >> picked
> >> >> > > > > > > > hbase.lease.recovery.timeout, and that was the catch.
> >> >> > > > > > > >
> >> >> > > > > > > > Initially, by default, hbase.lease.recovery.timeout is
> >> set
> >> >> to
> >> >> > 15
> >> >> > > > > > minutes.
> >> >> > > > > > > > Not sure why, and wasn't able to find yet why, but
> >> getting
> >> >> this
> >> >> > > > down
> >> >> > > > > to
> >> >> > > > > > > one
> >> >> > > > > > > > minute (what's more than OK for us) I was able to get
> >> rid of
> >> >> > our
> >> >> > > > > issue.
> >> >> > > > > > > Not
> >> >> > > > > > > > also sure why this is not mentioned in MTTR section in
> >> HBase
> >> >> > > book,
> >> >> > > > as
> >> >> > > > > > > > obviously MTTR doesn't work at all with this default
> >> >> timeout,
> >> >> > at
> >> >> > > > > least
> >> >> > > > > > it
> >> >> > > > > > > > doesn't work the way we expected it to work.
> >> >> > > > > > > >
> >> >> > > > > > > > So thanks again for everyone being spammed with this,
> and
> >> >> > > specially
> >> >> > > > > > > thanks
> >> >> > > > > > > > to Nicolas pointing me to the right direction.
> >> >> > > > > > > >
> >> >> > > > > > > > On Mon, Mar 23, 2015 at 1:37 PM Nicolas Liochon <
> >> >> > > nkeywal@gmail.com
> >> >> > > > >
> >> >> > > > > > > wrote:
> >> >> > > > > > > >
> >> >> > > > > > > > > the attachments are rejected by the mailing list, can
> >> you
> >> >> put
> >> >> > > > then
> >> >> > > > > on
> >> >> > > > > > > > > pastebin?
> >> >> > > > > > > > >
> >> >> > > > > > > > > stale is mandatory (so it's good), but the issue here
> >> is
> >> >> just
> >> >> > > > > before.
> >> >> > > > > > > The
> >> >> > > > > > > > > region server needs to read the file. In order to be
> >> sure
> >> >> > that
> >> >> > > > > there
> >> >> > > > > > is
> >> >> > > > > > > > no
> >> >> > > > > > > > > data loss, it needs to "recover the lease", that
> means
> >> >> > ensuring
> >> >> > > > > that
> >> >> > > > > > > > nobody
> >> >> > > > > > > > > is writing the file. The regionserver calls the
> >> namenode
> >> >> to
> >> >> > do
> >> >> > > > this
> >> >> > > > > > > > > recoverLease. So there should be some info in the
> >> namenode
> >> >> > > logs.
> >> >> > > > > You
> >> >> > > > > > > have
> >> >> > > > > > > > > HDFS-4721 on your hdfs? The hbase book details (more
> or
> >> >> > > less...)
> >> >> > > > > this
> >> >> > > > > > > > > recoverLease stuff.
> >> >> > > > > > > > >
> >> >> > > > > > > > >
> >> >> > > > > > > > > On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <
> >> >> > > > > > dejan.menges@gmail.com
> >> >> > > > > > > >
> >> >> > > > > > > > > wrote:
> >> >> > > > > > > > >
> >> >> > > > > > > > > > And also, just checked -
> >> dfs.namenode.avoid.read.stale.
> >> >> > > > datanode
> >> >> > > > > and
> >> >> > > > > > > > > > dfs.namenode.avoid.write.stale.datanode
> >> >> > > > > > > > > > are both true, and dfs.namenode.stale.datanode.in
> >> terval
> >> >> is
> >> >> > > > set to
> >> >> > > > > > > > > default
> >> >> > > > > > > > > > 30000.
> >> >> > > > > > > > > >
> >> >> > > > > > > > > > On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <
> >> >> > > > > > > dejan.menges@gmail.com>
> >> >> > > > > > > > > > wrote:
> >> >> > > > > > > > > >
> >> >> > > > > > > > > > > Hi Nicolas,
> >> >> > > > > > > > > > >
> >> >> > > > > > > > > > > Please find log attached.
> >> >> > > > > > > > > > >
> >> >> > > > > > > > > > > As I see it now more clearly, it was trying to
> >> recover
> >> >> > HDFS
> >> >> > > > > WALs
> >> >> > > > > > > from
> >> >> > > > > > > > > > node
> >> >> > > > > > > > > > > that's dead:
> >> >> > > > > > > > > > >
> >> >> > > > > > > > > > > 2015-03-23 08:53:44,381 WARN
> >> >> > > > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils:
> >> >> > > > > > > > > > > Cannot recoverLease after trying for 900000ms
> >> >> > > > > > > > > > > (hbase.lease.recovery.timeout); continuing, but
> >> may be
> >> >> > > > > > > DATALOSS!!!;
> >> >> > > > > > > > > > > attempt=40 on
> >> >> > > > > > > > > > >
> >> >> > > > > > > > > >
> >> >> > file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intent
> >> >> > > > > > > > > ionally_get_down_by_getting_ne
> >> twork_down},60020,1426862900
> >> >> > > > > > > > > 506-splitting/{node_i_intentionally_get_down_by_gett
> >> >> > > > > > > > > ing_network_down}%2C60020%2C14
> >> 26862900506.1427096924508
> >> >> > > > > > > > > > > after 908210ms
> >> >> > > > > > > > > > >
> >> >> > > > > > > > > > > And as you can see from the log, it tried 40
> times,
> >> >> what
> >> >> > > took
> >> >> > > > > it
> >> >> > > > > > > > > exactly
> >> >> > > > > > > > > > > 15 minutes.
> >> >> > > > > > > > > > >
> >> >> > > > > > > > > > > There's probably some parameter to tune to cut it
> >> of
> >> >> from
> >> >> > > 40
> >> >> > > > > > times
> >> >> > > > > > > /
> >> >> > > > > > > > 15
> >> >> > > > > > > > > > > minutes to something more useful, as for 15
> >> minutes we
> >> >> > > don't
> >> >> > > > > have
> >> >> > > > > > > our
> >> >> > > > > > > > > > > regions available, and HDFS have however
> >> replication
> >> >> > factor
> >> >> > > > 3.
> >> >> > > > > > > > > > >
> >> >> > > > > > > > > > > Googling, if I figure out what's this I will post
> >> it
> >> >> > here.
> >> >> > > > Will
> >> >> > > > > > > also
> >> >> > > > > > > > > > > appreciate if someone knows how to cut this down.
> >> >> > > > > > > > > > >
> >> >> > > > > > > > > > > Thanks,
> >> >> > > > > > > > > > >
> >> >> > > > > > > > > > > Dejan
> >> >> > > > > > > > > > >
> >> >> > > > > > > > > > > On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <
> >> >> > > > > > nkeywal@gmail.com
> >> >> > > > > > > >
> >> >> > > > > > > > > > wrote:
> >> >> > > > > > > > > > >
> >> >> > > > > > > > > > >> The split is done by the region servers (the
> >> master
> >> >> > > > > > coordinates).
> >> >> > > > > > > Is
> >> >> > > > > > > > > > there
> >> >> > > > > > > > > > >> some interesting stuff in their logs?
> >> >> > > > > > > > > > >>
> >> >> > > > > > > > > > >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <
> >> >> > > > > > > > dejan.menges@gmail.com
> >> >> > > > > > > > > >
> >> >> > > > > > > > > > >> wrote:
> >> >> > > > > > > > > > >>
> >> >> > > > > > > > > > >> > With client issue was that it was retrying
> >> >> connecting
> >> >> > to
> >> >> > > > the
> >> >> > > > > > > same
> >> >> > > > > > > > > > region
> >> >> > > > > > > > > > >> > servers even when they were reassigned.
> >> Lowering it
> >> >> > down
> >> >> > > > > > helped
> >> >> > > > > > > in
> >> >> > > > > > > > > > this
> >> >> > > > > > > > > > >> > specific use case, but yes, we still want
> >> servers
> >> >> to
> >> >> > > > > > reallocate
> >> >> > > > > > > > > > quickly.
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > What got me here:
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > http://hbase.apache.org/book.html#mttr
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > I basically set configuration exactly the same
> >> way
> >> >> as
> >> >> > > it's
> >> >> > > > > > > > explained
> >> >> > > > > > > > > > >> here.
> >> >> > > > > > > > > > >> > *zookeeper.session.timeout* is (and was
> before)
> >> >> 60000
> >> >> > > (one
> >> >> > > > > > > > minute).
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > So basically what happens is: - simulating
> >> network
> >> >> > > issues
> >> >> > > > we
> >> >> > > > > > had
> >> >> > > > > > > > > > >> recently).
> >> >> > > > > > > > > > >> > - After short time I see in HBase that my
> >> >> RegionServer
> >> >> > > is
> >> >> > > > > > dead,
> >> >> > > > > > > > and
> >> >> > > > > > > > > as
> >> >> > > > > > > > > > >> > total number of regions I see previous total
> >> minus
> >> >> > > number
> >> >> > > > of
> >> >> > > > > > > > regions
> >> >> > > > > > > > > > >> that
> >> >> > > > > > > > > > >> > were hosted on the node hosting RegionServer
> >> that
> >> >> just
> >> >> > > > > > > > > 'disappeared'.
> >> >> > > > > > > > > > >> > - In this point I want my clus
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > - I have test cluster consisting of four
> nodes,
> >> >> every
> >> >> > > node
> >> >> > > > > > being
> >> >> > > > > > > > > > >> DataNode
> >> >> > > > > > > > > > >> > and RegionServer.
> >> >> > > > > > > > > > >> > - I simulate network partition on one (connect
> >> to
> >> >> it
> >> >> > > > through
> >> >> > > > > > > > console
> >> >> > > > > > > > > > and
> >> >> > > > > > > > > > >> > take network interface downter to recover as
> >> soon
> >> >> as
> >> >> > > > > possible,
> >> >> > > > > > > to
> >> >> > > > > > > > > > start
> >> >> > > > > > > > > > >> > serving missing regions.
> >> >> > > > > > > > > > >> > - First thing I see in HMaster logs are:
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > 2015-03-20 14:17:26,015 INFO
> >> >> > > > > > > > > > >> >
> >> >> org.apache.hadoop.hbase.zookeeper.RegionServerTracker:
> >> >> > > > > > > > RegionServer
> >> >> > > > > > > > > > >> > ephemeral node deleted, processing expiration
> >> >> > > > > > > > > > >> > [{name_of_node_I_took_down},60
> >> 020,1426860403261]
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > 2015-03-20 14:17:26,067 INFO
> >> >> > > > > > > > > > >> >
> >> >> > > > > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler:
> >> >> > > > > > > > > > Splitting
> >> >> > > > > > > > > > >> > logs for
> >> >> > {name_of_node_I_took_down},60020,1426860403261
> >> >> > > > > before
> >> >> > > > > > > > > > >> assignment.
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > 2015-03-20 14:17:26,105 INFO
> >> >> > > > > > > > > > >> >
> org.apache.hadoop.hbase.master.SplitLogManager:
> >> >> dead
> >> >> > > > > splitlog
> >> >> > > > > > > > > workers
> >> >> > > > > > > > > > [
> >> >> > > > > > > > > > >> >
> {name_of_node_I_took_down},60020,1426860403261]
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > 2015-03-20 14:17:26,107 INFO
> >> >> > > > > > > > > > >> >
> org.apache.hadoop.hbase.master.SplitLogManager:
> >> >> > started
> >> >> > > > > > > splitting
> >> >> > > > > > > > 1
> >> >> > > > > > > > > > >> logs in
> >> >> > > > > > > > > > >> >
> >> >> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_
> >> >> > > > I_
> >> >> > > > > > > > > took_down}
> >> >> > > > > > > > > > >> > ,60020,1426860403261-splitting]
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > 2015-03-20 14:17:26,150 INFO
> >> >> > > > > > > > > > >> >
> org.apache.hadoop.hbase.master.SplitLogManager:
> >> >> task
> >> >> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F
> >> >> > > > > > > > > > >> >
> >> >> > > > > >
> {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting
> >> %2F
> >> >> > > > > > > > > > >> >
> >> >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
> >> >> > > > 14268
> >> >> > > > > > > > > 60404905
> >> >> > > > > > > > > > >> > acquired by {fqdn_of_another_live_hnode},
> >> >> > > > 60020,1426859445623
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > 2015-03-20 14:17:26,182 INFO
> >> >> > > > > > > > > > >> >
> org.apache.hadoop.hbase.master.SplitLogManager:
> >> >> total
> >> >> > > > tasks
> >> >> > > > > =
> >> >> > > > > > 1
> >> >> > > > > > > > > > >> unassigned
> >> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> >> >> > > > 2F{name_of_node_I_took_down}
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> >
> >> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> >> >> > > > took_
> >> >> > > > > > > > > > >>
> >> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> >> >> > > > > > > > > > >> > = 1426861046182 last_version = 2
> >> cur_worker_name =
> >> >> > > > > > > > > > >> >
> {fqdn_of_another_live_node},60020,1426859445623
> >> >> > status =
> >> >> > > > > > > > > in_progress
> >> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch =
> installed
> >> = 1
> >> >> > done
> >> >> > > > = 0
> >> >> > > > > > > > error =
> >> >> > > > > > > > > > 0}
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > 2015-03-20 14:17:31,183 INFO
> >> >> > > > > > > > > > >> >
> org.apache.hadoop.hbase.master.SplitLogManager:
> >> >> total
> >> >> > > > tasks
> >> >> > > > > =
> >> >> > > > > > 1
> >> >> > > > > > > > > > >> unassigned
> >> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> >> >> > > > 2F{name_of_node_I_took_down}
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> >
> >> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> >> >> > > > took_
> >> >> > > > > > > > > > >>
> >> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> >> >> > > > > > > > > > >> > = 1426861046182 last_version = 2
> >> cur_worker_name =
> >> >> > > > > > > > > > >> >
> {fqdn_of_another_live_node},60020,1426859445623
> >> >> > status =
> >> >> > > > > > > > > in_progress
> >> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch =
> installed
> >> = 1
> >> >> > done
> >> >> > > > = 0
> >> >> > > > > > > > error =
> >> >> > > > > > > > > > 0}
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > 2015-03-20 14:17:36,184 INFO
> >> >> > > > > > > > > > >> >
> org.apache.hadoop.hbase.master.SplitLogManager:
> >> >> total
> >> >> > > > tasks
> >> >> > > > > =
> >> >> > > > > > 1
> >> >> > > > > > > > > > >> unassigned
> >> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> >> >> > > > 2F{name_of_node_I_took_down}
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> >
> >> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> >> >> > > > took_
> >> >> > > > > > > > > > >>
> >> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> >> >> > > > > > > > > > >> > = 1426861046182 last_version = 2
> >> cur_worker_name =
> >> >> > > > > > > > > > >> >
> {fqdn_of_another_live_node},60020,1426859445623
> >> >> > status =
> >> >> > > > > > > > > in_progress
> >> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch =
> installed
> >> = 1
> >> >> > done
> >> >> > > > = 0
> >> >> > > > > > > > error =
> >> >> > > > > > > > > > 0}
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > 2015-03-20 14:17:42,185 INFO
> >> >> > > > > > > > > > >> >
> org.apache.hadoop.hbase.master.SplitLogManager:
> >> >> total
> >> >> > > > tasks
> >> >> > > > > =
> >> >> > > > > > 1
> >> >> > > > > > > > > > >> unassigned
> >> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> >> >> > > > 2F{name_of_node_I_took_down}
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> >
> >> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> >> >> > > > took_
> >> >> > > > > > > > > > >>
> >> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> >> >> > > > > > > > > > >> > = 1426861046182 last_version = 2
> >> cur_worker_name =
> >> >> > > > > > > > > > >> >
> {fqdn_of_another_live_node},60020,1426859445623
> >> >> > status =
> >> >> > > > > > > > > in_progress
> >> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch =
> installed
> >> = 1
> >> >> > done
> >> >> > > > = 0
> >> >> > > > > > > > error =
> >> >> > > > > > > > > > 0}
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > 2015-03-20 14:17:48,184 INFO
> >> >> > > > > > > > > > >> >
> org.apache.hadoop.hbase.master.SplitLogManager:
> >> >> total
> >> >> > > > tasks
> >> >> > > > > =
> >> >> > > > > > 1
> >> >> > > > > > > > > > >> unassigned
> >> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> >> >> > > > 2F{name_of_node_I_took_down}
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> >
> >> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> >> >> > > > took_
> >> >> > > > > > > > > > >>
> >> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> >> >> > > > > > > > > > >> > = 1426861046182 last_version = 2
> >> cur_worker_name =
> >> >> > > > > > > > > > >> >
> {fqdn_of_another_live_node},60020,1426859445623
> >> >> > status =
> >> >> > > > > > > > > in_progress
> >> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch =
> installed
> >> = 1
> >> >> > done
> >> >> > > > = 0
> >> >> > > > > > > > error =
> >> >> > > > > > > > > > 0}
> >> >> > > > > > > > > > >> > In the meantime, In hbase...out log I got
> this:
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > ==>
> >> >> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out
> >> >> > <==
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > java.io.IOException: Call to
> >> >> > > > > > > > > > >> >
> >> >> > > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_
> >> >> > > > dow
> >> >> > > > > > > > > n}:60020
> >> >> > > > > > > > > > >> > failed on local exception:
> >> >> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> >> >> > > > CallTimeoutException:
> >> >> > > > > > > Call
> >> >> > > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > >
> org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
> >> >> > > > > > > > > > >> t.java:1532)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:
> >> >> > > > > > > 1502)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> >
> >> >> > > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(
> >> >> > > > Rpc
> >> >> > > > > > > > > > >> Client.java:1684)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> >> >> > > > BlockingRpcChannelImpl
> >> >> > > > > > > > > ementati
> >> >> > > > > > > > > > >> on.callBlockingMethod(RpcClient.java:1737)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> >
> >> >> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$
> >> >> > > > > > > > > AdminService$
> >> >> > > > > > > > > > >> BlockingStub.getRegionInfo(Adm
> >> inProtos.java:20806)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > org.apache.hadoop.hbase.client.HBaseAdmin.
> >> >> > > > getCompactionState
> >> >> > > > > > > > > > >> (HBaseAdmin.java:2524)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> >
> >> >> org.apache.hadoop.hbase.generated.master.table_jsp._
> >> >> > > > jspServi
> >> >> > > > > > > > > > >> ce(table_jsp.java:167)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.
> >> >> > > > > > > > > java:98)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > >
> javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > >
> org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
> >> >> > > > > > > > > > >> .java:511)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> >
> >> >> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> >> >> > > > doFilte
> >> >> > > > > > > > > > >> r(ServletHandler.java:1221)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> >
> org.apache.hadoop.http.lib.StaticUserWebFilter$
> >> >> > > > StaticUserFil
> >> >> > > > > > > > > > >> ter.doFilter(StaticUserWebFilter.java:109)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> >
> >> >> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> >> >> > > > doFilte
> >> >> > > > > > > > > > >> r(ServletHandler.java:1212)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> >
> >> >> org.apache.hadoop.http.HttpServer$QuotingInputFilter.
> >> >> > > > doFilte
> >> >> > > > > > > > > > >> r(HttpServer.java:1081)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> >
> >> >> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> >> >> > > > doFilte
> >> >> > > > > > > > > > >> r(ServletHandler.java:1212)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > >
> >> >> > > > > >
> >> >> >
> org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> >
> >> >> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> >> >> > > > doFilte
> >> >> > > > > > > > > > >> r(ServletHandler.java:1212)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > >
> org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
> >> >> > > > > > > > > > >> er.java:399)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> > org.mortbay.jetty.security.Sec
> >> urityHandler.handle(
> >> >> > > > SecurityHa
> >> >> > > > > > > > > > >> ndler.java:216)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > >
> org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
> >> >> > > > > > > > > > >> er.java:182)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > >
> org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
> >> >> > > > > > > > > > >> er.java:766)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > >
> org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
> >> >> > > > > > > > > > >> java:450)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > org.mortbay.jetty.handler.Cont
> >> extHandlerCollection.
> >> >> > > > handle(Co
> >> >> > > > > > > > > > >> ntextHandlerCollection.java:230)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > >
> org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
> >> >> > > > > > > > > > >> er.java:152)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at org.mortbay.jetty.Server.handl
> >> e(Server.java:326)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > >
> org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
> >> >> > > > > > > > > > >> n.java:542)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > org.mortbay.jetty.HttpConnecti
> >> on$RequestHandler.
> >> >> > > > headerComple
> >> >> > > > > > > > > > >> te(HttpConnection.java:928)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > >
> >> >> > org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > >
> >> >> > org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > org.mortbay.io.nio.SelectChannelEndPoint.run(
> >> >> > > > SelectChannelEn
> >> >> > > > > > > > > > >> dPoint.java:410)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> >
> org.mortbay.thread.QueuedThreadPool$PoolThread.
> >> run(
> >> >> > > > > > > > > > >> QueuedThreadPool.java:582)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > Caused by: org.apache.hadoop.hbase.ipc.Rp
> >> cClient$
> >> >> > > > > > > > > CallTimeoutException:
> >> >> > > > > > > > > > >> Call
> >> >> > > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.Rp
> >> cClient$Connection.
> >> >> > > > cleanupCall
> >> >> > > > > > > > > > >> s(RpcClient.java:1234)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.Rp
> >> cClient$Connection.
> >> >> > > > readRespons
> >> >> > > > > > > > > > >> e(RpcClient.java:1171)
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > at
> >> >> > > > > >
> org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
> >> >> > > > > > > > > > >> nt.java:751)
> >> >> > > > > > > > > > >> > Beside this same issue, please note that first
> >> >> message
> >> >> > > was
> >> >> > > > > at
> >> >> > > > > > > > > > 2015-03-20
> >> >> > > > > > > > > > >> > 14:17:26,015. And then (we got to the point
> >> when it
> >> >> > > > started
> >> >> > > > > > > > > > transition):
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > 2015-03-20 14:32:35,059 INFO
> >> >> > > > > > > > > > >> >
> org.apache.hadoop.hbase.master.SplitLogManager:
> >> >> task
> >> >> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F
> >> >> > > > > > > > > > >> >
> >> >> > > > > >
> {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting
> >> %2F
> >> >> > > > > > > > > > >> >
> >> >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
> >> >> > > > 14268
> >> >> > > > > > > > > 60404905
> >> >> > > > > > > > > > >> > entered state: DONE
> >> >> > > > > > {fqdn_of_new_live_node},60020,1426859445623
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > 2015-03-20 14:32:35,109 INFO
> >> >> > > > > > > > > > >> >
> org.apache.hadoop.hbase.master.SplitLogManager:
> >> >> Done
> >> >> > > > > splitting
> >> >> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F{name_o
> >> f_node_I_took_down}
> >> >> > > > > > > > > > >> >
> >> >> > > > > >
> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_do
> >> wn}
> >> >> > > > > > > > > > >> > %252C60020%252C1426860403261.1426860404905
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > 2015-03-20 14:32:35,190 INFO
> >> >> > > > > > > > > > >> >
> org.apache.hadoop.hbase.master.SplitLogManager:
> >> >> > finished
> >> >> > > > > > > splitting
> >> >> > > > > > > > > > >> (more
> >> >> > > > > > > > > > >> > than or equal to) 9 bytes in 1 log files in
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> >
> >> >> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_
> >> >> > > > of_
> >> >> > > > > > > > > > >> node_I_took_down},60020,1426860403261-splitting]
> >> >> > > > > > > > > > >> > in 909083ms
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> >> >> > > > org.apache.hadoop.hbase.master
> >> >> > > > > > > > > > >> .RegionStates:
> >> >> > > > > > > > > > >> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c
> >> >> > > > state=OPEN,
> >> >> > > > > > > > > > >> > ts=1426860639088,
> >> >> > > > > > > > > > server={name_of_node_I_took_do
> >> wn},60020,1426860403261}
> >> >> > > > > > > > > > >> to
> >> >> > > > > > > > > > >> > {0e7cc87a4ef6c47a779186f5bf79a01c
> >> state=OFFLINE,
> >> >> > > > > > > ts=1426861955191,
> >> >> > > > > > > > > > >> server=
> >> >> > > > > > > > > > >> >
> {name_of_node_I_took_down},60020,1426860403261}
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> >> >> > > > org.apache.hadoop.hbase.master
> >> >> > > > > > > > > > >> .RegionStates:
> >> >> > > > > > > > > > >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
> >> >> > > > > > > > > > >> {name_of_node_I_took_down}
> >> >> > > > > > > > > > >> > ,60020,1426860403261
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> >> >> > > > org.apache.hadoop.hbase.master
> >> >> > > > > > > > > > >> .RegionStates:
> >> >> > > > > > > > > > >> > Transitioned {25ab6e7b42e36ddaa723d71be5954543
> >> >> > > > state=OPEN,
> >> >> > > > > > > > > > >> > ts=1426860641783,
> >> >> > > > > > > > > > server={name_of_node_I_took_do
> >> wn},60020,1426860403261}
> >> >> > > > > > > > > > >> to
> >> >> > > > > > > > > > >> > {25ab6e7b42e36ddaa723d71be5954543
> >> state=OFFLINE,
> >> >> > > > > > > ts=1426861955191,
> >> >> > > > > > > > > > >> server=
> >> >> > > > > > > > > > >> >
> {name_of_node_I_took_down},60020,1426860403261}
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> >> >> > > > org.apache.hadoop.hbase.master
> >> >> > > > > > > > > > >> .RegionStates:
> >> >> > > > > > > > > > >> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
> >> >> > > > > > > > > > >> {name_of_node_I_took_down}
> >> >> > > > > > > > > > >> > ,60020,1426860403261
> >> >> > > > > > > > > > >> > At this point, note that it finished
> >> >> SplitLogManager
> >> >> > > task
> >> >> > > > at
> >> >> > > > > > > > > 14:32:35
> >> >> > > > > > > > > > >> and
> >> >> > > > > > > > > > >> > started transitioning just after that. So this
> >> is
> >> >> 15
> >> >> > > > minutes
> >> >> > > > > > > that
> >> >> > > > > > > > > I'm
> >> >> > > > > > > > > > >> > talking about.
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > What am I missing?
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas
> Liochon
> >> <
> >> >> > > > > > > > nkeywal@gmail.com>
> >> >> > > > > > > > > > >> wrote:
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > > You've changed the value of
> >> >> hbase.zookeeper.timeout
> >> >> > to
> >> >> > > > 15
> >> >> > > > > > > > > minutes? A
> >> >> > > > > > > > > > >> very
> >> >> > > > > > > > > > >> > > reasonable target is 1 minute before
> >> relocating
> >> >> the
> >> >> > > > > regions.
> >> >> > > > > > > > > That's
> >> >> > > > > > > > > > >> the
> >> >> > > > > > > > > > >> > > default iirc. You can push it to 20s, but
> then
> >> >> > > > > > > > > gc-stopping-the-world
> >> >> > > > > > > > > > >> > > becomes more of an issue. 15 minutes is
> >> really a
> >> >> > lot.
> >> >> > > > The
> >> >> > > > > > hdfs
> >> >> > > > > > > > > stale
> >> >> > > > > > > > > > >> mode
> >> >> > > > > > > > > > >> > > must always be used, with a lower timeout
> than
> >> >> the
> >> >> > > hbase
> >> >> > > > > > one.
> >> >> > > > > > > > > > >> > >
> >> >> > > > > > > > > > >> > > Client side there should be nothing to do
> >> >> (excepted
> >> >> > > for
> >> >> > > > > > > advanced
> >> >> > > > > > > > > > >> stuff);
> >> >> > > > > > > > > > >> > at
> >> >> > > > > > > > > > >> > > each retry the client checks the location of
> >> the
> >> >> > > > regions.
> >> >> > > > > If
> >> >> > > > > > > you
> >> >> > > > > > > > > > lower
> >> >> > > > > > > > > > >> > the
> >> >> > > > > > > > > > >> > > number of retry the client will fail sooner,
> >> but
> >> >> > > usually
> >> >> > > > > you
> >> >> > > > > > > > don't
> >> >> > > > > > > > > > >> want
> >> >> > > > > > > > > > >> > the
> >> >> > > > > > > > > > >> > > client to fail, you want the servers to
> >> >> reallocate
> >> >> > > > > quickly.
> >> >> > > > > > > > > > >> > >
> >> >> > > > > > > > > > >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan
> Menges
> >> <
> >> >> > > > > > > > > > dejan.menges@gmail.com
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >> > > wrote:
> >> >> > > > > > > > > > >> > >
> >> >> > > > > > > > > > >> > > > Hi,
> >> >> > > > > > > > > > >> > > >
> >> >> > > > > > > > > > >> > > > Sorry for little bit late update, but
> >> managed
> >> >> to
> >> >> > > > narrow
> >> >> > > > > it
> >> >> > > > > > > > > little
> >> >> > > > > > > > > > >> bit
> >> >> > > > > > > > > > >> > > down.
> >> >> > > > > > > > > > >> > > >
> >> >> > > > > > > > > > >> > > > We didn't update yet, as we are using
> >> >> Hortonworks
> >> >> > > > > > > distribution
> >> >> > > > > > > > > > right
> >> >> > > > > > > > > > >> > now,
> >> >> > > > > > > > > > >> > > > and even if we update we will get 0.98.4.
> >> >> However,
> >> >> > > > looks
> >> >> > > > > > > that
> >> >> > > > > > > > > > issue
> >> >> > > > > > > > > > >> > here
> >> >> > > > > > > > > > >> > > > was in our use case and configuration
> (still
> >> >> > looking
> >> >> > > > > into
> >> >> > > > > > > it).
> >> >> > > > > > > > > > >> > > >
> >> >> > > > > > > > > > >> > > > Basically, initially I saw that when one
> >> server
> >> >> > goes
> >> >> > > > > down,
> >> >> > > > > > > we
> >> >> > > > > > > > > > start
> >> >> > > > > > > > > > >> > > having
> >> >> > > > > > > > > > >> > > > performance issues in general, but it
> >> managed
> >> >> to
> >> >> > be
> >> >> > > on
> >> >> > > > > our
> >> >> > > > > > > > > client
> >> >> > > > > > > > > > >> side,
> >> >> > > > > > > > > > >> > > due
> >> >> > > > > > > > > > >> > > > to caching, and clients were trying to
> >> >> reconnect
> >> >> > to
> >> >> > > > > nodes
> >> >> > > > > > > that
> >> >> > > > > > > > > > were
> >> >> > > > > > > > > > >> > > offline
> >> >> > > > > > > > > > >> > > > and later trying to get regions from those
> >> >> nodes
> >> >> > > too.
> >> >> > > > > This
> >> >> > > > > > > is
> >> >> > > > > > > > > > >> basically
> >> >> > > > > > > > > > >> > > why
> >> >> > > > > > > > > > >> > > > on server side I didn't manage to see
> >> anything
> >> >> in
> >> >> > > logs
> >> >> > > > > > that
> >> >> > > > > > > > > would
> >> >> > > > > > > > > > >> be at
> >> >> > > > > > > > > > >> > > > least little bit interesting and point me
> >> into
> >> >> > > desired
> >> >> > > > > > > > > direction.
> >> >> > > > > > > > > > >> > > >
> >> >> > > > > > > > > > >> > > > Another question that popped up to me is -
> >> in
> >> >> case
> >> >> > > > > server
> >> >> > > > > > is
> >> >> > > > > > > > > down
> >> >> > > > > > > > > > >> (and
> >> >> > > > > > > > > > >> > > with
> >> >> > > > > > > > > > >> > > > it DataNode and HRegionServer it was
> >> hosting) -
> >> >> > > what's
> >> >> > > > > > > optimal
> >> >> > > > > > > > > > time
> >> >> > > > > > > > > > >> to
> >> >> > > > > > > > > > >> > > set
> >> >> > > > > > > > > > >> > > > for HMaster to consider server dead
> reassign
> >> >> > regions
> >> >> > > > > > > somewhere
> >> >> > > > > > > > > > >> else, as
> >> >> > > > > > > > > > >> > > > this is another performance bottleneck we
> >> hit
> >> >> > during
> >> >> > > > > > > inability
> >> >> > > > > > > > > to
> >> >> > > > > > > > > > >> > access
> >> >> > > > > > > > > > >> > > > regions? In our case it's configured to 15
> >> >> > minutes,
> >> >> > > > and
> >> >> > > > > > > simple
> >> >> > > > > > > > > > logic
> >> >> > > > > > > > > > >> > > tells
> >> >> > > > > > > > > > >> > > > me if you want it earlier then configure
> >> lower
> >> >> > > number
> >> >> > > > of
> >> >> > > > > > > > > retries,
> >> >> > > > > > > > > > >> but
> >> >> > > > > > > > > > >> > > issue
> >> >> > > > > > > > > > >> > > > is as always in details, so not sure if
> >> anyone
> >> >> > knows
> >> >> > > > > some
> >> >> > > > > > > > better
> >> >> > > > > > > > > > >> math
> >> >> > > > > > > > > > >> > for
> >> >> > > > > > > > > > >> > > > this?
> >> >> > > > > > > > > > >> > > >
> >> >> > > > > > > > > > >> > > > And last question - is it possible to
> >> manually
> >> >> > force
> >> >> > > > > HBase
> >> >> > > > > > > to
> >> >> > > > > > > > > > >> reassign
> >> >> > > > > > > > > > >> > > > regions? In this case, while HMaster is
> >> >> retrying
> >> >> > to
> >> >> > > > > > contact
> >> >> > > > > > > > node
> >> >> > > > > > > > > > >> that's
> >> >> > > > > > > > > > >> > > > dead, it's impossible to force it using
> >> >> 'balancer'
> >> >> > > > > > command.
> >> >> > > > > > > > > > >> > > >
> >> >> > > > > > > > > > >> > > > Thanks a lot!
> >> >> > > > > > > > > > >> > > >
> >> >> > > > > > > > > > >> > > > Dejan
> >> >> > > > > > > > > > >> > > >
> >> >> > > > > > > > > > >> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan
> >> Menges <
> >> >> > > > > > > > > > >> dejan.menges@gmail.com>
> >> >> > > > > > > > > > >> > > > wrote:
> >> >> > > > > > > > > > >> > > >
> >> >> > > > > > > > > > >> > > > > Hi,
> >> >> > > > > > > > > > >> > > > >
> >> >> > > > > > > > > > >> > > > > To be very honest - there's no
> particular
> >> >> reason
> >> >> > > why
> >> >> > > > > we
> >> >> > > > > > > > stick
> >> >> > > > > > > > > to
> >> >> > > > > > > > > > >> this
> >> >> > > > > > > > > > >> > > > one,
> >> >> > > > > > > > > > >> > > > > beside just lack of time currently to go
> >> >> through
> >> >> > > > > upgrade
> >> >> > > > > > > > > > process,
> >> >> > > > > > > > > > >> but
> >> >> > > > > > > > > > >> > > > looks
> >> >> > > > > > > > > > >> > > > > to me that's going to be next step.
> >> >> > > > > > > > > > >> > > > >
> >> >> > > > > > > > > > >> > > > > Had a crazy day, didn't have time to go
> >> >> through
> >> >> > > all
> >> >> > > > > logs
> >> >> > > > > > > > > again,
> >> >> > > > > > > > > > >> plus
> >> >> > > > > > > > > > >> > > one
> >> >> > > > > > > > > > >> > > > > of the machines (last one where we had
> >> this
> >> >> > issue)
> >> >> > > > is
> >> >> > > > > > > fully
> >> >> > > > > > > > > > >> > > reprovisioned
> >> >> > > > > > > > > > >> > > > > yesterday so I don't have logs from
> there
> >> >> > anymore.
> >> >> > > > > > > > > > >> > > > >
> >> >> > > > > > > > > > >> > > > > Beside upgrading,  what I will talk
> about
> >> >> today,
> >> >> > > can
> >> >> > > > > you
> >> >> > > > > > > > just
> >> >> > > > > > > > > > >> point
> >> >> > > > > > > > > > >> > me
> >> >> > > > > > > > > > >> > > to
> >> >> > > > > > > > > > >> > > > > the specific RPC issue in 0.98.0? Thing
> is
> >> >> that
> >> >> > we
> >> >> > > > > have
> >> >> > > > > > > some
> >> >> > > > > > > > > > >> strange
> >> >> > > > > > > > > > >> > > > > moments with RPC in this case, and just
> >> want
> >> >> to
> >> >> > > see
> >> >> > > > if
> >> >> > > > > > > > that's
> >> >> > > > > > > > > > the
> >> >> > > > > > > > > > >> > same
> >> >> > > > > > > > > > >> > > > > thing (and we were even suspecting to
> >> RPC).
> >> >> > > > > > > > > > >> > > > >
> >> >> > > > > > > > > > >> > > > > Thanks a lot!
> >> >> > > > > > > > > > >> > > > > Dejan
> >> >> > > > > > > > > > >> > > > >
> >> >> > > > > > > > > > >> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew
> >> >> Purtell
> >> >> > <
> >> >> > > > > > > > > > >> apurtell@apache.org
> >> >> > > > > > > > > > >> > >
> >> >> > > > > > > > > > >> > > > > wrote:
> >> >> > > > > > > > > > >> > > > >
> >> >> > > > > > > > > > >> > > > >> Is there a particular reason why you
> are
> >> >> using
> >> >> > > > HBase
> >> >> > > > > > > > 0.98.0?
> >> >> > > > > > > > > > The
> >> >> > > > > > > > > > >> > > latest
> >> >> > > > > > > > > > >> > > > >> 0.98 release is 0.98.11. There's a
> known
> >> >> > > > performance
> >> >> > > > > > > issue
> >> >> > > > > > > > > with
> >> >> > > > > > > > > > >> > 0.98.0
> >> >> > > > > > > > > > >> > > > >> pertaining to RPC that was fixed in
> later
> >> >> > > releases,
> >> >> > > > > you
> >> >> > > > > > > > > should
> >> >> > > > > > > > > > >> move
> >> >> > > > > > > > > > >> > up
> >> >> > > > > > > > > > >> > > > >> from
> >> >> > > > > > > > > > >> > > > >> 0.98.0. In addition hundreds of
> >> improvements
> >> >> > and
> >> >> > > > bug
> >> >> > > > > > > fixes
> >> >> > > > > > > > > have
> >> >> > > > > > > > > > >> gone
> >> >> > > > > > > > > > >> > > > into
> >> >> > > > > > > > > > >> > > > >> the ten releases since 0.98.0.
> >> >> > > > > > > > > > >> > > > >>
> >> >> > > > > > > > > > >> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan
> >> >> Menges <
> >> >> > > > > > > > > > >> > dejan.menges@gmail.com
> >> >> > > > > > > > > > >> > > >
> >> >> > > > > > > > > > >> > > > >> wrote:
> >> >> > > > > > > > > > >> > > > >>
> >> >> > > > > > > > > > >> > > > >> > Hi All,
> >> >> > > > > > > > > > >> > > > >> >
> >> >> > > > > > > > > > >> > > > >> > We have a strange issue with HBase
> >> >> > performance
> >> >> > > > > > (overall
> >> >> > > > > > > > > > cluster
> >> >> > > > > > > > > > >> > > > >> > performance) in case one of datanodes
> >> in
> >> >> the
> >> >> > > > > cluster
> >> >> > > > > > > > > > >> unexpectedly
> >> >> > > > > > > > > > >> > > goes
> >> >> > > > > > > > > > >> > > > >> > down.
> >> >> > > > > > > > > > >> > > > >> >
> >> >> > > > > > > > > > >> > > > >> > So scenario is like follows:
> >> >> > > > > > > > > > >> > > > >> > - Cluster works fine, it's stable.
> >> >> > > > > > > > > > >> > > > >> > - One DataNode unexpectedly goes down
> >> (PSU
> >> >> > > issue,
> >> >> > > > > > > network
> >> >> > > > > > > > > > >> issue,
> >> >> > > > > > > > > > >> > > > >> anything)
> >> >> > > > > > > > > > >> > > > >> > - Whole HBase cluster goes down
> >> >> (performance
> >> >> > > > > becomes
> >> >> > > > > > so
> >> >> > > > > > > > bad
> >> >> > > > > > > > > > >> that
> >> >> > > > > > > > > > >> > we
> >> >> > > > > > > > > > >> > > > >> have to
> >> >> > > > > > > > > > >> > > > >> > restart all RegionServers to get it
> >> back
> >> >> to
> >> >> > > > life).
> >> >> > > > > > > > > > >> > > > >> >
> >> >> > > > > > > > > > >> > > > >> > Most funny and latest issue that
> >> happened
> >> >> was
> >> >> > > > that
> >> >> > > > > we
> >> >> > > > > > > > added
> >> >> > > > > > > > > > new
> >> >> > > > > > > > > > >> > node
> >> >> > > > > > > > > > >> > > > to
> >> >> > > > > > > > > > >> > > > >> the
> >> >> > > > > > > > > > >> > > > >> > cluster (having 8 x 4T SATA disks)
> and
> >> we
> >> >> > left
> >> >> > > > just
> >> >> > > > > > > > > DataNode
> >> >> > > > > > > > > > >> > running
> >> >> > > > > > > > > > >> > > > on
> >> >> > > > > > > > > > >> > > > >> it
> >> >> > > > > > > > > > >> > > > >> > to give it couple of days to get some
> >> >> data.
> >> >> > At
> >> >> > > > some
> >> >> > > > > > > point
> >> >> > > > > > > > > in
> >> >> > > > > > > > > > >> time,
> >> >> > > > > > > > > > >> > > due
> >> >> > > > > > > > > > >> > > > >> to
> >> >> > > > > > > > > > >> > > > >> > hardware issue, server rebooted
> (twice
> >> >> during
> >> >> > > > three
> >> >> > > > > > > > hours)
> >> >> > > > > > > > > in
> >> >> > > > > > > > > > >> > moment
> >> >> > > > > > > > > > >> > > > >> when
> >> >> > > > > > > > > > >> > > > >> > it had maybe 5% of data it would have
> >> in a
> >> >> > > couple
> >> >> > > > > of
> >> >> > > > > > > > days.
> >> >> > > > > > > > > > >> Nothing
> >> >> > > > > > > > > > >> > > > else
> >> >> > > > > > > > > > >> > > > >> > beside DataNode was running, and once
> >> it
> >> >> went
> >> >> > > > down,
> >> >> > > > > > it
> >> >> > > > > > > > > > affected
> >> >> > > > > > > > > > >> > > > literary
> >> >> > > > > > > > > > >> > > > >> > everything, and restarting
> >> RegionServers
> >> >> in
> >> >> > the
> >> >> > > > end
> >> >> > > > > > > fixed
> >> >> > > > > > > > > it.
> >> >> > > > > > > > > > >> > > > >> >
> >> >> > > > > > > > > > >> > > > >> > We are using HBase 0.98.0 with Hadoop
> >> >> 2.4.0
> >> >> > > > > > > > > > >> > > > >> >
> >> >> > > > > > > > > > >> > > > >>
> >> >> > > > > > > > > > >> > > > >>
> >> >> > > > > > > > > > >> > > > >>
> >> >> > > > > > > > > > >> > > > >> --
> >> >> > > > > > > > > > >> > > > >> Best regards,
> >> >> > > > > > > > > > >> > > > >>
> >> >> > > > > > > > > > >> > > > >>    - Andy
> >> >> > > > > > > > > > >> > > > >>
> >> >> > > > > > > > > > >> > > > >> Problems worthy of attack prove their
> >> worth
> >> >> by
> >> >> > > > > hitting
> >> >> > > > > > > > back.
> >> >> > > > > > > > > -
> >> >> > > > > > > > > > >> Piet
> >> >> > > > > > > > > > >> > > Hein
> >> >> > > > > > > > > > >> > > > >> (via Tom White)
> >> >> > > > > > > > > > >> > > > >>
> >> >> > > > > > > > > > >> > > > >
> >> >> > > > > > > > > > >> > > > >
> >> >> > > > > > > > > > >> > > >
> >> >> > > > > > > > > > >> > >
> >> >> > > > > > > > > > >> >
> >> >> > > > > > > > > > >>
> >> >> > > > > > > > > > >
> >> >> > > > > > > > > >
> >> >> > > > > > > > >
> >> >> > > > > > > >
> >> >> > > > > > >
> >> >> > > > > >
> >> >> > > > >
> >> >> > > >
> >> >> > >
> >> >> >
> >> >>
> >> >
> >> >
> >>
> >
>

Re: Strange issue when DataNode goes down

Posted by Dejan Menges <de...@gmail.com>.
...and I also got sure that it's applied with hdfs getconf -confKey...

On Mon, Mar 23, 2015 at 5:31 PM Dejan Menges <de...@gmail.com> wrote:

> It was true all the time, together with dfs.namenode.avoid.read.stale.
> datanode.
>
> On Mon, Mar 23, 2015 at 5:29 PM Nicolas Liochon <nk...@gmail.com> wrote:
>
>> Actually, double checking the final patch in HDFS-4721, the stale mode is
>> taken in account. Bryan is right, it's worth checking the namenodes
>> config.
>> Especially, dfs.namenode.avoid.write.stale.datanode must be set to true
>> on
>> the namenode.
>>
>> On Mon, Mar 23, 2015 at 5:08 PM, Nicolas Liochon <nk...@gmail.com>
>> wrote:
>>
>> > stale should not help for recoverLease: it helps for reads, but it's the
>> > step after lease recovery.
>> > It's not needed in recoverLease because the recoverLease in hdfs just
>> > sorts the datanode by the heartbeat time, so, usually the stale datanode
>> > will be the last one of the list.
>> >
>> >
>> >
>> >
>> > On Mon, Mar 23, 2015 at 4:38 PM, Bryan Beaudreault <
>> > bbeaudreault@hubspot.com> wrote:
>> >
>> >> @Nicholas, I see, thanks.  I'll keep the settings at default.  So
>> really
>> >> if
>> >> everything else is configured properly you should never reach the lease
>> >> recovery timeout in any failure scenarios.  It seems that the staleness
>> >> check would be the thing that prevents this, correct?  I'm surprised it
>> >> didn't help Dejan.
>> >>
>> >> On Mon, Mar 23, 2015 at 11:20 AM, Nicolas Liochon <nk...@gmail.com>
>> >> wrote:
>> >>
>> >> > @bryan: yes, you can change hbase.lease.recovery.timeout if you
>> changed
>> >> he
>> >> > hdfs settings. But this setting is really for desperate cases. The
>> >> recover
>> >> > Lease should have succeeded before. As well, if you depend on
>> >> > hbase.lease.recovery.timeout, it means that you're wasting recovery
>> >> time:
>> >> > the lease should be recovered in a few seconds.
>> >> >
>> >> > On Mon, Mar 23, 2015 at 3:59 PM, Dejan Menges <
>> dejan.menges@gmail.com>
>> >> > wrote:
>> >> >
>> >> > > Interesting discussion I also found, gives me some more light on
>> what
>> >> > > Nicolas is talking about -
>> >> > https://issues.apache.org/jira/browse/HDFS-3703
>> >> > >
>> >> > > On Mon, Mar 23, 2015 at 3:53 PM Bryan Beaudreault <
>> >> > > bbeaudreault@hubspot.com>
>> >> > > wrote:
>> >> > >
>> >> > > > So it is safe to set hbase.lease.recovery.timeout lower if you
>> also
>> >> > > > set heartbeat.recheck.interval lower (lowering that 10.5 min dead
>> >> node
>> >> > > > timer)?  Or is it recommended to not touch either of those?
>> >> > > >
>> >> > > > Reading the above with interest, thanks for digging in here guys.
>> >> > > >
>> >> > > > On Mon, Mar 23, 2015 at 10:13 AM, Nicolas Liochon <
>> >> nkeywal@gmail.com>
>> >> > > > wrote:
>> >> > > >
>> >> > > > > If the node is actually down it's fine. But the node may not be
>> >> that
>> >> > > down
>> >> > > > > (CAP theorem here); and then it's looking for trouble.
>> >> > > > > HDFS, by default declare a node as dead after 10:30. 15 minutes
>> >> is an
>> >> > > > extra
>> >> > > > > security. It seems your hdfs settings are different (or there
>> is a
>> >> > > > bug...).
>> >> > > > > There should be some info in the hdfs logs.
>> >> > > > >
>> >> > > > > On Mon, Mar 23, 2015 at 3:05 PM, Dejan Menges <
>> >> > dejan.menges@gmail.com>
>> >> > > > > wrote:
>> >> > > > >
>> >> > > > > > Will take a look.
>> >> > > > > >
>> >> > > > > > Actually, if node is down (someone unplugged network cable,
>> it
>> >> just
>> >> > > > died,
>> >> > > > > > whatever) what's chance it's going to become live so write
>> can
>> >> > > > continue?
>> >> > > > > On
>> >> > > > > > the other side, HBase is not starting recovery trying to
>> contact
>> >> > node
>> >> > > > > which
>> >> > > > > > is not there anymore, and even elected as dead on Namenode
>> side
>> >> > > > (another
>> >> > > > > > thing I didn't understood quite good).
>> >> > > > > >
>> >> > > > > > So what I was expecting is that as soon as Namenode decided
>> >> node is
>> >> > > > dead,
>> >> > > > > > that it would be enough for RegionServer to stop trying to
>> >> recover
>> >> > > from
>> >> > > > > the
>> >> > > > > > dead node, but it wasn't the case. Also, this whole MTTR
>> >> article in
>> >> > > > HBase
>> >> > > > > > book doesn't work at all with this parameter set to it's
>> default
>> >> > > value
>> >> > > > > (15
>> >> > > > > > minutes).
>> >> > > > > >
>> >> > > > > > So I'm still struggling to figure out what's drawback
>> exactly on
>> >> > > this?
>> >> > > > > >
>> >> > > > > > On Mon, Mar 23, 2015 at 2:50 PM Nicolas Liochon <
>> >> nkeywal@gmail.com
>> >> > >
>> >> > > > > wrote:
>> >> > > > > >
>> >> > > > > > > Thanks for the explanation. There is an issue if you modify
>> >> this
>> >> > > > > setting
>> >> > > > > > > however.
>> >> > > > > > > hbase tries to recover the lease (i.e. be sure that nobody
>> is
>> >> > > > writing)
>> >> > > > > > > If you change hbase.lease.recovery.timeout hbase will start
>> >> the
>> >> > > > > recovery
>> >> > > > > > > (i.e. start to read) even if it's not sure that nobody's
>> >> writing.
>> >> > > > That
>> >> > > > > > > means there is a dataloss risk.
>> >> > > > > > > Basically, you must not see this warning: WARN
>> >> > > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils: Cannot
>> recoverLease
>> >> > after
>> >> > > > > > trying
>> >> > > > > > > for[]
>> >> > > > > > >
>> >> > > > > > > The recoverLease must succeed. The fact that it does not
>> >> after X
>> >> > > > tries
>> >> > > > > is
>> >> > > > > > > strange.
>> >> > > > > > > There may be a mistmatch between the hbase parameters and
>> the
>> >> > hdfs
>> >> > > > > ones.
>> >> > > > > > > You may need to have a look at the comments in
>> >> FSHDFSUtils.java
>> >> > > > > > >
>> >> > > > > > >
>> >> > > > > > >
>> >> > > > > > >
>> >> > > > > > > On Mon, Mar 23, 2015 at 2:15 PM, Dejan Menges <
>> >> > > > dejan.menges@gmail.com>
>> >> > > > > > > wrote:
>> >> > > > > > >
>> >> > > > > > > > I found the issue and fixed it, and will try to explain
>> here
>> >> > what
>> >> > > > was
>> >> > > > > > > > exactly in our case in case someone else finds this
>> >> interesting
>> >> > > > too.
>> >> > > > > > > >
>> >> > > > > > > > So initially, we had (couple of times) some network and
>> >> > hardware
>> >> > > > > issues
>> >> > > > > > > in
>> >> > > > > > > > our datacenters. When one server would die (literary
>> die, we
>> >> > had
>> >> > > > some
>> >> > > > > > > issue
>> >> > > > > > > > with PSUs) we saw issues with overall cluster
>> performance on
>> >> > > HBase
>> >> > > > > > side.
>> >> > > > > > > As
>> >> > > > > > > > cluster is quite big and live, it was also quite hard to
>> >> figure
>> >> > > out
>> >> > > > > > exact
>> >> > > > > > > > root cause and how to fix the stuff we wanted to fix.
>> >> > > > > > > >
>> >> > > > > > > > So I set up another cluster, four nodes (with DataNode
>> and
>> >> > > > > > RegionServer)
>> >> > > > > > > > and two other nodes with HMaster and Namenode in HA,
>> using
>> >> same
>> >> > > > stuff
>> >> > > > > > we
>> >> > > > > > > > use on production. We pumped some data into it, and I was
>> >> able
>> >> > to
>> >> > > > > > > reproduce
>> >> > > > > > > > same issue last week on it. What I tried to do is to cut
>> one
>> >> > > server
>> >> > > > > > (shut
>> >> > > > > > > > down it's interface) when all is good with cluster, when
>> we
>> >> > have
>> >> > > > > load,
>> >> > > > > > > and
>> >> > > > > > > > see what's going to happen.
>> >> > > > > > > >
>> >> > > > > > > > On Friday, after Nicolas mentioned, I started taking a
>> look
>> >> in
>> >> > > > HBase
>> >> > > > > > logs
>> >> > > > > > > > on the node which was mentioned in HMaster log as the one
>> >> > taking
>> >> > > > over
>> >> > > > > > > > regions for the dead server. Basically what I was able to
>> >> > observe
>> >> > > > was
>> >> > > > > > 15
>> >> > > > > > > > minutes time (+- couple of seconds, what was also
>> >> interesting,
>> >> > > and
>> >> > > > > will
>> >> > > > > > > got
>> >> > > > > > > > later to that) between HMaster figures out that one of
>> it's
>> >> > > > > > RegionServers
>> >> > > > > > > > is dead, and the time one of the mentioned nodes starts
>> >> taking
>> >> > > over
>> >> > > > > > those
>> >> > > > > > > > regions and they start appearing in HMaster's Web UI.
>> >> > > > > > > >
>> >> > > > > > > > I then set up everything like mentioned here
>> >> > > > > > > > http://hbase.apache.org/book.html#mttr - but still had
>> >> exactly
>> >> > > the
>> >> > > > > > same
>> >> > > > > > > > issues. Went over (again and again) all currently
>> configured
>> >> > > stuff,
>> >> > > > > but
>> >> > > > > > > > still had the same issue.
>> >> > > > > > > >
>> >> > > > > > > > Then I started looking into HDFS. Opened NameNode UI, saw
>> >> all
>> >> > is
>> >> > > > > good,
>> >> > > > > > > took
>> >> > > > > > > > one node down, was also looking RegionServer logs in the
>> >> same
>> >> > > time
>> >> > > > -
>> >> > > > > > and
>> >> > > > > > > I
>> >> > > > > > > > also see that it took ~15 minutes for Namenode to elect
>> dead
>> >> > node
>> >> > > > as
>> >> > > > > > > dead.
>> >> > > > > > > > Somehow in the same moment regions started getting back
>> to
>> >> > life.
>> >> > > I
>> >> > > > > > > remember
>> >> > > > > > > > in some older versions dfs timeout checks and check
>> retries.
>> >> > > Looked
>> >> > > > > > into
>> >> > > > > > > > defaults for our Hadoop version -
>> >> > > > > > > >
>> >> > > > > > > > http://hadoop.apache.org/docs/
>> r2.4.1/hadoop-project-dist/
>> >> > > > > > > hadoop-hdfs/hdfs-default.xml
>> >> > > > > > > > - and saw there that there's no recheck parameter
>> anymore.
>> >> > > Strange,
>> >> > > > > as
>> >> > > > > > on
>> >> > > > > > > > StackOverflow I found post from month ago, for newer
>> version
>> >> > than
>> >> > > > we
>> >> > > > > > use
>> >> > > > > > > > (we use 2.4.1, guy was using 2.6 -
>> dfs.namenode.heartbeat.
>> >> > > > > > > recheck-interval)
>> >> > > > > > > > I set it to 10 seconds as he mentioned, having checks
>> every
>> >> > three
>> >> > > > > > seconds
>> >> > > > > > > > (default) and got DataNode elected as dead in ~45
>> seconds,
>> >> as
>> >> > he
>> >> > > > > > > mentioned.
>> >> > > > > > > > Not sure why this parameter is not documented, but
>> >> obviously it
>> >> > > > > works.
>> >> > > > > > > >
>> >> > > > > > > > Then figured out it still didn't fix our HBase failover
>> >> issue.
>> >> > I
>> >> > > > was
>> >> > > > > > > > looking into HBase book again and again, and then saw
>> this
>> >> > part:
>> >> > > > > > > >
>> >> > > > > > > > "How much time we allow elapse between calls to recover
>> >> lease.
>> >> > > > Should
>> >> > > > > > be
>> >> > > > > > > > larger than the dfs timeout."
>> >> > > > > > > >
>> >> > > > > > > > This was description for hbase.lease.recovery.dfs.timeo
>> ut.
>> >> > Wasn't
>> >> > > > > sure
>> >> > > > > > > from
>> >> > > > > > > > the comment what of all timeouts that's possible to set
>> in
>> >> > > > > Hadoop/HBase
>> >> > > > > > > and
>> >> > > > > > > > that has something to do with DFS is this all about. But
>> >> picked
>> >> > > > > > > > hbase.lease.recovery.timeout, and that was the catch.
>> >> > > > > > > >
>> >> > > > > > > > Initially, by default, hbase.lease.recovery.timeout is
>> set
>> >> to
>> >> > 15
>> >> > > > > > minutes.
>> >> > > > > > > > Not sure why, and wasn't able to find yet why, but
>> getting
>> >> this
>> >> > > > down
>> >> > > > > to
>> >> > > > > > > one
>> >> > > > > > > > minute (what's more than OK for us) I was able to get
>> rid of
>> >> > our
>> >> > > > > issue.
>> >> > > > > > > Not
>> >> > > > > > > > also sure why this is not mentioned in MTTR section in
>> HBase
>> >> > > book,
>> >> > > > as
>> >> > > > > > > > obviously MTTR doesn't work at all with this default
>> >> timeout,
>> >> > at
>> >> > > > > least
>> >> > > > > > it
>> >> > > > > > > > doesn't work the way we expected it to work.
>> >> > > > > > > >
>> >> > > > > > > > So thanks again for everyone being spammed with this, and
>> >> > > specially
>> >> > > > > > > thanks
>> >> > > > > > > > to Nicolas pointing me to the right direction.
>> >> > > > > > > >
>> >> > > > > > > > On Mon, Mar 23, 2015 at 1:37 PM Nicolas Liochon <
>> >> > > nkeywal@gmail.com
>> >> > > > >
>> >> > > > > > > wrote:
>> >> > > > > > > >
>> >> > > > > > > > > the attachments are rejected by the mailing list, can
>> you
>> >> put
>> >> > > > then
>> >> > > > > on
>> >> > > > > > > > > pastebin?
>> >> > > > > > > > >
>> >> > > > > > > > > stale is mandatory (so it's good), but the issue here
>> is
>> >> just
>> >> > > > > before.
>> >> > > > > > > The
>> >> > > > > > > > > region server needs to read the file. In order to be
>> sure
>> >> > that
>> >> > > > > there
>> >> > > > > > is
>> >> > > > > > > > no
>> >> > > > > > > > > data loss, it needs to "recover the lease", that means
>> >> > ensuring
>> >> > > > > that
>> >> > > > > > > > nobody
>> >> > > > > > > > > is writing the file. The regionserver calls the
>> namenode
>> >> to
>> >> > do
>> >> > > > this
>> >> > > > > > > > > recoverLease. So there should be some info in the
>> namenode
>> >> > > logs.
>> >> > > > > You
>> >> > > > > > > have
>> >> > > > > > > > > HDFS-4721 on your hdfs? The hbase book details (more or
>> >> > > less...)
>> >> > > > > this
>> >> > > > > > > > > recoverLease stuff.
>> >> > > > > > > > >
>> >> > > > > > > > >
>> >> > > > > > > > > On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <
>> >> > > > > > dejan.menges@gmail.com
>> >> > > > > > > >
>> >> > > > > > > > > wrote:
>> >> > > > > > > > >
>> >> > > > > > > > > > And also, just checked -
>> dfs.namenode.avoid.read.stale.
>> >> > > > datanode
>> >> > > > > and
>> >> > > > > > > > > > dfs.namenode.avoid.write.stale.datanode
>> >> > > > > > > > > > are both true, and dfs.namenode.stale.datanode.in
>> terval
>> >> is
>> >> > > > set to
>> >> > > > > > > > > default
>> >> > > > > > > > > > 30000.
>> >> > > > > > > > > >
>> >> > > > > > > > > > On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <
>> >> > > > > > > dejan.menges@gmail.com>
>> >> > > > > > > > > > wrote:
>> >> > > > > > > > > >
>> >> > > > > > > > > > > Hi Nicolas,
>> >> > > > > > > > > > >
>> >> > > > > > > > > > > Please find log attached.
>> >> > > > > > > > > > >
>> >> > > > > > > > > > > As I see it now more clearly, it was trying to
>> recover
>> >> > HDFS
>> >> > > > > WALs
>> >> > > > > > > from
>> >> > > > > > > > > > node
>> >> > > > > > > > > > > that's dead:
>> >> > > > > > > > > > >
>> >> > > > > > > > > > > 2015-03-23 08:53:44,381 WARN
>> >> > > > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils:
>> >> > > > > > > > > > > Cannot recoverLease after trying for 900000ms
>> >> > > > > > > > > > > (hbase.lease.recovery.timeout); continuing, but
>> may be
>> >> > > > > > > DATALOSS!!!;
>> >> > > > > > > > > > > attempt=40 on
>> >> > > > > > > > > > >
>> >> > > > > > > > > >
>> >> > file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intent
>> >> > > > > > > > > ionally_get_down_by_getting_ne
>> twork_down},60020,1426862900
>> >> > > > > > > > > 506-splitting/{node_i_intentionally_get_down_by_gett
>> >> > > > > > > > > ing_network_down}%2C60020%2C14
>> 26862900506.1427096924508
>> >> > > > > > > > > > > after 908210ms
>> >> > > > > > > > > > >
>> >> > > > > > > > > > > And as you can see from the log, it tried 40 times,
>> >> what
>> >> > > took
>> >> > > > > it
>> >> > > > > > > > > exactly
>> >> > > > > > > > > > > 15 minutes.
>> >> > > > > > > > > > >
>> >> > > > > > > > > > > There's probably some parameter to tune to cut it
>> of
>> >> from
>> >> > > 40
>> >> > > > > > times
>> >> > > > > > > /
>> >> > > > > > > > 15
>> >> > > > > > > > > > > minutes to something more useful, as for 15
>> minutes we
>> >> > > don't
>> >> > > > > have
>> >> > > > > > > our
>> >> > > > > > > > > > > regions available, and HDFS have however
>> replication
>> >> > factor
>> >> > > > 3.
>> >> > > > > > > > > > >
>> >> > > > > > > > > > > Googling, if I figure out what's this I will post
>> it
>> >> > here.
>> >> > > > Will
>> >> > > > > > > also
>> >> > > > > > > > > > > appreciate if someone knows how to cut this down.
>> >> > > > > > > > > > >
>> >> > > > > > > > > > > Thanks,
>> >> > > > > > > > > > >
>> >> > > > > > > > > > > Dejan
>> >> > > > > > > > > > >
>> >> > > > > > > > > > > On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <
>> >> > > > > > nkeywal@gmail.com
>> >> > > > > > > >
>> >> > > > > > > > > > wrote:
>> >> > > > > > > > > > >
>> >> > > > > > > > > > >> The split is done by the region servers (the
>> master
>> >> > > > > > coordinates).
>> >> > > > > > > Is
>> >> > > > > > > > > > there
>> >> > > > > > > > > > >> some interesting stuff in their logs?
>> >> > > > > > > > > > >>
>> >> > > > > > > > > > >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <
>> >> > > > > > > > dejan.menges@gmail.com
>> >> > > > > > > > > >
>> >> > > > > > > > > > >> wrote:
>> >> > > > > > > > > > >>
>> >> > > > > > > > > > >> > With client issue was that it was retrying
>> >> connecting
>> >> > to
>> >> > > > the
>> >> > > > > > > same
>> >> > > > > > > > > > region
>> >> > > > > > > > > > >> > servers even when they were reassigned.
>> Lowering it
>> >> > down
>> >> > > > > > helped
>> >> > > > > > > in
>> >> > > > > > > > > > this
>> >> > > > > > > > > > >> > specific use case, but yes, we still want
>> servers
>> >> to
>> >> > > > > > reallocate
>> >> > > > > > > > > > quickly.
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > What got me here:
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > http://hbase.apache.org/book.html#mttr
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > I basically set configuration exactly the same
>> way
>> >> as
>> >> > > it's
>> >> > > > > > > > explained
>> >> > > > > > > > > > >> here.
>> >> > > > > > > > > > >> > *zookeeper.session.timeout* is (and was before)
>> >> 60000
>> >> > > (one
>> >> > > > > > > > minute).
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > So basically what happens is: - simulating
>> network
>> >> > > issues
>> >> > > > we
>> >> > > > > > had
>> >> > > > > > > > > > >> recently).
>> >> > > > > > > > > > >> > - After short time I see in HBase that my
>> >> RegionServer
>> >> > > is
>> >> > > > > > dead,
>> >> > > > > > > > and
>> >> > > > > > > > > as
>> >> > > > > > > > > > >> > total number of regions I see previous total
>> minus
>> >> > > number
>> >> > > > of
>> >> > > > > > > > regions
>> >> > > > > > > > > > >> that
>> >> > > > > > > > > > >> > were hosted on the node hosting RegionServer
>> that
>> >> just
>> >> > > > > > > > > 'disappeared'.
>> >> > > > > > > > > > >> > - In this point I want my clus
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > - I have test cluster consisting of four nodes,
>> >> every
>> >> > > node
>> >> > > > > > being
>> >> > > > > > > > > > >> DataNode
>> >> > > > > > > > > > >> > and RegionServer.
>> >> > > > > > > > > > >> > - I simulate network partition on one (connect
>> to
>> >> it
>> >> > > > through
>> >> > > > > > > > console
>> >> > > > > > > > > > and
>> >> > > > > > > > > > >> > take network interface downter to recover as
>> soon
>> >> as
>> >> > > > > possible,
>> >> > > > > > > to
>> >> > > > > > > > > > start
>> >> > > > > > > > > > >> > serving missing regions.
>> >> > > > > > > > > > >> > - First thing I see in HMaster logs are:
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > 2015-03-20 14:17:26,015 INFO
>> >> > > > > > > > > > >> >
>> >> org.apache.hadoop.hbase.zookeeper.RegionServerTracker:
>> >> > > > > > > > RegionServer
>> >> > > > > > > > > > >> > ephemeral node deleted, processing expiration
>> >> > > > > > > > > > >> > [{name_of_node_I_took_down},60
>> 020,1426860403261]
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > 2015-03-20 14:17:26,067 INFO
>> >> > > > > > > > > > >> >
>> >> > > > > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler:
>> >> > > > > > > > > > Splitting
>> >> > > > > > > > > > >> > logs for
>> >> > {name_of_node_I_took_down},60020,1426860403261
>> >> > > > > before
>> >> > > > > > > > > > >> assignment.
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > 2015-03-20 14:17:26,105 INFO
>> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> >> dead
>> >> > > > > splitlog
>> >> > > > > > > > > workers
>> >> > > > > > > > > > [
>> >> > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261]
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > 2015-03-20 14:17:26,107 INFO
>> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> >> > started
>> >> > > > > > > splitting
>> >> > > > > > > > 1
>> >> > > > > > > > > > >> logs in
>> >> > > > > > > > > > >> >
>> >> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_
>> >> > > > I_
>> >> > > > > > > > > took_down}
>> >> > > > > > > > > > >> > ,60020,1426860403261-splitting]
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > 2015-03-20 14:17:26,150 INFO
>> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> >> task
>> >> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F
>> >> > > > > > > > > > >> >
>> >> > > > > > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting
>> %2F
>> >> > > > > > > > > > >> >
>> >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
>> >> > > > 14268
>> >> > > > > > > > > 60404905
>> >> > > > > > > > > > >> > acquired by {fqdn_of_another_live_hnode},
>> >> > > > 60020,1426859445623
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > 2015-03-20 14:17:26,182 INFO
>> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> >> total
>> >> > > > tasks
>> >> > > > > =
>> >> > > > > > 1
>> >> > > > > > > > > > >> unassigned
>> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
>> >> > > > 2F{name_of_node_I_took_down}
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> >
>> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
>> >> > > > took_
>> >> > > > > > > > > > >>
>> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
>> >> > > > > > > > > > >> > = 1426861046182 last_version = 2
>> cur_worker_name =
>> >> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
>> >> > status =
>> >> > > > > > > > > in_progress
>> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed
>> = 1
>> >> > done
>> >> > > > = 0
>> >> > > > > > > > error =
>> >> > > > > > > > > > 0}
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > 2015-03-20 14:17:31,183 INFO
>> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> >> total
>> >> > > > tasks
>> >> > > > > =
>> >> > > > > > 1
>> >> > > > > > > > > > >> unassigned
>> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
>> >> > > > 2F{name_of_node_I_took_down}
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> >
>> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
>> >> > > > took_
>> >> > > > > > > > > > >>
>> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
>> >> > > > > > > > > > >> > = 1426861046182 last_version = 2
>> cur_worker_name =
>> >> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
>> >> > status =
>> >> > > > > > > > > in_progress
>> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed
>> = 1
>> >> > done
>> >> > > > = 0
>> >> > > > > > > > error =
>> >> > > > > > > > > > 0}
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > 2015-03-20 14:17:36,184 INFO
>> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> >> total
>> >> > > > tasks
>> >> > > > > =
>> >> > > > > > 1
>> >> > > > > > > > > > >> unassigned
>> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
>> >> > > > 2F{name_of_node_I_took_down}
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> >
>> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
>> >> > > > took_
>> >> > > > > > > > > > >>
>> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
>> >> > > > > > > > > > >> > = 1426861046182 last_version = 2
>> cur_worker_name =
>> >> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
>> >> > status =
>> >> > > > > > > > > in_progress
>> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed
>> = 1
>> >> > done
>> >> > > > = 0
>> >> > > > > > > > error =
>> >> > > > > > > > > > 0}
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > 2015-03-20 14:17:42,185 INFO
>> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> >> total
>> >> > > > tasks
>> >> > > > > =
>> >> > > > > > 1
>> >> > > > > > > > > > >> unassigned
>> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
>> >> > > > 2F{name_of_node_I_took_down}
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> >
>> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
>> >> > > > took_
>> >> > > > > > > > > > >>
>> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
>> >> > > > > > > > > > >> > = 1426861046182 last_version = 2
>> cur_worker_name =
>> >> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
>> >> > status =
>> >> > > > > > > > > in_progress
>> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed
>> = 1
>> >> > done
>> >> > > > = 0
>> >> > > > > > > > error =
>> >> > > > > > > > > > 0}
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > 2015-03-20 14:17:48,184 INFO
>> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> >> total
>> >> > > > tasks
>> >> > > > > =
>> >> > > > > > 1
>> >> > > > > > > > > > >> unassigned
>> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
>> >> > > > 2F{name_of_node_I_took_down}
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> >
>> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
>> >> > > > took_
>> >> > > > > > > > > > >>
>> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
>> >> > > > > > > > > > >> > = 1426861046182 last_version = 2
>> cur_worker_name =
>> >> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
>> >> > status =
>> >> > > > > > > > > in_progress
>> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed
>> = 1
>> >> > done
>> >> > > > = 0
>> >> > > > > > > > error =
>> >> > > > > > > > > > 0}
>> >> > > > > > > > > > >> > In the meantime, In hbase...out log I got this:
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > ==>
>> >> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out
>> >> > <==
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > java.io.IOException: Call to
>> >> > > > > > > > > > >> >
>> >> > > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_
>> >> > > > dow
>> >> > > > > > > > > n}:60020
>> >> > > > > > > > > > >> > failed on local exception:
>> >> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
>> >> > > > CallTimeoutException:
>> >> > > > > > > Call
>> >> > > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
>> >> > > > > > > > > > >> t.java:1532)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:
>> >> > > > > > > 1502)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> >
>> >> > > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(
>> >> > > > Rpc
>> >> > > > > > > > > > >> Client.java:1684)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
>> >> > > > BlockingRpcChannelImpl
>> >> > > > > > > > > ementati
>> >> > > > > > > > > > >> on.callBlockingMethod(RpcClient.java:1737)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> >
>> >> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$
>> >> > > > > > > > > AdminService$
>> >> > > > > > > > > > >> BlockingStub.getRegionInfo(Adm
>> inProtos.java:20806)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > org.apache.hadoop.hbase.client.HBaseAdmin.
>> >> > > > getCompactionState
>> >> > > > > > > > > > >> (HBaseAdmin.java:2524)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> >
>> >> org.apache.hadoop.hbase.generated.master.table_jsp._
>> >> > > > jspServi
>> >> > > > > > > > > > >> ce(table_jsp.java:167)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.
>> >> > > > > > > > > java:98)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
>> >> > > > > > > > > > >> .java:511)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> >
>> >> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
>> >> > > > doFilte
>> >> > > > > > > > > > >> r(ServletHandler.java:1221)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > org.apache.hadoop.http.lib.StaticUserWebFilter$
>> >> > > > StaticUserFil
>> >> > > > > > > > > > >> ter.doFilter(StaticUserWebFilter.java:109)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> >
>> >> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
>> >> > > > doFilte
>> >> > > > > > > > > > >> r(ServletHandler.java:1212)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> >
>> >> org.apache.hadoop.http.HttpServer$QuotingInputFilter.
>> >> > > > doFilte
>> >> > > > > > > > > > >> r(HttpServer.java:1081)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> >
>> >> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
>> >> > > > doFilte
>> >> > > > > > > > > > >> r(ServletHandler.java:1212)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > >
>> >> > > > > >
>> >> > org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> >
>> >> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
>> >> > > > doFilte
>> >> > > > > > > > > > >> r(ServletHandler.java:1212)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
>> >> > > > > > > > > > >> er.java:399)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> > org.mortbay.jetty.security.Sec
>> urityHandler.handle(
>> >> > > > SecurityHa
>> >> > > > > > > > > > >> ndler.java:216)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
>> >> > > > > > > > > > >> er.java:182)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
>> >> > > > > > > > > > >> er.java:766)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
>> >> > > > > > > > > > >> java:450)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > org.mortbay.jetty.handler.Cont
>> extHandlerCollection.
>> >> > > > handle(Co
>> >> > > > > > > > > > >> ntextHandlerCollection.java:230)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
>> >> > > > > > > > > > >> er.java:152)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at org.mortbay.jetty.Server.handl
>> e(Server.java:326)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
>> >> > > > > > > > > > >> n.java:542)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > org.mortbay.jetty.HttpConnecti
>> on$RequestHandler.
>> >> > > > headerComple
>> >> > > > > > > > > > >> te(HttpConnection.java:928)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > >
>> >> > org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > >
>> >> > org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > org.mortbay.io.nio.SelectChannelEndPoint.run(
>> >> > > > SelectChannelEn
>> >> > > > > > > > > > >> dPoint.java:410)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > org.mortbay.thread.QueuedThreadPool$PoolThread.
>> run(
>> >> > > > > > > > > > >> QueuedThreadPool.java:582)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > Caused by: org.apache.hadoop.hbase.ipc.Rp
>> cClient$
>> >> > > > > > > > > CallTimeoutException:
>> >> > > > > > > > > > >> Call
>> >> > > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.Rp
>> cClient$Connection.
>> >> > > > cleanupCall
>> >> > > > > > > > > > >> s(RpcClient.java:1234)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.Rp
>> cClient$Connection.
>> >> > > > readRespons
>> >> > > > > > > > > > >> e(RpcClient.java:1171)
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > at
>> >> > > > > > org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
>> >> > > > > > > > > > >> nt.java:751)
>> >> > > > > > > > > > >> > Beside this same issue, please note that first
>> >> message
>> >> > > was
>> >> > > > > at
>> >> > > > > > > > > > 2015-03-20
>> >> > > > > > > > > > >> > 14:17:26,015. And then (we got to the point
>> when it
>> >> > > > started
>> >> > > > > > > > > > transition):
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > 2015-03-20 14:32:35,059 INFO
>> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> >> task
>> >> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F
>> >> > > > > > > > > > >> >
>> >> > > > > > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting
>> %2F
>> >> > > > > > > > > > >> >
>> >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
>> >> > > > 14268
>> >> > > > > > > > > 60404905
>> >> > > > > > > > > > >> > entered state: DONE
>> >> > > > > > {fqdn_of_new_live_node},60020,1426859445623
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > 2015-03-20 14:32:35,109 INFO
>> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> >> Done
>> >> > > > > splitting
>> >> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F{name_o
>> f_node_I_took_down}
>> >> > > > > > > > > > >> >
>> >> > > > > > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_do
>> wn}
>> >> > > > > > > > > > >> > %252C60020%252C1426860403261.1426860404905
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > 2015-03-20 14:32:35,190 INFO
>> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> >> > finished
>> >> > > > > > > splitting
>> >> > > > > > > > > > >> (more
>> >> > > > > > > > > > >> > than or equal to) 9 bytes in 1 log files in
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> >
>> >> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_
>> >> > > > of_
>> >> > > > > > > > > > >> node_I_took_down},60020,1426860403261-splitting]
>> >> > > > > > > > > > >> > in 909083ms
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
>> >> > > > org.apache.hadoop.hbase.master
>> >> > > > > > > > > > >> .RegionStates:
>> >> > > > > > > > > > >> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c
>> >> > > > state=OPEN,
>> >> > > > > > > > > > >> > ts=1426860639088,
>> >> > > > > > > > > > server={name_of_node_I_took_do
>> wn},60020,1426860403261}
>> >> > > > > > > > > > >> to
>> >> > > > > > > > > > >> > {0e7cc87a4ef6c47a779186f5bf79a01c
>> state=OFFLINE,
>> >> > > > > > > ts=1426861955191,
>> >> > > > > > > > > > >> server=
>> >> > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
>> >> > > > org.apache.hadoop.hbase.master
>> >> > > > > > > > > > >> .RegionStates:
>> >> > > > > > > > > > >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
>> >> > > > > > > > > > >> {name_of_node_I_took_down}
>> >> > > > > > > > > > >> > ,60020,1426860403261
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
>> >> > > > org.apache.hadoop.hbase.master
>> >> > > > > > > > > > >> .RegionStates:
>> >> > > > > > > > > > >> > Transitioned {25ab6e7b42e36ddaa723d71be5954543
>> >> > > > state=OPEN,
>> >> > > > > > > > > > >> > ts=1426860641783,
>> >> > > > > > > > > > server={name_of_node_I_took_do
>> wn},60020,1426860403261}
>> >> > > > > > > > > > >> to
>> >> > > > > > > > > > >> > {25ab6e7b42e36ddaa723d71be5954543
>> state=OFFLINE,
>> >> > > > > > > ts=1426861955191,
>> >> > > > > > > > > > >> server=
>> >> > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
>> >> > > > org.apache.hadoop.hbase.master
>> >> > > > > > > > > > >> .RegionStates:
>> >> > > > > > > > > > >> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
>> >> > > > > > > > > > >> {name_of_node_I_took_down}
>> >> > > > > > > > > > >> > ,60020,1426860403261
>> >> > > > > > > > > > >> > At this point, note that it finished
>> >> SplitLogManager
>> >> > > task
>> >> > > > at
>> >> > > > > > > > > 14:32:35
>> >> > > > > > > > > > >> and
>> >> > > > > > > > > > >> > started transitioning just after that. So this
>> is
>> >> 15
>> >> > > > minutes
>> >> > > > > > > that
>> >> > > > > > > > > I'm
>> >> > > > > > > > > > >> > talking about.
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > What am I missing?
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon
>> <
>> >> > > > > > > > nkeywal@gmail.com>
>> >> > > > > > > > > > >> wrote:
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > > You've changed the value of
>> >> hbase.zookeeper.timeout
>> >> > to
>> >> > > > 15
>> >> > > > > > > > > minutes? A
>> >> > > > > > > > > > >> very
>> >> > > > > > > > > > >> > > reasonable target is 1 minute before
>> relocating
>> >> the
>> >> > > > > regions.
>> >> > > > > > > > > That's
>> >> > > > > > > > > > >> the
>> >> > > > > > > > > > >> > > default iirc. You can push it to 20s, but then
>> >> > > > > > > > > gc-stopping-the-world
>> >> > > > > > > > > > >> > > becomes more of an issue. 15 minutes is
>> really a
>> >> > lot.
>> >> > > > The
>> >> > > > > > hdfs
>> >> > > > > > > > > stale
>> >> > > > > > > > > > >> mode
>> >> > > > > > > > > > >> > > must always be used, with a lower timeout than
>> >> the
>> >> > > hbase
>> >> > > > > > one.
>> >> > > > > > > > > > >> > >
>> >> > > > > > > > > > >> > > Client side there should be nothing to do
>> >> (excepted
>> >> > > for
>> >> > > > > > > advanced
>> >> > > > > > > > > > >> stuff);
>> >> > > > > > > > > > >> > at
>> >> > > > > > > > > > >> > > each retry the client checks the location of
>> the
>> >> > > > regions.
>> >> > > > > If
>> >> > > > > > > you
>> >> > > > > > > > > > lower
>> >> > > > > > > > > > >> > the
>> >> > > > > > > > > > >> > > number of retry the client will fail sooner,
>> but
>> >> > > usually
>> >> > > > > you
>> >> > > > > > > > don't
>> >> > > > > > > > > > >> want
>> >> > > > > > > > > > >> > the
>> >> > > > > > > > > > >> > > client to fail, you want the servers to
>> >> reallocate
>> >> > > > > quickly.
>> >> > > > > > > > > > >> > >
>> >> > > > > > > > > > >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges
>> <
>> >> > > > > > > > > > dejan.menges@gmail.com
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >> > > wrote:
>> >> > > > > > > > > > >> > >
>> >> > > > > > > > > > >> > > > Hi,
>> >> > > > > > > > > > >> > > >
>> >> > > > > > > > > > >> > > > Sorry for little bit late update, but
>> managed
>> >> to
>> >> > > > narrow
>> >> > > > > it
>> >> > > > > > > > > little
>> >> > > > > > > > > > >> bit
>> >> > > > > > > > > > >> > > down.
>> >> > > > > > > > > > >> > > >
>> >> > > > > > > > > > >> > > > We didn't update yet, as we are using
>> >> Hortonworks
>> >> > > > > > > distribution
>> >> > > > > > > > > > right
>> >> > > > > > > > > > >> > now,
>> >> > > > > > > > > > >> > > > and even if we update we will get 0.98.4.
>> >> However,
>> >> > > > looks
>> >> > > > > > > that
>> >> > > > > > > > > > issue
>> >> > > > > > > > > > >> > here
>> >> > > > > > > > > > >> > > > was in our use case and configuration (still
>> >> > looking
>> >> > > > > into
>> >> > > > > > > it).
>> >> > > > > > > > > > >> > > >
>> >> > > > > > > > > > >> > > > Basically, initially I saw that when one
>> server
>> >> > goes
>> >> > > > > down,
>> >> > > > > > > we
>> >> > > > > > > > > > start
>> >> > > > > > > > > > >> > > having
>> >> > > > > > > > > > >> > > > performance issues in general, but it
>> managed
>> >> to
>> >> > be
>> >> > > on
>> >> > > > > our
>> >> > > > > > > > > client
>> >> > > > > > > > > > >> side,
>> >> > > > > > > > > > >> > > due
>> >> > > > > > > > > > >> > > > to caching, and clients were trying to
>> >> reconnect
>> >> > to
>> >> > > > > nodes
>> >> > > > > > > that
>> >> > > > > > > > > > were
>> >> > > > > > > > > > >> > > offline
>> >> > > > > > > > > > >> > > > and later trying to get regions from those
>> >> nodes
>> >> > > too.
>> >> > > > > This
>> >> > > > > > > is
>> >> > > > > > > > > > >> basically
>> >> > > > > > > > > > >> > > why
>> >> > > > > > > > > > >> > > > on server side I didn't manage to see
>> anything
>> >> in
>> >> > > logs
>> >> > > > > > that
>> >> > > > > > > > > would
>> >> > > > > > > > > > >> be at
>> >> > > > > > > > > > >> > > > least little bit interesting and point me
>> into
>> >> > > desired
>> >> > > > > > > > > direction.
>> >> > > > > > > > > > >> > > >
>> >> > > > > > > > > > >> > > > Another question that popped up to me is -
>> in
>> >> case
>> >> > > > > server
>> >> > > > > > is
>> >> > > > > > > > > down
>> >> > > > > > > > > > >> (and
>> >> > > > > > > > > > >> > > with
>> >> > > > > > > > > > >> > > > it DataNode and HRegionServer it was
>> hosting) -
>> >> > > what's
>> >> > > > > > > optimal
>> >> > > > > > > > > > time
>> >> > > > > > > > > > >> to
>> >> > > > > > > > > > >> > > set
>> >> > > > > > > > > > >> > > > for HMaster to consider server dead reassign
>> >> > regions
>> >> > > > > > > somewhere
>> >> > > > > > > > > > >> else, as
>> >> > > > > > > > > > >> > > > this is another performance bottleneck we
>> hit
>> >> > during
>> >> > > > > > > inability
>> >> > > > > > > > > to
>> >> > > > > > > > > > >> > access
>> >> > > > > > > > > > >> > > > regions? In our case it's configured to 15
>> >> > minutes,
>> >> > > > and
>> >> > > > > > > simple
>> >> > > > > > > > > > logic
>> >> > > > > > > > > > >> > > tells
>> >> > > > > > > > > > >> > > > me if you want it earlier then configure
>> lower
>> >> > > number
>> >> > > > of
>> >> > > > > > > > > retries,
>> >> > > > > > > > > > >> but
>> >> > > > > > > > > > >> > > issue
>> >> > > > > > > > > > >> > > > is as always in details, so not sure if
>> anyone
>> >> > knows
>> >> > > > > some
>> >> > > > > > > > better
>> >> > > > > > > > > > >> math
>> >> > > > > > > > > > >> > for
>> >> > > > > > > > > > >> > > > this?
>> >> > > > > > > > > > >> > > >
>> >> > > > > > > > > > >> > > > And last question - is it possible to
>> manually
>> >> > force
>> >> > > > > HBase
>> >> > > > > > > to
>> >> > > > > > > > > > >> reassign
>> >> > > > > > > > > > >> > > > regions? In this case, while HMaster is
>> >> retrying
>> >> > to
>> >> > > > > > contact
>> >> > > > > > > > node
>> >> > > > > > > > > > >> that's
>> >> > > > > > > > > > >> > > > dead, it's impossible to force it using
>> >> 'balancer'
>> >> > > > > > command.
>> >> > > > > > > > > > >> > > >
>> >> > > > > > > > > > >> > > > Thanks a lot!
>> >> > > > > > > > > > >> > > >
>> >> > > > > > > > > > >> > > > Dejan
>> >> > > > > > > > > > >> > > >
>> >> > > > > > > > > > >> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan
>> Menges <
>> >> > > > > > > > > > >> dejan.menges@gmail.com>
>> >> > > > > > > > > > >> > > > wrote:
>> >> > > > > > > > > > >> > > >
>> >> > > > > > > > > > >> > > > > Hi,
>> >> > > > > > > > > > >> > > > >
>> >> > > > > > > > > > >> > > > > To be very honest - there's no particular
>> >> reason
>> >> > > why
>> >> > > > > we
>> >> > > > > > > > stick
>> >> > > > > > > > > to
>> >> > > > > > > > > > >> this
>> >> > > > > > > > > > >> > > > one,
>> >> > > > > > > > > > >> > > > > beside just lack of time currently to go
>> >> through
>> >> > > > > upgrade
>> >> > > > > > > > > > process,
>> >> > > > > > > > > > >> but
>> >> > > > > > > > > > >> > > > looks
>> >> > > > > > > > > > >> > > > > to me that's going to be next step.
>> >> > > > > > > > > > >> > > > >
>> >> > > > > > > > > > >> > > > > Had a crazy day, didn't have time to go
>> >> through
>> >> > > all
>> >> > > > > logs
>> >> > > > > > > > > again,
>> >> > > > > > > > > > >> plus
>> >> > > > > > > > > > >> > > one
>> >> > > > > > > > > > >> > > > > of the machines (last one where we had
>> this
>> >> > issue)
>> >> > > > is
>> >> > > > > > > fully
>> >> > > > > > > > > > >> > > reprovisioned
>> >> > > > > > > > > > >> > > > > yesterday so I don't have logs from there
>> >> > anymore.
>> >> > > > > > > > > > >> > > > >
>> >> > > > > > > > > > >> > > > > Beside upgrading,  what I will talk about
>> >> today,
>> >> > > can
>> >> > > > > you
>> >> > > > > > > > just
>> >> > > > > > > > > > >> point
>> >> > > > > > > > > > >> > me
>> >> > > > > > > > > > >> > > to
>> >> > > > > > > > > > >> > > > > the specific RPC issue in 0.98.0? Thing is
>> >> that
>> >> > we
>> >> > > > > have
>> >> > > > > > > some
>> >> > > > > > > > > > >> strange
>> >> > > > > > > > > > >> > > > > moments with RPC in this case, and just
>> want
>> >> to
>> >> > > see
>> >> > > > if
>> >> > > > > > > > that's
>> >> > > > > > > > > > the
>> >> > > > > > > > > > >> > same
>> >> > > > > > > > > > >> > > > > thing (and we were even suspecting to
>> RPC).
>> >> > > > > > > > > > >> > > > >
>> >> > > > > > > > > > >> > > > > Thanks a lot!
>> >> > > > > > > > > > >> > > > > Dejan
>> >> > > > > > > > > > >> > > > >
>> >> > > > > > > > > > >> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew
>> >> Purtell
>> >> > <
>> >> > > > > > > > > > >> apurtell@apache.org
>> >> > > > > > > > > > >> > >
>> >> > > > > > > > > > >> > > > > wrote:
>> >> > > > > > > > > > >> > > > >
>> >> > > > > > > > > > >> > > > >> Is there a particular reason why you are
>> >> using
>> >> > > > HBase
>> >> > > > > > > > 0.98.0?
>> >> > > > > > > > > > The
>> >> > > > > > > > > > >> > > latest
>> >> > > > > > > > > > >> > > > >> 0.98 release is 0.98.11. There's a known
>> >> > > > performance
>> >> > > > > > > issue
>> >> > > > > > > > > with
>> >> > > > > > > > > > >> > 0.98.0
>> >> > > > > > > > > > >> > > > >> pertaining to RPC that was fixed in later
>> >> > > releases,
>> >> > > > > you
>> >> > > > > > > > > should
>> >> > > > > > > > > > >> move
>> >> > > > > > > > > > >> > up
>> >> > > > > > > > > > >> > > > >> from
>> >> > > > > > > > > > >> > > > >> 0.98.0. In addition hundreds of
>> improvements
>> >> > and
>> >> > > > bug
>> >> > > > > > > fixes
>> >> > > > > > > > > have
>> >> > > > > > > > > > >> gone
>> >> > > > > > > > > > >> > > > into
>> >> > > > > > > > > > >> > > > >> the ten releases since 0.98.0.
>> >> > > > > > > > > > >> > > > >>
>> >> > > > > > > > > > >> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan
>> >> Menges <
>> >> > > > > > > > > > >> > dejan.menges@gmail.com
>> >> > > > > > > > > > >> > > >
>> >> > > > > > > > > > >> > > > >> wrote:
>> >> > > > > > > > > > >> > > > >>
>> >> > > > > > > > > > >> > > > >> > Hi All,
>> >> > > > > > > > > > >> > > > >> >
>> >> > > > > > > > > > >> > > > >> > We have a strange issue with HBase
>> >> > performance
>> >> > > > > > (overall
>> >> > > > > > > > > > cluster
>> >> > > > > > > > > > >> > > > >> > performance) in case one of datanodes
>> in
>> >> the
>> >> > > > > cluster
>> >> > > > > > > > > > >> unexpectedly
>> >> > > > > > > > > > >> > > goes
>> >> > > > > > > > > > >> > > > >> > down.
>> >> > > > > > > > > > >> > > > >> >
>> >> > > > > > > > > > >> > > > >> > So scenario is like follows:
>> >> > > > > > > > > > >> > > > >> > - Cluster works fine, it's stable.
>> >> > > > > > > > > > >> > > > >> > - One DataNode unexpectedly goes down
>> (PSU
>> >> > > issue,
>> >> > > > > > > network
>> >> > > > > > > > > > >> issue,
>> >> > > > > > > > > > >> > > > >> anything)
>> >> > > > > > > > > > >> > > > >> > - Whole HBase cluster goes down
>> >> (performance
>> >> > > > > becomes
>> >> > > > > > so
>> >> > > > > > > > bad
>> >> > > > > > > > > > >> that
>> >> > > > > > > > > > >> > we
>> >> > > > > > > > > > >> > > > >> have to
>> >> > > > > > > > > > >> > > > >> > restart all RegionServers to get it
>> back
>> >> to
>> >> > > > life).
>> >> > > > > > > > > > >> > > > >> >
>> >> > > > > > > > > > >> > > > >> > Most funny and latest issue that
>> happened
>> >> was
>> >> > > > that
>> >> > > > > we
>> >> > > > > > > > added
>> >> > > > > > > > > > new
>> >> > > > > > > > > > >> > node
>> >> > > > > > > > > > >> > > > to
>> >> > > > > > > > > > >> > > > >> the
>> >> > > > > > > > > > >> > > > >> > cluster (having 8 x 4T SATA disks) and
>> we
>> >> > left
>> >> > > > just
>> >> > > > > > > > > DataNode
>> >> > > > > > > > > > >> > running
>> >> > > > > > > > > > >> > > > on
>> >> > > > > > > > > > >> > > > >> it
>> >> > > > > > > > > > >> > > > >> > to give it couple of days to get some
>> >> data.
>> >> > At
>> >> > > > some
>> >> > > > > > > point
>> >> > > > > > > > > in
>> >> > > > > > > > > > >> time,
>> >> > > > > > > > > > >> > > due
>> >> > > > > > > > > > >> > > > >> to
>> >> > > > > > > > > > >> > > > >> > hardware issue, server rebooted (twice
>> >> during
>> >> > > > three
>> >> > > > > > > > hours)
>> >> > > > > > > > > in
>> >> > > > > > > > > > >> > moment
>> >> > > > > > > > > > >> > > > >> when
>> >> > > > > > > > > > >> > > > >> > it had maybe 5% of data it would have
>> in a
>> >> > > couple
>> >> > > > > of
>> >> > > > > > > > days.
>> >> > > > > > > > > > >> Nothing
>> >> > > > > > > > > > >> > > > else
>> >> > > > > > > > > > >> > > > >> > beside DataNode was running, and once
>> it
>> >> went
>> >> > > > down,
>> >> > > > > > it
>> >> > > > > > > > > > affected
>> >> > > > > > > > > > >> > > > literary
>> >> > > > > > > > > > >> > > > >> > everything, and restarting
>> RegionServers
>> >> in
>> >> > the
>> >> > > > end
>> >> > > > > > > fixed
>> >> > > > > > > > > it.
>> >> > > > > > > > > > >> > > > >> >
>> >> > > > > > > > > > >> > > > >> > We are using HBase 0.98.0 with Hadoop
>> >> 2.4.0
>> >> > > > > > > > > > >> > > > >> >
>> >> > > > > > > > > > >> > > > >>
>> >> > > > > > > > > > >> > > > >>
>> >> > > > > > > > > > >> > > > >>
>> >> > > > > > > > > > >> > > > >> --
>> >> > > > > > > > > > >> > > > >> Best regards,
>> >> > > > > > > > > > >> > > > >>
>> >> > > > > > > > > > >> > > > >>    - Andy
>> >> > > > > > > > > > >> > > > >>
>> >> > > > > > > > > > >> > > > >> Problems worthy of attack prove their
>> worth
>> >> by
>> >> > > > > hitting
>> >> > > > > > > > back.
>> >> > > > > > > > > -
>> >> > > > > > > > > > >> Piet
>> >> > > > > > > > > > >> > > Hein
>> >> > > > > > > > > > >> > > > >> (via Tom White)
>> >> > > > > > > > > > >> > > > >>
>> >> > > > > > > > > > >> > > > >
>> >> > > > > > > > > > >> > > > >
>> >> > > > > > > > > > >> > > >
>> >> > > > > > > > > > >> > >
>> >> > > > > > > > > > >> >
>> >> > > > > > > > > > >>
>> >> > > > > > > > > > >
>> >> > > > > > > > > >
>> >> > > > > > > > >
>> >> > > > > > > >
>> >> > > > > > >
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> >
>> >
>>
>

Re: Strange issue when DataNode goes down

Posted by Dejan Menges <de...@gmail.com>.
It was true all the time, together with
dfs.namenode.avoid.read.stale.datanode.

On Mon, Mar 23, 2015 at 5:29 PM Nicolas Liochon <nk...@gmail.com> wrote:

> Actually, double checking the final patch in HDFS-4721, the stale mode is
> taken in account. Bryan is right, it's worth checking the namenodes config.
> Especially, dfs.namenode.avoid.write.stale.datanode must be set to true on
> the namenode.
>
> On Mon, Mar 23, 2015 at 5:08 PM, Nicolas Liochon <nk...@gmail.com>
> wrote:
>
> > stale should not help for recoverLease: it helps for reads, but it's the
> > step after lease recovery.
> > It's not needed in recoverLease because the recoverLease in hdfs just
> > sorts the datanode by the heartbeat time, so, usually the stale datanode
> > will be the last one of the list.
> >
> >
> >
> >
> > On Mon, Mar 23, 2015 at 4:38 PM, Bryan Beaudreault <
> > bbeaudreault@hubspot.com> wrote:
> >
> >> @Nicholas, I see, thanks.  I'll keep the settings at default.  So really
> >> if
> >> everything else is configured properly you should never reach the lease
> >> recovery timeout in any failure scenarios.  It seems that the staleness
> >> check would be the thing that prevents this, correct?  I'm surprised it
> >> didn't help Dejan.
> >>
> >> On Mon, Mar 23, 2015 at 11:20 AM, Nicolas Liochon <nk...@gmail.com>
> >> wrote:
> >>
> >> > @bryan: yes, you can change hbase.lease.recovery.timeout if you
> changed
> >> he
> >> > hdfs settings. But this setting is really for desperate cases. The
> >> recover
> >> > Lease should have succeeded before. As well, if you depend on
> >> > hbase.lease.recovery.timeout, it means that you're wasting recovery
> >> time:
> >> > the lease should be recovered in a few seconds.
> >> >
> >> > On Mon, Mar 23, 2015 at 3:59 PM, Dejan Menges <dejan.menges@gmail.com
> >
> >> > wrote:
> >> >
> >> > > Interesting discussion I also found, gives me some more light on
> what
> >> > > Nicolas is talking about -
> >> > https://issues.apache.org/jira/browse/HDFS-3703
> >> > >
> >> > > On Mon, Mar 23, 2015 at 3:53 PM Bryan Beaudreault <
> >> > > bbeaudreault@hubspot.com>
> >> > > wrote:
> >> > >
> >> > > > So it is safe to set hbase.lease.recovery.timeout lower if you
> also
> >> > > > set heartbeat.recheck.interval lower (lowering that 10.5 min dead
> >> node
> >> > > > timer)?  Or is it recommended to not touch either of those?
> >> > > >
> >> > > > Reading the above with interest, thanks for digging in here guys.
> >> > > >
> >> > > > On Mon, Mar 23, 2015 at 10:13 AM, Nicolas Liochon <
> >> nkeywal@gmail.com>
> >> > > > wrote:
> >> > > >
> >> > > > > If the node is actually down it's fine. But the node may not be
> >> that
> >> > > down
> >> > > > > (CAP theorem here); and then it's looking for trouble.
> >> > > > > HDFS, by default declare a node as dead after 10:30. 15 minutes
> >> is an
> >> > > > extra
> >> > > > > security. It seems your hdfs settings are different (or there
> is a
> >> > > > bug...).
> >> > > > > There should be some info in the hdfs logs.
> >> > > > >
> >> > > > > On Mon, Mar 23, 2015 at 3:05 PM, Dejan Menges <
> >> > dejan.menges@gmail.com>
> >> > > > > wrote:
> >> > > > >
> >> > > > > > Will take a look.
> >> > > > > >
> >> > > > > > Actually, if node is down (someone unplugged network cable, it
> >> just
> >> > > > died,
> >> > > > > > whatever) what's chance it's going to become live so write can
> >> > > > continue?
> >> > > > > On
> >> > > > > > the other side, HBase is not starting recovery trying to
> contact
> >> > node
> >> > > > > which
> >> > > > > > is not there anymore, and even elected as dead on Namenode
> side
> >> > > > (another
> >> > > > > > thing I didn't understood quite good).
> >> > > > > >
> >> > > > > > So what I was expecting is that as soon as Namenode decided
> >> node is
> >> > > > dead,
> >> > > > > > that it would be enough for RegionServer to stop trying to
> >> recover
> >> > > from
> >> > > > > the
> >> > > > > > dead node, but it wasn't the case. Also, this whole MTTR
> >> article in
> >> > > > HBase
> >> > > > > > book doesn't work at all with this parameter set to it's
> default
> >> > > value
> >> > > > > (15
> >> > > > > > minutes).
> >> > > > > >
> >> > > > > > So I'm still struggling to figure out what's drawback exactly
> on
> >> > > this?
> >> > > > > >
> >> > > > > > On Mon, Mar 23, 2015 at 2:50 PM Nicolas Liochon <
> >> nkeywal@gmail.com
> >> > >
> >> > > > > wrote:
> >> > > > > >
> >> > > > > > > Thanks for the explanation. There is an issue if you modify
> >> this
> >> > > > > setting
> >> > > > > > > however.
> >> > > > > > > hbase tries to recover the lease (i.e. be sure that nobody
> is
> >> > > > writing)
> >> > > > > > > If you change hbase.lease.recovery.timeout hbase will start
> >> the
> >> > > > > recovery
> >> > > > > > > (i.e. start to read) even if it's not sure that nobody's
> >> writing.
> >> > > > That
> >> > > > > > > means there is a dataloss risk.
> >> > > > > > > Basically, you must not see this warning: WARN
> >> > > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils: Cannot
> recoverLease
> >> > after
> >> > > > > > trying
> >> > > > > > > for[]
> >> > > > > > >
> >> > > > > > > The recoverLease must succeed. The fact that it does not
> >> after X
> >> > > > tries
> >> > > > > is
> >> > > > > > > strange.
> >> > > > > > > There may be a mistmatch between the hbase parameters and
> the
> >> > hdfs
> >> > > > > ones.
> >> > > > > > > You may need to have a look at the comments in
> >> FSHDFSUtils.java
> >> > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > On Mon, Mar 23, 2015 at 2:15 PM, Dejan Menges <
> >> > > > dejan.menges@gmail.com>
> >> > > > > > > wrote:
> >> > > > > > >
> >> > > > > > > > I found the issue and fixed it, and will try to explain
> here
> >> > what
> >> > > > was
> >> > > > > > > > exactly in our case in case someone else finds this
> >> interesting
> >> > > > too.
> >> > > > > > > >
> >> > > > > > > > So initially, we had (couple of times) some network and
> >> > hardware
> >> > > > > issues
> >> > > > > > > in
> >> > > > > > > > our datacenters. When one server would die (literary die,
> we
> >> > had
> >> > > > some
> >> > > > > > > issue
> >> > > > > > > > with PSUs) we saw issues with overall cluster performance
> on
> >> > > HBase
> >> > > > > > side.
> >> > > > > > > As
> >> > > > > > > > cluster is quite big and live, it was also quite hard to
> >> figure
> >> > > out
> >> > > > > > exact
> >> > > > > > > > root cause and how to fix the stuff we wanted to fix.
> >> > > > > > > >
> >> > > > > > > > So I set up another cluster, four nodes (with DataNode and
> >> > > > > > RegionServer)
> >> > > > > > > > and two other nodes with HMaster and Namenode in HA, using
> >> same
> >> > > > stuff
> >> > > > > > we
> >> > > > > > > > use on production. We pumped some data into it, and I was
> >> able
> >> > to
> >> > > > > > > reproduce
> >> > > > > > > > same issue last week on it. What I tried to do is to cut
> one
> >> > > server
> >> > > > > > (shut
> >> > > > > > > > down it's interface) when all is good with cluster, when
> we
> >> > have
> >> > > > > load,
> >> > > > > > > and
> >> > > > > > > > see what's going to happen.
> >> > > > > > > >
> >> > > > > > > > On Friday, after Nicolas mentioned, I started taking a
> look
> >> in
> >> > > > HBase
> >> > > > > > logs
> >> > > > > > > > on the node which was mentioned in HMaster log as the one
> >> > taking
> >> > > > over
> >> > > > > > > > regions for the dead server. Basically what I was able to
> >> > observe
> >> > > > was
> >> > > > > > 15
> >> > > > > > > > minutes time (+- couple of seconds, what was also
> >> interesting,
> >> > > and
> >> > > > > will
> >> > > > > > > got
> >> > > > > > > > later to that) between HMaster figures out that one of
> it's
> >> > > > > > RegionServers
> >> > > > > > > > is dead, and the time one of the mentioned nodes starts
> >> taking
> >> > > over
> >> > > > > > those
> >> > > > > > > > regions and they start appearing in HMaster's Web UI.
> >> > > > > > > >
> >> > > > > > > > I then set up everything like mentioned here
> >> > > > > > > > http://hbase.apache.org/book.html#mttr - but still had
> >> exactly
> >> > > the
> >> > > > > > same
> >> > > > > > > > issues. Went over (again and again) all currently
> configured
> >> > > stuff,
> >> > > > > but
> >> > > > > > > > still had the same issue.
> >> > > > > > > >
> >> > > > > > > > Then I started looking into HDFS. Opened NameNode UI, saw
> >> all
> >> > is
> >> > > > > good,
> >> > > > > > > took
> >> > > > > > > > one node down, was also looking RegionServer logs in the
> >> same
> >> > > time
> >> > > > -
> >> > > > > > and
> >> > > > > > > I
> >> > > > > > > > also see that it took ~15 minutes for Namenode to elect
> dead
> >> > node
> >> > > > as
> >> > > > > > > dead.
> >> > > > > > > > Somehow in the same moment regions started getting back to
> >> > life.
> >> > > I
> >> > > > > > > remember
> >> > > > > > > > in some older versions dfs timeout checks and check
> retries.
> >> > > Looked
> >> > > > > > into
> >> > > > > > > > defaults for our Hadoop version -
> >> > > > > > > >
> >> > > > > > > > http://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/
> >> > > > > > > hadoop-hdfs/hdfs-default.xml
> >> > > > > > > > - and saw there that there's no recheck parameter anymore.
> >> > > Strange,
> >> > > > > as
> >> > > > > > on
> >> > > > > > > > StackOverflow I found post from month ago, for newer
> version
> >> > than
> >> > > > we
> >> > > > > > use
> >> > > > > > > > (we use 2.4.1, guy was using 2.6 - dfs.namenode.heartbeat.
> >> > > > > > > recheck-interval)
> >> > > > > > > > I set it to 10 seconds as he mentioned, having checks
> every
> >> > three
> >> > > > > > seconds
> >> > > > > > > > (default) and got DataNode elected as dead in ~45 seconds,
> >> as
> >> > he
> >> > > > > > > mentioned.
> >> > > > > > > > Not sure why this parameter is not documented, but
> >> obviously it
> >> > > > > works.
> >> > > > > > > >
> >> > > > > > > > Then figured out it still didn't fix our HBase failover
> >> issue.
> >> > I
> >> > > > was
> >> > > > > > > > looking into HBase book again and again, and then saw this
> >> > part:
> >> > > > > > > >
> >> > > > > > > > "How much time we allow elapse between calls to recover
> >> lease.
> >> > > > Should
> >> > > > > > be
> >> > > > > > > > larger than the dfs timeout."
> >> > > > > > > >
> >> > > > > > > > This was description for hbase.lease.recovery.dfs.
> timeout.
> >> > Wasn't
> >> > > > > sure
> >> > > > > > > from
> >> > > > > > > > the comment what of all timeouts that's possible to set in
> >> > > > > Hadoop/HBase
> >> > > > > > > and
> >> > > > > > > > that has something to do with DFS is this all about. But
> >> picked
> >> > > > > > > > hbase.lease.recovery.timeout, and that was the catch.
> >> > > > > > > >
> >> > > > > > > > Initially, by default, hbase.lease.recovery.timeout is set
> >> to
> >> > 15
> >> > > > > > minutes.
> >> > > > > > > > Not sure why, and wasn't able to find yet why, but getting
> >> this
> >> > > > down
> >> > > > > to
> >> > > > > > > one
> >> > > > > > > > minute (what's more than OK for us) I was able to get rid
> of
> >> > our
> >> > > > > issue.
> >> > > > > > > Not
> >> > > > > > > > also sure why this is not mentioned in MTTR section in
> HBase
> >> > > book,
> >> > > > as
> >> > > > > > > > obviously MTTR doesn't work at all with this default
> >> timeout,
> >> > at
> >> > > > > least
> >> > > > > > it
> >> > > > > > > > doesn't work the way we expected it to work.
> >> > > > > > > >
> >> > > > > > > > So thanks again for everyone being spammed with this, and
> >> > > specially
> >> > > > > > > thanks
> >> > > > > > > > to Nicolas pointing me to the right direction.
> >> > > > > > > >
> >> > > > > > > > On Mon, Mar 23, 2015 at 1:37 PM Nicolas Liochon <
> >> > > nkeywal@gmail.com
> >> > > > >
> >> > > > > > > wrote:
> >> > > > > > > >
> >> > > > > > > > > the attachments are rejected by the mailing list, can
> you
> >> put
> >> > > > then
> >> > > > > on
> >> > > > > > > > > pastebin?
> >> > > > > > > > >
> >> > > > > > > > > stale is mandatory (so it's good), but the issue here is
> >> just
> >> > > > > before.
> >> > > > > > > The
> >> > > > > > > > > region server needs to read the file. In order to be
> sure
> >> > that
> >> > > > > there
> >> > > > > > is
> >> > > > > > > > no
> >> > > > > > > > > data loss, it needs to "recover the lease", that means
> >> > ensuring
> >> > > > > that
> >> > > > > > > > nobody
> >> > > > > > > > > is writing the file. The regionserver calls the namenode
> >> to
> >> > do
> >> > > > this
> >> > > > > > > > > recoverLease. So there should be some info in the
> namenode
> >> > > logs.
> >> > > > > You
> >> > > > > > > have
> >> > > > > > > > > HDFS-4721 on your hdfs? The hbase book details (more or
> >> > > less...)
> >> > > > > this
> >> > > > > > > > > recoverLease stuff.
> >> > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > > > On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <
> >> > > > > > dejan.menges@gmail.com
> >> > > > > > > >
> >> > > > > > > > > wrote:
> >> > > > > > > > >
> >> > > > > > > > > > And also, just checked -
> dfs.namenode.avoid.read.stale.
> >> > > > datanode
> >> > > > > and
> >> > > > > > > > > > dfs.namenode.avoid.write.stale.datanode
> >> > > > > > > > > > are both true, and dfs.namenode.stale.datanode.
> interval
> >> is
> >> > > > set to
> >> > > > > > > > > default
> >> > > > > > > > > > 30000.
> >> > > > > > > > > >
> >> > > > > > > > > > On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <
> >> > > > > > > dejan.menges@gmail.com>
> >> > > > > > > > > > wrote:
> >> > > > > > > > > >
> >> > > > > > > > > > > Hi Nicolas,
> >> > > > > > > > > > >
> >> > > > > > > > > > > Please find log attached.
> >> > > > > > > > > > >
> >> > > > > > > > > > > As I see it now more clearly, it was trying to
> recover
> >> > HDFS
> >> > > > > WALs
> >> > > > > > > from
> >> > > > > > > > > > node
> >> > > > > > > > > > > that's dead:
> >> > > > > > > > > > >
> >> > > > > > > > > > > 2015-03-23 08:53:44,381 WARN
> >> > > > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils:
> >> > > > > > > > > > > Cannot recoverLease after trying for 900000ms
> >> > > > > > > > > > > (hbase.lease.recovery.timeout); continuing, but
> may be
> >> > > > > > > DATALOSS!!!;
> >> > > > > > > > > > > attempt=40 on
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intent
> >> > > > > > > > > ionally_get_down_by_getting_
> network_down},60020,1426862900
> >> > > > > > > > > 506-splitting/{node_i_intentionally_get_down_by_gett
> >> > > > > > > > > ing_network_down}%2C60020%2C1426862900506.1427096924508
> >> > > > > > > > > > > after 908210ms
> >> > > > > > > > > > >
> >> > > > > > > > > > > And as you can see from the log, it tried 40 times,
> >> what
> >> > > took
> >> > > > > it
> >> > > > > > > > > exactly
> >> > > > > > > > > > > 15 minutes.
> >> > > > > > > > > > >
> >> > > > > > > > > > > There's probably some parameter to tune to cut it of
> >> from
> >> > > 40
> >> > > > > > times
> >> > > > > > > /
> >> > > > > > > > 15
> >> > > > > > > > > > > minutes to something more useful, as for 15 minutes
> we
> >> > > don't
> >> > > > > have
> >> > > > > > > our
> >> > > > > > > > > > > regions available, and HDFS have however replication
> >> > factor
> >> > > > 3.
> >> > > > > > > > > > >
> >> > > > > > > > > > > Googling, if I figure out what's this I will post it
> >> > here.
> >> > > > Will
> >> > > > > > > also
> >> > > > > > > > > > > appreciate if someone knows how to cut this down.
> >> > > > > > > > > > >
> >> > > > > > > > > > > Thanks,
> >> > > > > > > > > > >
> >> > > > > > > > > > > Dejan
> >> > > > > > > > > > >
> >> > > > > > > > > > > On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <
> >> > > > > > nkeywal@gmail.com
> >> > > > > > > >
> >> > > > > > > > > > wrote:
> >> > > > > > > > > > >
> >> > > > > > > > > > >> The split is done by the region servers (the master
> >> > > > > > coordinates).
> >> > > > > > > Is
> >> > > > > > > > > > there
> >> > > > > > > > > > >> some interesting stuff in their logs?
> >> > > > > > > > > > >>
> >> > > > > > > > > > >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <
> >> > > > > > > > dejan.menges@gmail.com
> >> > > > > > > > > >
> >> > > > > > > > > > >> wrote:
> >> > > > > > > > > > >>
> >> > > > > > > > > > >> > With client issue was that it was retrying
> >> connecting
> >> > to
> >> > > > the
> >> > > > > > > same
> >> > > > > > > > > > region
> >> > > > > > > > > > >> > servers even when they were reassigned. Lowering
> it
> >> > down
> >> > > > > > helped
> >> > > > > > > in
> >> > > > > > > > > > this
> >> > > > > > > > > > >> > specific use case, but yes, we still want servers
> >> to
> >> > > > > > reallocate
> >> > > > > > > > > > quickly.
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > What got me here:
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > http://hbase.apache.org/book.html#mttr
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > I basically set configuration exactly the same
> way
> >> as
> >> > > it's
> >> > > > > > > > explained
> >> > > > > > > > > > >> here.
> >> > > > > > > > > > >> > *zookeeper.session.timeout* is (and was before)
> >> 60000
> >> > > (one
> >> > > > > > > > minute).
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > So basically what happens is: - simulating
> network
> >> > > issues
> >> > > > we
> >> > > > > > had
> >> > > > > > > > > > >> recently).
> >> > > > > > > > > > >> > - After short time I see in HBase that my
> >> RegionServer
> >> > > is
> >> > > > > > dead,
> >> > > > > > > > and
> >> > > > > > > > > as
> >> > > > > > > > > > >> > total number of regions I see previous total
> minus
> >> > > number
> >> > > > of
> >> > > > > > > > regions
> >> > > > > > > > > > >> that
> >> > > > > > > > > > >> > were hosted on the node hosting RegionServer that
> >> just
> >> > > > > > > > > 'disappeared'.
> >> > > > > > > > > > >> > - In this point I want my clus
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > - I have test cluster consisting of four nodes,
> >> every
> >> > > node
> >> > > > > > being
> >> > > > > > > > > > >> DataNode
> >> > > > > > > > > > >> > and RegionServer.
> >> > > > > > > > > > >> > - I simulate network partition on one (connect to
> >> it
> >> > > > through
> >> > > > > > > > console
> >> > > > > > > > > > and
> >> > > > > > > > > > >> > take network interface downter to recover as soon
> >> as
> >> > > > > possible,
> >> > > > > > > to
> >> > > > > > > > > > start
> >> > > > > > > > > > >> > serving missing regions.
> >> > > > > > > > > > >> > - First thing I see in HMaster logs are:
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > 2015-03-20 14:17:26,015 INFO
> >> > > > > > > > > > >> >
> >> org.apache.hadoop.hbase.zookeeper.RegionServerTracker:
> >> > > > > > > > RegionServer
> >> > > > > > > > > > >> > ephemeral node deleted, processing expiration
> >> > > > > > > > > > >> > [{name_of_node_I_took_down},60020,1426860403261]
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > 2015-03-20 14:17:26,067 INFO
> >> > > > > > > > > > >> >
> >> > > > > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler:
> >> > > > > > > > > > Splitting
> >> > > > > > > > > > >> > logs for
> >> > {name_of_node_I_took_down},60020,1426860403261
> >> > > > > before
> >> > > > > > > > > > >> assignment.
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > 2015-03-20 14:17:26,105 INFO
> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> >> dead
> >> > > > > splitlog
> >> > > > > > > > > workers
> >> > > > > > > > > > [
> >> > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261]
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > 2015-03-20 14:17:26,107 INFO
> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> >> > started
> >> > > > > > > splitting
> >> > > > > > > > 1
> >> > > > > > > > > > >> logs in
> >> > > > > > > > > > >> >
> >> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_
> >> > > > I_
> >> > > > > > > > > took_down}
> >> > > > > > > > > > >> > ,60020,1426860403261-splitting]
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > 2015-03-20 14:17:26,150 INFO
> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> >> task
> >> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F
> >> > > > > > > > > > >> >
> >> > > > > > {name_of_node_I_took_down}%2C60020%2C1426860403261-
> splitting%2F
> >> > > > > > > > > > >> >
> >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
> >> > > > 14268
> >> > > > > > > > > 60404905
> >> > > > > > > > > > >> > acquired by {fqdn_of_another_live_hnode},
> >> > > > 60020,1426859445623
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > 2015-03-20 14:17:26,182 INFO
> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> >> total
> >> > > > tasks
> >> > > > > =
> >> > > > > > 1
> >> > > > > > > > > > >> unassigned
> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> >> > > > 2F{name_of_node_I_took_down}
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> >> > > > took_
> >> > > > > > > > > > >>
> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> >> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name
> =
> >> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> >> > status =
> >> > > > > > > > > in_progress
> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed
> = 1
> >> > done
> >> > > > = 0
> >> > > > > > > > error =
> >> > > > > > > > > > 0}
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > 2015-03-20 14:17:31,183 INFO
> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> >> total
> >> > > > tasks
> >> > > > > =
> >> > > > > > 1
> >> > > > > > > > > > >> unassigned
> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> >> > > > 2F{name_of_node_I_took_down}
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> >> > > > took_
> >> > > > > > > > > > >>
> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> >> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name
> =
> >> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> >> > status =
> >> > > > > > > > > in_progress
> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed
> = 1
> >> > done
> >> > > > = 0
> >> > > > > > > > error =
> >> > > > > > > > > > 0}
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > 2015-03-20 14:17:36,184 INFO
> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> >> total
> >> > > > tasks
> >> > > > > =
> >> > > > > > 1
> >> > > > > > > > > > >> unassigned
> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> >> > > > 2F{name_of_node_I_took_down}
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> >> > > > took_
> >> > > > > > > > > > >>
> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> >> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name
> =
> >> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> >> > status =
> >> > > > > > > > > in_progress
> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed
> = 1
> >> > done
> >> > > > = 0
> >> > > > > > > > error =
> >> > > > > > > > > > 0}
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > 2015-03-20 14:17:42,185 INFO
> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> >> total
> >> > > > tasks
> >> > > > > =
> >> > > > > > 1
> >> > > > > > > > > > >> unassigned
> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> >> > > > 2F{name_of_node_I_took_down}
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> >> > > > took_
> >> > > > > > > > > > >>
> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> >> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name
> =
> >> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> >> > status =
> >> > > > > > > > > in_progress
> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed
> = 1
> >> > done
> >> > > > = 0
> >> > > > > > > > error =
> >> > > > > > > > > > 0}
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > 2015-03-20 14:17:48,184 INFO
> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> >> total
> >> > > > tasks
> >> > > > > =
> >> > > > > > 1
> >> > > > > > > > > > >> unassigned
> >> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> >> > > > 2F{name_of_node_I_took_down}
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> >> > > > took_
> >> > > > > > > > > > >>
> >> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> >> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name
> =
> >> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> >> > status =
> >> > > > > > > > > in_progress
> >> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed
> = 1
> >> > done
> >> > > > = 0
> >> > > > > > > > error =
> >> > > > > > > > > > 0}
> >> > > > > > > > > > >> > In the meantime, In hbase...out log I got this:
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > ==>
> >> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out
> >> > <==
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > java.io.IOException: Call to
> >> > > > > > > > > > >> >
> >> > > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_
> >> > > > dow
> >> > > > > > > > > n}:60020
> >> > > > > > > > > > >> > failed on local exception:
> >> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> >> > > > CallTimeoutException:
> >> > > > > > > Call
> >> > > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
> >> > > > > > > > > > >> t.java:1532)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:
> >> > > > > > > 1502)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> > > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(
> >> > > > Rpc
> >> > > > > > > > > > >> Client.java:1684)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> >> > > > BlockingRpcChannelImpl
> >> > > > > > > > > ementati
> >> > > > > > > > > > >> on.callBlockingMethod(RpcClient.java:1737)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$
> >> > > > > > > > > AdminService$
> >> > > > > > > > > > >> BlockingStub.getRegionInfo(AdminProtos.java:20806)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > org.apache.hadoop.hbase.client.HBaseAdmin.
> >> > > > getCompactionState
> >> > > > > > > > > > >> (HBaseAdmin.java:2524)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> org.apache.hadoop.hbase.generated.master.table_jsp._
> >> > > > jspServi
> >> > > > > > > > > > >> ce(table_jsp.java:167)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.
> >> > > > > > > > > java:98)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
> >> > > > > > > > > > >> .java:511)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> >> > > > doFilte
> >> > > > > > > > > > >> r(ServletHandler.java:1221)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > org.apache.hadoop.http.lib.StaticUserWebFilter$
> >> > > > StaticUserFil
> >> > > > > > > > > > >> ter.doFilter(StaticUserWebFilter.java:109)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> >> > > > doFilte
> >> > > > > > > > > > >> r(ServletHandler.java:1212)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> org.apache.hadoop.http.HttpServer$QuotingInputFilter.
> >> > > > doFilte
> >> > > > > > > > > > >> r(HttpServer.java:1081)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> >> > > > doFilte
> >> > > > > > > > > > >> r(ServletHandler.java:1212)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > >
> >> > > > > >
> >> > org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> >> > > > doFilte
> >> > > > > > > > > > >> r(ServletHandler.java:1212)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
> >> > > > > > > > > > >> er.java:399)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> > org.mortbay.jetty.security.
> SecurityHandler.handle(
> >> > > > SecurityHa
> >> > > > > > > > > > >> ndler.java:216)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
> >> > > > > > > > > > >> er.java:182)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
> >> > > > > > > > > > >> er.java:766)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
> >> > > > > > > > > > >> java:450)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > org.mortbay.jetty.handler.
> ContextHandlerCollection.
> >> > > > handle(Co
> >> > > > > > > > > > >> ntextHandlerCollection.java:230)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
> >> > > > > > > > > > >> er.java:152)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at org.mortbay.jetty.Server.
> handle(Server.java:326)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
> >> > > > > > > > > > >> n.java:542)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > org.mortbay.jetty.HttpConnection$RequestHandler.
> >> > > > headerComple
> >> > > > > > > > > > >> te(HttpConnection.java:928)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > >
> >> > org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > >
> >> > org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > org.mortbay.io.nio.SelectChannelEndPoint.run(
> >> > > > SelectChannelEn
> >> > > > > > > > > > >> dPoint.java:410)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > org.mortbay.thread.QueuedThreadPool$PoolThread.
> run(
> >> > > > > > > > > > >> QueuedThreadPool.java:582)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > Caused by: org.apache.hadoop.hbase.ipc.
> RpcClient$
> >> > > > > > > > > CallTimeoutException:
> >> > > > > > > > > > >> Call
> >> > > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.
> RpcClient$Connection.
> >> > > > cleanupCall
> >> > > > > > > > > > >> s(RpcClient.java:1234)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.
> RpcClient$Connection.
> >> > > > readRespons
> >> > > > > > > > > > >> e(RpcClient.java:1171)
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > at
> >> > > > > > org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
> >> > > > > > > > > > >> nt.java:751)
> >> > > > > > > > > > >> > Beside this same issue, please note that first
> >> message
> >> > > was
> >> > > > > at
> >> > > > > > > > > > 2015-03-20
> >> > > > > > > > > > >> > 14:17:26,015. And then (we got to the point when
> it
> >> > > > started
> >> > > > > > > > > > transition):
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > 2015-03-20 14:32:35,059 INFO
> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> >> task
> >> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F
> >> > > > > > > > > > >> >
> >> > > > > > {name_of_node_I_took_down}%2C60020%2C1426860403261-
> splitting%2F
> >> > > > > > > > > > >> >
> >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
> >> > > > 14268
> >> > > > > > > > > 60404905
> >> > > > > > > > > > >> > entered state: DONE
> >> > > > > > {fqdn_of_new_live_node},60020,1426859445623
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > 2015-03-20 14:32:35,109 INFO
> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> >> Done
> >> > > > > splitting
> >> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F{name_
> of_node_I_took_down}
> >> > > > > > > > > > >> >
> >> > > > > > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> took_down}
> >> > > > > > > > > > >> > %252C60020%252C1426860403261.1426860404905
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > 2015-03-20 14:32:35,190 INFO
> >> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> >> > finished
> >> > > > > > > splitting
> >> > > > > > > > > > >> (more
> >> > > > > > > > > > >> > than or equal to) 9 bytes in 1 log files in
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_
> >> > > > of_
> >> > > > > > > > > > >> node_I_took_down},60020,1426860403261-splitting]
> >> > > > > > > > > > >> > in 909083ms
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> >> > > > org.apache.hadoop.hbase.master
> >> > > > > > > > > > >> .RegionStates:
> >> > > > > > > > > > >> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c
> >> > > > state=OPEN,
> >> > > > > > > > > > >> > ts=1426860639088,
> >> > > > > > > > > > server={name_of_node_I_took_
> down},60020,1426860403261}
> >> > > > > > > > > > >> to
> >> > > > > > > > > > >> > {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE,
> >> > > > > > > ts=1426861955191,
> >> > > > > > > > > > >> server=
> >> > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> >> > > > org.apache.hadoop.hbase.master
> >> > > > > > > > > > >> .RegionStates:
> >> > > > > > > > > > >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
> >> > > > > > > > > > >> {name_of_node_I_took_down}
> >> > > > > > > > > > >> > ,60020,1426860403261
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> >> > > > org.apache.hadoop.hbase.master
> >> > > > > > > > > > >> .RegionStates:
> >> > > > > > > > > > >> > Transitioned {25ab6e7b42e36ddaa723d71be5954543
> >> > > > state=OPEN,
> >> > > > > > > > > > >> > ts=1426860641783,
> >> > > > > > > > > > server={name_of_node_I_took_
> down},60020,1426860403261}
> >> > > > > > > > > > >> to
> >> > > > > > > > > > >> > {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE,
> >> > > > > > > ts=1426861955191,
> >> > > > > > > > > > >> server=
> >> > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> >> > > > org.apache.hadoop.hbase.master
> >> > > > > > > > > > >> .RegionStates:
> >> > > > > > > > > > >> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
> >> > > > > > > > > > >> {name_of_node_I_took_down}
> >> > > > > > > > > > >> > ,60020,1426860403261
> >> > > > > > > > > > >> > At this point, note that it finished
> >> SplitLogManager
> >> > > task
> >> > > > at
> >> > > > > > > > > 14:32:35
> >> > > > > > > > > > >> and
> >> > > > > > > > > > >> > started transitioning just after that. So this is
> >> 15
> >> > > > minutes
> >> > > > > > > that
> >> > > > > > > > > I'm
> >> > > > > > > > > > >> > talking about.
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > What am I missing?
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <
> >> > > > > > > > nkeywal@gmail.com>
> >> > > > > > > > > > >> wrote:
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > > You've changed the value of
> >> hbase.zookeeper.timeout
> >> > to
> >> > > > 15
> >> > > > > > > > > minutes? A
> >> > > > > > > > > > >> very
> >> > > > > > > > > > >> > > reasonable target is 1 minute before relocating
> >> the
> >> > > > > regions.
> >> > > > > > > > > That's
> >> > > > > > > > > > >> the
> >> > > > > > > > > > >> > > default iirc. You can push it to 20s, but then
> >> > > > > > > > > gc-stopping-the-world
> >> > > > > > > > > > >> > > becomes more of an issue. 15 minutes is really
> a
> >> > lot.
> >> > > > The
> >> > > > > > hdfs
> >> > > > > > > > > stale
> >> > > > > > > > > > >> mode
> >> > > > > > > > > > >> > > must always be used, with a lower timeout than
> >> the
> >> > > hbase
> >> > > > > > one.
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> > > Client side there should be nothing to do
> >> (excepted
> >> > > for
> >> > > > > > > advanced
> >> > > > > > > > > > >> stuff);
> >> > > > > > > > > > >> > at
> >> > > > > > > > > > >> > > each retry the client checks the location of
> the
> >> > > > regions.
> >> > > > > If
> >> > > > > > > you
> >> > > > > > > > > > lower
> >> > > > > > > > > > >> > the
> >> > > > > > > > > > >> > > number of retry the client will fail sooner,
> but
> >> > > usually
> >> > > > > you
> >> > > > > > > > don't
> >> > > > > > > > > > >> want
> >> > > > > > > > > > >> > the
> >> > > > > > > > > > >> > > client to fail, you want the servers to
> >> reallocate
> >> > > > > quickly.
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <
> >> > > > > > > > > > dejan.menges@gmail.com
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >> > > wrote:
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> > > > Hi,
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > Sorry for little bit late update, but managed
> >> to
> >> > > > narrow
> >> > > > > it
> >> > > > > > > > > little
> >> > > > > > > > > > >> bit
> >> > > > > > > > > > >> > > down.
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > We didn't update yet, as we are using
> >> Hortonworks
> >> > > > > > > distribution
> >> > > > > > > > > > right
> >> > > > > > > > > > >> > now,
> >> > > > > > > > > > >> > > > and even if we update we will get 0.98.4.
> >> However,
> >> > > > looks
> >> > > > > > > that
> >> > > > > > > > > > issue
> >> > > > > > > > > > >> > here
> >> > > > > > > > > > >> > > > was in our use case and configuration (still
> >> > looking
> >> > > > > into
> >> > > > > > > it).
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > Basically, initially I saw that when one
> server
> >> > goes
> >> > > > > down,
> >> > > > > > > we
> >> > > > > > > > > > start
> >> > > > > > > > > > >> > > having
> >> > > > > > > > > > >> > > > performance issues in general, but it managed
> >> to
> >> > be
> >> > > on
> >> > > > > our
> >> > > > > > > > > client
> >> > > > > > > > > > >> side,
> >> > > > > > > > > > >> > > due
> >> > > > > > > > > > >> > > > to caching, and clients were trying to
> >> reconnect
> >> > to
> >> > > > > nodes
> >> > > > > > > that
> >> > > > > > > > > > were
> >> > > > > > > > > > >> > > offline
> >> > > > > > > > > > >> > > > and later trying to get regions from those
> >> nodes
> >> > > too.
> >> > > > > This
> >> > > > > > > is
> >> > > > > > > > > > >> basically
> >> > > > > > > > > > >> > > why
> >> > > > > > > > > > >> > > > on server side I didn't manage to see
> anything
> >> in
> >> > > logs
> >> > > > > > that
> >> > > > > > > > > would
> >> > > > > > > > > > >> be at
> >> > > > > > > > > > >> > > > least little bit interesting and point me
> into
> >> > > desired
> >> > > > > > > > > direction.
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > Another question that popped up to me is - in
> >> case
> >> > > > > server
> >> > > > > > is
> >> > > > > > > > > down
> >> > > > > > > > > > >> (and
> >> > > > > > > > > > >> > > with
> >> > > > > > > > > > >> > > > it DataNode and HRegionServer it was
> hosting) -
> >> > > what's
> >> > > > > > > optimal
> >> > > > > > > > > > time
> >> > > > > > > > > > >> to
> >> > > > > > > > > > >> > > set
> >> > > > > > > > > > >> > > > for HMaster to consider server dead reassign
> >> > regions
> >> > > > > > > somewhere
> >> > > > > > > > > > >> else, as
> >> > > > > > > > > > >> > > > this is another performance bottleneck we hit
> >> > during
> >> > > > > > > inability
> >> > > > > > > > > to
> >> > > > > > > > > > >> > access
> >> > > > > > > > > > >> > > > regions? In our case it's configured to 15
> >> > minutes,
> >> > > > and
> >> > > > > > > simple
> >> > > > > > > > > > logic
> >> > > > > > > > > > >> > > tells
> >> > > > > > > > > > >> > > > me if you want it earlier then configure
> lower
> >> > > number
> >> > > > of
> >> > > > > > > > > retries,
> >> > > > > > > > > > >> but
> >> > > > > > > > > > >> > > issue
> >> > > > > > > > > > >> > > > is as always in details, so not sure if
> anyone
> >> > knows
> >> > > > > some
> >> > > > > > > > better
> >> > > > > > > > > > >> math
> >> > > > > > > > > > >> > for
> >> > > > > > > > > > >> > > > this?
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > And last question - is it possible to
> manually
> >> > force
> >> > > > > HBase
> >> > > > > > > to
> >> > > > > > > > > > >> reassign
> >> > > > > > > > > > >> > > > regions? In this case, while HMaster is
> >> retrying
> >> > to
> >> > > > > > contact
> >> > > > > > > > node
> >> > > > > > > > > > >> that's
> >> > > > > > > > > > >> > > > dead, it's impossible to force it using
> >> 'balancer'
> >> > > > > > command.
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > Thanks a lot!
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > Dejan
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges
> <
> >> > > > > > > > > > >> dejan.menges@gmail.com>
> >> > > > > > > > > > >> > > > wrote:
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > > Hi,
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > > > To be very honest - there's no particular
> >> reason
> >> > > why
> >> > > > > we
> >> > > > > > > > stick
> >> > > > > > > > > to
> >> > > > > > > > > > >> this
> >> > > > > > > > > > >> > > > one,
> >> > > > > > > > > > >> > > > > beside just lack of time currently to go
> >> through
> >> > > > > upgrade
> >> > > > > > > > > > process,
> >> > > > > > > > > > >> but
> >> > > > > > > > > > >> > > > looks
> >> > > > > > > > > > >> > > > > to me that's going to be next step.
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > > > Had a crazy day, didn't have time to go
> >> through
> >> > > all
> >> > > > > logs
> >> > > > > > > > > again,
> >> > > > > > > > > > >> plus
> >> > > > > > > > > > >> > > one
> >> > > > > > > > > > >> > > > > of the machines (last one where we had this
> >> > issue)
> >> > > > is
> >> > > > > > > fully
> >> > > > > > > > > > >> > > reprovisioned
> >> > > > > > > > > > >> > > > > yesterday so I don't have logs from there
> >> > anymore.
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > > > Beside upgrading,  what I will talk about
> >> today,
> >> > > can
> >> > > > > you
> >> > > > > > > > just
> >> > > > > > > > > > >> point
> >> > > > > > > > > > >> > me
> >> > > > > > > > > > >> > > to
> >> > > > > > > > > > >> > > > > the specific RPC issue in 0.98.0? Thing is
> >> that
> >> > we
> >> > > > > have
> >> > > > > > > some
> >> > > > > > > > > > >> strange
> >> > > > > > > > > > >> > > > > moments with RPC in this case, and just
> want
> >> to
> >> > > see
> >> > > > if
> >> > > > > > > > that's
> >> > > > > > > > > > the
> >> > > > > > > > > > >> > same
> >> > > > > > > > > > >> > > > > thing (and we were even suspecting to RPC).
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > > > Thanks a lot!
> >> > > > > > > > > > >> > > > > Dejan
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew
> >> Purtell
> >> > <
> >> > > > > > > > > > >> apurtell@apache.org
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> > > > > wrote:
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > > >> Is there a particular reason why you are
> >> using
> >> > > > HBase
> >> > > > > > > > 0.98.0?
> >> > > > > > > > > > The
> >> > > > > > > > > > >> > > latest
> >> > > > > > > > > > >> > > > >> 0.98 release is 0.98.11. There's a known
> >> > > > performance
> >> > > > > > > issue
> >> > > > > > > > > with
> >> > > > > > > > > > >> > 0.98.0
> >> > > > > > > > > > >> > > > >> pertaining to RPC that was fixed in later
> >> > > releases,
> >> > > > > you
> >> > > > > > > > > should
> >> > > > > > > > > > >> move
> >> > > > > > > > > > >> > up
> >> > > > > > > > > > >> > > > >> from
> >> > > > > > > > > > >> > > > >> 0.98.0. In addition hundreds of
> improvements
> >> > and
> >> > > > bug
> >> > > > > > > fixes
> >> > > > > > > > > have
> >> > > > > > > > > > >> gone
> >> > > > > > > > > > >> > > > into
> >> > > > > > > > > > >> > > > >> the ten releases since 0.98.0.
> >> > > > > > > > > > >> > > > >>
> >> > > > > > > > > > >> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan
> >> Menges <
> >> > > > > > > > > > >> > dejan.menges@gmail.com
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > > > >> wrote:
> >> > > > > > > > > > >> > > > >>
> >> > > > > > > > > > >> > > > >> > Hi All,
> >> > > > > > > > > > >> > > > >> >
> >> > > > > > > > > > >> > > > >> > We have a strange issue with HBase
> >> > performance
> >> > > > > > (overall
> >> > > > > > > > > > cluster
> >> > > > > > > > > > >> > > > >> > performance) in case one of datanodes in
> >> the
> >> > > > > cluster
> >> > > > > > > > > > >> unexpectedly
> >> > > > > > > > > > >> > > goes
> >> > > > > > > > > > >> > > > >> > down.
> >> > > > > > > > > > >> > > > >> >
> >> > > > > > > > > > >> > > > >> > So scenario is like follows:
> >> > > > > > > > > > >> > > > >> > - Cluster works fine, it's stable.
> >> > > > > > > > > > >> > > > >> > - One DataNode unexpectedly goes down
> (PSU
> >> > > issue,
> >> > > > > > > network
> >> > > > > > > > > > >> issue,
> >> > > > > > > > > > >> > > > >> anything)
> >> > > > > > > > > > >> > > > >> > - Whole HBase cluster goes down
> >> (performance
> >> > > > > becomes
> >> > > > > > so
> >> > > > > > > > bad
> >> > > > > > > > > > >> that
> >> > > > > > > > > > >> > we
> >> > > > > > > > > > >> > > > >> have to
> >> > > > > > > > > > >> > > > >> > restart all RegionServers to get it back
> >> to
> >> > > > life).
> >> > > > > > > > > > >> > > > >> >
> >> > > > > > > > > > >> > > > >> > Most funny and latest issue that
> happened
> >> was
> >> > > > that
> >> > > > > we
> >> > > > > > > > added
> >> > > > > > > > > > new
> >> > > > > > > > > > >> > node
> >> > > > > > > > > > >> > > > to
> >> > > > > > > > > > >> > > > >> the
> >> > > > > > > > > > >> > > > >> > cluster (having 8 x 4T SATA disks) and
> we
> >> > left
> >> > > > just
> >> > > > > > > > > DataNode
> >> > > > > > > > > > >> > running
> >> > > > > > > > > > >> > > > on
> >> > > > > > > > > > >> > > > >> it
> >> > > > > > > > > > >> > > > >> > to give it couple of days to get some
> >> data.
> >> > At
> >> > > > some
> >> > > > > > > point
> >> > > > > > > > > in
> >> > > > > > > > > > >> time,
> >> > > > > > > > > > >> > > due
> >> > > > > > > > > > >> > > > >> to
> >> > > > > > > > > > >> > > > >> > hardware issue, server rebooted (twice
> >> during
> >> > > > three
> >> > > > > > > > hours)
> >> > > > > > > > > in
> >> > > > > > > > > > >> > moment
> >> > > > > > > > > > >> > > > >> when
> >> > > > > > > > > > >> > > > >> > it had maybe 5% of data it would have
> in a
> >> > > couple
> >> > > > > of
> >> > > > > > > > days.
> >> > > > > > > > > > >> Nothing
> >> > > > > > > > > > >> > > > else
> >> > > > > > > > > > >> > > > >> > beside DataNode was running, and once it
> >> went
> >> > > > down,
> >> > > > > > it
> >> > > > > > > > > > affected
> >> > > > > > > > > > >> > > > literary
> >> > > > > > > > > > >> > > > >> > everything, and restarting RegionServers
> >> in
> >> > the
> >> > > > end
> >> > > > > > > fixed
> >> > > > > > > > > it.
> >> > > > > > > > > > >> > > > >> >
> >> > > > > > > > > > >> > > > >> > We are using HBase 0.98.0 with Hadoop
> >> 2.4.0
> >> > > > > > > > > > >> > > > >> >
> >> > > > > > > > > > >> > > > >>
> >> > > > > > > > > > >> > > > >>
> >> > > > > > > > > > >> > > > >>
> >> > > > > > > > > > >> > > > >> --
> >> > > > > > > > > > >> > > > >> Best regards,
> >> > > > > > > > > > >> > > > >>
> >> > > > > > > > > > >> > > > >>    - Andy
> >> > > > > > > > > > >> > > > >>
> >> > > > > > > > > > >> > > > >> Problems worthy of attack prove their
> worth
> >> by
> >> > > > > hitting
> >> > > > > > > > back.
> >> > > > > > > > > -
> >> > > > > > > > > > >> Piet
> >> > > > > > > > > > >> > > Hein
> >> > > > > > > > > > >> > > > >> (via Tom White)
> >> > > > > > > > > > >> > > > >>
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > > >
> >> > > > > > > > > > >> > > >
> >> > > > > > > > > > >> > >
> >> > > > > > > > > > >> >
> >> > > > > > > > > > >>
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
> >
>

Re: Strange issue when DataNode goes down

Posted by Nicolas Liochon <nk...@gmail.com>.
Actually, double checking the final patch in HDFS-4721, the stale mode is
taken in account. Bryan is right, it's worth checking the namenodes config.
Especially, dfs.namenode.avoid.write.stale.datanode must be set to true on
the namenode.

On Mon, Mar 23, 2015 at 5:08 PM, Nicolas Liochon <nk...@gmail.com> wrote:

> stale should not help for recoverLease: it helps for reads, but it's the
> step after lease recovery.
> It's not needed in recoverLease because the recoverLease in hdfs just
> sorts the datanode by the heartbeat time, so, usually the stale datanode
> will be the last one of the list.
>
>
>
>
> On Mon, Mar 23, 2015 at 4:38 PM, Bryan Beaudreault <
> bbeaudreault@hubspot.com> wrote:
>
>> @Nicholas, I see, thanks.  I'll keep the settings at default.  So really
>> if
>> everything else is configured properly you should never reach the lease
>> recovery timeout in any failure scenarios.  It seems that the staleness
>> check would be the thing that prevents this, correct?  I'm surprised it
>> didn't help Dejan.
>>
>> On Mon, Mar 23, 2015 at 11:20 AM, Nicolas Liochon <nk...@gmail.com>
>> wrote:
>>
>> > @bryan: yes, you can change hbase.lease.recovery.timeout if you changed
>> he
>> > hdfs settings. But this setting is really for desperate cases. The
>> recover
>> > Lease should have succeeded before. As well, if you depend on
>> > hbase.lease.recovery.timeout, it means that you're wasting recovery
>> time:
>> > the lease should be recovered in a few seconds.
>> >
>> > On Mon, Mar 23, 2015 at 3:59 PM, Dejan Menges <de...@gmail.com>
>> > wrote:
>> >
>> > > Interesting discussion I also found, gives me some more light on what
>> > > Nicolas is talking about -
>> > https://issues.apache.org/jira/browse/HDFS-3703
>> > >
>> > > On Mon, Mar 23, 2015 at 3:53 PM Bryan Beaudreault <
>> > > bbeaudreault@hubspot.com>
>> > > wrote:
>> > >
>> > > > So it is safe to set hbase.lease.recovery.timeout lower if you also
>> > > > set heartbeat.recheck.interval lower (lowering that 10.5 min dead
>> node
>> > > > timer)?  Or is it recommended to not touch either of those?
>> > > >
>> > > > Reading the above with interest, thanks for digging in here guys.
>> > > >
>> > > > On Mon, Mar 23, 2015 at 10:13 AM, Nicolas Liochon <
>> nkeywal@gmail.com>
>> > > > wrote:
>> > > >
>> > > > > If the node is actually down it's fine. But the node may not be
>> that
>> > > down
>> > > > > (CAP theorem here); and then it's looking for trouble.
>> > > > > HDFS, by default declare a node as dead after 10:30. 15 minutes
>> is an
>> > > > extra
>> > > > > security. It seems your hdfs settings are different (or there is a
>> > > > bug...).
>> > > > > There should be some info in the hdfs logs.
>> > > > >
>> > > > > On Mon, Mar 23, 2015 at 3:05 PM, Dejan Menges <
>> > dejan.menges@gmail.com>
>> > > > > wrote:
>> > > > >
>> > > > > > Will take a look.
>> > > > > >
>> > > > > > Actually, if node is down (someone unplugged network cable, it
>> just
>> > > > died,
>> > > > > > whatever) what's chance it's going to become live so write can
>> > > > continue?
>> > > > > On
>> > > > > > the other side, HBase is not starting recovery trying to contact
>> > node
>> > > > > which
>> > > > > > is not there anymore, and even elected as dead on Namenode side
>> > > > (another
>> > > > > > thing I didn't understood quite good).
>> > > > > >
>> > > > > > So what I was expecting is that as soon as Namenode decided
>> node is
>> > > > dead,
>> > > > > > that it would be enough for RegionServer to stop trying to
>> recover
>> > > from
>> > > > > the
>> > > > > > dead node, but it wasn't the case. Also, this whole MTTR
>> article in
>> > > > HBase
>> > > > > > book doesn't work at all with this parameter set to it's default
>> > > value
>> > > > > (15
>> > > > > > minutes).
>> > > > > >
>> > > > > > So I'm still struggling to figure out what's drawback exactly on
>> > > this?
>> > > > > >
>> > > > > > On Mon, Mar 23, 2015 at 2:50 PM Nicolas Liochon <
>> nkeywal@gmail.com
>> > >
>> > > > > wrote:
>> > > > > >
>> > > > > > > Thanks for the explanation. There is an issue if you modify
>> this
>> > > > > setting
>> > > > > > > however.
>> > > > > > > hbase tries to recover the lease (i.e. be sure that nobody is
>> > > > writing)
>> > > > > > > If you change hbase.lease.recovery.timeout hbase will start
>> the
>> > > > > recovery
>> > > > > > > (i.e. start to read) even if it's not sure that nobody's
>> writing.
>> > > > That
>> > > > > > > means there is a dataloss risk.
>> > > > > > > Basically, you must not see this warning: WARN
>> > > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils: Cannot recoverLease
>> > after
>> > > > > > trying
>> > > > > > > for[]
>> > > > > > >
>> > > > > > > The recoverLease must succeed. The fact that it does not
>> after X
>> > > > tries
>> > > > > is
>> > > > > > > strange.
>> > > > > > > There may be a mistmatch between the hbase parameters and the
>> > hdfs
>> > > > > ones.
>> > > > > > > You may need to have a look at the comments in
>> FSHDFSUtils.java
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > > On Mon, Mar 23, 2015 at 2:15 PM, Dejan Menges <
>> > > > dejan.menges@gmail.com>
>> > > > > > > wrote:
>> > > > > > >
>> > > > > > > > I found the issue and fixed it, and will try to explain here
>> > what
>> > > > was
>> > > > > > > > exactly in our case in case someone else finds this
>> interesting
>> > > > too.
>> > > > > > > >
>> > > > > > > > So initially, we had (couple of times) some network and
>> > hardware
>> > > > > issues
>> > > > > > > in
>> > > > > > > > our datacenters. When one server would die (literary die, we
>> > had
>> > > > some
>> > > > > > > issue
>> > > > > > > > with PSUs) we saw issues with overall cluster performance on
>> > > HBase
>> > > > > > side.
>> > > > > > > As
>> > > > > > > > cluster is quite big and live, it was also quite hard to
>> figure
>> > > out
>> > > > > > exact
>> > > > > > > > root cause and how to fix the stuff we wanted to fix.
>> > > > > > > >
>> > > > > > > > So I set up another cluster, four nodes (with DataNode and
>> > > > > > RegionServer)
>> > > > > > > > and two other nodes with HMaster and Namenode in HA, using
>> same
>> > > > stuff
>> > > > > > we
>> > > > > > > > use on production. We pumped some data into it, and I was
>> able
>> > to
>> > > > > > > reproduce
>> > > > > > > > same issue last week on it. What I tried to do is to cut one
>> > > server
>> > > > > > (shut
>> > > > > > > > down it's interface) when all is good with cluster, when we
>> > have
>> > > > > load,
>> > > > > > > and
>> > > > > > > > see what's going to happen.
>> > > > > > > >
>> > > > > > > > On Friday, after Nicolas mentioned, I started taking a look
>> in
>> > > > HBase
>> > > > > > logs
>> > > > > > > > on the node which was mentioned in HMaster log as the one
>> > taking
>> > > > over
>> > > > > > > > regions for the dead server. Basically what I was able to
>> > observe
>> > > > was
>> > > > > > 15
>> > > > > > > > minutes time (+- couple of seconds, what was also
>> interesting,
>> > > and
>> > > > > will
>> > > > > > > got
>> > > > > > > > later to that) between HMaster figures out that one of it's
>> > > > > > RegionServers
>> > > > > > > > is dead, and the time one of the mentioned nodes starts
>> taking
>> > > over
>> > > > > > those
>> > > > > > > > regions and they start appearing in HMaster's Web UI.
>> > > > > > > >
>> > > > > > > > I then set up everything like mentioned here
>> > > > > > > > http://hbase.apache.org/book.html#mttr - but still had
>> exactly
>> > > the
>> > > > > > same
>> > > > > > > > issues. Went over (again and again) all currently configured
>> > > stuff,
>> > > > > but
>> > > > > > > > still had the same issue.
>> > > > > > > >
>> > > > > > > > Then I started looking into HDFS. Opened NameNode UI, saw
>> all
>> > is
>> > > > > good,
>> > > > > > > took
>> > > > > > > > one node down, was also looking RegionServer logs in the
>> same
>> > > time
>> > > > -
>> > > > > > and
>> > > > > > > I
>> > > > > > > > also see that it took ~15 minutes for Namenode to elect dead
>> > node
>> > > > as
>> > > > > > > dead.
>> > > > > > > > Somehow in the same moment regions started getting back to
>> > life.
>> > > I
>> > > > > > > remember
>> > > > > > > > in some older versions dfs timeout checks and check retries.
>> > > Looked
>> > > > > > into
>> > > > > > > > defaults for our Hadoop version -
>> > > > > > > >
>> > > > > > > > http://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/
>> > > > > > > hadoop-hdfs/hdfs-default.xml
>> > > > > > > > - and saw there that there's no recheck parameter anymore.
>> > > Strange,
>> > > > > as
>> > > > > > on
>> > > > > > > > StackOverflow I found post from month ago, for newer version
>> > than
>> > > > we
>> > > > > > use
>> > > > > > > > (we use 2.4.1, guy was using 2.6 - dfs.namenode.heartbeat.
>> > > > > > > recheck-interval)
>> > > > > > > > I set it to 10 seconds as he mentioned, having checks every
>> > three
>> > > > > > seconds
>> > > > > > > > (default) and got DataNode elected as dead in ~45 seconds,
>> as
>> > he
>> > > > > > > mentioned.
>> > > > > > > > Not sure why this parameter is not documented, but
>> obviously it
>> > > > > works.
>> > > > > > > >
>> > > > > > > > Then figured out it still didn't fix our HBase failover
>> issue.
>> > I
>> > > > was
>> > > > > > > > looking into HBase book again and again, and then saw this
>> > part:
>> > > > > > > >
>> > > > > > > > "How much time we allow elapse between calls to recover
>> lease.
>> > > > Should
>> > > > > > be
>> > > > > > > > larger than the dfs timeout."
>> > > > > > > >
>> > > > > > > > This was description for hbase.lease.recovery.dfs.timeout.
>> > Wasn't
>> > > > > sure
>> > > > > > > from
>> > > > > > > > the comment what of all timeouts that's possible to set in
>> > > > > Hadoop/HBase
>> > > > > > > and
>> > > > > > > > that has something to do with DFS is this all about. But
>> picked
>> > > > > > > > hbase.lease.recovery.timeout, and that was the catch.
>> > > > > > > >
>> > > > > > > > Initially, by default, hbase.lease.recovery.timeout is set
>> to
>> > 15
>> > > > > > minutes.
>> > > > > > > > Not sure why, and wasn't able to find yet why, but getting
>> this
>> > > > down
>> > > > > to
>> > > > > > > one
>> > > > > > > > minute (what's more than OK for us) I was able to get rid of
>> > our
>> > > > > issue.
>> > > > > > > Not
>> > > > > > > > also sure why this is not mentioned in MTTR section in HBase
>> > > book,
>> > > > as
>> > > > > > > > obviously MTTR doesn't work at all with this default
>> timeout,
>> > at
>> > > > > least
>> > > > > > it
>> > > > > > > > doesn't work the way we expected it to work.
>> > > > > > > >
>> > > > > > > > So thanks again for everyone being spammed with this, and
>> > > specially
>> > > > > > > thanks
>> > > > > > > > to Nicolas pointing me to the right direction.
>> > > > > > > >
>> > > > > > > > On Mon, Mar 23, 2015 at 1:37 PM Nicolas Liochon <
>> > > nkeywal@gmail.com
>> > > > >
>> > > > > > > wrote:
>> > > > > > > >
>> > > > > > > > > the attachments are rejected by the mailing list, can you
>> put
>> > > > then
>> > > > > on
>> > > > > > > > > pastebin?
>> > > > > > > > >
>> > > > > > > > > stale is mandatory (so it's good), but the issue here is
>> just
>> > > > > before.
>> > > > > > > The
>> > > > > > > > > region server needs to read the file. In order to be sure
>> > that
>> > > > > there
>> > > > > > is
>> > > > > > > > no
>> > > > > > > > > data loss, it needs to "recover the lease", that means
>> > ensuring
>> > > > > that
>> > > > > > > > nobody
>> > > > > > > > > is writing the file. The regionserver calls the namenode
>> to
>> > do
>> > > > this
>> > > > > > > > > recoverLease. So there should be some info in the namenode
>> > > logs.
>> > > > > You
>> > > > > > > have
>> > > > > > > > > HDFS-4721 on your hdfs? The hbase book details (more or
>> > > less...)
>> > > > > this
>> > > > > > > > > recoverLease stuff.
>> > > > > > > > >
>> > > > > > > > >
>> > > > > > > > > On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <
>> > > > > > dejan.menges@gmail.com
>> > > > > > > >
>> > > > > > > > > wrote:
>> > > > > > > > >
>> > > > > > > > > > And also, just checked - dfs.namenode.avoid.read.stale.
>> > > > datanode
>> > > > > and
>> > > > > > > > > > dfs.namenode.avoid.write.stale.datanode
>> > > > > > > > > > are both true, and dfs.namenode.stale.datanode.interval
>> is
>> > > > set to
>> > > > > > > > > default
>> > > > > > > > > > 30000.
>> > > > > > > > > >
>> > > > > > > > > > On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <
>> > > > > > > dejan.menges@gmail.com>
>> > > > > > > > > > wrote:
>> > > > > > > > > >
>> > > > > > > > > > > Hi Nicolas,
>> > > > > > > > > > >
>> > > > > > > > > > > Please find log attached.
>> > > > > > > > > > >
>> > > > > > > > > > > As I see it now more clearly, it was trying to recover
>> > HDFS
>> > > > > WALs
>> > > > > > > from
>> > > > > > > > > > node
>> > > > > > > > > > > that's dead:
>> > > > > > > > > > >
>> > > > > > > > > > > 2015-03-23 08:53:44,381 WARN
>> > > > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils:
>> > > > > > > > > > > Cannot recoverLease after trying for 900000ms
>> > > > > > > > > > > (hbase.lease.recovery.timeout); continuing, but may be
>> > > > > > > DATALOSS!!!;
>> > > > > > > > > > > attempt=40 on
>> > > > > > > > > > >
>> > > > > > > > > >
>> > file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intent
>> > > > > > > > > ionally_get_down_by_getting_network_down},60020,1426862900
>> > > > > > > > > 506-splitting/{node_i_intentionally_get_down_by_gett
>> > > > > > > > > ing_network_down}%2C60020%2C1426862900506.1427096924508
>> > > > > > > > > > > after 908210ms
>> > > > > > > > > > >
>> > > > > > > > > > > And as you can see from the log, it tried 40 times,
>> what
>> > > took
>> > > > > it
>> > > > > > > > > exactly
>> > > > > > > > > > > 15 minutes.
>> > > > > > > > > > >
>> > > > > > > > > > > There's probably some parameter to tune to cut it of
>> from
>> > > 40
>> > > > > > times
>> > > > > > > /
>> > > > > > > > 15
>> > > > > > > > > > > minutes to something more useful, as for 15 minutes we
>> > > don't
>> > > > > have
>> > > > > > > our
>> > > > > > > > > > > regions available, and HDFS have however replication
>> > factor
>> > > > 3.
>> > > > > > > > > > >
>> > > > > > > > > > > Googling, if I figure out what's this I will post it
>> > here.
>> > > > Will
>> > > > > > > also
>> > > > > > > > > > > appreciate if someone knows how to cut this down.
>> > > > > > > > > > >
>> > > > > > > > > > > Thanks,
>> > > > > > > > > > >
>> > > > > > > > > > > Dejan
>> > > > > > > > > > >
>> > > > > > > > > > > On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <
>> > > > > > nkeywal@gmail.com
>> > > > > > > >
>> > > > > > > > > > wrote:
>> > > > > > > > > > >
>> > > > > > > > > > >> The split is done by the region servers (the master
>> > > > > > coordinates).
>> > > > > > > Is
>> > > > > > > > > > there
>> > > > > > > > > > >> some interesting stuff in their logs?
>> > > > > > > > > > >>
>> > > > > > > > > > >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <
>> > > > > > > > dejan.menges@gmail.com
>> > > > > > > > > >
>> > > > > > > > > > >> wrote:
>> > > > > > > > > > >>
>> > > > > > > > > > >> > With client issue was that it was retrying
>> connecting
>> > to
>> > > > the
>> > > > > > > same
>> > > > > > > > > > region
>> > > > > > > > > > >> > servers even when they were reassigned. Lowering it
>> > down
>> > > > > > helped
>> > > > > > > in
>> > > > > > > > > > this
>> > > > > > > > > > >> > specific use case, but yes, we still want servers
>> to
>> > > > > > reallocate
>> > > > > > > > > > quickly.
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > What got me here:
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > http://hbase.apache.org/book.html#mttr
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > I basically set configuration exactly the same way
>> as
>> > > it's
>> > > > > > > > explained
>> > > > > > > > > > >> here.
>> > > > > > > > > > >> > *zookeeper.session.timeout* is (and was before)
>> 60000
>> > > (one
>> > > > > > > > minute).
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > So basically what happens is: - simulating network
>> > > issues
>> > > > we
>> > > > > > had
>> > > > > > > > > > >> recently).
>> > > > > > > > > > >> > - After short time I see in HBase that my
>> RegionServer
>> > > is
>> > > > > > dead,
>> > > > > > > > and
>> > > > > > > > > as
>> > > > > > > > > > >> > total number of regions I see previous total minus
>> > > number
>> > > > of
>> > > > > > > > regions
>> > > > > > > > > > >> that
>> > > > > > > > > > >> > were hosted on the node hosting RegionServer that
>> just
>> > > > > > > > > 'disappeared'.
>> > > > > > > > > > >> > - In this point I want my clus
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > - I have test cluster consisting of four nodes,
>> every
>> > > node
>> > > > > > being
>> > > > > > > > > > >> DataNode
>> > > > > > > > > > >> > and RegionServer.
>> > > > > > > > > > >> > - I simulate network partition on one (connect to
>> it
>> > > > through
>> > > > > > > > console
>> > > > > > > > > > and
>> > > > > > > > > > >> > take network interface downter to recover as soon
>> as
>> > > > > possible,
>> > > > > > > to
>> > > > > > > > > > start
>> > > > > > > > > > >> > serving missing regions.
>> > > > > > > > > > >> > - First thing I see in HMaster logs are:
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > 2015-03-20 14:17:26,015 INFO
>> > > > > > > > > > >> >
>> org.apache.hadoop.hbase.zookeeper.RegionServerTracker:
>> > > > > > > > RegionServer
>> > > > > > > > > > >> > ephemeral node deleted, processing expiration
>> > > > > > > > > > >> > [{name_of_node_I_took_down},60020,1426860403261]
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > 2015-03-20 14:17:26,067 INFO
>> > > > > > > > > > >> >
>> > > > > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler:
>> > > > > > > > > > Splitting
>> > > > > > > > > > >> > logs for
>> > {name_of_node_I_took_down},60020,1426860403261
>> > > > > before
>> > > > > > > > > > >> assignment.
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > 2015-03-20 14:17:26,105 INFO
>> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> dead
>> > > > > splitlog
>> > > > > > > > > workers
>> > > > > > > > > > [
>> > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261]
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > 2015-03-20 14:17:26,107 INFO
>> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> > started
>> > > > > > > splitting
>> > > > > > > > 1
>> > > > > > > > > > >> logs in
>> > > > > > > > > > >> >
>> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_
>> > > > I_
>> > > > > > > > > took_down}
>> > > > > > > > > > >> > ,60020,1426860403261-splitting]
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > 2015-03-20 14:17:26,150 INFO
>> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> task
>> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F
>> > > > > > > > > > >> >
>> > > > > > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
>> > > > > > > > > > >> >
>> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
>> > > > 14268
>> > > > > > > > > 60404905
>> > > > > > > > > > >> > acquired by {fqdn_of_another_live_hnode},
>> > > > 60020,1426859445623
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > 2015-03-20 14:17:26,182 INFO
>> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> total
>> > > > tasks
>> > > > > =
>> > > > > > 1
>> > > > > > > > > > >> unassigned
>> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
>> > > > 2F{name_of_node_I_took_down}
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
>> > > > took_
>> > > > > > > > > > >>
>> > > down}%252C60020%252C1426860403261.1426860404905=last_update
>> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
>> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
>> > status =
>> > > > > > > > > in_progress
>> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
>> > done
>> > > > = 0
>> > > > > > > > error =
>> > > > > > > > > > 0}
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > 2015-03-20 14:17:31,183 INFO
>> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> total
>> > > > tasks
>> > > > > =
>> > > > > > 1
>> > > > > > > > > > >> unassigned
>> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
>> > > > 2F{name_of_node_I_took_down}
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
>> > > > took_
>> > > > > > > > > > >>
>> > > down}%252C60020%252C1426860403261.1426860404905=last_update
>> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
>> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
>> > status =
>> > > > > > > > > in_progress
>> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
>> > done
>> > > > = 0
>> > > > > > > > error =
>> > > > > > > > > > 0}
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > 2015-03-20 14:17:36,184 INFO
>> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> total
>> > > > tasks
>> > > > > =
>> > > > > > 1
>> > > > > > > > > > >> unassigned
>> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
>> > > > 2F{name_of_node_I_took_down}
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
>> > > > took_
>> > > > > > > > > > >>
>> > > down}%252C60020%252C1426860403261.1426860404905=last_update
>> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
>> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
>> > status =
>> > > > > > > > > in_progress
>> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
>> > done
>> > > > = 0
>> > > > > > > > error =
>> > > > > > > > > > 0}
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > 2015-03-20 14:17:42,185 INFO
>> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> total
>> > > > tasks
>> > > > > =
>> > > > > > 1
>> > > > > > > > > > >> unassigned
>> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
>> > > > 2F{name_of_node_I_took_down}
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
>> > > > took_
>> > > > > > > > > > >>
>> > > down}%252C60020%252C1426860403261.1426860404905=last_update
>> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
>> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
>> > status =
>> > > > > > > > > in_progress
>> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
>> > done
>> > > > = 0
>> > > > > > > > error =
>> > > > > > > > > > 0}
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > 2015-03-20 14:17:48,184 INFO
>> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> total
>> > > > tasks
>> > > > > =
>> > > > > > 1
>> > > > > > > > > > >> unassigned
>> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
>> > > > 2F{name_of_node_I_took_down}
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
>> > > > took_
>> > > > > > > > > > >>
>> > > down}%252C60020%252C1426860403261.1426860404905=last_update
>> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
>> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
>> > status =
>> > > > > > > > > in_progress
>> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
>> > done
>> > > > = 0
>> > > > > > > > error =
>> > > > > > > > > > 0}
>> > > > > > > > > > >> > In the meantime, In hbase...out log I got this:
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > ==>
>> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out
>> > <==
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > java.io.IOException: Call to
>> > > > > > > > > > >> >
>> > > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_
>> > > > dow
>> > > > > > > > > n}:60020
>> > > > > > > > > > >> > failed on local exception:
>> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
>> > > > CallTimeoutException:
>> > > > > > > Call
>> > > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
>> > > > > > > > > > >> t.java:1532)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:
>> > > > > > > 1502)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> > > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(
>> > > > Rpc
>> > > > > > > > > > >> Client.java:1684)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
>> > > > BlockingRpcChannelImpl
>> > > > > > > > > ementati
>> > > > > > > > > > >> on.callBlockingMethod(RpcClient.java:1737)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$
>> > > > > > > > > AdminService$
>> > > > > > > > > > >> BlockingStub.getRegionInfo(AdminProtos.java:20806)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > org.apache.hadoop.hbase.client.HBaseAdmin.
>> > > > getCompactionState
>> > > > > > > > > > >> (HBaseAdmin.java:2524)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> org.apache.hadoop.hbase.generated.master.table_jsp._
>> > > > jspServi
>> > > > > > > > > > >> ce(table_jsp.java:167)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.
>> > > > > > > > > java:98)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
>> > > > > > > > > > >> .java:511)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
>> > > > doFilte
>> > > > > > > > > > >> r(ServletHandler.java:1221)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > org.apache.hadoop.http.lib.StaticUserWebFilter$
>> > > > StaticUserFil
>> > > > > > > > > > >> ter.doFilter(StaticUserWebFilter.java:109)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
>> > > > doFilte
>> > > > > > > > > > >> r(ServletHandler.java:1212)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> org.apache.hadoop.http.HttpServer$QuotingInputFilter.
>> > > > doFilte
>> > > > > > > > > > >> r(HttpServer.java:1081)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
>> > > > doFilte
>> > > > > > > > > > >> r(ServletHandler.java:1212)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > >
>> > > > > >
>> > org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
>> > > > doFilte
>> > > > > > > > > > >> r(ServletHandler.java:1212)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
>> > > > > > > > > > >> er.java:399)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> > org.mortbay.jetty.security.SecurityHandler.handle(
>> > > > SecurityHa
>> > > > > > > > > > >> ndler.java:216)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
>> > > > > > > > > > >> er.java:182)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
>> > > > > > > > > > >> er.java:766)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
>> > > > > > > > > > >> java:450)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > org.mortbay.jetty.handler.ContextHandlerCollection.
>> > > > handle(Co
>> > > > > > > > > > >> ntextHandlerCollection.java:230)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
>> > > > > > > > > > >> er.java:152)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at org.mortbay.jetty.Server.handle(Server.java:326)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
>> > > > > > > > > > >> n.java:542)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > org.mortbay.jetty.HttpConnection$RequestHandler.
>> > > > headerComple
>> > > > > > > > > > >> te(HttpConnection.java:928)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > >
>> > org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > >
>> > org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > org.mortbay.io.nio.SelectChannelEndPoint.run(
>> > > > SelectChannelEn
>> > > > > > > > > > >> dPoint.java:410)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > org.mortbay.thread.QueuedThreadPool$PoolThread.run(
>> > > > > > > > > > >> QueuedThreadPool.java:582)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > Caused by: org.apache.hadoop.hbase.ipc.RpcClient$
>> > > > > > > > > CallTimeoutException:
>> > > > > > > > > > >> Call
>> > > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.
>> > > > cleanupCall
>> > > > > > > > > > >> s(RpcClient.java:1234)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.
>> > > > readRespons
>> > > > > > > > > > >> e(RpcClient.java:1171)
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > at
>> > > > > > org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
>> > > > > > > > > > >> nt.java:751)
>> > > > > > > > > > >> > Beside this same issue, please note that first
>> message
>> > > was
>> > > > > at
>> > > > > > > > > > 2015-03-20
>> > > > > > > > > > >> > 14:17:26,015. And then (we got to the point when it
>> > > > started
>> > > > > > > > > > transition):
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > 2015-03-20 14:32:35,059 INFO
>> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> task
>> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F
>> > > > > > > > > > >> >
>> > > > > > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
>> > > > > > > > > > >> >
>> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
>> > > > 14268
>> > > > > > > > > 60404905
>> > > > > > > > > > >> > entered state: DONE
>> > > > > > {fqdn_of_new_live_node},60020,1426859445623
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > 2015-03-20 14:32:35,109 INFO
>> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> Done
>> > > > > splitting
>> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>> > > > > > > > > > >> >
>> > > > > > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}
>> > > > > > > > > > >> > %252C60020%252C1426860403261.1426860404905
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > 2015-03-20 14:32:35,190 INFO
>> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
>> > finished
>> > > > > > > splitting
>> > > > > > > > > > >> (more
>> > > > > > > > > > >> > than or equal to) 9 bytes in 1 log files in
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_
>> > > > of_
>> > > > > > > > > > >> node_I_took_down},60020,1426860403261-splitting]
>> > > > > > > > > > >> > in 909083ms
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
>> > > > org.apache.hadoop.hbase.master
>> > > > > > > > > > >> .RegionStates:
>> > > > > > > > > > >> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c
>> > > > state=OPEN,
>> > > > > > > > > > >> > ts=1426860639088,
>> > > > > > > > > > server={name_of_node_I_took_down},60020,1426860403261}
>> > > > > > > > > > >> to
>> > > > > > > > > > >> > {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE,
>> > > > > > > ts=1426861955191,
>> > > > > > > > > > >> server=
>> > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
>> > > > org.apache.hadoop.hbase.master
>> > > > > > > > > > >> .RegionStates:
>> > > > > > > > > > >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
>> > > > > > > > > > >> {name_of_node_I_took_down}
>> > > > > > > > > > >> > ,60020,1426860403261
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
>> > > > org.apache.hadoop.hbase.master
>> > > > > > > > > > >> .RegionStates:
>> > > > > > > > > > >> > Transitioned {25ab6e7b42e36ddaa723d71be5954543
>> > > > state=OPEN,
>> > > > > > > > > > >> > ts=1426860641783,
>> > > > > > > > > > server={name_of_node_I_took_down},60020,1426860403261}
>> > > > > > > > > > >> to
>> > > > > > > > > > >> > {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE,
>> > > > > > > ts=1426861955191,
>> > > > > > > > > > >> server=
>> > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
>> > > > org.apache.hadoop.hbase.master
>> > > > > > > > > > >> .RegionStates:
>> > > > > > > > > > >> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
>> > > > > > > > > > >> {name_of_node_I_took_down}
>> > > > > > > > > > >> > ,60020,1426860403261
>> > > > > > > > > > >> > At this point, note that it finished
>> SplitLogManager
>> > > task
>> > > > at
>> > > > > > > > > 14:32:35
>> > > > > > > > > > >> and
>> > > > > > > > > > >> > started transitioning just after that. So this is
>> 15
>> > > > minutes
>> > > > > > > that
>> > > > > > > > > I'm
>> > > > > > > > > > >> > talking about.
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > What am I missing?
>> > > > > > > > > > >> >
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <
>> > > > > > > > nkeywal@gmail.com>
>> > > > > > > > > > >> wrote:
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > > You've changed the value of
>> hbase.zookeeper.timeout
>> > to
>> > > > 15
>> > > > > > > > > minutes? A
>> > > > > > > > > > >> very
>> > > > > > > > > > >> > > reasonable target is 1 minute before relocating
>> the
>> > > > > regions.
>> > > > > > > > > That's
>> > > > > > > > > > >> the
>> > > > > > > > > > >> > > default iirc. You can push it to 20s, but then
>> > > > > > > > > gc-stopping-the-world
>> > > > > > > > > > >> > > becomes more of an issue. 15 minutes is really a
>> > lot.
>> > > > The
>> > > > > > hdfs
>> > > > > > > > > stale
>> > > > > > > > > > >> mode
>> > > > > > > > > > >> > > must always be used, with a lower timeout than
>> the
>> > > hbase
>> > > > > > one.
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> > > Client side there should be nothing to do
>> (excepted
>> > > for
>> > > > > > > advanced
>> > > > > > > > > > >> stuff);
>> > > > > > > > > > >> > at
>> > > > > > > > > > >> > > each retry the client checks the location of the
>> > > > regions.
>> > > > > If
>> > > > > > > you
>> > > > > > > > > > lower
>> > > > > > > > > > >> > the
>> > > > > > > > > > >> > > number of retry the client will fail sooner, but
>> > > usually
>> > > > > you
>> > > > > > > > don't
>> > > > > > > > > > >> want
>> > > > > > > > > > >> > the
>> > > > > > > > > > >> > > client to fail, you want the servers to
>> reallocate
>> > > > > quickly.
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <
>> > > > > > > > > > dejan.menges@gmail.com
>> > > > > > > > > > >> >
>> > > > > > > > > > >> > > wrote:
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> > > > Hi,
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > Sorry for little bit late update, but managed
>> to
>> > > > narrow
>> > > > > it
>> > > > > > > > > little
>> > > > > > > > > > >> bit
>> > > > > > > > > > >> > > down.
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > We didn't update yet, as we are using
>> Hortonworks
>> > > > > > > distribution
>> > > > > > > > > > right
>> > > > > > > > > > >> > now,
>> > > > > > > > > > >> > > > and even if we update we will get 0.98.4.
>> However,
>> > > > looks
>> > > > > > > that
>> > > > > > > > > > issue
>> > > > > > > > > > >> > here
>> > > > > > > > > > >> > > > was in our use case and configuration (still
>> > looking
>> > > > > into
>> > > > > > > it).
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > Basically, initially I saw that when one server
>> > goes
>> > > > > down,
>> > > > > > > we
>> > > > > > > > > > start
>> > > > > > > > > > >> > > having
>> > > > > > > > > > >> > > > performance issues in general, but it managed
>> to
>> > be
>> > > on
>> > > > > our
>> > > > > > > > > client
>> > > > > > > > > > >> side,
>> > > > > > > > > > >> > > due
>> > > > > > > > > > >> > > > to caching, and clients were trying to
>> reconnect
>> > to
>> > > > > nodes
>> > > > > > > that
>> > > > > > > > > > were
>> > > > > > > > > > >> > > offline
>> > > > > > > > > > >> > > > and later trying to get regions from those
>> nodes
>> > > too.
>> > > > > This
>> > > > > > > is
>> > > > > > > > > > >> basically
>> > > > > > > > > > >> > > why
>> > > > > > > > > > >> > > > on server side I didn't manage to see anything
>> in
>> > > logs
>> > > > > > that
>> > > > > > > > > would
>> > > > > > > > > > >> be at
>> > > > > > > > > > >> > > > least little bit interesting and point me into
>> > > desired
>> > > > > > > > > direction.
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > Another question that popped up to me is - in
>> case
>> > > > > server
>> > > > > > is
>> > > > > > > > > down
>> > > > > > > > > > >> (and
>> > > > > > > > > > >> > > with
>> > > > > > > > > > >> > > > it DataNode and HRegionServer it was hosting) -
>> > > what's
>> > > > > > > optimal
>> > > > > > > > > > time
>> > > > > > > > > > >> to
>> > > > > > > > > > >> > > set
>> > > > > > > > > > >> > > > for HMaster to consider server dead reassign
>> > regions
>> > > > > > > somewhere
>> > > > > > > > > > >> else, as
>> > > > > > > > > > >> > > > this is another performance bottleneck we hit
>> > during
>> > > > > > > inability
>> > > > > > > > > to
>> > > > > > > > > > >> > access
>> > > > > > > > > > >> > > > regions? In our case it's configured to 15
>> > minutes,
>> > > > and
>> > > > > > > simple
>> > > > > > > > > > logic
>> > > > > > > > > > >> > > tells
>> > > > > > > > > > >> > > > me if you want it earlier then configure lower
>> > > number
>> > > > of
>> > > > > > > > > retries,
>> > > > > > > > > > >> but
>> > > > > > > > > > >> > > issue
>> > > > > > > > > > >> > > > is as always in details, so not sure if anyone
>> > knows
>> > > > > some
>> > > > > > > > better
>> > > > > > > > > > >> math
>> > > > > > > > > > >> > for
>> > > > > > > > > > >> > > > this?
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > And last question - is it possible to manually
>> > force
>> > > > > HBase
>> > > > > > > to
>> > > > > > > > > > >> reassign
>> > > > > > > > > > >> > > > regions? In this case, while HMaster is
>> retrying
>> > to
>> > > > > > contact
>> > > > > > > > node
>> > > > > > > > > > >> that's
>> > > > > > > > > > >> > > > dead, it's impossible to force it using
>> 'balancer'
>> > > > > > command.
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > Thanks a lot!
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > Dejan
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <
>> > > > > > > > > > >> dejan.menges@gmail.com>
>> > > > > > > > > > >> > > > wrote:
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > > Hi,
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > > > To be very honest - there's no particular
>> reason
>> > > why
>> > > > > we
>> > > > > > > > stick
>> > > > > > > > > to
>> > > > > > > > > > >> this
>> > > > > > > > > > >> > > > one,
>> > > > > > > > > > >> > > > > beside just lack of time currently to go
>> through
>> > > > > upgrade
>> > > > > > > > > > process,
>> > > > > > > > > > >> but
>> > > > > > > > > > >> > > > looks
>> > > > > > > > > > >> > > > > to me that's going to be next step.
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > > > Had a crazy day, didn't have time to go
>> through
>> > > all
>> > > > > logs
>> > > > > > > > > again,
>> > > > > > > > > > >> plus
>> > > > > > > > > > >> > > one
>> > > > > > > > > > >> > > > > of the machines (last one where we had this
>> > issue)
>> > > > is
>> > > > > > > fully
>> > > > > > > > > > >> > > reprovisioned
>> > > > > > > > > > >> > > > > yesterday so I don't have logs from there
>> > anymore.
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > > > Beside upgrading,  what I will talk about
>> today,
>> > > can
>> > > > > you
>> > > > > > > > just
>> > > > > > > > > > >> point
>> > > > > > > > > > >> > me
>> > > > > > > > > > >> > > to
>> > > > > > > > > > >> > > > > the specific RPC issue in 0.98.0? Thing is
>> that
>> > we
>> > > > > have
>> > > > > > > some
>> > > > > > > > > > >> strange
>> > > > > > > > > > >> > > > > moments with RPC in this case, and just want
>> to
>> > > see
>> > > > if
>> > > > > > > > that's
>> > > > > > > > > > the
>> > > > > > > > > > >> > same
>> > > > > > > > > > >> > > > > thing (and we were even suspecting to RPC).
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > > > Thanks a lot!
>> > > > > > > > > > >> > > > > Dejan
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew
>> Purtell
>> > <
>> > > > > > > > > > >> apurtell@apache.org
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> > > > > wrote:
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > > >> Is there a particular reason why you are
>> using
>> > > > HBase
>> > > > > > > > 0.98.0?
>> > > > > > > > > > The
>> > > > > > > > > > >> > > latest
>> > > > > > > > > > >> > > > >> 0.98 release is 0.98.11. There's a known
>> > > > performance
>> > > > > > > issue
>> > > > > > > > > with
>> > > > > > > > > > >> > 0.98.0
>> > > > > > > > > > >> > > > >> pertaining to RPC that was fixed in later
>> > > releases,
>> > > > > you
>> > > > > > > > > should
>> > > > > > > > > > >> move
>> > > > > > > > > > >> > up
>> > > > > > > > > > >> > > > >> from
>> > > > > > > > > > >> > > > >> 0.98.0. In addition hundreds of improvements
>> > and
>> > > > bug
>> > > > > > > fixes
>> > > > > > > > > have
>> > > > > > > > > > >> gone
>> > > > > > > > > > >> > > > into
>> > > > > > > > > > >> > > > >> the ten releases since 0.98.0.
>> > > > > > > > > > >> > > > >>
>> > > > > > > > > > >> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan
>> Menges <
>> > > > > > > > > > >> > dejan.menges@gmail.com
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > > > >> wrote:
>> > > > > > > > > > >> > > > >>
>> > > > > > > > > > >> > > > >> > Hi All,
>> > > > > > > > > > >> > > > >> >
>> > > > > > > > > > >> > > > >> > We have a strange issue with HBase
>> > performance
>> > > > > > (overall
>> > > > > > > > > > cluster
>> > > > > > > > > > >> > > > >> > performance) in case one of datanodes in
>> the
>> > > > > cluster
>> > > > > > > > > > >> unexpectedly
>> > > > > > > > > > >> > > goes
>> > > > > > > > > > >> > > > >> > down.
>> > > > > > > > > > >> > > > >> >
>> > > > > > > > > > >> > > > >> > So scenario is like follows:
>> > > > > > > > > > >> > > > >> > - Cluster works fine, it's stable.
>> > > > > > > > > > >> > > > >> > - One DataNode unexpectedly goes down (PSU
>> > > issue,
>> > > > > > > network
>> > > > > > > > > > >> issue,
>> > > > > > > > > > >> > > > >> anything)
>> > > > > > > > > > >> > > > >> > - Whole HBase cluster goes down
>> (performance
>> > > > > becomes
>> > > > > > so
>> > > > > > > > bad
>> > > > > > > > > > >> that
>> > > > > > > > > > >> > we
>> > > > > > > > > > >> > > > >> have to
>> > > > > > > > > > >> > > > >> > restart all RegionServers to get it back
>> to
>> > > > life).
>> > > > > > > > > > >> > > > >> >
>> > > > > > > > > > >> > > > >> > Most funny and latest issue that happened
>> was
>> > > > that
>> > > > > we
>> > > > > > > > added
>> > > > > > > > > > new
>> > > > > > > > > > >> > node
>> > > > > > > > > > >> > > > to
>> > > > > > > > > > >> > > > >> the
>> > > > > > > > > > >> > > > >> > cluster (having 8 x 4T SATA disks) and we
>> > left
>> > > > just
>> > > > > > > > > DataNode
>> > > > > > > > > > >> > running
>> > > > > > > > > > >> > > > on
>> > > > > > > > > > >> > > > >> it
>> > > > > > > > > > >> > > > >> > to give it couple of days to get some
>> data.
>> > At
>> > > > some
>> > > > > > > point
>> > > > > > > > > in
>> > > > > > > > > > >> time,
>> > > > > > > > > > >> > > due
>> > > > > > > > > > >> > > > >> to
>> > > > > > > > > > >> > > > >> > hardware issue, server rebooted (twice
>> during
>> > > > three
>> > > > > > > > hours)
>> > > > > > > > > in
>> > > > > > > > > > >> > moment
>> > > > > > > > > > >> > > > >> when
>> > > > > > > > > > >> > > > >> > it had maybe 5% of data it would have in a
>> > > couple
>> > > > > of
>> > > > > > > > days.
>> > > > > > > > > > >> Nothing
>> > > > > > > > > > >> > > > else
>> > > > > > > > > > >> > > > >> > beside DataNode was running, and once it
>> went
>> > > > down,
>> > > > > > it
>> > > > > > > > > > affected
>> > > > > > > > > > >> > > > literary
>> > > > > > > > > > >> > > > >> > everything, and restarting RegionServers
>> in
>> > the
>> > > > end
>> > > > > > > fixed
>> > > > > > > > > it.
>> > > > > > > > > > >> > > > >> >
>> > > > > > > > > > >> > > > >> > We are using HBase 0.98.0 with Hadoop
>> 2.4.0
>> > > > > > > > > > >> > > > >> >
>> > > > > > > > > > >> > > > >>
>> > > > > > > > > > >> > > > >>
>> > > > > > > > > > >> > > > >>
>> > > > > > > > > > >> > > > >> --
>> > > > > > > > > > >> > > > >> Best regards,
>> > > > > > > > > > >> > > > >>
>> > > > > > > > > > >> > > > >>    - Andy
>> > > > > > > > > > >> > > > >>
>> > > > > > > > > > >> > > > >> Problems worthy of attack prove their worth
>> by
>> > > > > hitting
>> > > > > > > > back.
>> > > > > > > > > -
>> > > > > > > > > > >> Piet
>> > > > > > > > > > >> > > Hein
>> > > > > > > > > > >> > > > >> (via Tom White)
>> > > > > > > > > > >> > > > >>
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > > >
>> > > > > > > > > > >> > > >
>> > > > > > > > > > >> > >
>> > > > > > > > > > >> >
>> > > > > > > > > > >>
>> > > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>
>

Re: Strange issue when DataNode goes down

Posted by Nicolas Liochon <nk...@gmail.com>.
stale should not help for recoverLease: it helps for reads, but it's the
step after lease recovery.
It's not needed in recoverLease because the recoverLease in hdfs just sorts
the datanode by the heartbeat time, so, usually the stale datanode will be
the last one of the list.



On Mon, Mar 23, 2015 at 4:38 PM, Bryan Beaudreault <bbeaudreault@hubspot.com
> wrote:

> @Nicholas, I see, thanks.  I'll keep the settings at default.  So really if
> everything else is configured properly you should never reach the lease
> recovery timeout in any failure scenarios.  It seems that the staleness
> check would be the thing that prevents this, correct?  I'm surprised it
> didn't help Dejan.
>
> On Mon, Mar 23, 2015 at 11:20 AM, Nicolas Liochon <nk...@gmail.com>
> wrote:
>
> > @bryan: yes, you can change hbase.lease.recovery.timeout if you changed
> he
> > hdfs settings. But this setting is really for desperate cases. The
> recover
> > Lease should have succeeded before. As well, if you depend on
> > hbase.lease.recovery.timeout, it means that you're wasting recovery time:
> > the lease should be recovered in a few seconds.
> >
> > On Mon, Mar 23, 2015 at 3:59 PM, Dejan Menges <de...@gmail.com>
> > wrote:
> >
> > > Interesting discussion I also found, gives me some more light on what
> > > Nicolas is talking about -
> > https://issues.apache.org/jira/browse/HDFS-3703
> > >
> > > On Mon, Mar 23, 2015 at 3:53 PM Bryan Beaudreault <
> > > bbeaudreault@hubspot.com>
> > > wrote:
> > >
> > > > So it is safe to set hbase.lease.recovery.timeout lower if you also
> > > > set heartbeat.recheck.interval lower (lowering that 10.5 min dead
> node
> > > > timer)?  Or is it recommended to not touch either of those?
> > > >
> > > > Reading the above with interest, thanks for digging in here guys.
> > > >
> > > > On Mon, Mar 23, 2015 at 10:13 AM, Nicolas Liochon <nkeywal@gmail.com
> >
> > > > wrote:
> > > >
> > > > > If the node is actually down it's fine. But the node may not be
> that
> > > down
> > > > > (CAP theorem here); and then it's looking for trouble.
> > > > > HDFS, by default declare a node as dead after 10:30. 15 minutes is
> an
> > > > extra
> > > > > security. It seems your hdfs settings are different (or there is a
> > > > bug...).
> > > > > There should be some info in the hdfs logs.
> > > > >
> > > > > On Mon, Mar 23, 2015 at 3:05 PM, Dejan Menges <
> > dejan.menges@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Will take a look.
> > > > > >
> > > > > > Actually, if node is down (someone unplugged network cable, it
> just
> > > > died,
> > > > > > whatever) what's chance it's going to become live so write can
> > > > continue?
> > > > > On
> > > > > > the other side, HBase is not starting recovery trying to contact
> > node
> > > > > which
> > > > > > is not there anymore, and even elected as dead on Namenode side
> > > > (another
> > > > > > thing I didn't understood quite good).
> > > > > >
> > > > > > So what I was expecting is that as soon as Namenode decided node
> is
> > > > dead,
> > > > > > that it would be enough for RegionServer to stop trying to
> recover
> > > from
> > > > > the
> > > > > > dead node, but it wasn't the case. Also, this whole MTTR article
> in
> > > > HBase
> > > > > > book doesn't work at all with this parameter set to it's default
> > > value
> > > > > (15
> > > > > > minutes).
> > > > > >
> > > > > > So I'm still struggling to figure out what's drawback exactly on
> > > this?
> > > > > >
> > > > > > On Mon, Mar 23, 2015 at 2:50 PM Nicolas Liochon <
> nkeywal@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > Thanks for the explanation. There is an issue if you modify
> this
> > > > > setting
> > > > > > > however.
> > > > > > > hbase tries to recover the lease (i.e. be sure that nobody is
> > > > writing)
> > > > > > > If you change hbase.lease.recovery.timeout hbase will start the
> > > > > recovery
> > > > > > > (i.e. start to read) even if it's not sure that nobody's
> writing.
> > > > That
> > > > > > > means there is a dataloss risk.
> > > > > > > Basically, you must not see this warning: WARN
> > > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils: Cannot recoverLease
> > after
> > > > > > trying
> > > > > > > for[]
> > > > > > >
> > > > > > > The recoverLease must succeed. The fact that it does not after
> X
> > > > tries
> > > > > is
> > > > > > > strange.
> > > > > > > There may be a mistmatch between the hbase parameters and the
> > hdfs
> > > > > ones.
> > > > > > > You may need to have a look at the comments in FSHDFSUtils.java
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Mon, Mar 23, 2015 at 2:15 PM, Dejan Menges <
> > > > dejan.menges@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > I found the issue and fixed it, and will try to explain here
> > what
> > > > was
> > > > > > > > exactly in our case in case someone else finds this
> interesting
> > > > too.
> > > > > > > >
> > > > > > > > So initially, we had (couple of times) some network and
> > hardware
> > > > > issues
> > > > > > > in
> > > > > > > > our datacenters. When one server would die (literary die, we
> > had
> > > > some
> > > > > > > issue
> > > > > > > > with PSUs) we saw issues with overall cluster performance on
> > > HBase
> > > > > > side.
> > > > > > > As
> > > > > > > > cluster is quite big and live, it was also quite hard to
> figure
> > > out
> > > > > > exact
> > > > > > > > root cause and how to fix the stuff we wanted to fix.
> > > > > > > >
> > > > > > > > So I set up another cluster, four nodes (with DataNode and
> > > > > > RegionServer)
> > > > > > > > and two other nodes with HMaster and Namenode in HA, using
> same
> > > > stuff
> > > > > > we
> > > > > > > > use on production. We pumped some data into it, and I was
> able
> > to
> > > > > > > reproduce
> > > > > > > > same issue last week on it. What I tried to do is to cut one
> > > server
> > > > > > (shut
> > > > > > > > down it's interface) when all is good with cluster, when we
> > have
> > > > > load,
> > > > > > > and
> > > > > > > > see what's going to happen.
> > > > > > > >
> > > > > > > > On Friday, after Nicolas mentioned, I started taking a look
> in
> > > > HBase
> > > > > > logs
> > > > > > > > on the node which was mentioned in HMaster log as the one
> > taking
> > > > over
> > > > > > > > regions for the dead server. Basically what I was able to
> > observe
> > > > was
> > > > > > 15
> > > > > > > > minutes time (+- couple of seconds, what was also
> interesting,
> > > and
> > > > > will
> > > > > > > got
> > > > > > > > later to that) between HMaster figures out that one of it's
> > > > > > RegionServers
> > > > > > > > is dead, and the time one of the mentioned nodes starts
> taking
> > > over
> > > > > > those
> > > > > > > > regions and they start appearing in HMaster's Web UI.
> > > > > > > >
> > > > > > > > I then set up everything like mentioned here
> > > > > > > > http://hbase.apache.org/book.html#mttr - but still had
> exactly
> > > the
> > > > > > same
> > > > > > > > issues. Went over (again and again) all currently configured
> > > stuff,
> > > > > but
> > > > > > > > still had the same issue.
> > > > > > > >
> > > > > > > > Then I started looking into HDFS. Opened NameNode UI, saw all
> > is
> > > > > good,
> > > > > > > took
> > > > > > > > one node down, was also looking RegionServer logs in the same
> > > time
> > > > -
> > > > > > and
> > > > > > > I
> > > > > > > > also see that it took ~15 minutes for Namenode to elect dead
> > node
> > > > as
> > > > > > > dead.
> > > > > > > > Somehow in the same moment regions started getting back to
> > life.
> > > I
> > > > > > > remember
> > > > > > > > in some older versions dfs timeout checks and check retries.
> > > Looked
> > > > > > into
> > > > > > > > defaults for our Hadoop version -
> > > > > > > >
> > > > > > > > http://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/
> > > > > > > hadoop-hdfs/hdfs-default.xml
> > > > > > > > - and saw there that there's no recheck parameter anymore.
> > > Strange,
> > > > > as
> > > > > > on
> > > > > > > > StackOverflow I found post from month ago, for newer version
> > than
> > > > we
> > > > > > use
> > > > > > > > (we use 2.4.1, guy was using 2.6 - dfs.namenode.heartbeat.
> > > > > > > recheck-interval)
> > > > > > > > I set it to 10 seconds as he mentioned, having checks every
> > three
> > > > > > seconds
> > > > > > > > (default) and got DataNode elected as dead in ~45 seconds, as
> > he
> > > > > > > mentioned.
> > > > > > > > Not sure why this parameter is not documented, but obviously
> it
> > > > > works.
> > > > > > > >
> > > > > > > > Then figured out it still didn't fix our HBase failover
> issue.
> > I
> > > > was
> > > > > > > > looking into HBase book again and again, and then saw this
> > part:
> > > > > > > >
> > > > > > > > "How much time we allow elapse between calls to recover
> lease.
> > > > Should
> > > > > > be
> > > > > > > > larger than the dfs timeout."
> > > > > > > >
> > > > > > > > This was description for hbase.lease.recovery.dfs.timeout.
> > Wasn't
> > > > > sure
> > > > > > > from
> > > > > > > > the comment what of all timeouts that's possible to set in
> > > > > Hadoop/HBase
> > > > > > > and
> > > > > > > > that has something to do with DFS is this all about. But
> picked
> > > > > > > > hbase.lease.recovery.timeout, and that was the catch.
> > > > > > > >
> > > > > > > > Initially, by default, hbase.lease.recovery.timeout is set to
> > 15
> > > > > > minutes.
> > > > > > > > Not sure why, and wasn't able to find yet why, but getting
> this
> > > > down
> > > > > to
> > > > > > > one
> > > > > > > > minute (what's more than OK for us) I was able to get rid of
> > our
> > > > > issue.
> > > > > > > Not
> > > > > > > > also sure why this is not mentioned in MTTR section in HBase
> > > book,
> > > > as
> > > > > > > > obviously MTTR doesn't work at all with this default timeout,
> > at
> > > > > least
> > > > > > it
> > > > > > > > doesn't work the way we expected it to work.
> > > > > > > >
> > > > > > > > So thanks again for everyone being spammed with this, and
> > > specially
> > > > > > > thanks
> > > > > > > > to Nicolas pointing me to the right direction.
> > > > > > > >
> > > > > > > > On Mon, Mar 23, 2015 at 1:37 PM Nicolas Liochon <
> > > nkeywal@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > the attachments are rejected by the mailing list, can you
> put
> > > > then
> > > > > on
> > > > > > > > > pastebin?
> > > > > > > > >
> > > > > > > > > stale is mandatory (so it's good), but the issue here is
> just
> > > > > before.
> > > > > > > The
> > > > > > > > > region server needs to read the file. In order to be sure
> > that
> > > > > there
> > > > > > is
> > > > > > > > no
> > > > > > > > > data loss, it needs to "recover the lease", that means
> > ensuring
> > > > > that
> > > > > > > > nobody
> > > > > > > > > is writing the file. The regionserver calls the namenode to
> > do
> > > > this
> > > > > > > > > recoverLease. So there should be some info in the namenode
> > > logs.
> > > > > You
> > > > > > > have
> > > > > > > > > HDFS-4721 on your hdfs? The hbase book details (more or
> > > less...)
> > > > > this
> > > > > > > > > recoverLease stuff.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <
> > > > > > dejan.menges@gmail.com
> > > > > > > >
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > And also, just checked - dfs.namenode.avoid.read.stale.
> > > > datanode
> > > > > and
> > > > > > > > > > dfs.namenode.avoid.write.stale.datanode
> > > > > > > > > > are both true, and dfs.namenode.stale.datanode.interval
> is
> > > > set to
> > > > > > > > > default
> > > > > > > > > > 30000.
> > > > > > > > > >
> > > > > > > > > > On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <
> > > > > > > dejan.menges@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi Nicolas,
> > > > > > > > > > >
> > > > > > > > > > > Please find log attached.
> > > > > > > > > > >
> > > > > > > > > > > As I see it now more clearly, it was trying to recover
> > HDFS
> > > > > WALs
> > > > > > > from
> > > > > > > > > > node
> > > > > > > > > > > that's dead:
> > > > > > > > > > >
> > > > > > > > > > > 2015-03-23 08:53:44,381 WARN
> > > > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils:
> > > > > > > > > > > Cannot recoverLease after trying for 900000ms
> > > > > > > > > > > (hbase.lease.recovery.timeout); continuing, but may be
> > > > > > > DATALOSS!!!;
> > > > > > > > > > > attempt=40 on
> > > > > > > > > > >
> > > > > > > > > >
> > file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intent
> > > > > > > > > ionally_get_down_by_getting_network_down},60020,1426862900
> > > > > > > > > 506-splitting/{node_i_intentionally_get_down_by_gett
> > > > > > > > > ing_network_down}%2C60020%2C1426862900506.1427096924508
> > > > > > > > > > > after 908210ms
> > > > > > > > > > >
> > > > > > > > > > > And as you can see from the log, it tried 40 times,
> what
> > > took
> > > > > it
> > > > > > > > > exactly
> > > > > > > > > > > 15 minutes.
> > > > > > > > > > >
> > > > > > > > > > > There's probably some parameter to tune to cut it of
> from
> > > 40
> > > > > > times
> > > > > > > /
> > > > > > > > 15
> > > > > > > > > > > minutes to something more useful, as for 15 minutes we
> > > don't
> > > > > have
> > > > > > > our
> > > > > > > > > > > regions available, and HDFS have however replication
> > factor
> > > > 3.
> > > > > > > > > > >
> > > > > > > > > > > Googling, if I figure out what's this I will post it
> > here.
> > > > Will
> > > > > > > also
> > > > > > > > > > > appreciate if someone knows how to cut this down.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > Dejan
> > > > > > > > > > >
> > > > > > > > > > > On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <
> > > > > > nkeywal@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > >> The split is done by the region servers (the master
> > > > > > coordinates).
> > > > > > > Is
> > > > > > > > > > there
> > > > > > > > > > >> some interesting stuff in their logs?
> > > > > > > > > > >>
> > > > > > > > > > >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <
> > > > > > > > dejan.menges@gmail.com
> > > > > > > > > >
> > > > > > > > > > >> wrote:
> > > > > > > > > > >>
> > > > > > > > > > >> > With client issue was that it was retrying
> connecting
> > to
> > > > the
> > > > > > > same
> > > > > > > > > > region
> > > > > > > > > > >> > servers even when they were reassigned. Lowering it
> > down
> > > > > > helped
> > > > > > > in
> > > > > > > > > > this
> > > > > > > > > > >> > specific use case, but yes, we still want servers to
> > > > > > reallocate
> > > > > > > > > > quickly.
> > > > > > > > > > >> >
> > > > > > > > > > >> > What got me here:
> > > > > > > > > > >> >
> > > > > > > > > > >> > http://hbase.apache.org/book.html#mttr
> > > > > > > > > > >> >
> > > > > > > > > > >> > I basically set configuration exactly the same way
> as
> > > it's
> > > > > > > > explained
> > > > > > > > > > >> here.
> > > > > > > > > > >> > *zookeeper.session.timeout* is (and was before)
> 60000
> > > (one
> > > > > > > > minute).
> > > > > > > > > > >> >
> > > > > > > > > > >> > So basically what happens is: - simulating network
> > > issues
> > > > we
> > > > > > had
> > > > > > > > > > >> recently).
> > > > > > > > > > >> > - After short time I see in HBase that my
> RegionServer
> > > is
> > > > > > dead,
> > > > > > > > and
> > > > > > > > > as
> > > > > > > > > > >> > total number of regions I see previous total minus
> > > number
> > > > of
> > > > > > > > regions
> > > > > > > > > > >> that
> > > > > > > > > > >> > were hosted on the node hosting RegionServer that
> just
> > > > > > > > > 'disappeared'.
> > > > > > > > > > >> > - In this point I want my clus
> > > > > > > > > > >> >
> > > > > > > > > > >> > - I have test cluster consisting of four nodes,
> every
> > > node
> > > > > > being
> > > > > > > > > > >> DataNode
> > > > > > > > > > >> > and RegionServer.
> > > > > > > > > > >> > - I simulate network partition on one (connect to it
> > > > through
> > > > > > > > console
> > > > > > > > > > and
> > > > > > > > > > >> > take network interface downter to recover as soon as
> > > > > possible,
> > > > > > > to
> > > > > > > > > > start
> > > > > > > > > > >> > serving missing regions.
> > > > > > > > > > >> > - First thing I see in HMaster logs are:
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:26,015 INFO
> > > > > > > > > > >> >
> org.apache.hadoop.hbase.zookeeper.RegionServerTracker:
> > > > > > > > RegionServer
> > > > > > > > > > >> > ephemeral node deleted, processing expiration
> > > > > > > > > > >> > [{name_of_node_I_took_down},60020,1426860403261]
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:26,067 INFO
> > > > > > > > > > >> >
> > > > > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler:
> > > > > > > > > > Splitting
> > > > > > > > > > >> > logs for
> > {name_of_node_I_took_down},60020,1426860403261
> > > > > before
> > > > > > > > > > >> assignment.
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:26,105 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: dead
> > > > > splitlog
> > > > > > > > > workers
> > > > > > > > > > [
> > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261]
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:26,107 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> > started
> > > > > > > splitting
> > > > > > > > 1
> > > > > > > > > > >> logs in
> > > > > > > > > > >> >
> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_
> > > > I_
> > > > > > > > > took_down}
> > > > > > > > > > >> > ,60020,1426860403261-splitting]
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:26,150 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F
> > > > > > > > > > >> >
> > > > > > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > > > > > > > >> >
> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
> > > > 14268
> > > > > > > > > 60404905
> > > > > > > > > > >> > acquired by {fqdn_of_another_live_hnode},
> > > > 60020,1426859445623
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:26,182 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> total
> > > > tasks
> > > > > =
> > > > > > 1
> > > > > > > > > > >> unassigned
> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > > 2F{name_of_node_I_took_down}
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > > > took_
> > > > > > > > > > >>
> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> > status =
> > > > > > > > > in_progress
> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
> > done
> > > > = 0
> > > > > > > > error =
> > > > > > > > > > 0}
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:31,183 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> total
> > > > tasks
> > > > > =
> > > > > > 1
> > > > > > > > > > >> unassigned
> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > > 2F{name_of_node_I_took_down}
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > > > took_
> > > > > > > > > > >>
> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> > status =
> > > > > > > > > in_progress
> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
> > done
> > > > = 0
> > > > > > > > error =
> > > > > > > > > > 0}
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:36,184 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> total
> > > > tasks
> > > > > =
> > > > > > 1
> > > > > > > > > > >> unassigned
> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > > 2F{name_of_node_I_took_down}
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > > > took_
> > > > > > > > > > >>
> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> > status =
> > > > > > > > > in_progress
> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
> > done
> > > > = 0
> > > > > > > > error =
> > > > > > > > > > 0}
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:42,185 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> total
> > > > tasks
> > > > > =
> > > > > > 1
> > > > > > > > > > >> unassigned
> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > > 2F{name_of_node_I_took_down}
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > > > took_
> > > > > > > > > > >>
> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> > status =
> > > > > > > > > in_progress
> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
> > done
> > > > = 0
> > > > > > > > error =
> > > > > > > > > > 0}
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:48,184 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> total
> > > > tasks
> > > > > =
> > > > > > 1
> > > > > > > > > > >> unassigned
> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > > 2F{name_of_node_I_took_down}
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > > > took_
> > > > > > > > > > >>
> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> > status =
> > > > > > > > > in_progress
> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
> > done
> > > > = 0
> > > > > > > > error =
> > > > > > > > > > 0}
> > > > > > > > > > >> > In the meantime, In hbase...out log I got this:
> > > > > > > > > > >> >
> > > > > > > > > > >> > ==> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out
> > <==
> > > > > > > > > > >> >
> > > > > > > > > > >> > java.io.IOException: Call to
> > > > > > > > > > >> >
> > > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_
> > > > dow
> > > > > > > > > n}:60020
> > > > > > > > > > >> > failed on local exception:
> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> > > > CallTimeoutException:
> > > > > > > Call
> > > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
> > > > > > > > > > >> t.java:1532)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:
> > > > > > > 1502)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> > > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(
> > > > Rpc
> > > > > > > > > > >> Client.java:1684)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> > > > BlockingRpcChannelImpl
> > > > > > > > > ementati
> > > > > > > > > > >> on.callBlockingMethod(RpcClient.java:1737)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$
> > > > > > > > > AdminService$
> > > > > > > > > > >> BlockingStub.getRegionInfo(AdminProtos.java:20806)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.apache.hadoop.hbase.client.HBaseAdmin.
> > > > getCompactionState
> > > > > > > > > > >> (HBaseAdmin.java:2524)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.apache.hadoop.hbase.generated.master.table_jsp._
> > > > jspServi
> > > > > > > > > > >> ce(table_jsp.java:167)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.
> > > > > > > > > java:98)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
> > > > > > > > > > >> .java:511)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> > > > doFilte
> > > > > > > > > > >> r(ServletHandler.java:1221)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.apache.hadoop.http.lib.StaticUserWebFilter$
> > > > StaticUserFil
> > > > > > > > > > >> ter.doFilter(StaticUserWebFilter.java:109)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> > > > doFilte
> > > > > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> org.apache.hadoop.http.HttpServer$QuotingInputFilter.
> > > > doFilte
> > > > > > > > > > >> r(HttpServer.java:1081)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> > > > doFilte
> > > > > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > >
> > > > > >
> > org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> > > > doFilte
> > > > > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
> > > > > > > > > > >> er.java:399)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> > org.mortbay.jetty.security.SecurityHandler.handle(
> > > > SecurityHa
> > > > > > > > > > >> ndler.java:216)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
> > > > > > > > > > >> er.java:182)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
> > > > > > > > > > >> er.java:766)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
> > > > > > > > > > >> java:450)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.mortbay.jetty.handler.ContextHandlerCollection.
> > > > handle(Co
> > > > > > > > > > >> ntextHandlerCollection.java:230)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
> > > > > > > > > > >> er.java:152)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at org.mortbay.jetty.Server.handle(Server.java:326)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
> > > > > > > > > > >> n.java:542)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.mortbay.jetty.HttpConnection$RequestHandler.
> > > > headerComple
> > > > > > > > > > >> te(HttpConnection.java:928)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > >
> > org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > >
> > org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.mortbay.io.nio.SelectChannelEndPoint.run(
> > > > SelectChannelEn
> > > > > > > > > > >> dPoint.java:410)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.mortbay.thread.QueuedThreadPool$PoolThread.run(
> > > > > > > > > > >> QueuedThreadPool.java:582)
> > > > > > > > > > >> >
> > > > > > > > > > >> > Caused by: org.apache.hadoop.hbase.ipc.RpcClient$
> > > > > > > > > CallTimeoutException:
> > > > > > > > > > >> Call
> > > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.
> > > > cleanupCall
> > > > > > > > > > >> s(RpcClient.java:1234)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.
> > > > readRespons
> > > > > > > > > > >> e(RpcClient.java:1171)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
> > > > > > > > > > >> nt.java:751)
> > > > > > > > > > >> > Beside this same issue, please note that first
> message
> > > was
> > > > > at
> > > > > > > > > > 2015-03-20
> > > > > > > > > > >> > 14:17:26,015. And then (we got to the point when it
> > > > started
> > > > > > > > > > transition):
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:32:35,059 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F
> > > > > > > > > > >> >
> > > > > > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > > > > > > > >> >
> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
> > > > 14268
> > > > > > > > > 60404905
> > > > > > > > > > >> > entered state: DONE
> > > > > > {fqdn_of_new_live_node},60020,1426859445623
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:32:35,109 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: Done
> > > > > splitting
> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > > > > > > >> >
> > > > > > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}
> > > > > > > > > > >> > %252C60020%252C1426860403261.1426860404905
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:32:35,190 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> > finished
> > > > > > > splitting
> > > > > > > > > > >> (more
> > > > > > > > > > >> > than or equal to) 9 bytes in 1 log files in
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_
> > > > of_
> > > > > > > > > > >> node_I_took_down},60020,1426860403261-splitting]
> > > > > > > > > > >> > in 909083ms
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > > > org.apache.hadoop.hbase.master
> > > > > > > > > > >> .RegionStates:
> > > > > > > > > > >> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c
> > > > state=OPEN,
> > > > > > > > > > >> > ts=1426860639088,
> > > > > > > > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > > > >> to
> > > > > > > > > > >> > {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE,
> > > > > > > ts=1426861955191,
> > > > > > > > > > >> server=
> > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > > > org.apache.hadoop.hbase.master
> > > > > > > > > > >> .RegionStates:
> > > > > > > > > > >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
> > > > > > > > > > >> {name_of_node_I_took_down}
> > > > > > > > > > >> > ,60020,1426860403261
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > > > org.apache.hadoop.hbase.master
> > > > > > > > > > >> .RegionStates:
> > > > > > > > > > >> > Transitioned {25ab6e7b42e36ddaa723d71be5954543
> > > > state=OPEN,
> > > > > > > > > > >> > ts=1426860641783,
> > > > > > > > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > > > >> to
> > > > > > > > > > >> > {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE,
> > > > > > > ts=1426861955191,
> > > > > > > > > > >> server=
> > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > > > org.apache.hadoop.hbase.master
> > > > > > > > > > >> .RegionStates:
> > > > > > > > > > >> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
> > > > > > > > > > >> {name_of_node_I_took_down}
> > > > > > > > > > >> > ,60020,1426860403261
> > > > > > > > > > >> > At this point, note that it finished SplitLogManager
> > > task
> > > > at
> > > > > > > > > 14:32:35
> > > > > > > > > > >> and
> > > > > > > > > > >> > started transitioning just after that. So this is 15
> > > > minutes
> > > > > > > that
> > > > > > > > > I'm
> > > > > > > > > > >> > talking about.
> > > > > > > > > > >> >
> > > > > > > > > > >> > What am I missing?
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> > > > > > > > > > >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <
> > > > > > > > nkeywal@gmail.com>
> > > > > > > > > > >> wrote:
> > > > > > > > > > >> >
> > > > > > > > > > >> > > You've changed the value of
> hbase.zookeeper.timeout
> > to
> > > > 15
> > > > > > > > > minutes? A
> > > > > > > > > > >> very
> > > > > > > > > > >> > > reasonable target is 1 minute before relocating
> the
> > > > > regions.
> > > > > > > > > That's
> > > > > > > > > > >> the
> > > > > > > > > > >> > > default iirc. You can push it to 20s, but then
> > > > > > > > > gc-stopping-the-world
> > > > > > > > > > >> > > becomes more of an issue. 15 minutes is really a
> > lot.
> > > > The
> > > > > > hdfs
> > > > > > > > > stale
> > > > > > > > > > >> mode
> > > > > > > > > > >> > > must always be used, with a lower timeout than the
> > > hbase
> > > > > > one.
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > Client side there should be nothing to do
> (excepted
> > > for
> > > > > > > advanced
> > > > > > > > > > >> stuff);
> > > > > > > > > > >> > at
> > > > > > > > > > >> > > each retry the client checks the location of the
> > > > regions.
> > > > > If
> > > > > > > you
> > > > > > > > > > lower
> > > > > > > > > > >> > the
> > > > > > > > > > >> > > number of retry the client will fail sooner, but
> > > usually
> > > > > you
> > > > > > > > don't
> > > > > > > > > > >> want
> > > > > > > > > > >> > the
> > > > > > > > > > >> > > client to fail, you want the servers to reallocate
> > > > > quickly.
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <
> > > > > > > > > > dejan.menges@gmail.com
> > > > > > > > > > >> >
> > > > > > > > > > >> > > wrote:
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > > Hi,
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > Sorry for little bit late update, but managed to
> > > > narrow
> > > > > it
> > > > > > > > > little
> > > > > > > > > > >> bit
> > > > > > > > > > >> > > down.
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > We didn't update yet, as we are using
> Hortonworks
> > > > > > > distribution
> > > > > > > > > > right
> > > > > > > > > > >> > now,
> > > > > > > > > > >> > > > and even if we update we will get 0.98.4.
> However,
> > > > looks
> > > > > > > that
> > > > > > > > > > issue
> > > > > > > > > > >> > here
> > > > > > > > > > >> > > > was in our use case and configuration (still
> > looking
> > > > > into
> > > > > > > it).
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > Basically, initially I saw that when one server
> > goes
> > > > > down,
> > > > > > > we
> > > > > > > > > > start
> > > > > > > > > > >> > > having
> > > > > > > > > > >> > > > performance issues in general, but it managed to
> > be
> > > on
> > > > > our
> > > > > > > > > client
> > > > > > > > > > >> side,
> > > > > > > > > > >> > > due
> > > > > > > > > > >> > > > to caching, and clients were trying to reconnect
> > to
> > > > > nodes
> > > > > > > that
> > > > > > > > > > were
> > > > > > > > > > >> > > offline
> > > > > > > > > > >> > > > and later trying to get regions from those nodes
> > > too.
> > > > > This
> > > > > > > is
> > > > > > > > > > >> basically
> > > > > > > > > > >> > > why
> > > > > > > > > > >> > > > on server side I didn't manage to see anything
> in
> > > logs
> > > > > > that
> > > > > > > > > would
> > > > > > > > > > >> be at
> > > > > > > > > > >> > > > least little bit interesting and point me into
> > > desired
> > > > > > > > > direction.
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > Another question that popped up to me is - in
> case
> > > > > server
> > > > > > is
> > > > > > > > > down
> > > > > > > > > > >> (and
> > > > > > > > > > >> > > with
> > > > > > > > > > >> > > > it DataNode and HRegionServer it was hosting) -
> > > what's
> > > > > > > optimal
> > > > > > > > > > time
> > > > > > > > > > >> to
> > > > > > > > > > >> > > set
> > > > > > > > > > >> > > > for HMaster to consider server dead reassign
> > regions
> > > > > > > somewhere
> > > > > > > > > > >> else, as
> > > > > > > > > > >> > > > this is another performance bottleneck we hit
> > during
> > > > > > > inability
> > > > > > > > > to
> > > > > > > > > > >> > access
> > > > > > > > > > >> > > > regions? In our case it's configured to 15
> > minutes,
> > > > and
> > > > > > > simple
> > > > > > > > > > logic
> > > > > > > > > > >> > > tells
> > > > > > > > > > >> > > > me if you want it earlier then configure lower
> > > number
> > > > of
> > > > > > > > > retries,
> > > > > > > > > > >> but
> > > > > > > > > > >> > > issue
> > > > > > > > > > >> > > > is as always in details, so not sure if anyone
> > knows
> > > > > some
> > > > > > > > better
> > > > > > > > > > >> math
> > > > > > > > > > >> > for
> > > > > > > > > > >> > > > this?
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > And last question - is it possible to manually
> > force
> > > > > HBase
> > > > > > > to
> > > > > > > > > > >> reassign
> > > > > > > > > > >> > > > regions? In this case, while HMaster is retrying
> > to
> > > > > > contact
> > > > > > > > node
> > > > > > > > > > >> that's
> > > > > > > > > > >> > > > dead, it's impossible to force it using
> 'balancer'
> > > > > > command.
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > Thanks a lot!
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > Dejan
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <
> > > > > > > > > > >> dejan.menges@gmail.com>
> > > > > > > > > > >> > > > wrote:
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > > Hi,
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > To be very honest - there's no particular
> reason
> > > why
> > > > > we
> > > > > > > > stick
> > > > > > > > > to
> > > > > > > > > > >> this
> > > > > > > > > > >> > > > one,
> > > > > > > > > > >> > > > > beside just lack of time currently to go
> through
> > > > > upgrade
> > > > > > > > > > process,
> > > > > > > > > > >> but
> > > > > > > > > > >> > > > looks
> > > > > > > > > > >> > > > > to me that's going to be next step.
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > Had a crazy day, didn't have time to go
> through
> > > all
> > > > > logs
> > > > > > > > > again,
> > > > > > > > > > >> plus
> > > > > > > > > > >> > > one
> > > > > > > > > > >> > > > > of the machines (last one where we had this
> > issue)
> > > > is
> > > > > > > fully
> > > > > > > > > > >> > > reprovisioned
> > > > > > > > > > >> > > > > yesterday so I don't have logs from there
> > anymore.
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > Beside upgrading,  what I will talk about
> today,
> > > can
> > > > > you
> > > > > > > > just
> > > > > > > > > > >> point
> > > > > > > > > > >> > me
> > > > > > > > > > >> > > to
> > > > > > > > > > >> > > > > the specific RPC issue in 0.98.0? Thing is
> that
> > we
> > > > > have
> > > > > > > some
> > > > > > > > > > >> strange
> > > > > > > > > > >> > > > > moments with RPC in this case, and just want
> to
> > > see
> > > > if
> > > > > > > > that's
> > > > > > > > > > the
> > > > > > > > > > >> > same
> > > > > > > > > > >> > > > > thing (and we were even suspecting to RPC).
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > Thanks a lot!
> > > > > > > > > > >> > > > > Dejan
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew
> Purtell
> > <
> > > > > > > > > > >> apurtell@apache.org
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > > > wrote:
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > >> Is there a particular reason why you are
> using
> > > > HBase
> > > > > > > > 0.98.0?
> > > > > > > > > > The
> > > > > > > > > > >> > > latest
> > > > > > > > > > >> > > > >> 0.98 release is 0.98.11. There's a known
> > > > performance
> > > > > > > issue
> > > > > > > > > with
> > > > > > > > > > >> > 0.98.0
> > > > > > > > > > >> > > > >> pertaining to RPC that was fixed in later
> > > releases,
> > > > > you
> > > > > > > > > should
> > > > > > > > > > >> move
> > > > > > > > > > >> > up
> > > > > > > > > > >> > > > >> from
> > > > > > > > > > >> > > > >> 0.98.0. In addition hundreds of improvements
> > and
> > > > bug
> > > > > > > fixes
> > > > > > > > > have
> > > > > > > > > > >> gone
> > > > > > > > > > >> > > > into
> > > > > > > > > > >> > > > >> the ten releases since 0.98.0.
> > > > > > > > > > >> > > > >>
> > > > > > > > > > >> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan
> Menges <
> > > > > > > > > > >> > dejan.menges@gmail.com
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > >> wrote:
> > > > > > > > > > >> > > > >>
> > > > > > > > > > >> > > > >> > Hi All,
> > > > > > > > > > >> > > > >> >
> > > > > > > > > > >> > > > >> > We have a strange issue with HBase
> > performance
> > > > > > (overall
> > > > > > > > > > cluster
> > > > > > > > > > >> > > > >> > performance) in case one of datanodes in
> the
> > > > > cluster
> > > > > > > > > > >> unexpectedly
> > > > > > > > > > >> > > goes
> > > > > > > > > > >> > > > >> > down.
> > > > > > > > > > >> > > > >> >
> > > > > > > > > > >> > > > >> > So scenario is like follows:
> > > > > > > > > > >> > > > >> > - Cluster works fine, it's stable.
> > > > > > > > > > >> > > > >> > - One DataNode unexpectedly goes down (PSU
> > > issue,
> > > > > > > network
> > > > > > > > > > >> issue,
> > > > > > > > > > >> > > > >> anything)
> > > > > > > > > > >> > > > >> > - Whole HBase cluster goes down
> (performance
> > > > > becomes
> > > > > > so
> > > > > > > > bad
> > > > > > > > > > >> that
> > > > > > > > > > >> > we
> > > > > > > > > > >> > > > >> have to
> > > > > > > > > > >> > > > >> > restart all RegionServers to get it back to
> > > > life).
> > > > > > > > > > >> > > > >> >
> > > > > > > > > > >> > > > >> > Most funny and latest issue that happened
> was
> > > > that
> > > > > we
> > > > > > > > added
> > > > > > > > > > new
> > > > > > > > > > >> > node
> > > > > > > > > > >> > > > to
> > > > > > > > > > >> > > > >> the
> > > > > > > > > > >> > > > >> > cluster (having 8 x 4T SATA disks) and we
> > left
> > > > just
> > > > > > > > > DataNode
> > > > > > > > > > >> > running
> > > > > > > > > > >> > > > on
> > > > > > > > > > >> > > > >> it
> > > > > > > > > > >> > > > >> > to give it couple of days to get some data.
> > At
> > > > some
> > > > > > > point
> > > > > > > > > in
> > > > > > > > > > >> time,
> > > > > > > > > > >> > > due
> > > > > > > > > > >> > > > >> to
> > > > > > > > > > >> > > > >> > hardware issue, server rebooted (twice
> during
> > > > three
> > > > > > > > hours)
> > > > > > > > > in
> > > > > > > > > > >> > moment
> > > > > > > > > > >> > > > >> when
> > > > > > > > > > >> > > > >> > it had maybe 5% of data it would have in a
> > > couple
> > > > > of
> > > > > > > > days.
> > > > > > > > > > >> Nothing
> > > > > > > > > > >> > > > else
> > > > > > > > > > >> > > > >> > beside DataNode was running, and once it
> went
> > > > down,
> > > > > > it
> > > > > > > > > > affected
> > > > > > > > > > >> > > > literary
> > > > > > > > > > >> > > > >> > everything, and restarting RegionServers in
> > the
> > > > end
> > > > > > > fixed
> > > > > > > > > it.
> > > > > > > > > > >> > > > >> >
> > > > > > > > > > >> > > > >> > We are using HBase 0.98.0 with Hadoop 2.4.0
> > > > > > > > > > >> > > > >> >
> > > > > > > > > > >> > > > >>
> > > > > > > > > > >> > > > >>
> > > > > > > > > > >> > > > >>
> > > > > > > > > > >> > > > >> --
> > > > > > > > > > >> > > > >> Best regards,
> > > > > > > > > > >> > > > >>
> > > > > > > > > > >> > > > >>    - Andy
> > > > > > > > > > >> > > > >>
> > > > > > > > > > >> > > > >> Problems worthy of attack prove their worth
> by
> > > > > hitting
> > > > > > > > back.
> > > > > > > > > -
> > > > > > > > > > >> Piet
> > > > > > > > > > >> > > Hein
> > > > > > > > > > >> > > > >> (via Tom White)
> > > > > > > > > > >> > > > >>
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > >
> > > > > > > > > > >> >
> > > > > > > > > > >>
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Strange issue when DataNode goes down

Posted by Bryan Beaudreault <bb...@hubspot.com>.
@Dejan, I've had staleness configured on my cluster for a while, but
haven't needed it.  Looking more closely at it thanks to this thread, I
noticed though that I was missing two critical parameters.

Considering you just now set this up, I'll guess that you probably didn't
miss this (the docs used to be much less organized).  But it won't work
unless the namenodes have dfs.namenode.avoid.read.stale.datanode
and dfs.namenode.avoid.write.stale.datanode set to true.

Just thought I'd throw it out there.

On Mon, Mar 23, 2015 at 11:51 AM, Dejan Menges <de...@gmail.com>
wrote:

> I'm surprised by this as well. Staleness was configured also, MTTR from
> HBase book as described, and in this specific case - when machine really
> dies - even when NN makes DataNode dead, HBase was trying to replay WALs
> from dead node until timeout reached.
>
> Still reading this HDFS-3703, trying to get more context, but looks to me
> so far that this is one of those things where you should decide what are
> your own acceptable tradeoffs.
>
> On Mon, Mar 23, 2015 at 4:40 PM Bryan Beaudreault <
> bbeaudreault@hubspot.com>
> wrote:
>
> > @Nicholas, I see, thanks.  I'll keep the settings at default.  So really
> if
> > everything else is configured properly you should never reach the lease
> > recovery timeout in any failure scenarios.  It seems that the staleness
> > check would be the thing that prevents this, correct?  I'm surprised it
> > didn't help Dejan.
> >
> > On Mon, Mar 23, 2015 at 11:20 AM, Nicolas Liochon <nk...@gmail.com>
> > wrote:
> >
> > > @bryan: yes, you can change hbase.lease.recovery.timeout if you changed
> > he
> > > hdfs settings. But this setting is really for desperate cases. The
> > recover
> > > Lease should have succeeded before. As well, if you depend on
> > > hbase.lease.recovery.timeout, it means that you're wasting recovery
> time:
> > > the lease should be recovered in a few seconds.
> > >
> > > On Mon, Mar 23, 2015 at 3:59 PM, Dejan Menges <de...@gmail.com>
> > > wrote:
> > >
> > > > Interesting discussion I also found, gives me some more light on what
> > > > Nicolas is talking about -
> > > https://issues.apache.org/jira/browse/HDFS-3703
> > > >
> > > > On Mon, Mar 23, 2015 at 3:53 PM Bryan Beaudreault <
> > > > bbeaudreault@hubspot.com>
> > > > wrote:
> > > >
> > > > > So it is safe to set hbase.lease.recovery.timeout lower if you also
> > > > > set heartbeat.recheck.interval lower (lowering that 10.5 min dead
> > node
> > > > > timer)?  Or is it recommended to not touch either of those?
> > > > >
> > > > > Reading the above with interest, thanks for digging in here guys.
> > > > >
> > > > > On Mon, Mar 23, 2015 at 10:13 AM, Nicolas Liochon <
> nkeywal@gmail.com
> > >
> > > > > wrote:
> > > > >
> > > > > > If the node is actually down it's fine. But the node may not be
> > that
> > > > down
> > > > > > (CAP theorem here); and then it's looking for trouble.
> > > > > > HDFS, by default declare a node as dead after 10:30. 15 minutes
> is
> > an
> > > > > extra
> > > > > > security. It seems your hdfs settings are different (or there is
> a
> > > > > bug...).
> > > > > > There should be some info in the hdfs logs.
> > > > > >
> > > > > > On Mon, Mar 23, 2015 at 3:05 PM, Dejan Menges <
> > > dejan.menges@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Will take a look.
> > > > > > >
> > > > > > > Actually, if node is down (someone unplugged network cable, it
> > just
> > > > > died,
> > > > > > > whatever) what's chance it's going to become live so write can
> > > > > continue?
> > > > > > On
> > > > > > > the other side, HBase is not starting recovery trying to
> contact
> > > node
> > > > > > which
> > > > > > > is not there anymore, and even elected as dead on Namenode side
> > > > > (another
> > > > > > > thing I didn't understood quite good).
> > > > > > >
> > > > > > > So what I was expecting is that as soon as Namenode decided
> node
> > is
> > > > > dead,
> > > > > > > that it would be enough for RegionServer to stop trying to
> > recover
> > > > from
> > > > > > the
> > > > > > > dead node, but it wasn't the case. Also, this whole MTTR
> article
> > in
> > > > > HBase
> > > > > > > book doesn't work at all with this parameter set to it's
> default
> > > > value
> > > > > > (15
> > > > > > > minutes).
> > > > > > >
> > > > > > > So I'm still struggling to figure out what's drawback exactly
> on
> > > > this?
> > > > > > >
> > > > > > > On Mon, Mar 23, 2015 at 2:50 PM Nicolas Liochon <
> > nkeywal@gmail.com
> > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > > Thanks for the explanation. There is an issue if you modify
> > this
> > > > > > setting
> > > > > > > > however.
> > > > > > > > hbase tries to recover the lease (i.e. be sure that nobody is
> > > > > writing)
> > > > > > > > If you change hbase.lease.recovery.timeout hbase will start
> the
> > > > > > recovery
> > > > > > > > (i.e. start to read) even if it's not sure that nobody's
> > writing.
> > > > > That
> > > > > > > > means there is a dataloss risk.
> > > > > > > > Basically, you must not see this warning: WARN
> > > > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils: Cannot recoverLease
> > > after
> > > > > > > trying
> > > > > > > > for[]
> > > > > > > >
> > > > > > > > The recoverLease must succeed. The fact that it does not
> after
> > X
> > > > > tries
> > > > > > is
> > > > > > > > strange.
> > > > > > > > There may be a mistmatch between the hbase parameters and the
> > > hdfs
> > > > > > ones.
> > > > > > > > You may need to have a look at the comments in
> FSHDFSUtils.java
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > On Mon, Mar 23, 2015 at 2:15 PM, Dejan Menges <
> > > > > dejan.menges@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > I found the issue and fixed it, and will try to explain
> here
> > > what
> > > > > was
> > > > > > > > > exactly in our case in case someone else finds this
> > interesting
> > > > > too.
> > > > > > > > >
> > > > > > > > > So initially, we had (couple of times) some network and
> > > hardware
> > > > > > issues
> > > > > > > > in
> > > > > > > > > our datacenters. When one server would die (literary die,
> we
> > > had
> > > > > some
> > > > > > > > issue
> > > > > > > > > with PSUs) we saw issues with overall cluster performance
> on
> > > > HBase
> > > > > > > side.
> > > > > > > > As
> > > > > > > > > cluster is quite big and live, it was also quite hard to
> > figure
> > > > out
> > > > > > > exact
> > > > > > > > > root cause and how to fix the stuff we wanted to fix.
> > > > > > > > >
> > > > > > > > > So I set up another cluster, four nodes (with DataNode and
> > > > > > > RegionServer)
> > > > > > > > > and two other nodes with HMaster and Namenode in HA, using
> > same
> > > > > stuff
> > > > > > > we
> > > > > > > > > use on production. We pumped some data into it, and I was
> > able
> > > to
> > > > > > > > reproduce
> > > > > > > > > same issue last week on it. What I tried to do is to cut
> one
> > > > server
> > > > > > > (shut
> > > > > > > > > down it's interface) when all is good with cluster, when we
> > > have
> > > > > > load,
> > > > > > > > and
> > > > > > > > > see what's going to happen.
> > > > > > > > >
> > > > > > > > > On Friday, after Nicolas mentioned, I started taking a look
> > in
> > > > > HBase
> > > > > > > logs
> > > > > > > > > on the node which was mentioned in HMaster log as the one
> > > taking
> > > > > over
> > > > > > > > > regions for the dead server. Basically what I was able to
> > > observe
> > > > > was
> > > > > > > 15
> > > > > > > > > minutes time (+- couple of seconds, what was also
> > interesting,
> > > > and
> > > > > > will
> > > > > > > > got
> > > > > > > > > later to that) between HMaster figures out that one of it's
> > > > > > > RegionServers
> > > > > > > > > is dead, and the time one of the mentioned nodes starts
> > taking
> > > > over
> > > > > > > those
> > > > > > > > > regions and they start appearing in HMaster's Web UI.
> > > > > > > > >
> > > > > > > > > I then set up everything like mentioned here
> > > > > > > > > http://hbase.apache.org/book.html#mttr - but still had
> > exactly
> > > > the
> > > > > > > same
> > > > > > > > > issues. Went over (again and again) all currently
> configured
> > > > stuff,
> > > > > > but
> > > > > > > > > still had the same issue.
> > > > > > > > >
> > > > > > > > > Then I started looking into HDFS. Opened NameNode UI, saw
> all
> > > is
> > > > > > good,
> > > > > > > > took
> > > > > > > > > one node down, was also looking RegionServer logs in the
> same
> > > > time
> > > > > -
> > > > > > > and
> > > > > > > > I
> > > > > > > > > also see that it took ~15 minutes for Namenode to elect
> dead
> > > node
> > > > > as
> > > > > > > > dead.
> > > > > > > > > Somehow in the same moment regions started getting back to
> > > life.
> > > > I
> > > > > > > > remember
> > > > > > > > > in some older versions dfs timeout checks and check
> retries.
> > > > Looked
> > > > > > > into
> > > > > > > > > defaults for our Hadoop version -
> > > > > > > > >
> > > > > > > > > http://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/
> > > > > > > > hadoop-hdfs/hdfs-default.xml
> > > > > > > > > - and saw there that there's no recheck parameter anymore.
> > > > Strange,
> > > > > > as
> > > > > > > on
> > > > > > > > > StackOverflow I found post from month ago, for newer
> version
> > > than
> > > > > we
> > > > > > > use
> > > > > > > > > (we use 2.4.1, guy was using 2.6 - dfs.namenode.heartbeat.
> > > > > > > > recheck-interval)
> > > > > > > > > I set it to 10 seconds as he mentioned, having checks every
> > > three
> > > > > > > seconds
> > > > > > > > > (default) and got DataNode elected as dead in ~45 seconds,
> as
> > > he
> > > > > > > > mentioned.
> > > > > > > > > Not sure why this parameter is not documented, but
> obviously
> > it
> > > > > > works.
> > > > > > > > >
> > > > > > > > > Then figured out it still didn't fix our HBase failover
> > issue.
> > > I
> > > > > was
> > > > > > > > > looking into HBase book again and again, and then saw this
> > > part:
> > > > > > > > >
> > > > > > > > > "How much time we allow elapse between calls to recover
> > lease.
> > > > > Should
> > > > > > > be
> > > > > > > > > larger than the dfs timeout."
> > > > > > > > >
> > > > > > > > > This was description for hbase.lease.recovery.dfs.timeout.
> > > Wasn't
> > > > > > sure
> > > > > > > > from
> > > > > > > > > the comment what of all timeouts that's possible to set in
> > > > > > Hadoop/HBase
> > > > > > > > and
> > > > > > > > > that has something to do with DFS is this all about. But
> > picked
> > > > > > > > > hbase.lease.recovery.timeout, and that was the catch.
> > > > > > > > >
> > > > > > > > > Initially, by default, hbase.lease.recovery.timeout is set
> to
> > > 15
> > > > > > > minutes.
> > > > > > > > > Not sure why, and wasn't able to find yet why, but getting
> > this
> > > > > down
> > > > > > to
> > > > > > > > one
> > > > > > > > > minute (what's more than OK for us) I was able to get rid
> of
> > > our
> > > > > > issue.
> > > > > > > > Not
> > > > > > > > > also sure why this is not mentioned in MTTR section in
> HBase
> > > > book,
> > > > > as
> > > > > > > > > obviously MTTR doesn't work at all with this default
> timeout,
> > > at
> > > > > > least
> > > > > > > it
> > > > > > > > > doesn't work the way we expected it to work.
> > > > > > > > >
> > > > > > > > > So thanks again for everyone being spammed with this, and
> > > > specially
> > > > > > > > thanks
> > > > > > > > > to Nicolas pointing me to the right direction.
> > > > > > > > >
> > > > > > > > > On Mon, Mar 23, 2015 at 1:37 PM Nicolas Liochon <
> > > > nkeywal@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > the attachments are rejected by the mailing list, can you
> > put
> > > > > then
> > > > > > on
> > > > > > > > > > pastebin?
> > > > > > > > > >
> > > > > > > > > > stale is mandatory (so it's good), but the issue here is
> > just
> > > > > > before.
> > > > > > > > The
> > > > > > > > > > region server needs to read the file. In order to be sure
> > > that
> > > > > > there
> > > > > > > is
> > > > > > > > > no
> > > > > > > > > > data loss, it needs to "recover the lease", that means
> > > ensuring
> > > > > > that
> > > > > > > > > nobody
> > > > > > > > > > is writing the file. The regionserver calls the namenode
> to
> > > do
> > > > > this
> > > > > > > > > > recoverLease. So there should be some info in the
> namenode
> > > > logs.
> > > > > > You
> > > > > > > > have
> > > > > > > > > > HDFS-4721 on your hdfs? The hbase book details (more or
> > > > less...)
> > > > > > this
> > > > > > > > > > recoverLease stuff.
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <
> > > > > > > dejan.menges@gmail.com
> > > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > And also, just checked - dfs.namenode.avoid.read.stale.
> > > > > datanode
> > > > > > and
> > > > > > > > > > > dfs.namenode.avoid.write.stale.datanode
> > > > > > > > > > > are both true, and dfs.namenode.stale.datanode.interval
> > is
> > > > > set to
> > > > > > > > > > default
> > > > > > > > > > > 30000.
> > > > > > > > > > >
> > > > > > > > > > > On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <
> > > > > > > > dejan.menges@gmail.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hi Nicolas,
> > > > > > > > > > > >
> > > > > > > > > > > > Please find log attached.
> > > > > > > > > > > >
> > > > > > > > > > > > As I see it now more clearly, it was trying to
> recover
> > > HDFS
> > > > > > WALs
> > > > > > > > from
> > > > > > > > > > > node
> > > > > > > > > > > > that's dead:
> > > > > > > > > > > >
> > > > > > > > > > > > 2015-03-23 08:53:44,381 WARN
> > > > > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils:
> > > > > > > > > > > > Cannot recoverLease after trying for 900000ms
> > > > > > > > > > > > (hbase.lease.recovery.timeout); continuing, but may
> be
> > > > > > > > DATALOSS!!!;
> > > > > > > > > > > > attempt=40 on
> > > > > > > > > > > >
> > > > > > > > > > >
> > > file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intent
> > > > > > > > > >
> ionally_get_down_by_getting_network_down},60020,1426862900
> > > > > > > > > > 506-splitting/{node_i_intentionally_get_down_by_gett
> > > > > > > > > > ing_network_down}%2C60020%2C1426862900506.1427096924508
> > > > > > > > > > > > after 908210ms
> > > > > > > > > > > >
> > > > > > > > > > > > And as you can see from the log, it tried 40 times,
> > what
> > > > took
> > > > > > it
> > > > > > > > > > exactly
> > > > > > > > > > > > 15 minutes.
> > > > > > > > > > > >
> > > > > > > > > > > > There's probably some parameter to tune to cut it of
> > from
> > > > 40
> > > > > > > times
> > > > > > > > /
> > > > > > > > > 15
> > > > > > > > > > > > minutes to something more useful, as for 15 minutes
> we
> > > > don't
> > > > > > have
> > > > > > > > our
> > > > > > > > > > > > regions available, and HDFS have however replication
> > > factor
> > > > > 3.
> > > > > > > > > > > >
> > > > > > > > > > > > Googling, if I figure out what's this I will post it
> > > here.
> > > > > Will
> > > > > > > > also
> > > > > > > > > > > > appreciate if someone knows how to cut this down.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > >
> > > > > > > > > > > > Dejan
> > > > > > > > > > > >
> > > > > > > > > > > > On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <
> > > > > > > nkeywal@gmail.com
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > >> The split is done by the region servers (the master
> > > > > > > coordinates).
> > > > > > > > Is
> > > > > > > > > > > there
> > > > > > > > > > > >> some interesting stuff in their logs?
> > > > > > > > > > > >>
> > > > > > > > > > > >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <
> > > > > > > > > dejan.menges@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > >> wrote:
> > > > > > > > > > > >>
> > > > > > > > > > > >> > With client issue was that it was retrying
> > connecting
> > > to
> > > > > the
> > > > > > > > same
> > > > > > > > > > > region
> > > > > > > > > > > >> > servers even when they were reassigned. Lowering
> it
> > > down
> > > > > > > helped
> > > > > > > > in
> > > > > > > > > > > this
> > > > > > > > > > > >> > specific use case, but yes, we still want servers
> to
> > > > > > > reallocate
> > > > > > > > > > > quickly.
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > What got me here:
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > http://hbase.apache.org/book.html#mttr
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > I basically set configuration exactly the same way
> > as
> > > > it's
> > > > > > > > > explained
> > > > > > > > > > > >> here.
> > > > > > > > > > > >> > *zookeeper.session.timeout* is (and was before)
> > 60000
> > > > (one
> > > > > > > > > minute).
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > So basically what happens is: - simulating network
> > > > issues
> > > > > we
> > > > > > > had
> > > > > > > > > > > >> recently).
> > > > > > > > > > > >> > - After short time I see in HBase that my
> > RegionServer
> > > > is
> > > > > > > dead,
> > > > > > > > > and
> > > > > > > > > > as
> > > > > > > > > > > >> > total number of regions I see previous total minus
> > > > number
> > > > > of
> > > > > > > > > regions
> > > > > > > > > > > >> that
> > > > > > > > > > > >> > were hosted on the node hosting RegionServer that
> > just
> > > > > > > > > > 'disappeared'.
> > > > > > > > > > > >> > - In this point I want my clus
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > - I have test cluster consisting of four nodes,
> > every
> > > > node
> > > > > > > being
> > > > > > > > > > > >> DataNode
> > > > > > > > > > > >> > and RegionServer.
> > > > > > > > > > > >> > - I simulate network partition on one (connect to
> it
> > > > > through
> > > > > > > > > console
> > > > > > > > > > > and
> > > > > > > > > > > >> > take network interface downter to recover as soon
> as
> > > > > > possible,
> > > > > > > > to
> > > > > > > > > > > start
> > > > > > > > > > > >> > serving missing regions.
> > > > > > > > > > > >> > - First thing I see in HMaster logs are:
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 2015-03-20 14:17:26,015 INFO
> > > > > > > > > > > >> > org.apache.hadoop.hbase.
> > zookeeper.RegionServerTracker:
> > > > > > > > > RegionServer
> > > > > > > > > > > >> > ephemeral node deleted, processing expiration
> > > > > > > > > > > >> > [{name_of_node_I_took_down},60020,1426860403261]
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 2015-03-20 14:17:26,067 INFO
> > > > > > > > > > > >> >
> > > > > > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler:
> > > > > > > > > > > Splitting
> > > > > > > > > > > >> > logs for
> > > {name_of_node_I_took_down},60020,1426860403261
> > > > > > before
> > > > > > > > > > > >> assignment.
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 2015-03-20 14:17:26,105 INFO
> > > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> > dead
> > > > > > splitlog
> > > > > > > > > > workers
> > > > > > > > > > > [
> > > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261]
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 2015-03-20 14:17:26,107 INFO
> > > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> > > started
> > > > > > > > splitting
> > > > > > > > > 1
> > > > > > > > > > > >> logs in
> > > > > > > > > > > >> >
> > > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_
> > > > > I_
> > > > > > > > > > took_down}
> > > > > > > > > > > >> > ,60020,1426860403261-splitting]
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 2015-03-20 14:17:26,150 INFO
> > > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> > task
> > > > > > > > > > > >> > /hbase/splitWAL/WALs%2F
> > > > > > > > > > > >> >
> > > > > > > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > > > > > > > > >> >
> > > {name_of_node_I_took_down}%252C60020%252C1426860403261.
> > > > > 14268
> > > > > > > > > > 60404905
> > > > > > > > > > > >> > acquired by {fqdn_of_another_live_hnode},
> > > > > 60020,1426859445623
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 2015-03-20 14:17:26,182 INFO
> > > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> > total
> > > > > tasks
> > > > > > =
> > > > > > > 1
> > > > > > > > > > > >> unassigned
> > > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > > > 2F{name_of_node_I_took_down}
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > %2C60020%2C1426860403261-
> > splitting%2F{name_of_node_I_
> > > > > took_
> > > > > > > > > > > >>
> > > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> > > status =
> > > > > > > > > > in_progress
> > > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed =
> 1
> > > done
> > > > > = 0
> > > > > > > > > error =
> > > > > > > > > > > 0}
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 2015-03-20 14:17:31,183 INFO
> > > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> > total
> > > > > tasks
> > > > > > =
> > > > > > > 1
> > > > > > > > > > > >> unassigned
> > > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > > > 2F{name_of_node_I_took_down}
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > %2C60020%2C1426860403261-
> > splitting%2F{name_of_node_I_
> > > > > took_
> > > > > > > > > > > >>
> > > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> > > status =
> > > > > > > > > > in_progress
> > > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed =
> 1
> > > done
> > > > > = 0
> > > > > > > > > error =
> > > > > > > > > > > 0}
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 2015-03-20 14:17:36,184 INFO
> > > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> > total
> > > > > tasks
> > > > > > =
> > > > > > > 1
> > > > > > > > > > > >> unassigned
> > > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > > > 2F{name_of_node_I_took_down}
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > %2C60020%2C1426860403261-
> > splitting%2F{name_of_node_I_
> > > > > took_
> > > > > > > > > > > >>
> > > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> > > status =
> > > > > > > > > > in_progress
> > > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed =
> 1
> > > done
> > > > > = 0
> > > > > > > > > error =
> > > > > > > > > > > 0}
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 2015-03-20 14:17:42,185 INFO
> > > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> > total
> > > > > tasks
> > > > > > =
> > > > > > > 1
> > > > > > > > > > > >> unassigned
> > > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > > > 2F{name_of_node_I_took_down}
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > %2C60020%2C1426860403261-
> > splitting%2F{name_of_node_I_
> > > > > took_
> > > > > > > > > > > >>
> > > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> > > status =
> > > > > > > > > > in_progress
> > > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed =
> 1
> > > done
> > > > > = 0
> > > > > > > > > error =
> > > > > > > > > > > 0}
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 2015-03-20 14:17:48,184 INFO
> > > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> > total
> > > > > tasks
> > > > > > =
> > > > > > > 1
> > > > > > > > > > > >> unassigned
> > > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > > > 2F{name_of_node_I_took_down}
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > %2C60020%2C1426860403261-
> > splitting%2F{name_of_node_I_
> > > > > took_
> > > > > > > > > > > >>
> > > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> > > status =
> > > > > > > > > > in_progress
> > > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed =
> 1
> > > done
> > > > > = 0
> > > > > > > > > error =
> > > > > > > > > > > 0}
> > > > > > > > > > > >> > In the meantime, In hbase...out log I got this:
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > ==> hbase-hbase-master-{fqdn_of_
> > my_hmaster_node}.out
> > > <==
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > java.io.IOException: Call to
> > > > > > > > > > > >> >
> > > > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_
> > > > > dow
> > > > > > > > > > n}:60020
> > > > > > > > > > > >> > failed on local exception:
> > > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> > > > > CallTimeoutException:
> > > > > > > > Call
> > > > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
> > > > > > > > > > > >> t.java:1532)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:
> > > > > > > > 1502)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> >
> > > > > > > > > > > >> >
> > > > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(
> > > > > Rpc
> > > > > > > > > > > >> Client.java:1684)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> > > > > BlockingRpcChannelImpl
> > > > > > > > > > ementati
> > > > > > > > > > > >> on.callBlockingMethod(RpcClient.java:1737)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> >
> > > > > > > > > > > >> >
> > > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$
> > > > > > > > > > AdminService$
> > > > > > > > > > > >> BlockingStub.getRegionInfo(AdminProtos.java:20806)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > org.apache.hadoop.hbase.client.HBaseAdmin.
> > > > > getCompactionState
> > > > > > > > > > > >> (HBaseAdmin.java:2524)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > org.apache.hadoop.hbase.
> > generated.master.table_jsp._
> > > > > jspServi
> > > > > > > > > > > >> ce(table_jsp.java:167)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.
> > > > > > > > > > java:98)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
> > > > > > > > > > > >> .java:511)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > org.mortbay.jetty.servlet.
> > ServletHandler$CachedChain.
> > > > > doFilte
> > > > > > > > > > > >> r(ServletHandler.java:1221)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > org.apache.hadoop.http.lib.StaticUserWebFilter$
> > > > > StaticUserFil
> > > > > > > > > > > >> ter.doFilter(StaticUserWebFilter.java:109)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > org.mortbay.jetty.servlet.
> > ServletHandler$CachedChain.
> > > > > doFilte
> > > > > > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > org.apache.hadoop.http.
> > HttpServer$QuotingInputFilter.
> > > > > doFilte
> > > > > > > > > > > >> r(HttpServer.java:1081)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > org.mortbay.jetty.servlet.
> > ServletHandler$CachedChain.
> > > > > doFilte
> > > > > > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > >
> > > > > > >
> > > org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > org.mortbay.jetty.servlet.
> > ServletHandler$CachedChain.
> > > > > doFilte
> > > > > > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
> > > > > > > > > > > >> er.java:399)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> > org.mortbay.jetty.security.SecurityHandler.handle(
> > > > > SecurityHa
> > > > > > > > > > > >> ndler.java:216)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
> > > > > > > > > > > >> er.java:182)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
> > > > > > > > > > > >> er.java:766)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
> > > > > > > > > > > >> java:450)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> >
> > > > > > > > > > > >> >
> org.mortbay.jetty.handler.ContextHandlerCollection.
> > > > > handle(Co
> > > > > > > > > > > >> ntextHandlerCollection.java:230)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
> > > > > > > > > > > >> er.java:152)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> org.mortbay.jetty.Server.handle(Server.java:326)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
> > > > > > > > > > > >> n.java:542)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > org.mortbay.jetty.HttpConnection$RequestHandler.
> > > > > headerComple
> > > > > > > > > > > >> te(HttpConnection.java:928)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > >
> > > org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > >
> > > org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > org.mortbay.io.nio.SelectChannelEndPoint.run(
> > > > > SelectChannelEn
> > > > > > > > > > > >> dPoint.java:410)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> >
> > > > > > > > > > > >> >
> org.mortbay.thread.QueuedThreadPool$PoolThread.run(
> > > > > > > > > > > >> QueuedThreadPool.java:582)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > Caused by: org.apache.hadoop.hbase.ipc.RpcClient$
> > > > > > > > > > CallTimeoutException:
> > > > > > > > > > > >> Call
> > > > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.
> > > > > cleanupCall
> > > > > > > > > > > >> s(RpcClient.java:1234)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.
> > > > > readRespons
> > > > > > > > > > > >> e(RpcClient.java:1171)
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > at
> > > > > > > org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
> > > > > > > > > > > >> nt.java:751)
> > > > > > > > > > > >> > Beside this same issue, please note that first
> > message
> > > > was
> > > > > > at
> > > > > > > > > > > 2015-03-20
> > > > > > > > > > > >> > 14:17:26,015. And then (we got to the point when
> it
> > > > > started
> > > > > > > > > > > transition):
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 2015-03-20 14:32:35,059 INFO
> > > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> > task
> > > > > > > > > > > >> > /hbase/splitWAL/WALs%2F
> > > > > > > > > > > >> >
> > > > > > > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > > > > > > > > >> >
> > > {name_of_node_I_took_down}%252C60020%252C1426860403261.
> > > > > 14268
> > > > > > > > > > 60404905
> > > > > > > > > > > >> > entered state: DONE
> > > > > > > {fqdn_of_new_live_node},60020,1426859445623
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 2015-03-20 14:32:35,109 INFO
> > > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> > Done
> > > > > > splitting
> > > > > > > > > > > >> > /hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > > > > > > > >> >
> > > > > > > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}
> > > > > > > > > > > >> > %252C60020%252C1426860403261.1426860404905
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 2015-03-20 14:32:35,190 INFO
> > > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> > > finished
> > > > > > > > splitting
> > > > > > > > > > > >> (more
> > > > > > > > > > > >> > than or equal to) 9 bytes in 1 log files in
> > > > > > > > > > > >> >
> > > > > > > > > > > >> >
> > > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_
> > > > > of_
> > > > > > > > > > > >> node_I_took_down},60020,1426860403261-splitting]
> > > > > > > > > > > >> > in 909083ms
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > > > > org.apache.hadoop.hbase.master
> > > > > > > > > > > >> .RegionStates:
> > > > > > > > > > > >> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c
> > > > > state=OPEN,
> > > > > > > > > > > >> > ts=1426860639088,
> > > > > > > > > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > > > > >> to
> > > > > > > > > > > >> > {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE,
> > > > > > > > ts=1426861955191,
> > > > > > > > > > > >> server=
> > > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > > > > org.apache.hadoop.hbase.master
> > > > > > > > > > > >> .RegionStates:
> > > > > > > > > > > >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
> > > > > > > > > > > >> {name_of_node_I_took_down}
> > > > > > > > > > > >> > ,60020,1426860403261
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > > > > org.apache.hadoop.hbase.master
> > > > > > > > > > > >> .RegionStates:
> > > > > > > > > > > >> > Transitioned {25ab6e7b42e36ddaa723d71be5954543
> > > > > state=OPEN,
> > > > > > > > > > > >> > ts=1426860641783,
> > > > > > > > > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > > > > >> to
> > > > > > > > > > > >> > {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE,
> > > > > > > > ts=1426861955191,
> > > > > > > > > > > >> server=
> > > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > > > > org.apache.hadoop.hbase.master
> > > > > > > > > > > >> .RegionStates:
> > > > > > > > > > > >> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
> > > > > > > > > > > >> {name_of_node_I_took_down}
> > > > > > > > > > > >> > ,60020,1426860403261
> > > > > > > > > > > >> > At this point, note that it finished
> SplitLogManager
> > > > task
> > > > > at
> > > > > > > > > > 14:32:35
> > > > > > > > > > > >> and
> > > > > > > > > > > >> > started transitioning just after that. So this is
> 15
> > > > > minutes
> > > > > > > > that
> > > > > > > > > > I'm
> > > > > > > > > > > >> > talking about.
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > What am I missing?
> > > > > > > > > > > >> >
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <
> > > > > > > > > nkeywal@gmail.com>
> > > > > > > > > > > >> wrote:
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > > You've changed the value of
> > hbase.zookeeper.timeout
> > > to
> > > > > 15
> > > > > > > > > > minutes? A
> > > > > > > > > > > >> very
> > > > > > > > > > > >> > > reasonable target is 1 minute before relocating
> > the
> > > > > > regions.
> > > > > > > > > > That's
> > > > > > > > > > > >> the
> > > > > > > > > > > >> > > default iirc. You can push it to 20s, but then
> > > > > > > > > > gc-stopping-the-world
> > > > > > > > > > > >> > > becomes more of an issue. 15 minutes is really a
> > > lot.
> > > > > The
> > > > > > > hdfs
> > > > > > > > > > stale
> > > > > > > > > > > >> mode
> > > > > > > > > > > >> > > must always be used, with a lower timeout than
> the
> > > > hbase
> > > > > > > one.
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > Client side there should be nothing to do
> > (excepted
> > > > for
> > > > > > > > advanced
> > > > > > > > > > > >> stuff);
> > > > > > > > > > > >> > at
> > > > > > > > > > > >> > > each retry the client checks the location of the
> > > > > regions.
> > > > > > If
> > > > > > > > you
> > > > > > > > > > > lower
> > > > > > > > > > > >> > the
> > > > > > > > > > > >> > > number of retry the client will fail sooner, but
> > > > usually
> > > > > > you
> > > > > > > > > don't
> > > > > > > > > > > >> want
> > > > > > > > > > > >> > the
> > > > > > > > > > > >> > > client to fail, you want the servers to
> reallocate
> > > > > > quickly.
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <
> > > > > > > > > > > dejan.menges@gmail.com
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > > wrote:
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > > Hi,
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > Sorry for little bit late update, but managed
> to
> > > > > narrow
> > > > > > it
> > > > > > > > > > little
> > > > > > > > > > > >> bit
> > > > > > > > > > > >> > > down.
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > We didn't update yet, as we are using
> > Hortonworks
> > > > > > > > distribution
> > > > > > > > > > > right
> > > > > > > > > > > >> > now,
> > > > > > > > > > > >> > > > and even if we update we will get 0.98.4.
> > However,
> > > > > looks
> > > > > > > > that
> > > > > > > > > > > issue
> > > > > > > > > > > >> > here
> > > > > > > > > > > >> > > > was in our use case and configuration (still
> > > looking
> > > > > > into
> > > > > > > > it).
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > Basically, initially I saw that when one
> server
> > > goes
> > > > > > down,
> > > > > > > > we
> > > > > > > > > > > start
> > > > > > > > > > > >> > > having
> > > > > > > > > > > >> > > > performance issues in general, but it managed
> to
> > > be
> > > > on
> > > > > > our
> > > > > > > > > > client
> > > > > > > > > > > >> side,
> > > > > > > > > > > >> > > due
> > > > > > > > > > > >> > > > to caching, and clients were trying to
> reconnect
> > > to
> > > > > > nodes
> > > > > > > > that
> > > > > > > > > > > were
> > > > > > > > > > > >> > > offline
> > > > > > > > > > > >> > > > and later trying to get regions from those
> nodes
> > > > too.
> > > > > > This
> > > > > > > > is
> > > > > > > > > > > >> basically
> > > > > > > > > > > >> > > why
> > > > > > > > > > > >> > > > on server side I didn't manage to see anything
> > in
> > > > logs
> > > > > > > that
> > > > > > > > > > would
> > > > > > > > > > > >> be at
> > > > > > > > > > > >> > > > least little bit interesting and point me into
> > > > desired
> > > > > > > > > > direction.
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > Another question that popped up to me is - in
> > case
> > > > > > server
> > > > > > > is
> > > > > > > > > > down
> > > > > > > > > > > >> (and
> > > > > > > > > > > >> > > with
> > > > > > > > > > > >> > > > it DataNode and HRegionServer it was hosting)
> -
> > > > what's
> > > > > > > > optimal
> > > > > > > > > > > time
> > > > > > > > > > > >> to
> > > > > > > > > > > >> > > set
> > > > > > > > > > > >> > > > for HMaster to consider server dead reassign
> > > regions
> > > > > > > > somewhere
> > > > > > > > > > > >> else, as
> > > > > > > > > > > >> > > > this is another performance bottleneck we hit
> > > during
> > > > > > > > inability
> > > > > > > > > > to
> > > > > > > > > > > >> > access
> > > > > > > > > > > >> > > > regions? In our case it's configured to 15
> > > minutes,
> > > > > and
> > > > > > > > simple
> > > > > > > > > > > logic
> > > > > > > > > > > >> > > tells
> > > > > > > > > > > >> > > > me if you want it earlier then configure lower
> > > > number
> > > > > of
> > > > > > > > > > retries,
> > > > > > > > > > > >> but
> > > > > > > > > > > >> > > issue
> > > > > > > > > > > >> > > > is as always in details, so not sure if anyone
> > > knows
> > > > > > some
> > > > > > > > > better
> > > > > > > > > > > >> math
> > > > > > > > > > > >> > for
> > > > > > > > > > > >> > > > this?
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > And last question - is it possible to manually
> > > force
> > > > > > HBase
> > > > > > > > to
> > > > > > > > > > > >> reassign
> > > > > > > > > > > >> > > > regions? In this case, while HMaster is
> retrying
> > > to
> > > > > > > contact
> > > > > > > > > node
> > > > > > > > > > > >> that's
> > > > > > > > > > > >> > > > dead, it's impossible to force it using
> > 'balancer'
> > > > > > > command.
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > Thanks a lot!
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > Dejan
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <
> > > > > > > > > > > >> dejan.menges@gmail.com>
> > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > > Hi,
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > > > To be very honest - there's no particular
> > reason
> > > > why
> > > > > > we
> > > > > > > > > stick
> > > > > > > > > > to
> > > > > > > > > > > >> this
> > > > > > > > > > > >> > > > one,
> > > > > > > > > > > >> > > > > beside just lack of time currently to go
> > through
> > > > > > upgrade
> > > > > > > > > > > process,
> > > > > > > > > > > >> but
> > > > > > > > > > > >> > > > looks
> > > > > > > > > > > >> > > > > to me that's going to be next step.
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > > > Had a crazy day, didn't have time to go
> > through
> > > > all
> > > > > > logs
> > > > > > > > > > again,
> > > > > > > > > > > >> plus
> > > > > > > > > > > >> > > one
> > > > > > > > > > > >> > > > > of the machines (last one where we had this
> > > issue)
> > > > > is
> > > > > > > > fully
> > > > > > > > > > > >> > > reprovisioned
> > > > > > > > > > > >> > > > > yesterday so I don't have logs from there
> > > anymore.
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > > > Beside upgrading,  what I will talk about
> > today,
> > > > can
> > > > > > you
> > > > > > > > > just
> > > > > > > > > > > >> point
> > > > > > > > > > > >> > me
> > > > > > > > > > > >> > > to
> > > > > > > > > > > >> > > > > the specific RPC issue in 0.98.0? Thing is
> > that
> > > we
> > > > > > have
> > > > > > > > some
> > > > > > > > > > > >> strange
> > > > > > > > > > > >> > > > > moments with RPC in this case, and just want
> > to
> > > > see
> > > > > if
> > > > > > > > > that's
> > > > > > > > > > > the
> > > > > > > > > > > >> > same
> > > > > > > > > > > >> > > > > thing (and we were even suspecting to RPC).
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > > > Thanks a lot!
> > > > > > > > > > > >> > > > > Dejan
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew
> > Purtell
> > > <
> > > > > > > > > > > >> apurtell@apache.org
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > > > wrote:
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > > >> Is there a particular reason why you are
> > using
> > > > > HBase
> > > > > > > > > 0.98.0?
> > > > > > > > > > > The
> > > > > > > > > > > >> > > latest
> > > > > > > > > > > >> > > > >> 0.98 release is 0.98.11. There's a known
> > > > > performance
> > > > > > > > issue
> > > > > > > > > > with
> > > > > > > > > > > >> > 0.98.0
> > > > > > > > > > > >> > > > >> pertaining to RPC that was fixed in later
> > > > releases,
> > > > > > you
> > > > > > > > > > should
> > > > > > > > > > > >> move
> > > > > > > > > > > >> > up
> > > > > > > > > > > >> > > > >> from
> > > > > > > > > > > >> > > > >> 0.98.0. In addition hundreds of
> improvements
> > > and
> > > > > bug
> > > > > > > > fixes
> > > > > > > > > > have
> > > > > > > > > > > >> gone
> > > > > > > > > > > >> > > > into
> > > > > > > > > > > >> > > > >> the ten releases since 0.98.0.
> > > > > > > > > > > >> > > > >>
> > > > > > > > > > > >> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan
> > Menges <
> > > > > > > > > > > >> > dejan.menges@gmail.com
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > >> wrote:
> > > > > > > > > > > >> > > > >>
> > > > > > > > > > > >> > > > >> > Hi All,
> > > > > > > > > > > >> > > > >> >
> > > > > > > > > > > >> > > > >> > We have a strange issue with HBase
> > > performance
> > > > > > > (overall
> > > > > > > > > > > cluster
> > > > > > > > > > > >> > > > >> > performance) in case one of datanodes in
> > the
> > > > > > cluster
> > > > > > > > > > > >> unexpectedly
> > > > > > > > > > > >> > > goes
> > > > > > > > > > > >> > > > >> > down.
> > > > > > > > > > > >> > > > >> >
> > > > > > > > > > > >> > > > >> > So scenario is like follows:
> > > > > > > > > > > >> > > > >> > - Cluster works fine, it's stable.
> > > > > > > > > > > >> > > > >> > - One DataNode unexpectedly goes down
> (PSU
> > > > issue,
> > > > > > > > network
> > > > > > > > > > > >> issue,
> > > > > > > > > > > >> > > > >> anything)
> > > > > > > > > > > >> > > > >> > - Whole HBase cluster goes down
> > (performance
> > > > > > becomes
> > > > > > > so
> > > > > > > > > bad
> > > > > > > > > > > >> that
> > > > > > > > > > > >> > we
> > > > > > > > > > > >> > > > >> have to
> > > > > > > > > > > >> > > > >> > restart all RegionServers to get it back
> to
> > > > > life).
> > > > > > > > > > > >> > > > >> >
> > > > > > > > > > > >> > > > >> > Most funny and latest issue that happened
> > was
> > > > > that
> > > > > > we
> > > > > > > > > added
> > > > > > > > > > > new
> > > > > > > > > > > >> > node
> > > > > > > > > > > >> > > > to
> > > > > > > > > > > >> > > > >> the
> > > > > > > > > > > >> > > > >> > cluster (having 8 x 4T SATA disks) and we
> > > left
> > > > > just
> > > > > > > > > > DataNode
> > > > > > > > > > > >> > running
> > > > > > > > > > > >> > > > on
> > > > > > > > > > > >> > > > >> it
> > > > > > > > > > > >> > > > >> > to give it couple of days to get some
> data.
> > > At
> > > > > some
> > > > > > > > point
> > > > > > > > > > in
> > > > > > > > > > > >> time,
> > > > > > > > > > > >> > > due
> > > > > > > > > > > >> > > > >> to
> > > > > > > > > > > >> > > > >> > hardware issue, server rebooted (twice
> > during
> > > > > three
> > > > > > > > > hours)
> > > > > > > > > > in
> > > > > > > > > > > >> > moment
> > > > > > > > > > > >> > > > >> when
> > > > > > > > > > > >> > > > >> > it had maybe 5% of data it would have in
> a
> > > > couple
> > > > > > of
> > > > > > > > > days.
> > > > > > > > > > > >> Nothing
> > > > > > > > > > > >> > > > else
> > > > > > > > > > > >> > > > >> > beside DataNode was running, and once it
> > went
> > > > > down,
> > > > > > > it
> > > > > > > > > > > affected
> > > > > > > > > > > >> > > > literary
> > > > > > > > > > > >> > > > >> > everything, and restarting RegionServers
> in
> > > the
> > > > > end
> > > > > > > > fixed
> > > > > > > > > > it.
> > > > > > > > > > > >> > > > >> >
> > > > > > > > > > > >> > > > >> > We are using HBase 0.98.0 with Hadoop
> 2.4.0
> > > > > > > > > > > >> > > > >> >
> > > > > > > > > > > >> > > > >>
> > > > > > > > > > > >> > > > >>
> > > > > > > > > > > >> > > > >>
> > > > > > > > > > > >> > > > >> --
> > > > > > > > > > > >> > > > >> Best regards,
> > > > > > > > > > > >> > > > >>
> > > > > > > > > > > >> > > > >>    - Andy
> > > > > > > > > > > >> > > > >>
> > > > > > > > > > > >> > > > >> Problems worthy of attack prove their worth
> > by
> > > > > > hitting
> > > > > > > > > back.
> > > > > > > > > > -
> > > > > > > > > > > >> Piet
> > > > > > > > > > > >> > > Hein
> > > > > > > > > > > >> > > > >> (via Tom White)
> > > > > > > > > > > >> > > > >>
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> >
> > > > > > > > > > > >>
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Strange issue when DataNode goes down

Posted by Dejan Menges <de...@gmail.com>.
I'm surprised by this as well. Staleness was configured also, MTTR from
HBase book as described, and in this specific case - when machine really
dies - even when NN makes DataNode dead, HBase was trying to replay WALs
from dead node until timeout reached.

Still reading this HDFS-3703, trying to get more context, but looks to me
so far that this is one of those things where you should decide what are
your own acceptable tradeoffs.

On Mon, Mar 23, 2015 at 4:40 PM Bryan Beaudreault <bb...@hubspot.com>
wrote:

> @Nicholas, I see, thanks.  I'll keep the settings at default.  So really if
> everything else is configured properly you should never reach the lease
> recovery timeout in any failure scenarios.  It seems that the staleness
> check would be the thing that prevents this, correct?  I'm surprised it
> didn't help Dejan.
>
> On Mon, Mar 23, 2015 at 11:20 AM, Nicolas Liochon <nk...@gmail.com>
> wrote:
>
> > @bryan: yes, you can change hbase.lease.recovery.timeout if you changed
> he
> > hdfs settings. But this setting is really for desperate cases. The
> recover
> > Lease should have succeeded before. As well, if you depend on
> > hbase.lease.recovery.timeout, it means that you're wasting recovery time:
> > the lease should be recovered in a few seconds.
> >
> > On Mon, Mar 23, 2015 at 3:59 PM, Dejan Menges <de...@gmail.com>
> > wrote:
> >
> > > Interesting discussion I also found, gives me some more light on what
> > > Nicolas is talking about -
> > https://issues.apache.org/jira/browse/HDFS-3703
> > >
> > > On Mon, Mar 23, 2015 at 3:53 PM Bryan Beaudreault <
> > > bbeaudreault@hubspot.com>
> > > wrote:
> > >
> > > > So it is safe to set hbase.lease.recovery.timeout lower if you also
> > > > set heartbeat.recheck.interval lower (lowering that 10.5 min dead
> node
> > > > timer)?  Or is it recommended to not touch either of those?
> > > >
> > > > Reading the above with interest, thanks for digging in here guys.
> > > >
> > > > On Mon, Mar 23, 2015 at 10:13 AM, Nicolas Liochon <nkeywal@gmail.com
> >
> > > > wrote:
> > > >
> > > > > If the node is actually down it's fine. But the node may not be
> that
> > > down
> > > > > (CAP theorem here); and then it's looking for trouble.
> > > > > HDFS, by default declare a node as dead after 10:30. 15 minutes is
> an
> > > > extra
> > > > > security. It seems your hdfs settings are different (or there is a
> > > > bug...).
> > > > > There should be some info in the hdfs logs.
> > > > >
> > > > > On Mon, Mar 23, 2015 at 3:05 PM, Dejan Menges <
> > dejan.menges@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Will take a look.
> > > > > >
> > > > > > Actually, if node is down (someone unplugged network cable, it
> just
> > > > died,
> > > > > > whatever) what's chance it's going to become live so write can
> > > > continue?
> > > > > On
> > > > > > the other side, HBase is not starting recovery trying to contact
> > node
> > > > > which
> > > > > > is not there anymore, and even elected as dead on Namenode side
> > > > (another
> > > > > > thing I didn't understood quite good).
> > > > > >
> > > > > > So what I was expecting is that as soon as Namenode decided node
> is
> > > > dead,
> > > > > > that it would be enough for RegionServer to stop trying to
> recover
> > > from
> > > > > the
> > > > > > dead node, but it wasn't the case. Also, this whole MTTR article
> in
> > > > HBase
> > > > > > book doesn't work at all with this parameter set to it's default
> > > value
> > > > > (15
> > > > > > minutes).
> > > > > >
> > > > > > So I'm still struggling to figure out what's drawback exactly on
> > > this?
> > > > > >
> > > > > > On Mon, Mar 23, 2015 at 2:50 PM Nicolas Liochon <
> nkeywal@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > Thanks for the explanation. There is an issue if you modify
> this
> > > > > setting
> > > > > > > however.
> > > > > > > hbase tries to recover the lease (i.e. be sure that nobody is
> > > > writing)
> > > > > > > If you change hbase.lease.recovery.timeout hbase will start the
> > > > > recovery
> > > > > > > (i.e. start to read) even if it's not sure that nobody's
> writing.
> > > > That
> > > > > > > means there is a dataloss risk.
> > > > > > > Basically, you must not see this warning: WARN
> > > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils: Cannot recoverLease
> > after
> > > > > > trying
> > > > > > > for[]
> > > > > > >
> > > > > > > The recoverLease must succeed. The fact that it does not after
> X
> > > > tries
> > > > > is
> > > > > > > strange.
> > > > > > > There may be a mistmatch between the hbase parameters and the
> > hdfs
> > > > > ones.
> > > > > > > You may need to have a look at the comments in FSHDFSUtils.java
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Mon, Mar 23, 2015 at 2:15 PM, Dejan Menges <
> > > > dejan.menges@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > I found the issue and fixed it, and will try to explain here
> > what
> > > > was
> > > > > > > > exactly in our case in case someone else finds this
> interesting
> > > > too.
> > > > > > > >
> > > > > > > > So initially, we had (couple of times) some network and
> > hardware
> > > > > issues
> > > > > > > in
> > > > > > > > our datacenters. When one server would die (literary die, we
> > had
> > > > some
> > > > > > > issue
> > > > > > > > with PSUs) we saw issues with overall cluster performance on
> > > HBase
> > > > > > side.
> > > > > > > As
> > > > > > > > cluster is quite big and live, it was also quite hard to
> figure
> > > out
> > > > > > exact
> > > > > > > > root cause and how to fix the stuff we wanted to fix.
> > > > > > > >
> > > > > > > > So I set up another cluster, four nodes (with DataNode and
> > > > > > RegionServer)
> > > > > > > > and two other nodes with HMaster and Namenode in HA, using
> same
> > > > stuff
> > > > > > we
> > > > > > > > use on production. We pumped some data into it, and I was
> able
> > to
> > > > > > > reproduce
> > > > > > > > same issue last week on it. What I tried to do is to cut one
> > > server
> > > > > > (shut
> > > > > > > > down it's interface) when all is good with cluster, when we
> > have
> > > > > load,
> > > > > > > and
> > > > > > > > see what's going to happen.
> > > > > > > >
> > > > > > > > On Friday, after Nicolas mentioned, I started taking a look
> in
> > > > HBase
> > > > > > logs
> > > > > > > > on the node which was mentioned in HMaster log as the one
> > taking
> > > > over
> > > > > > > > regions for the dead server. Basically what I was able to
> > observe
> > > > was
> > > > > > 15
> > > > > > > > minutes time (+- couple of seconds, what was also
> interesting,
> > > and
> > > > > will
> > > > > > > got
> > > > > > > > later to that) between HMaster figures out that one of it's
> > > > > > RegionServers
> > > > > > > > is dead, and the time one of the mentioned nodes starts
> taking
> > > over
> > > > > > those
> > > > > > > > regions and they start appearing in HMaster's Web UI.
> > > > > > > >
> > > > > > > > I then set up everything like mentioned here
> > > > > > > > http://hbase.apache.org/book.html#mttr - but still had
> exactly
> > > the
> > > > > > same
> > > > > > > > issues. Went over (again and again) all currently configured
> > > stuff,
> > > > > but
> > > > > > > > still had the same issue.
> > > > > > > >
> > > > > > > > Then I started looking into HDFS. Opened NameNode UI, saw all
> > is
> > > > > good,
> > > > > > > took
> > > > > > > > one node down, was also looking RegionServer logs in the same
> > > time
> > > > -
> > > > > > and
> > > > > > > I
> > > > > > > > also see that it took ~15 minutes for Namenode to elect dead
> > node
> > > > as
> > > > > > > dead.
> > > > > > > > Somehow in the same moment regions started getting back to
> > life.
> > > I
> > > > > > > remember
> > > > > > > > in some older versions dfs timeout checks and check retries.
> > > Looked
> > > > > > into
> > > > > > > > defaults for our Hadoop version -
> > > > > > > >
> > > > > > > > http://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/
> > > > > > > hadoop-hdfs/hdfs-default.xml
> > > > > > > > - and saw there that there's no recheck parameter anymore.
> > > Strange,
> > > > > as
> > > > > > on
> > > > > > > > StackOverflow I found post from month ago, for newer version
> > than
> > > > we
> > > > > > use
> > > > > > > > (we use 2.4.1, guy was using 2.6 - dfs.namenode.heartbeat.
> > > > > > > recheck-interval)
> > > > > > > > I set it to 10 seconds as he mentioned, having checks every
> > three
> > > > > > seconds
> > > > > > > > (default) and got DataNode elected as dead in ~45 seconds, as
> > he
> > > > > > > mentioned.
> > > > > > > > Not sure why this parameter is not documented, but obviously
> it
> > > > > works.
> > > > > > > >
> > > > > > > > Then figured out it still didn't fix our HBase failover
> issue.
> > I
> > > > was
> > > > > > > > looking into HBase book again and again, and then saw this
> > part:
> > > > > > > >
> > > > > > > > "How much time we allow elapse between calls to recover
> lease.
> > > > Should
> > > > > > be
> > > > > > > > larger than the dfs timeout."
> > > > > > > >
> > > > > > > > This was description for hbase.lease.recovery.dfs.timeout.
> > Wasn't
> > > > > sure
> > > > > > > from
> > > > > > > > the comment what of all timeouts that's possible to set in
> > > > > Hadoop/HBase
> > > > > > > and
> > > > > > > > that has something to do with DFS is this all about. But
> picked
> > > > > > > > hbase.lease.recovery.timeout, and that was the catch.
> > > > > > > >
> > > > > > > > Initially, by default, hbase.lease.recovery.timeout is set to
> > 15
> > > > > > minutes.
> > > > > > > > Not sure why, and wasn't able to find yet why, but getting
> this
> > > > down
> > > > > to
> > > > > > > one
> > > > > > > > minute (what's more than OK for us) I was able to get rid of
> > our
> > > > > issue.
> > > > > > > Not
> > > > > > > > also sure why this is not mentioned in MTTR section in HBase
> > > book,
> > > > as
> > > > > > > > obviously MTTR doesn't work at all with this default timeout,
> > at
> > > > > least
> > > > > > it
> > > > > > > > doesn't work the way we expected it to work.
> > > > > > > >
> > > > > > > > So thanks again for everyone being spammed with this, and
> > > specially
> > > > > > > thanks
> > > > > > > > to Nicolas pointing me to the right direction.
> > > > > > > >
> > > > > > > > On Mon, Mar 23, 2015 at 1:37 PM Nicolas Liochon <
> > > nkeywal@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > the attachments are rejected by the mailing list, can you
> put
> > > > then
> > > > > on
> > > > > > > > > pastebin?
> > > > > > > > >
> > > > > > > > > stale is mandatory (so it's good), but the issue here is
> just
> > > > > before.
> > > > > > > The
> > > > > > > > > region server needs to read the file. In order to be sure
> > that
> > > > > there
> > > > > > is
> > > > > > > > no
> > > > > > > > > data loss, it needs to "recover the lease", that means
> > ensuring
> > > > > that
> > > > > > > > nobody
> > > > > > > > > is writing the file. The regionserver calls the namenode to
> > do
> > > > this
> > > > > > > > > recoverLease. So there should be some info in the namenode
> > > logs.
> > > > > You
> > > > > > > have
> > > > > > > > > HDFS-4721 on your hdfs? The hbase book details (more or
> > > less...)
> > > > > this
> > > > > > > > > recoverLease stuff.
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <
> > > > > > dejan.menges@gmail.com
> > > > > > > >
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > And also, just checked - dfs.namenode.avoid.read.stale.
> > > > datanode
> > > > > and
> > > > > > > > > > dfs.namenode.avoid.write.stale.datanode
> > > > > > > > > > are both true, and dfs.namenode.stale.datanode.interval
> is
> > > > set to
> > > > > > > > > default
> > > > > > > > > > 30000.
> > > > > > > > > >
> > > > > > > > > > On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <
> > > > > > > dejan.menges@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Hi Nicolas,
> > > > > > > > > > >
> > > > > > > > > > > Please find log attached.
> > > > > > > > > > >
> > > > > > > > > > > As I see it now more clearly, it was trying to recover
> > HDFS
> > > > > WALs
> > > > > > > from
> > > > > > > > > > node
> > > > > > > > > > > that's dead:
> > > > > > > > > > >
> > > > > > > > > > > 2015-03-23 08:53:44,381 WARN
> > > > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils:
> > > > > > > > > > > Cannot recoverLease after trying for 900000ms
> > > > > > > > > > > (hbase.lease.recovery.timeout); continuing, but may be
> > > > > > > DATALOSS!!!;
> > > > > > > > > > > attempt=40 on
> > > > > > > > > > >
> > > > > > > > > >
> > file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intent
> > > > > > > > > ionally_get_down_by_getting_network_down},60020,1426862900
> > > > > > > > > 506-splitting/{node_i_intentionally_get_down_by_gett
> > > > > > > > > ing_network_down}%2C60020%2C1426862900506.1427096924508
> > > > > > > > > > > after 908210ms
> > > > > > > > > > >
> > > > > > > > > > > And as you can see from the log, it tried 40 times,
> what
> > > took
> > > > > it
> > > > > > > > > exactly
> > > > > > > > > > > 15 minutes.
> > > > > > > > > > >
> > > > > > > > > > > There's probably some parameter to tune to cut it of
> from
> > > 40
> > > > > > times
> > > > > > > /
> > > > > > > > 15
> > > > > > > > > > > minutes to something more useful, as for 15 minutes we
> > > don't
> > > > > have
> > > > > > > our
> > > > > > > > > > > regions available, and HDFS have however replication
> > factor
> > > > 3.
> > > > > > > > > > >
> > > > > > > > > > > Googling, if I figure out what's this I will post it
> > here.
> > > > Will
> > > > > > > also
> > > > > > > > > > > appreciate if someone knows how to cut this down.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > Dejan
> > > > > > > > > > >
> > > > > > > > > > > On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <
> > > > > > nkeywal@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > >> The split is done by the region servers (the master
> > > > > > coordinates).
> > > > > > > Is
> > > > > > > > > > there
> > > > > > > > > > >> some interesting stuff in their logs?
> > > > > > > > > > >>
> > > > > > > > > > >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <
> > > > > > > > dejan.menges@gmail.com
> > > > > > > > > >
> > > > > > > > > > >> wrote:
> > > > > > > > > > >>
> > > > > > > > > > >> > With client issue was that it was retrying
> connecting
> > to
> > > > the
> > > > > > > same
> > > > > > > > > > region
> > > > > > > > > > >> > servers even when they were reassigned. Lowering it
> > down
> > > > > > helped
> > > > > > > in
> > > > > > > > > > this
> > > > > > > > > > >> > specific use case, but yes, we still want servers to
> > > > > > reallocate
> > > > > > > > > > quickly.
> > > > > > > > > > >> >
> > > > > > > > > > >> > What got me here:
> > > > > > > > > > >> >
> > > > > > > > > > >> > http://hbase.apache.org/book.html#mttr
> > > > > > > > > > >> >
> > > > > > > > > > >> > I basically set configuration exactly the same way
> as
> > > it's
> > > > > > > > explained
> > > > > > > > > > >> here.
> > > > > > > > > > >> > *zookeeper.session.timeout* is (and was before)
> 60000
> > > (one
> > > > > > > > minute).
> > > > > > > > > > >> >
> > > > > > > > > > >> > So basically what happens is: - simulating network
> > > issues
> > > > we
> > > > > > had
> > > > > > > > > > >> recently).
> > > > > > > > > > >> > - After short time I see in HBase that my
> RegionServer
> > > is
> > > > > > dead,
> > > > > > > > and
> > > > > > > > > as
> > > > > > > > > > >> > total number of regions I see previous total minus
> > > number
> > > > of
> > > > > > > > regions
> > > > > > > > > > >> that
> > > > > > > > > > >> > were hosted on the node hosting RegionServer that
> just
> > > > > > > > > 'disappeared'.
> > > > > > > > > > >> > - In this point I want my clus
> > > > > > > > > > >> >
> > > > > > > > > > >> > - I have test cluster consisting of four nodes,
> every
> > > node
> > > > > > being
> > > > > > > > > > >> DataNode
> > > > > > > > > > >> > and RegionServer.
> > > > > > > > > > >> > - I simulate network partition on one (connect to it
> > > > through
> > > > > > > > console
> > > > > > > > > > and
> > > > > > > > > > >> > take network interface downter to recover as soon as
> > > > > possible,
> > > > > > > to
> > > > > > > > > > start
> > > > > > > > > > >> > serving missing regions.
> > > > > > > > > > >> > - First thing I see in HMaster logs are:
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:26,015 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.
> zookeeper.RegionServerTracker:
> > > > > > > > RegionServer
> > > > > > > > > > >> > ephemeral node deleted, processing expiration
> > > > > > > > > > >> > [{name_of_node_I_took_down},60020,1426860403261]
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:26,067 INFO
> > > > > > > > > > >> >
> > > > > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler:
> > > > > > > > > > Splitting
> > > > > > > > > > >> > logs for
> > {name_of_node_I_took_down},60020,1426860403261
> > > > > before
> > > > > > > > > > >> assignment.
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:26,105 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> dead
> > > > > splitlog
> > > > > > > > > workers
> > > > > > > > > > [
> > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261]
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:26,107 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> > started
> > > > > > > splitting
> > > > > > > > 1
> > > > > > > > > > >> logs in
> > > > > > > > > > >> >
> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_
> > > > I_
> > > > > > > > > took_down}
> > > > > > > > > > >> > ,60020,1426860403261-splitting]
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:26,150 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> task
> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F
> > > > > > > > > > >> >
> > > > > > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > > > > > > > >> >
> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
> > > > 14268
> > > > > > > > > 60404905
> > > > > > > > > > >> > acquired by {fqdn_of_another_live_hnode},
> > > > 60020,1426859445623
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:26,182 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> total
> > > > tasks
> > > > > =
> > > > > > 1
> > > > > > > > > > >> unassigned
> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > > 2F{name_of_node_I_took_down}
> > > > > > > > > > >> >
> > > > > > > > > > >> > %2C60020%2C1426860403261-
> splitting%2F{name_of_node_I_
> > > > took_
> > > > > > > > > > >>
> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> > status =
> > > > > > > > > in_progress
> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
> > done
> > > > = 0
> > > > > > > > error =
> > > > > > > > > > 0}
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:31,183 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> total
> > > > tasks
> > > > > =
> > > > > > 1
> > > > > > > > > > >> unassigned
> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > > 2F{name_of_node_I_took_down}
> > > > > > > > > > >> >
> > > > > > > > > > >> > %2C60020%2C1426860403261-
> splitting%2F{name_of_node_I_
> > > > took_
> > > > > > > > > > >>
> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> > status =
> > > > > > > > > in_progress
> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
> > done
> > > > = 0
> > > > > > > > error =
> > > > > > > > > > 0}
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:36,184 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> total
> > > > tasks
> > > > > =
> > > > > > 1
> > > > > > > > > > >> unassigned
> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > > 2F{name_of_node_I_took_down}
> > > > > > > > > > >> >
> > > > > > > > > > >> > %2C60020%2C1426860403261-
> splitting%2F{name_of_node_I_
> > > > took_
> > > > > > > > > > >>
> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> > status =
> > > > > > > > > in_progress
> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
> > done
> > > > = 0
> > > > > > > > error =
> > > > > > > > > > 0}
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:42,185 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> total
> > > > tasks
> > > > > =
> > > > > > 1
> > > > > > > > > > >> unassigned
> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > > 2F{name_of_node_I_took_down}
> > > > > > > > > > >> >
> > > > > > > > > > >> > %2C60020%2C1426860403261-
> splitting%2F{name_of_node_I_
> > > > took_
> > > > > > > > > > >>
> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> > status =
> > > > > > > > > in_progress
> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
> > done
> > > > = 0
> > > > > > > > error =
> > > > > > > > > > 0}
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:17:48,184 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> total
> > > > tasks
> > > > > =
> > > > > > 1
> > > > > > > > > > >> unassigned
> > > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > > 2F{name_of_node_I_took_down}
> > > > > > > > > > >> >
> > > > > > > > > > >> > %2C60020%2C1426860403261-
> splitting%2F{name_of_node_I_
> > > > took_
> > > > > > > > > > >>
> > > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> > status =
> > > > > > > > > in_progress
> > > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
> > done
> > > > = 0
> > > > > > > > error =
> > > > > > > > > > 0}
> > > > > > > > > > >> > In the meantime, In hbase...out log I got this:
> > > > > > > > > > >> >
> > > > > > > > > > >> > ==> hbase-hbase-master-{fqdn_of_
> my_hmaster_node}.out
> > <==
> > > > > > > > > > >> >
> > > > > > > > > > >> > java.io.IOException: Call to
> > > > > > > > > > >> >
> > > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_
> > > > dow
> > > > > > > > > n}:60020
> > > > > > > > > > >> > failed on local exception:
> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> > > > CallTimeoutException:
> > > > > > > Call
> > > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
> > > > > > > > > > >> t.java:1532)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:
> > > > > > > 1502)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> > > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(
> > > > Rpc
> > > > > > > > > > >> Client.java:1684)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> > > > BlockingRpcChannelImpl
> > > > > > > > > ementati
> > > > > > > > > > >> on.callBlockingMethod(RpcClient.java:1737)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$
> > > > > > > > > AdminService$
> > > > > > > > > > >> BlockingStub.getRegionInfo(AdminProtos.java:20806)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.apache.hadoop.hbase.client.HBaseAdmin.
> > > > getCompactionState
> > > > > > > > > > >> (HBaseAdmin.java:2524)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.apache.hadoop.hbase.
> generated.master.table_jsp._
> > > > jspServi
> > > > > > > > > > >> ce(table_jsp.java:167)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.
> > > > > > > > > java:98)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
> > > > > > > > > > >> .java:511)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.mortbay.jetty.servlet.
> ServletHandler$CachedChain.
> > > > doFilte
> > > > > > > > > > >> r(ServletHandler.java:1221)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.apache.hadoop.http.lib.StaticUserWebFilter$
> > > > StaticUserFil
> > > > > > > > > > >> ter.doFilter(StaticUserWebFilter.java:109)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.mortbay.jetty.servlet.
> ServletHandler$CachedChain.
> > > > doFilte
> > > > > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.apache.hadoop.http.
> HttpServer$QuotingInputFilter.
> > > > doFilte
> > > > > > > > > > >> r(HttpServer.java:1081)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.mortbay.jetty.servlet.
> ServletHandler$CachedChain.
> > > > doFilte
> > > > > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > >
> > > > > >
> > org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.mortbay.jetty.servlet.
> ServletHandler$CachedChain.
> > > > doFilte
> > > > > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
> > > > > > > > > > >> er.java:399)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> > org.mortbay.jetty.security.SecurityHandler.handle(
> > > > SecurityHa
> > > > > > > > > > >> ndler.java:216)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
> > > > > > > > > > >> er.java:182)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
> > > > > > > > > > >> er.java:766)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
> > > > > > > > > > >> java:450)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.mortbay.jetty.handler.ContextHandlerCollection.
> > > > handle(Co
> > > > > > > > > > >> ntextHandlerCollection.java:230)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
> > > > > > > > > > >> er.java:152)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at org.mortbay.jetty.Server.handle(Server.java:326)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
> > > > > > > > > > >> n.java:542)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.mortbay.jetty.HttpConnection$RequestHandler.
> > > > headerComple
> > > > > > > > > > >> te(HttpConnection.java:928)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > >
> > org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > >
> > org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.mortbay.io.nio.SelectChannelEndPoint.run(
> > > > SelectChannelEn
> > > > > > > > > > >> dPoint.java:410)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.mortbay.thread.QueuedThreadPool$PoolThread.run(
> > > > > > > > > > >> QueuedThreadPool.java:582)
> > > > > > > > > > >> >
> > > > > > > > > > >> > Caused by: org.apache.hadoop.hbase.ipc.RpcClient$
> > > > > > > > > CallTimeoutException:
> > > > > > > > > > >> Call
> > > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.
> > > > cleanupCall
> > > > > > > > > > >> s(RpcClient.java:1234)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > > > > > >> >
> > > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.
> > > > readRespons
> > > > > > > > > > >> e(RpcClient.java:1171)
> > > > > > > > > > >> >
> > > > > > > > > > >> > at
> > > > > > org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
> > > > > > > > > > >> nt.java:751)
> > > > > > > > > > >> > Beside this same issue, please note that first
> message
> > > was
> > > > > at
> > > > > > > > > > 2015-03-20
> > > > > > > > > > >> > 14:17:26,015. And then (we got to the point when it
> > > > started
> > > > > > > > > > transition):
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:32:35,059 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> task
> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F
> > > > > > > > > > >> >
> > > > > > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > > > > > > > >> >
> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
> > > > 14268
> > > > > > > > > 60404905
> > > > > > > > > > >> > entered state: DONE
> > > > > > {fqdn_of_new_live_node},60020,1426859445623
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:32:35,109 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> Done
> > > > > splitting
> > > > > > > > > > >> > /hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > > > > > > >> >
> > > > > > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}
> > > > > > > > > > >> > %252C60020%252C1426860403261.1426860404905
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:32:35,190 INFO
> > > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> > finished
> > > > > > > splitting
> > > > > > > > > > >> (more
> > > > > > > > > > >> > than or equal to) 9 bytes in 1 log files in
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_
> > > > of_
> > > > > > > > > > >> node_I_took_down},60020,1426860403261-splitting]
> > > > > > > > > > >> > in 909083ms
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > > > org.apache.hadoop.hbase.master
> > > > > > > > > > >> .RegionStates:
> > > > > > > > > > >> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c
> > > > state=OPEN,
> > > > > > > > > > >> > ts=1426860639088,
> > > > > > > > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > > > >> to
> > > > > > > > > > >> > {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE,
> > > > > > > ts=1426861955191,
> > > > > > > > > > >> server=
> > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > > > org.apache.hadoop.hbase.master
> > > > > > > > > > >> .RegionStates:
> > > > > > > > > > >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
> > > > > > > > > > >> {name_of_node_I_took_down}
> > > > > > > > > > >> > ,60020,1426860403261
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > > > org.apache.hadoop.hbase.master
> > > > > > > > > > >> .RegionStates:
> > > > > > > > > > >> > Transitioned {25ab6e7b42e36ddaa723d71be5954543
> > > > state=OPEN,
> > > > > > > > > > >> > ts=1426860641783,
> > > > > > > > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > > > >> to
> > > > > > > > > > >> > {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE,
> > > > > > > ts=1426861955191,
> > > > > > > > > > >> server=
> > > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > > > >> >
> > > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > > > org.apache.hadoop.hbase.master
> > > > > > > > > > >> .RegionStates:
> > > > > > > > > > >> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
> > > > > > > > > > >> {name_of_node_I_took_down}
> > > > > > > > > > >> > ,60020,1426860403261
> > > > > > > > > > >> > At this point, note that it finished SplitLogManager
> > > task
> > > > at
> > > > > > > > > 14:32:35
> > > > > > > > > > >> and
> > > > > > > > > > >> > started transitioning just after that. So this is 15
> > > > minutes
> > > > > > > that
> > > > > > > > > I'm
> > > > > > > > > > >> > talking about.
> > > > > > > > > > >> >
> > > > > > > > > > >> > What am I missing?
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> > > > > > > > > > >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <
> > > > > > > > nkeywal@gmail.com>
> > > > > > > > > > >> wrote:
> > > > > > > > > > >> >
> > > > > > > > > > >> > > You've changed the value of
> hbase.zookeeper.timeout
> > to
> > > > 15
> > > > > > > > > minutes? A
> > > > > > > > > > >> very
> > > > > > > > > > >> > > reasonable target is 1 minute before relocating
> the
> > > > > regions.
> > > > > > > > > That's
> > > > > > > > > > >> the
> > > > > > > > > > >> > > default iirc. You can push it to 20s, but then
> > > > > > > > > gc-stopping-the-world
> > > > > > > > > > >> > > becomes more of an issue. 15 minutes is really a
> > lot.
> > > > The
> > > > > > hdfs
> > > > > > > > > stale
> > > > > > > > > > >> mode
> > > > > > > > > > >> > > must always be used, with a lower timeout than the
> > > hbase
> > > > > > one.
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > Client side there should be nothing to do
> (excepted
> > > for
> > > > > > > advanced
> > > > > > > > > > >> stuff);
> > > > > > > > > > >> > at
> > > > > > > > > > >> > > each retry the client checks the location of the
> > > > regions.
> > > > > If
> > > > > > > you
> > > > > > > > > > lower
> > > > > > > > > > >> > the
> > > > > > > > > > >> > > number of retry the client will fail sooner, but
> > > usually
> > > > > you
> > > > > > > > don't
> > > > > > > > > > >> want
> > > > > > > > > > >> > the
> > > > > > > > > > >> > > client to fail, you want the servers to reallocate
> > > > > quickly.
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <
> > > > > > > > > > dejan.menges@gmail.com
> > > > > > > > > > >> >
> > > > > > > > > > >> > > wrote:
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > > Hi,
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > Sorry for little bit late update, but managed to
> > > > narrow
> > > > > it
> > > > > > > > > little
> > > > > > > > > > >> bit
> > > > > > > > > > >> > > down.
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > We didn't update yet, as we are using
> Hortonworks
> > > > > > > distribution
> > > > > > > > > > right
> > > > > > > > > > >> > now,
> > > > > > > > > > >> > > > and even if we update we will get 0.98.4.
> However,
> > > > looks
> > > > > > > that
> > > > > > > > > > issue
> > > > > > > > > > >> > here
> > > > > > > > > > >> > > > was in our use case and configuration (still
> > looking
> > > > > into
> > > > > > > it).
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > Basically, initially I saw that when one server
> > goes
> > > > > down,
> > > > > > > we
> > > > > > > > > > start
> > > > > > > > > > >> > > having
> > > > > > > > > > >> > > > performance issues in general, but it managed to
> > be
> > > on
> > > > > our
> > > > > > > > > client
> > > > > > > > > > >> side,
> > > > > > > > > > >> > > due
> > > > > > > > > > >> > > > to caching, and clients were trying to reconnect
> > to
> > > > > nodes
> > > > > > > that
> > > > > > > > > > were
> > > > > > > > > > >> > > offline
> > > > > > > > > > >> > > > and later trying to get regions from those nodes
> > > too.
> > > > > This
> > > > > > > is
> > > > > > > > > > >> basically
> > > > > > > > > > >> > > why
> > > > > > > > > > >> > > > on server side I didn't manage to see anything
> in
> > > logs
> > > > > > that
> > > > > > > > > would
> > > > > > > > > > >> be at
> > > > > > > > > > >> > > > least little bit interesting and point me into
> > > desired
> > > > > > > > > direction.
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > Another question that popped up to me is - in
> case
> > > > > server
> > > > > > is
> > > > > > > > > down
> > > > > > > > > > >> (and
> > > > > > > > > > >> > > with
> > > > > > > > > > >> > > > it DataNode and HRegionServer it was hosting) -
> > > what's
> > > > > > > optimal
> > > > > > > > > > time
> > > > > > > > > > >> to
> > > > > > > > > > >> > > set
> > > > > > > > > > >> > > > for HMaster to consider server dead reassign
> > regions
> > > > > > > somewhere
> > > > > > > > > > >> else, as
> > > > > > > > > > >> > > > this is another performance bottleneck we hit
> > during
> > > > > > > inability
> > > > > > > > > to
> > > > > > > > > > >> > access
> > > > > > > > > > >> > > > regions? In our case it's configured to 15
> > minutes,
> > > > and
> > > > > > > simple
> > > > > > > > > > logic
> > > > > > > > > > >> > > tells
> > > > > > > > > > >> > > > me if you want it earlier then configure lower
> > > number
> > > > of
> > > > > > > > > retries,
> > > > > > > > > > >> but
> > > > > > > > > > >> > > issue
> > > > > > > > > > >> > > > is as always in details, so not sure if anyone
> > knows
> > > > > some
> > > > > > > > better
> > > > > > > > > > >> math
> > > > > > > > > > >> > for
> > > > > > > > > > >> > > > this?
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > And last question - is it possible to manually
> > force
> > > > > HBase
> > > > > > > to
> > > > > > > > > > >> reassign
> > > > > > > > > > >> > > > regions? In this case, while HMaster is retrying
> > to
> > > > > > contact
> > > > > > > > node
> > > > > > > > > > >> that's
> > > > > > > > > > >> > > > dead, it's impossible to force it using
> 'balancer'
> > > > > > command.
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > Thanks a lot!
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > Dejan
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <
> > > > > > > > > > >> dejan.menges@gmail.com>
> > > > > > > > > > >> > > > wrote:
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > > Hi,
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > To be very honest - there's no particular
> reason
> > > why
> > > > > we
> > > > > > > > stick
> > > > > > > > > to
> > > > > > > > > > >> this
> > > > > > > > > > >> > > > one,
> > > > > > > > > > >> > > > > beside just lack of time currently to go
> through
> > > > > upgrade
> > > > > > > > > > process,
> > > > > > > > > > >> but
> > > > > > > > > > >> > > > looks
> > > > > > > > > > >> > > > > to me that's going to be next step.
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > Had a crazy day, didn't have time to go
> through
> > > all
> > > > > logs
> > > > > > > > > again,
> > > > > > > > > > >> plus
> > > > > > > > > > >> > > one
> > > > > > > > > > >> > > > > of the machines (last one where we had this
> > issue)
> > > > is
> > > > > > > fully
> > > > > > > > > > >> > > reprovisioned
> > > > > > > > > > >> > > > > yesterday so I don't have logs from there
> > anymore.
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > Beside upgrading,  what I will talk about
> today,
> > > can
> > > > > you
> > > > > > > > just
> > > > > > > > > > >> point
> > > > > > > > > > >> > me
> > > > > > > > > > >> > > to
> > > > > > > > > > >> > > > > the specific RPC issue in 0.98.0? Thing is
> that
> > we
> > > > > have
> > > > > > > some
> > > > > > > > > > >> strange
> > > > > > > > > > >> > > > > moments with RPC in this case, and just want
> to
> > > see
> > > > if
> > > > > > > > that's
> > > > > > > > > > the
> > > > > > > > > > >> > same
> > > > > > > > > > >> > > > > thing (and we were even suspecting to RPC).
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > Thanks a lot!
> > > > > > > > > > >> > > > > Dejan
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew
> Purtell
> > <
> > > > > > > > > > >> apurtell@apache.org
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > > > wrote:
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > >> Is there a particular reason why you are
> using
> > > > HBase
> > > > > > > > 0.98.0?
> > > > > > > > > > The
> > > > > > > > > > >> > > latest
> > > > > > > > > > >> > > > >> 0.98 release is 0.98.11. There's a known
> > > > performance
> > > > > > > issue
> > > > > > > > > with
> > > > > > > > > > >> > 0.98.0
> > > > > > > > > > >> > > > >> pertaining to RPC that was fixed in later
> > > releases,
> > > > > you
> > > > > > > > > should
> > > > > > > > > > >> move
> > > > > > > > > > >> > up
> > > > > > > > > > >> > > > >> from
> > > > > > > > > > >> > > > >> 0.98.0. In addition hundreds of improvements
> > and
> > > > bug
> > > > > > > fixes
> > > > > > > > > have
> > > > > > > > > > >> gone
> > > > > > > > > > >> > > > into
> > > > > > > > > > >> > > > >> the ten releases since 0.98.0.
> > > > > > > > > > >> > > > >>
> > > > > > > > > > >> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan
> Menges <
> > > > > > > > > > >> > dejan.menges@gmail.com
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > >> wrote:
> > > > > > > > > > >> > > > >>
> > > > > > > > > > >> > > > >> > Hi All,
> > > > > > > > > > >> > > > >> >
> > > > > > > > > > >> > > > >> > We have a strange issue with HBase
> > performance
> > > > > > (overall
> > > > > > > > > > cluster
> > > > > > > > > > >> > > > >> > performance) in case one of datanodes in
> the
> > > > > cluster
> > > > > > > > > > >> unexpectedly
> > > > > > > > > > >> > > goes
> > > > > > > > > > >> > > > >> > down.
> > > > > > > > > > >> > > > >> >
> > > > > > > > > > >> > > > >> > So scenario is like follows:
> > > > > > > > > > >> > > > >> > - Cluster works fine, it's stable.
> > > > > > > > > > >> > > > >> > - One DataNode unexpectedly goes down (PSU
> > > issue,
> > > > > > > network
> > > > > > > > > > >> issue,
> > > > > > > > > > >> > > > >> anything)
> > > > > > > > > > >> > > > >> > - Whole HBase cluster goes down
> (performance
> > > > > becomes
> > > > > > so
> > > > > > > > bad
> > > > > > > > > > >> that
> > > > > > > > > > >> > we
> > > > > > > > > > >> > > > >> have to
> > > > > > > > > > >> > > > >> > restart all RegionServers to get it back to
> > > > life).
> > > > > > > > > > >> > > > >> >
> > > > > > > > > > >> > > > >> > Most funny and latest issue that happened
> was
> > > > that
> > > > > we
> > > > > > > > added
> > > > > > > > > > new
> > > > > > > > > > >> > node
> > > > > > > > > > >> > > > to
> > > > > > > > > > >> > > > >> the
> > > > > > > > > > >> > > > >> > cluster (having 8 x 4T SATA disks) and we
> > left
> > > > just
> > > > > > > > > DataNode
> > > > > > > > > > >> > running
> > > > > > > > > > >> > > > on
> > > > > > > > > > >> > > > >> it
> > > > > > > > > > >> > > > >> > to give it couple of days to get some data.
> > At
> > > > some
> > > > > > > point
> > > > > > > > > in
> > > > > > > > > > >> time,
> > > > > > > > > > >> > > due
> > > > > > > > > > >> > > > >> to
> > > > > > > > > > >> > > > >> > hardware issue, server rebooted (twice
> during
> > > > three
> > > > > > > > hours)
> > > > > > > > > in
> > > > > > > > > > >> > moment
> > > > > > > > > > >> > > > >> when
> > > > > > > > > > >> > > > >> > it had maybe 5% of data it would have in a
> > > couple
> > > > > of
> > > > > > > > days.
> > > > > > > > > > >> Nothing
> > > > > > > > > > >> > > > else
> > > > > > > > > > >> > > > >> > beside DataNode was running, and once it
> went
> > > > down,
> > > > > > it
> > > > > > > > > > affected
> > > > > > > > > > >> > > > literary
> > > > > > > > > > >> > > > >> > everything, and restarting RegionServers in
> > the
> > > > end
> > > > > > > fixed
> > > > > > > > > it.
> > > > > > > > > > >> > > > >> >
> > > > > > > > > > >> > > > >> > We are using HBase 0.98.0 with Hadoop 2.4.0
> > > > > > > > > > >> > > > >> >
> > > > > > > > > > >> > > > >>
> > > > > > > > > > >> > > > >>
> > > > > > > > > > >> > > > >>
> > > > > > > > > > >> > > > >> --
> > > > > > > > > > >> > > > >> Best regards,
> > > > > > > > > > >> > > > >>
> > > > > > > > > > >> > > > >>    - Andy
> > > > > > > > > > >> > > > >>
> > > > > > > > > > >> > > > >> Problems worthy of attack prove their worth
> by
> > > > > hitting
> > > > > > > > back.
> > > > > > > > > -
> > > > > > > > > > >> Piet
> > > > > > > > > > >> > > Hein
> > > > > > > > > > >> > > > >> (via Tom White)
> > > > > > > > > > >> > > > >>
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > >
> > > > > > > > > > >> >
> > > > > > > > > > >>
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Strange issue when DataNode goes down

Posted by Bryan Beaudreault <bb...@hubspot.com>.
@Nicholas, I see, thanks.  I'll keep the settings at default.  So really if
everything else is configured properly you should never reach the lease
recovery timeout in any failure scenarios.  It seems that the staleness
check would be the thing that prevents this, correct?  I'm surprised it
didn't help Dejan.

On Mon, Mar 23, 2015 at 11:20 AM, Nicolas Liochon <nk...@gmail.com> wrote:

> @bryan: yes, you can change hbase.lease.recovery.timeout if you changed he
> hdfs settings. But this setting is really for desperate cases. The recover
> Lease should have succeeded before. As well, if you depend on
> hbase.lease.recovery.timeout, it means that you're wasting recovery time:
> the lease should be recovered in a few seconds.
>
> On Mon, Mar 23, 2015 at 3:59 PM, Dejan Menges <de...@gmail.com>
> wrote:
>
> > Interesting discussion I also found, gives me some more light on what
> > Nicolas is talking about -
> https://issues.apache.org/jira/browse/HDFS-3703
> >
> > On Mon, Mar 23, 2015 at 3:53 PM Bryan Beaudreault <
> > bbeaudreault@hubspot.com>
> > wrote:
> >
> > > So it is safe to set hbase.lease.recovery.timeout lower if you also
> > > set heartbeat.recheck.interval lower (lowering that 10.5 min dead node
> > > timer)?  Or is it recommended to not touch either of those?
> > >
> > > Reading the above with interest, thanks for digging in here guys.
> > >
> > > On Mon, Mar 23, 2015 at 10:13 AM, Nicolas Liochon <nk...@gmail.com>
> > > wrote:
> > >
> > > > If the node is actually down it's fine. But the node may not be that
> > down
> > > > (CAP theorem here); and then it's looking for trouble.
> > > > HDFS, by default declare a node as dead after 10:30. 15 minutes is an
> > > extra
> > > > security. It seems your hdfs settings are different (or there is a
> > > bug...).
> > > > There should be some info in the hdfs logs.
> > > >
> > > > On Mon, Mar 23, 2015 at 3:05 PM, Dejan Menges <
> dejan.menges@gmail.com>
> > > > wrote:
> > > >
> > > > > Will take a look.
> > > > >
> > > > > Actually, if node is down (someone unplugged network cable, it just
> > > died,
> > > > > whatever) what's chance it's going to become live so write can
> > > continue?
> > > > On
> > > > > the other side, HBase is not starting recovery trying to contact
> node
> > > > which
> > > > > is not there anymore, and even elected as dead on Namenode side
> > > (another
> > > > > thing I didn't understood quite good).
> > > > >
> > > > > So what I was expecting is that as soon as Namenode decided node is
> > > dead,
> > > > > that it would be enough for RegionServer to stop trying to recover
> > from
> > > > the
> > > > > dead node, but it wasn't the case. Also, this whole MTTR article in
> > > HBase
> > > > > book doesn't work at all with this parameter set to it's default
> > value
> > > > (15
> > > > > minutes).
> > > > >
> > > > > So I'm still struggling to figure out what's drawback exactly on
> > this?
> > > > >
> > > > > On Mon, Mar 23, 2015 at 2:50 PM Nicolas Liochon <nkeywal@gmail.com
> >
> > > > wrote:
> > > > >
> > > > > > Thanks for the explanation. There is an issue if you modify this
> > > > setting
> > > > > > however.
> > > > > > hbase tries to recover the lease (i.e. be sure that nobody is
> > > writing)
> > > > > > If you change hbase.lease.recovery.timeout hbase will start the
> > > > recovery
> > > > > > (i.e. start to read) even if it's not sure that nobody's writing.
> > > That
> > > > > > means there is a dataloss risk.
> > > > > > Basically, you must not see this warning: WARN
> > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils: Cannot recoverLease
> after
> > > > > trying
> > > > > > for[]
> > > > > >
> > > > > > The recoverLease must succeed. The fact that it does not after X
> > > tries
> > > > is
> > > > > > strange.
> > > > > > There may be a mistmatch between the hbase parameters and the
> hdfs
> > > > ones.
> > > > > > You may need to have a look at the comments in FSHDFSUtils.java
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Mon, Mar 23, 2015 at 2:15 PM, Dejan Menges <
> > > dejan.menges@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > I found the issue and fixed it, and will try to explain here
> what
> > > was
> > > > > > > exactly in our case in case someone else finds this interesting
> > > too.
> > > > > > >
> > > > > > > So initially, we had (couple of times) some network and
> hardware
> > > > issues
> > > > > > in
> > > > > > > our datacenters. When one server would die (literary die, we
> had
> > > some
> > > > > > issue
> > > > > > > with PSUs) we saw issues with overall cluster performance on
> > HBase
> > > > > side.
> > > > > > As
> > > > > > > cluster is quite big and live, it was also quite hard to figure
> > out
> > > > > exact
> > > > > > > root cause and how to fix the stuff we wanted to fix.
> > > > > > >
> > > > > > > So I set up another cluster, four nodes (with DataNode and
> > > > > RegionServer)
> > > > > > > and two other nodes with HMaster and Namenode in HA, using same
> > > stuff
> > > > > we
> > > > > > > use on production. We pumped some data into it, and I was able
> to
> > > > > > reproduce
> > > > > > > same issue last week on it. What I tried to do is to cut one
> > server
> > > > > (shut
> > > > > > > down it's interface) when all is good with cluster, when we
> have
> > > > load,
> > > > > > and
> > > > > > > see what's going to happen.
> > > > > > >
> > > > > > > On Friday, after Nicolas mentioned, I started taking a look in
> > > HBase
> > > > > logs
> > > > > > > on the node which was mentioned in HMaster log as the one
> taking
> > > over
> > > > > > > regions for the dead server. Basically what I was able to
> observe
> > > was
> > > > > 15
> > > > > > > minutes time (+- couple of seconds, what was also interesting,
> > and
> > > > will
> > > > > > got
> > > > > > > later to that) between HMaster figures out that one of it's
> > > > > RegionServers
> > > > > > > is dead, and the time one of the mentioned nodes starts taking
> > over
> > > > > those
> > > > > > > regions and they start appearing in HMaster's Web UI.
> > > > > > >
> > > > > > > I then set up everything like mentioned here
> > > > > > > http://hbase.apache.org/book.html#mttr - but still had exactly
> > the
> > > > > same
> > > > > > > issues. Went over (again and again) all currently configured
> > stuff,
> > > > but
> > > > > > > still had the same issue.
> > > > > > >
> > > > > > > Then I started looking into HDFS. Opened NameNode UI, saw all
> is
> > > > good,
> > > > > > took
> > > > > > > one node down, was also looking RegionServer logs in the same
> > time
> > > -
> > > > > and
> > > > > > I
> > > > > > > also see that it took ~15 minutes for Namenode to elect dead
> node
> > > as
> > > > > > dead.
> > > > > > > Somehow in the same moment regions started getting back to
> life.
> > I
> > > > > > remember
> > > > > > > in some older versions dfs timeout checks and check retries.
> > Looked
> > > > > into
> > > > > > > defaults for our Hadoop version -
> > > > > > >
> > > > > > > http://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/
> > > > > > hadoop-hdfs/hdfs-default.xml
> > > > > > > - and saw there that there's no recheck parameter anymore.
> > Strange,
> > > > as
> > > > > on
> > > > > > > StackOverflow I found post from month ago, for newer version
> than
> > > we
> > > > > use
> > > > > > > (we use 2.4.1, guy was using 2.6 - dfs.namenode.heartbeat.
> > > > > > recheck-interval)
> > > > > > > I set it to 10 seconds as he mentioned, having checks every
> three
> > > > > seconds
> > > > > > > (default) and got DataNode elected as dead in ~45 seconds, as
> he
> > > > > > mentioned.
> > > > > > > Not sure why this parameter is not documented, but obviously it
> > > > works.
> > > > > > >
> > > > > > > Then figured out it still didn't fix our HBase failover issue.
> I
> > > was
> > > > > > > looking into HBase book again and again, and then saw this
> part:
> > > > > > >
> > > > > > > "How much time we allow elapse between calls to recover lease.
> > > Should
> > > > > be
> > > > > > > larger than the dfs timeout."
> > > > > > >
> > > > > > > This was description for hbase.lease.recovery.dfs.timeout.
> Wasn't
> > > > sure
> > > > > > from
> > > > > > > the comment what of all timeouts that's possible to set in
> > > > Hadoop/HBase
> > > > > > and
> > > > > > > that has something to do with DFS is this all about. But picked
> > > > > > > hbase.lease.recovery.timeout, and that was the catch.
> > > > > > >
> > > > > > > Initially, by default, hbase.lease.recovery.timeout is set to
> 15
> > > > > minutes.
> > > > > > > Not sure why, and wasn't able to find yet why, but getting this
> > > down
> > > > to
> > > > > > one
> > > > > > > minute (what's more than OK for us) I was able to get rid of
> our
> > > > issue.
> > > > > > Not
> > > > > > > also sure why this is not mentioned in MTTR section in HBase
> > book,
> > > as
> > > > > > > obviously MTTR doesn't work at all with this default timeout,
> at
> > > > least
> > > > > it
> > > > > > > doesn't work the way we expected it to work.
> > > > > > >
> > > > > > > So thanks again for everyone being spammed with this, and
> > specially
> > > > > > thanks
> > > > > > > to Nicolas pointing me to the right direction.
> > > > > > >
> > > > > > > On Mon, Mar 23, 2015 at 1:37 PM Nicolas Liochon <
> > nkeywal@gmail.com
> > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > > the attachments are rejected by the mailing list, can you put
> > > then
> > > > on
> > > > > > > > pastebin?
> > > > > > > >
> > > > > > > > stale is mandatory (so it's good), but the issue here is just
> > > > before.
> > > > > > The
> > > > > > > > region server needs to read the file. In order to be sure
> that
> > > > there
> > > > > is
> > > > > > > no
> > > > > > > > data loss, it needs to "recover the lease", that means
> ensuring
> > > > that
> > > > > > > nobody
> > > > > > > > is writing the file. The regionserver calls the namenode to
> do
> > > this
> > > > > > > > recoverLease. So there should be some info in the namenode
> > logs.
> > > > You
> > > > > > have
> > > > > > > > HDFS-4721 on your hdfs? The hbase book details (more or
> > less...)
> > > > this
> > > > > > > > recoverLease stuff.
> > > > > > > >
> > > > > > > >
> > > > > > > > On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <
> > > > > dejan.menges@gmail.com
> > > > > > >
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > And also, just checked - dfs.namenode.avoid.read.stale.
> > > datanode
> > > > and
> > > > > > > > > dfs.namenode.avoid.write.stale.datanode
> > > > > > > > > are both true, and dfs.namenode.stale.datanode.interval is
> > > set to
> > > > > > > > default
> > > > > > > > > 30000.
> > > > > > > > >
> > > > > > > > > On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <
> > > > > > dejan.menges@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Hi Nicolas,
> > > > > > > > > >
> > > > > > > > > > Please find log attached.
> > > > > > > > > >
> > > > > > > > > > As I see it now more clearly, it was trying to recover
> HDFS
> > > > WALs
> > > > > > from
> > > > > > > > > node
> > > > > > > > > > that's dead:
> > > > > > > > > >
> > > > > > > > > > 2015-03-23 08:53:44,381 WARN
> > > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils:
> > > > > > > > > > Cannot recoverLease after trying for 900000ms
> > > > > > > > > > (hbase.lease.recovery.timeout); continuing, but may be
> > > > > > DATALOSS!!!;
> > > > > > > > > > attempt=40 on
> > > > > > > > > >
> > > > > > > > >
> file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intent
> > > > > > > > ionally_get_down_by_getting_network_down},60020,1426862900
> > > > > > > > 506-splitting/{node_i_intentionally_get_down_by_gett
> > > > > > > > ing_network_down}%2C60020%2C1426862900506.1427096924508
> > > > > > > > > > after 908210ms
> > > > > > > > > >
> > > > > > > > > > And as you can see from the log, it tried 40 times, what
> > took
> > > > it
> > > > > > > > exactly
> > > > > > > > > > 15 minutes.
> > > > > > > > > >
> > > > > > > > > > There's probably some parameter to tune to cut it of from
> > 40
> > > > > times
> > > > > > /
> > > > > > > 15
> > > > > > > > > > minutes to something more useful, as for 15 minutes we
> > don't
> > > > have
> > > > > > our
> > > > > > > > > > regions available, and HDFS have however replication
> factor
> > > 3.
> > > > > > > > > >
> > > > > > > > > > Googling, if I figure out what's this I will post it
> here.
> > > Will
> > > > > > also
> > > > > > > > > > appreciate if someone knows how to cut this down.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > Dejan
> > > > > > > > > >
> > > > > > > > > > On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <
> > > > > nkeywal@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > >> The split is done by the region servers (the master
> > > > > coordinates).
> > > > > > Is
> > > > > > > > > there
> > > > > > > > > >> some interesting stuff in their logs?
> > > > > > > > > >>
> > > > > > > > > >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <
> > > > > > > dejan.menges@gmail.com
> > > > > > > > >
> > > > > > > > > >> wrote:
> > > > > > > > > >>
> > > > > > > > > >> > With client issue was that it was retrying connecting
> to
> > > the
> > > > > > same
> > > > > > > > > region
> > > > > > > > > >> > servers even when they were reassigned. Lowering it
> down
> > > > > helped
> > > > > > in
> > > > > > > > > this
> > > > > > > > > >> > specific use case, but yes, we still want servers to
> > > > > reallocate
> > > > > > > > > quickly.
> > > > > > > > > >> >
> > > > > > > > > >> > What got me here:
> > > > > > > > > >> >
> > > > > > > > > >> > http://hbase.apache.org/book.html#mttr
> > > > > > > > > >> >
> > > > > > > > > >> > I basically set configuration exactly the same way as
> > it's
> > > > > > > explained
> > > > > > > > > >> here.
> > > > > > > > > >> > *zookeeper.session.timeout* is (and was before) 60000
> > (one
> > > > > > > minute).
> > > > > > > > > >> >
> > > > > > > > > >> > So basically what happens is: - simulating network
> > issues
> > > we
> > > > > had
> > > > > > > > > >> recently).
> > > > > > > > > >> > - After short time I see in HBase that my RegionServer
> > is
> > > > > dead,
> > > > > > > and
> > > > > > > > as
> > > > > > > > > >> > total number of regions I see previous total minus
> > number
> > > of
> > > > > > > regions
> > > > > > > > > >> that
> > > > > > > > > >> > were hosted on the node hosting RegionServer that just
> > > > > > > > 'disappeared'.
> > > > > > > > > >> > - In this point I want my clus
> > > > > > > > > >> >
> > > > > > > > > >> > - I have test cluster consisting of four nodes, every
> > node
> > > > > being
> > > > > > > > > >> DataNode
> > > > > > > > > >> > and RegionServer.
> > > > > > > > > >> > - I simulate network partition on one (connect to it
> > > through
> > > > > > > console
> > > > > > > > > and
> > > > > > > > > >> > take network interface downter to recover as soon as
> > > > possible,
> > > > > > to
> > > > > > > > > start
> > > > > > > > > >> > serving missing regions.
> > > > > > > > > >> > - First thing I see in HMaster logs are:
> > > > > > > > > >> >
> > > > > > > > > >> > 2015-03-20 14:17:26,015 INFO
> > > > > > > > > >> > org.apache.hadoop.hbase.zookeeper.RegionServerTracker:
> > > > > > > RegionServer
> > > > > > > > > >> > ephemeral node deleted, processing expiration
> > > > > > > > > >> > [{name_of_node_I_took_down},60020,1426860403261]
> > > > > > > > > >> >
> > > > > > > > > >> > 2015-03-20 14:17:26,067 INFO
> > > > > > > > > >> >
> > > > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler:
> > > > > > > > > Splitting
> > > > > > > > > >> > logs for
> {name_of_node_I_took_down},60020,1426860403261
> > > > before
> > > > > > > > > >> assignment.
> > > > > > > > > >> >
> > > > > > > > > >> > 2015-03-20 14:17:26,105 INFO
> > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: dead
> > > > splitlog
> > > > > > > > workers
> > > > > > > > > [
> > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261]
> > > > > > > > > >> >
> > > > > > > > > >> > 2015-03-20 14:17:26,107 INFO
> > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> started
> > > > > > splitting
> > > > > > > 1
> > > > > > > > > >> logs in
> > > > > > > > > >> >
> [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_
> > > I_
> > > > > > > > took_down}
> > > > > > > > > >> > ,60020,1426860403261-splitting]
> > > > > > > > > >> >
> > > > > > > > > >> > 2015-03-20 14:17:26,150 INFO
> > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > > > > > > > > >> > /hbase/splitWAL/WALs%2F
> > > > > > > > > >> >
> > > > > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > > > > > > >> >
> {name_of_node_I_took_down}%252C60020%252C1426860403261.
> > > 14268
> > > > > > > > 60404905
> > > > > > > > > >> > acquired by {fqdn_of_another_live_hnode},
> > > 60020,1426859445623
> > > > > > > > > >> >
> > > > > > > > > >> > 2015-03-20 14:17:26,182 INFO
> > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total
> > > tasks
> > > > =
> > > > > 1
> > > > > > > > > >> unassigned
> > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > 2F{name_of_node_I_took_down}
> > > > > > > > > >> >
> > > > > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > > took_
> > > > > > > > > >>
> > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> status =
> > > > > > > > in_progress
> > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
> done
> > > = 0
> > > > > > > error =
> > > > > > > > > 0}
> > > > > > > > > >> >
> > > > > > > > > >> > 2015-03-20 14:17:31,183 INFO
> > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total
> > > tasks
> > > > =
> > > > > 1
> > > > > > > > > >> unassigned
> > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > 2F{name_of_node_I_took_down}
> > > > > > > > > >> >
> > > > > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > > took_
> > > > > > > > > >>
> > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> status =
> > > > > > > > in_progress
> > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
> done
> > > = 0
> > > > > > > error =
> > > > > > > > > 0}
> > > > > > > > > >> >
> > > > > > > > > >> > 2015-03-20 14:17:36,184 INFO
> > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total
> > > tasks
> > > > =
> > > > > 1
> > > > > > > > > >> unassigned
> > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > 2F{name_of_node_I_took_down}
> > > > > > > > > >> >
> > > > > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > > took_
> > > > > > > > > >>
> > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> status =
> > > > > > > > in_progress
> > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
> done
> > > = 0
> > > > > > > error =
> > > > > > > > > 0}
> > > > > > > > > >> >
> > > > > > > > > >> > 2015-03-20 14:17:42,185 INFO
> > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total
> > > tasks
> > > > =
> > > > > 1
> > > > > > > > > >> unassigned
> > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > 2F{name_of_node_I_took_down}
> > > > > > > > > >> >
> > > > > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > > took_
> > > > > > > > > >>
> > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> status =
> > > > > > > > in_progress
> > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
> done
> > > = 0
> > > > > > > error =
> > > > > > > > > 0}
> > > > > > > > > >> >
> > > > > > > > > >> > 2015-03-20 14:17:48,184 INFO
> > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total
> > > tasks
> > > > =
> > > > > 1
> > > > > > > > > >> unassigned
> > > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > > 2F{name_of_node_I_took_down}
> > > > > > > > > >> >
> > > > > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > > took_
> > > > > > > > > >>
> > down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623
> status =
> > > > > > > > in_progress
> > > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1
> done
> > > = 0
> > > > > > > error =
> > > > > > > > > 0}
> > > > > > > > > >> > In the meantime, In hbase...out log I got this:
> > > > > > > > > >> >
> > > > > > > > > >> > ==> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out
> <==
> > > > > > > > > >> >
> > > > > > > > > >> > java.io.IOException: Call to
> > > > > > > > > >> >
> > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_
> > > dow
> > > > > > > > n}:60020
> > > > > > > > > >> > failed on local exception:
> > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> > > CallTimeoutException:
> > > > > > Call
> > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
> > > > > > > > > >> t.java:1532)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:
> > > > > > 1502)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> >
> > > > > > > > > >> >
> > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(
> > > Rpc
> > > > > > > > > >> Client.java:1684)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> >
> > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> > > BlockingRpcChannelImpl
> > > > > > > > ementati
> > > > > > > > > >> on.callBlockingMethod(RpcClient.java:1737)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> >
> > > > > > > > > >> >
> org.apache.hadoop.hbase.protobuf.generated.AdminProtos$
> > > > > > > > AdminService$
> > > > > > > > > >> BlockingStub.getRegionInfo(AdminProtos.java:20806)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> >
> > > > > > > > > >> > org.apache.hadoop.hbase.client.HBaseAdmin.
> > > getCompactionState
> > > > > > > > > >> (HBaseAdmin.java:2524)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> >
> > > > > > > > > >> > org.apache.hadoop.hbase.generated.master.table_jsp._
> > > jspServi
> > > > > > > > > >> ce(table_jsp.java:167)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.
> > > > > > > > java:98)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
> > > > > > > > > >> .java:511)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> >
> > > > > > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> > > doFilte
> > > > > > > > > >> r(ServletHandler.java:1221)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> >
> > > > > > > > > >> > org.apache.hadoop.http.lib.StaticUserWebFilter$
> > > StaticUserFil
> > > > > > > > > >> ter.doFilter(StaticUserWebFilter.java:109)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> >
> > > > > > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> > > doFilte
> > > > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> >
> > > > > > > > > >> > org.apache.hadoop.http.HttpServer$QuotingInputFilter.
> > > doFilte
> > > > > > > > > >> r(HttpServer.java:1081)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> >
> > > > > > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> > > doFilte
> > > > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > >
> > > > >
> org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> >
> > > > > > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> > > doFilte
> > > > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
> > > > > > > > > >> er.java:399)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> > org.mortbay.jetty.security.SecurityHandler.handle(
> > > SecurityHa
> > > > > > > > > >> ndler.java:216)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
> > > > > > > > > >> er.java:182)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
> > > > > > > > > >> er.java:766)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
> > > > > > > > > >> java:450)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> >
> > > > > > > > > >> > org.mortbay.jetty.handler.ContextHandlerCollection.
> > > handle(Co
> > > > > > > > > >> ntextHandlerCollection.java:230)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
> > > > > > > > > >> er.java:152)
> > > > > > > > > >> >
> > > > > > > > > >> > at org.mortbay.jetty.Server.handle(Server.java:326)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
> > > > > > > > > >> n.java:542)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> >
> > > > > > > > > >> > org.mortbay.jetty.HttpConnection$RequestHandler.
> > > headerComple
> > > > > > > > > >> te(HttpConnection.java:928)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > >
> org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > >
> org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> >
> > > > > > > > > >> > org.mortbay.io.nio.SelectChannelEndPoint.run(
> > > SelectChannelEn
> > > > > > > > > >> dPoint.java:410)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> >
> > > > > > > > > >> > org.mortbay.thread.QueuedThreadPool$PoolThread.run(
> > > > > > > > > >> QueuedThreadPool.java:582)
> > > > > > > > > >> >
> > > > > > > > > >> > Caused by: org.apache.hadoop.hbase.ipc.RpcClient$
> > > > > > > > CallTimeoutException:
> > > > > > > > > >> Call
> > > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> >
> > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.
> > > cleanupCall
> > > > > > > > > >> s(RpcClient.java:1234)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > > > > > >> >
> > > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.
> > > readRespons
> > > > > > > > > >> e(RpcClient.java:1171)
> > > > > > > > > >> >
> > > > > > > > > >> > at
> > > > > org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
> > > > > > > > > >> nt.java:751)
> > > > > > > > > >> > Beside this same issue, please note that first message
> > was
> > > > at
> > > > > > > > > 2015-03-20
> > > > > > > > > >> > 14:17:26,015. And then (we got to the point when it
> > > started
> > > > > > > > > transition):
> > > > > > > > > >> >
> > > > > > > > > >> > 2015-03-20 14:32:35,059 INFO
> > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > > > > > > > > >> > /hbase/splitWAL/WALs%2F
> > > > > > > > > >> >
> > > > > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > > > > > > >> >
> {name_of_node_I_took_down}%252C60020%252C1426860403261.
> > > 14268
> > > > > > > > 60404905
> > > > > > > > > >> > entered state: DONE
> > > > > {fqdn_of_new_live_node},60020,1426859445623
> > > > > > > > > >> >
> > > > > > > > > >> > 2015-03-20 14:32:35,109 INFO
> > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: Done
> > > > splitting
> > > > > > > > > >> > /hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > > > > > >> >
> > > > > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}
> > > > > > > > > >> > %252C60020%252C1426860403261.1426860404905
> > > > > > > > > >> >
> > > > > > > > > >> > 2015-03-20 14:32:35,190 INFO
> > > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager:
> finished
> > > > > > splitting
> > > > > > > > > >> (more
> > > > > > > > > >> > than or equal to) 9 bytes in 1 log files in
> > > > > > > > > >> >
> > > > > > > > > >> >
> [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_
> > > of_
> > > > > > > > > >> node_I_took_down},60020,1426860403261-splitting]
> > > > > > > > > >> > in 909083ms
> > > > > > > > > >> >
> > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > > org.apache.hadoop.hbase.master
> > > > > > > > > >> .RegionStates:
> > > > > > > > > >> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c
> > > state=OPEN,
> > > > > > > > > >> > ts=1426860639088,
> > > > > > > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > > >> to
> > > > > > > > > >> > {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE,
> > > > > > ts=1426861955191,
> > > > > > > > > >> server=
> > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > > >> >
> > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > > org.apache.hadoop.hbase.master
> > > > > > > > > >> .RegionStates:
> > > > > > > > > >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
> > > > > > > > > >> {name_of_node_I_took_down}
> > > > > > > > > >> > ,60020,1426860403261
> > > > > > > > > >> >
> > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > > org.apache.hadoop.hbase.master
> > > > > > > > > >> .RegionStates:
> > > > > > > > > >> > Transitioned {25ab6e7b42e36ddaa723d71be5954543
> > > state=OPEN,
> > > > > > > > > >> > ts=1426860641783,
> > > > > > > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > > >> to
> > > > > > > > > >> > {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE,
> > > > > > ts=1426861955191,
> > > > > > > > > >> server=
> > > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > > >> >
> > > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > > org.apache.hadoop.hbase.master
> > > > > > > > > >> .RegionStates:
> > > > > > > > > >> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
> > > > > > > > > >> {name_of_node_I_took_down}
> > > > > > > > > >> > ,60020,1426860403261
> > > > > > > > > >> > At this point, note that it finished SplitLogManager
> > task
> > > at
> > > > > > > > 14:32:35
> > > > > > > > > >> and
> > > > > > > > > >> > started transitioning just after that. So this is 15
> > > minutes
> > > > > > that
> > > > > > > > I'm
> > > > > > > > > >> > talking about.
> > > > > > > > > >> >
> > > > > > > > > >> > What am I missing?
> > > > > > > > > >> >
> > > > > > > > > >> >
> > > > > > > > > >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <
> > > > > > > nkeywal@gmail.com>
> > > > > > > > > >> wrote:
> > > > > > > > > >> >
> > > > > > > > > >> > > You've changed the value of hbase.zookeeper.timeout
> to
> > > 15
> > > > > > > > minutes? A
> > > > > > > > > >> very
> > > > > > > > > >> > > reasonable target is 1 minute before relocating the
> > > > regions.
> > > > > > > > That's
> > > > > > > > > >> the
> > > > > > > > > >> > > default iirc. You can push it to 20s, but then
> > > > > > > > gc-stopping-the-world
> > > > > > > > > >> > > becomes more of an issue. 15 minutes is really a
> lot.
> > > The
> > > > > hdfs
> > > > > > > > stale
> > > > > > > > > >> mode
> > > > > > > > > >> > > must always be used, with a lower timeout than the
> > hbase
> > > > > one.
> > > > > > > > > >> > >
> > > > > > > > > >> > > Client side there should be nothing to do (excepted
> > for
> > > > > > advanced
> > > > > > > > > >> stuff);
> > > > > > > > > >> > at
> > > > > > > > > >> > > each retry the client checks the location of the
> > > regions.
> > > > If
> > > > > > you
> > > > > > > > > lower
> > > > > > > > > >> > the
> > > > > > > > > >> > > number of retry the client will fail sooner, but
> > usually
> > > > you
> > > > > > > don't
> > > > > > > > > >> want
> > > > > > > > > >> > the
> > > > > > > > > >> > > client to fail, you want the servers to reallocate
> > > > quickly.
> > > > > > > > > >> > >
> > > > > > > > > >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <
> > > > > > > > > dejan.menges@gmail.com
> > > > > > > > > >> >
> > > > > > > > > >> > > wrote:
> > > > > > > > > >> > >
> > > > > > > > > >> > > > Hi,
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > Sorry for little bit late update, but managed to
> > > narrow
> > > > it
> > > > > > > > little
> > > > > > > > > >> bit
> > > > > > > > > >> > > down.
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > We didn't update yet, as we are using Hortonworks
> > > > > > distribution
> > > > > > > > > right
> > > > > > > > > >> > now,
> > > > > > > > > >> > > > and even if we update we will get 0.98.4. However,
> > > looks
> > > > > > that
> > > > > > > > > issue
> > > > > > > > > >> > here
> > > > > > > > > >> > > > was in our use case and configuration (still
> looking
> > > > into
> > > > > > it).
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > Basically, initially I saw that when one server
> goes
> > > > down,
> > > > > > we
> > > > > > > > > start
> > > > > > > > > >> > > having
> > > > > > > > > >> > > > performance issues in general, but it managed to
> be
> > on
> > > > our
> > > > > > > > client
> > > > > > > > > >> side,
> > > > > > > > > >> > > due
> > > > > > > > > >> > > > to caching, and clients were trying to reconnect
> to
> > > > nodes
> > > > > > that
> > > > > > > > > were
> > > > > > > > > >> > > offline
> > > > > > > > > >> > > > and later trying to get regions from those nodes
> > too.
> > > > This
> > > > > > is
> > > > > > > > > >> basically
> > > > > > > > > >> > > why
> > > > > > > > > >> > > > on server side I didn't manage to see anything in
> > logs
> > > > > that
> > > > > > > > would
> > > > > > > > > >> be at
> > > > > > > > > >> > > > least little bit interesting and point me into
> > desired
> > > > > > > > direction.
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > Another question that popped up to me is - in case
> > > > server
> > > > > is
> > > > > > > > down
> > > > > > > > > >> (and
> > > > > > > > > >> > > with
> > > > > > > > > >> > > > it DataNode and HRegionServer it was hosting) -
> > what's
> > > > > > optimal
> > > > > > > > > time
> > > > > > > > > >> to
> > > > > > > > > >> > > set
> > > > > > > > > >> > > > for HMaster to consider server dead reassign
> regions
> > > > > > somewhere
> > > > > > > > > >> else, as
> > > > > > > > > >> > > > this is another performance bottleneck we hit
> during
> > > > > > inability
> > > > > > > > to
> > > > > > > > > >> > access
> > > > > > > > > >> > > > regions? In our case it's configured to 15
> minutes,
> > > and
> > > > > > simple
> > > > > > > > > logic
> > > > > > > > > >> > > tells
> > > > > > > > > >> > > > me if you want it earlier then configure lower
> > number
> > > of
> > > > > > > > retries,
> > > > > > > > > >> but
> > > > > > > > > >> > > issue
> > > > > > > > > >> > > > is as always in details, so not sure if anyone
> knows
> > > > some
> > > > > > > better
> > > > > > > > > >> math
> > > > > > > > > >> > for
> > > > > > > > > >> > > > this?
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > And last question - is it possible to manually
> force
> > > > HBase
> > > > > > to
> > > > > > > > > >> reassign
> > > > > > > > > >> > > > regions? In this case, while HMaster is retrying
> to
> > > > > contact
> > > > > > > node
> > > > > > > > > >> that's
> > > > > > > > > >> > > > dead, it's impossible to force it using 'balancer'
> > > > > command.
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > Thanks a lot!
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > Dejan
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <
> > > > > > > > > >> dejan.menges@gmail.com>
> > > > > > > > > >> > > > wrote:
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > > Hi,
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > To be very honest - there's no particular reason
> > why
> > > > we
> > > > > > > stick
> > > > > > > > to
> > > > > > > > > >> this
> > > > > > > > > >> > > > one,
> > > > > > > > > >> > > > > beside just lack of time currently to go through
> > > > upgrade
> > > > > > > > > process,
> > > > > > > > > >> but
> > > > > > > > > >> > > > looks
> > > > > > > > > >> > > > > to me that's going to be next step.
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > Had a crazy day, didn't have time to go through
> > all
> > > > logs
> > > > > > > > again,
> > > > > > > > > >> plus
> > > > > > > > > >> > > one
> > > > > > > > > >> > > > > of the machines (last one where we had this
> issue)
> > > is
> > > > > > fully
> > > > > > > > > >> > > reprovisioned
> > > > > > > > > >> > > > > yesterday so I don't have logs from there
> anymore.
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > Beside upgrading,  what I will talk about today,
> > can
> > > > you
> > > > > > > just
> > > > > > > > > >> point
> > > > > > > > > >> > me
> > > > > > > > > >> > > to
> > > > > > > > > >> > > > > the specific RPC issue in 0.98.0? Thing is that
> we
> > > > have
> > > > > > some
> > > > > > > > > >> strange
> > > > > > > > > >> > > > > moments with RPC in this case, and just want to
> > see
> > > if
> > > > > > > that's
> > > > > > > > > the
> > > > > > > > > >> > same
> > > > > > > > > >> > > > > thing (and we were even suspecting to RPC).
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > Thanks a lot!
> > > > > > > > > >> > > > > Dejan
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew Purtell
> <
> > > > > > > > > >> apurtell@apache.org
> > > > > > > > > >> > >
> > > > > > > > > >> > > > > wrote:
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > >> Is there a particular reason why you are using
> > > HBase
> > > > > > > 0.98.0?
> > > > > > > > > The
> > > > > > > > > >> > > latest
> > > > > > > > > >> > > > >> 0.98 release is 0.98.11. There's a known
> > > performance
> > > > > > issue
> > > > > > > > with
> > > > > > > > > >> > 0.98.0
> > > > > > > > > >> > > > >> pertaining to RPC that was fixed in later
> > releases,
> > > > you
> > > > > > > > should
> > > > > > > > > >> move
> > > > > > > > > >> > up
> > > > > > > > > >> > > > >> from
> > > > > > > > > >> > > > >> 0.98.0. In addition hundreds of improvements
> and
> > > bug
> > > > > > fixes
> > > > > > > > have
> > > > > > > > > >> gone
> > > > > > > > > >> > > > into
> > > > > > > > > >> > > > >> the ten releases since 0.98.0.
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <
> > > > > > > > > >> > dejan.menges@gmail.com
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > >> wrote:
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >> > Hi All,
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > We have a strange issue with HBase
> performance
> > > > > (overall
> > > > > > > > > cluster
> > > > > > > > > >> > > > >> > performance) in case one of datanodes in the
> > > > cluster
> > > > > > > > > >> unexpectedly
> > > > > > > > > >> > > goes
> > > > > > > > > >> > > > >> > down.
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > So scenario is like follows:
> > > > > > > > > >> > > > >> > - Cluster works fine, it's stable.
> > > > > > > > > >> > > > >> > - One DataNode unexpectedly goes down (PSU
> > issue,
> > > > > > network
> > > > > > > > > >> issue,
> > > > > > > > > >> > > > >> anything)
> > > > > > > > > >> > > > >> > - Whole HBase cluster goes down (performance
> > > > becomes
> > > > > so
> > > > > > > bad
> > > > > > > > > >> that
> > > > > > > > > >> > we
> > > > > > > > > >> > > > >> have to
> > > > > > > > > >> > > > >> > restart all RegionServers to get it back to
> > > life).
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > Most funny and latest issue that happened was
> > > that
> > > > we
> > > > > > > added
> > > > > > > > > new
> > > > > > > > > >> > node
> > > > > > > > > >> > > > to
> > > > > > > > > >> > > > >> the
> > > > > > > > > >> > > > >> > cluster (having 8 x 4T SATA disks) and we
> left
> > > just
> > > > > > > > DataNode
> > > > > > > > > >> > running
> > > > > > > > > >> > > > on
> > > > > > > > > >> > > > >> it
> > > > > > > > > >> > > > >> > to give it couple of days to get some data.
> At
> > > some
> > > > > > point
> > > > > > > > in
> > > > > > > > > >> time,
> > > > > > > > > >> > > due
> > > > > > > > > >> > > > >> to
> > > > > > > > > >> > > > >> > hardware issue, server rebooted (twice during
> > > three
> > > > > > > hours)
> > > > > > > > in
> > > > > > > > > >> > moment
> > > > > > > > > >> > > > >> when
> > > > > > > > > >> > > > >> > it had maybe 5% of data it would have in a
> > couple
> > > > of
> > > > > > > days.
> > > > > > > > > >> Nothing
> > > > > > > > > >> > > > else
> > > > > > > > > >> > > > >> > beside DataNode was running, and once it went
> > > down,
> > > > > it
> > > > > > > > > affected
> > > > > > > > > >> > > > literary
> > > > > > > > > >> > > > >> > everything, and restarting RegionServers in
> the
> > > end
> > > > > > fixed
> > > > > > > > it.
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >> > We are using HBase 0.98.0 with Hadoop 2.4.0
> > > > > > > > > >> > > > >> >
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >> --
> > > > > > > > > >> > > > >> Best regards,
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >>    - Andy
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >> Problems worthy of attack prove their worth by
> > > > hitting
> > > > > > > back.
> > > > > > > > -
> > > > > > > > > >> Piet
> > > > > > > > > >> > > Hein
> > > > > > > > > >> > > > >> (via Tom White)
> > > > > > > > > >> > > > >>
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > >
> > > > > > > > > >> >
> > > > > > > > > >>
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Strange issue when DataNode goes down

Posted by Nicolas Liochon <nk...@gmail.com>.
@bryan: yes, you can change hbase.lease.recovery.timeout if you changed he
hdfs settings. But this setting is really for desperate cases. The recover
Lease should have succeeded before. As well, if you depend on
hbase.lease.recovery.timeout, it means that you're wasting recovery time:
the lease should be recovered in a few seconds.

On Mon, Mar 23, 2015 at 3:59 PM, Dejan Menges <de...@gmail.com>
wrote:

> Interesting discussion I also found, gives me some more light on what
> Nicolas is talking about - https://issues.apache.org/jira/browse/HDFS-3703
>
> On Mon, Mar 23, 2015 at 3:53 PM Bryan Beaudreault <
> bbeaudreault@hubspot.com>
> wrote:
>
> > So it is safe to set hbase.lease.recovery.timeout lower if you also
> > set heartbeat.recheck.interval lower (lowering that 10.5 min dead node
> > timer)?  Or is it recommended to not touch either of those?
> >
> > Reading the above with interest, thanks for digging in here guys.
> >
> > On Mon, Mar 23, 2015 at 10:13 AM, Nicolas Liochon <nk...@gmail.com>
> > wrote:
> >
> > > If the node is actually down it's fine. But the node may not be that
> down
> > > (CAP theorem here); and then it's looking for trouble.
> > > HDFS, by default declare a node as dead after 10:30. 15 minutes is an
> > extra
> > > security. It seems your hdfs settings are different (or there is a
> > bug...).
> > > There should be some info in the hdfs logs.
> > >
> > > On Mon, Mar 23, 2015 at 3:05 PM, Dejan Menges <de...@gmail.com>
> > > wrote:
> > >
> > > > Will take a look.
> > > >
> > > > Actually, if node is down (someone unplugged network cable, it just
> > died,
> > > > whatever) what's chance it's going to become live so write can
> > continue?
> > > On
> > > > the other side, HBase is not starting recovery trying to contact node
> > > which
> > > > is not there anymore, and even elected as dead on Namenode side
> > (another
> > > > thing I didn't understood quite good).
> > > >
> > > > So what I was expecting is that as soon as Namenode decided node is
> > dead,
> > > > that it would be enough for RegionServer to stop trying to recover
> from
> > > the
> > > > dead node, but it wasn't the case. Also, this whole MTTR article in
> > HBase
> > > > book doesn't work at all with this parameter set to it's default
> value
> > > (15
> > > > minutes).
> > > >
> > > > So I'm still struggling to figure out what's drawback exactly on
> this?
> > > >
> > > > On Mon, Mar 23, 2015 at 2:50 PM Nicolas Liochon <nk...@gmail.com>
> > > wrote:
> > > >
> > > > > Thanks for the explanation. There is an issue if you modify this
> > > setting
> > > > > however.
> > > > > hbase tries to recover the lease (i.e. be sure that nobody is
> > writing)
> > > > > If you change hbase.lease.recovery.timeout hbase will start the
> > > recovery
> > > > > (i.e. start to read) even if it's not sure that nobody's writing.
> > That
> > > > > means there is a dataloss risk.
> > > > > Basically, you must not see this warning: WARN
> > > > > org.apache.hadoop.hbase.util.FSHDFSUtils: Cannot recoverLease after
> > > > trying
> > > > > for[]
> > > > >
> > > > > The recoverLease must succeed. The fact that it does not after X
> > tries
> > > is
> > > > > strange.
> > > > > There may be a mistmatch between the hbase parameters and the hdfs
> > > ones.
> > > > > You may need to have a look at the comments in FSHDFSUtils.java
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > On Mon, Mar 23, 2015 at 2:15 PM, Dejan Menges <
> > dejan.menges@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > I found the issue and fixed it, and will try to explain here what
> > was
> > > > > > exactly in our case in case someone else finds this interesting
> > too.
> > > > > >
> > > > > > So initially, we had (couple of times) some network and hardware
> > > issues
> > > > > in
> > > > > > our datacenters. When one server would die (literary die, we had
> > some
> > > > > issue
> > > > > > with PSUs) we saw issues with overall cluster performance on
> HBase
> > > > side.
> > > > > As
> > > > > > cluster is quite big and live, it was also quite hard to figure
> out
> > > > exact
> > > > > > root cause and how to fix the stuff we wanted to fix.
> > > > > >
> > > > > > So I set up another cluster, four nodes (with DataNode and
> > > > RegionServer)
> > > > > > and two other nodes with HMaster and Namenode in HA, using same
> > stuff
> > > > we
> > > > > > use on production. We pumped some data into it, and I was able to
> > > > > reproduce
> > > > > > same issue last week on it. What I tried to do is to cut one
> server
> > > > (shut
> > > > > > down it's interface) when all is good with cluster, when we have
> > > load,
> > > > > and
> > > > > > see what's going to happen.
> > > > > >
> > > > > > On Friday, after Nicolas mentioned, I started taking a look in
> > HBase
> > > > logs
> > > > > > on the node which was mentioned in HMaster log as the one taking
> > over
> > > > > > regions for the dead server. Basically what I was able to observe
> > was
> > > > 15
> > > > > > minutes time (+- couple of seconds, what was also interesting,
> and
> > > will
> > > > > got
> > > > > > later to that) between HMaster figures out that one of it's
> > > > RegionServers
> > > > > > is dead, and the time one of the mentioned nodes starts taking
> over
> > > > those
> > > > > > regions and they start appearing in HMaster's Web UI.
> > > > > >
> > > > > > I then set up everything like mentioned here
> > > > > > http://hbase.apache.org/book.html#mttr - but still had exactly
> the
> > > > same
> > > > > > issues. Went over (again and again) all currently configured
> stuff,
> > > but
> > > > > > still had the same issue.
> > > > > >
> > > > > > Then I started looking into HDFS. Opened NameNode UI, saw all is
> > > good,
> > > > > took
> > > > > > one node down, was also looking RegionServer logs in the same
> time
> > -
> > > > and
> > > > > I
> > > > > > also see that it took ~15 minutes for Namenode to elect dead node
> > as
> > > > > dead.
> > > > > > Somehow in the same moment regions started getting back to life.
> I
> > > > > remember
> > > > > > in some older versions dfs timeout checks and check retries.
> Looked
> > > > into
> > > > > > defaults for our Hadoop version -
> > > > > >
> > > > > > http://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/
> > > > > hadoop-hdfs/hdfs-default.xml
> > > > > > - and saw there that there's no recheck parameter anymore.
> Strange,
> > > as
> > > > on
> > > > > > StackOverflow I found post from month ago, for newer version than
> > we
> > > > use
> > > > > > (we use 2.4.1, guy was using 2.6 - dfs.namenode.heartbeat.
> > > > > recheck-interval)
> > > > > > I set it to 10 seconds as he mentioned, having checks every three
> > > > seconds
> > > > > > (default) and got DataNode elected as dead in ~45 seconds, as he
> > > > > mentioned.
> > > > > > Not sure why this parameter is not documented, but obviously it
> > > works.
> > > > > >
> > > > > > Then figured out it still didn't fix our HBase failover issue. I
> > was
> > > > > > looking into HBase book again and again, and then saw this part:
> > > > > >
> > > > > > "How much time we allow elapse between calls to recover lease.
> > Should
> > > > be
> > > > > > larger than the dfs timeout."
> > > > > >
> > > > > > This was description for hbase.lease.recovery.dfs.timeout. Wasn't
> > > sure
> > > > > from
> > > > > > the comment what of all timeouts that's possible to set in
> > > Hadoop/HBase
> > > > > and
> > > > > > that has something to do with DFS is this all about. But picked
> > > > > > hbase.lease.recovery.timeout, and that was the catch.
> > > > > >
> > > > > > Initially, by default, hbase.lease.recovery.timeout is set to 15
> > > > minutes.
> > > > > > Not sure why, and wasn't able to find yet why, but getting this
> > down
> > > to
> > > > > one
> > > > > > minute (what's more than OK for us) I was able to get rid of our
> > > issue.
> > > > > Not
> > > > > > also sure why this is not mentioned in MTTR section in HBase
> book,
> > as
> > > > > > obviously MTTR doesn't work at all with this default timeout, at
> > > least
> > > > it
> > > > > > doesn't work the way we expected it to work.
> > > > > >
> > > > > > So thanks again for everyone being spammed with this, and
> specially
> > > > > thanks
> > > > > > to Nicolas pointing me to the right direction.
> > > > > >
> > > > > > On Mon, Mar 23, 2015 at 1:37 PM Nicolas Liochon <
> nkeywal@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > the attachments are rejected by the mailing list, can you put
> > then
> > > on
> > > > > > > pastebin?
> > > > > > >
> > > > > > > stale is mandatory (so it's good), but the issue here is just
> > > before.
> > > > > The
> > > > > > > region server needs to read the file. In order to be sure that
> > > there
> > > > is
> > > > > > no
> > > > > > > data loss, it needs to "recover the lease", that means ensuring
> > > that
> > > > > > nobody
> > > > > > > is writing the file. The regionserver calls the namenode to do
> > this
> > > > > > > recoverLease. So there should be some info in the namenode
> logs.
> > > You
> > > > > have
> > > > > > > HDFS-4721 on your hdfs? The hbase book details (more or
> less...)
> > > this
> > > > > > > recoverLease stuff.
> > > > > > >
> > > > > > >
> > > > > > > On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <
> > > > dejan.menges@gmail.com
> > > > > >
> > > > > > > wrote:
> > > > > > >
> > > > > > > > And also, just checked - dfs.namenode.avoid.read.stale.
> > datanode
> > > and
> > > > > > > > dfs.namenode.avoid.write.stale.datanode
> > > > > > > > are both true, and dfs.namenode.stale.datanode.interval is
> > set to
> > > > > > > default
> > > > > > > > 30000.
> > > > > > > >
> > > > > > > > On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <
> > > > > dejan.menges@gmail.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi Nicolas,
> > > > > > > > >
> > > > > > > > > Please find log attached.
> > > > > > > > >
> > > > > > > > > As I see it now more clearly, it was trying to recover HDFS
> > > WALs
> > > > > from
> > > > > > > > node
> > > > > > > > > that's dead:
> > > > > > > > >
> > > > > > > > > 2015-03-23 08:53:44,381 WARN
> > > > > > org.apache.hadoop.hbase.util.FSHDFSUtils:
> > > > > > > > > Cannot recoverLease after trying for 900000ms
> > > > > > > > > (hbase.lease.recovery.timeout); continuing, but may be
> > > > > DATALOSS!!!;
> > > > > > > > > attempt=40 on
> > > > > > > > >
> > > > > > > > file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intent
> > > > > > > ionally_get_down_by_getting_network_down},60020,1426862900
> > > > > > > 506-splitting/{node_i_intentionally_get_down_by_gett
> > > > > > > ing_network_down}%2C60020%2C1426862900506.1427096924508
> > > > > > > > > after 908210ms
> > > > > > > > >
> > > > > > > > > And as you can see from the log, it tried 40 times, what
> took
> > > it
> > > > > > > exactly
> > > > > > > > > 15 minutes.
> > > > > > > > >
> > > > > > > > > There's probably some parameter to tune to cut it of from
> 40
> > > > times
> > > > > /
> > > > > > 15
> > > > > > > > > minutes to something more useful, as for 15 minutes we
> don't
> > > have
> > > > > our
> > > > > > > > > regions available, and HDFS have however replication factor
> > 3.
> > > > > > > > >
> > > > > > > > > Googling, if I figure out what's this I will post it here.
> > Will
> > > > > also
> > > > > > > > > appreciate if someone knows how to cut this down.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Dejan
> > > > > > > > >
> > > > > > > > > On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <
> > > > nkeywal@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > >> The split is done by the region servers (the master
> > > > coordinates).
> > > > > Is
> > > > > > > > there
> > > > > > > > >> some interesting stuff in their logs?
> > > > > > > > >>
> > > > > > > > >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <
> > > > > > dejan.menges@gmail.com
> > > > > > > >
> > > > > > > > >> wrote:
> > > > > > > > >>
> > > > > > > > >> > With client issue was that it was retrying connecting to
> > the
> > > > > same
> > > > > > > > region
> > > > > > > > >> > servers even when they were reassigned. Lowering it down
> > > > helped
> > > > > in
> > > > > > > > this
> > > > > > > > >> > specific use case, but yes, we still want servers to
> > > > reallocate
> > > > > > > > quickly.
> > > > > > > > >> >
> > > > > > > > >> > What got me here:
> > > > > > > > >> >
> > > > > > > > >> > http://hbase.apache.org/book.html#mttr
> > > > > > > > >> >
> > > > > > > > >> > I basically set configuration exactly the same way as
> it's
> > > > > > explained
> > > > > > > > >> here.
> > > > > > > > >> > *zookeeper.session.timeout* is (and was before) 60000
> (one
> > > > > > minute).
> > > > > > > > >> >
> > > > > > > > >> > So basically what happens is: - simulating network
> issues
> > we
> > > > had
> > > > > > > > >> recently).
> > > > > > > > >> > - After short time I see in HBase that my RegionServer
> is
> > > > dead,
> > > > > > and
> > > > > > > as
> > > > > > > > >> > total number of regions I see previous total minus
> number
> > of
> > > > > > regions
> > > > > > > > >> that
> > > > > > > > >> > were hosted on the node hosting RegionServer that just
> > > > > > > 'disappeared'.
> > > > > > > > >> > - In this point I want my clus
> > > > > > > > >> >
> > > > > > > > >> > - I have test cluster consisting of four nodes, every
> node
> > > > being
> > > > > > > > >> DataNode
> > > > > > > > >> > and RegionServer.
> > > > > > > > >> > - I simulate network partition on one (connect to it
> > through
> > > > > > console
> > > > > > > > and
> > > > > > > > >> > take network interface downter to recover as soon as
> > > possible,
> > > > > to
> > > > > > > > start
> > > > > > > > >> > serving missing regions.
> > > > > > > > >> > - First thing I see in HMaster logs are:
> > > > > > > > >> >
> > > > > > > > >> > 2015-03-20 14:17:26,015 INFO
> > > > > > > > >> > org.apache.hadoop.hbase.zookeeper.RegionServerTracker:
> > > > > > RegionServer
> > > > > > > > >> > ephemeral node deleted, processing expiration
> > > > > > > > >> > [{name_of_node_I_took_down},60020,1426860403261]
> > > > > > > > >> >
> > > > > > > > >> > 2015-03-20 14:17:26,067 INFO
> > > > > > > > >> >
> > > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler:
> > > > > > > > Splitting
> > > > > > > > >> > logs for {name_of_node_I_took_down},60020,1426860403261
> > > before
> > > > > > > > >> assignment.
> > > > > > > > >> >
> > > > > > > > >> > 2015-03-20 14:17:26,105 INFO
> > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: dead
> > > splitlog
> > > > > > > workers
> > > > > > > > [
> > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261]
> > > > > > > > >> >
> > > > > > > > >> > 2015-03-20 14:17:26,107 INFO
> > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: started
> > > > > splitting
> > > > > > 1
> > > > > > > > >> logs in
> > > > > > > > >> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_
> > I_
> > > > > > > took_down}
> > > > > > > > >> > ,60020,1426860403261-splitting]
> > > > > > > > >> >
> > > > > > > > >> > 2015-03-20 14:17:26,150 INFO
> > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > > > > > > > >> > /hbase/splitWAL/WALs%2F
> > > > > > > > >> >
> > > > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > > > > > >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
> > 14268
> > > > > > > 60404905
> > > > > > > > >> > acquired by {fqdn_of_another_live_hnode},
> > 60020,1426859445623
> > > > > > > > >> >
> > > > > > > > >> > 2015-03-20 14:17:26,182 INFO
> > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total
> > tasks
> > > =
> > > > 1
> > > > > > > > >> unassigned
> > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > 2F{name_of_node_I_took_down}
> > > > > > > > >> >
> > > > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > took_
> > > > > > > > >>
> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > > > > in_progress
> > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done
> > = 0
> > > > > > error =
> > > > > > > > 0}
> > > > > > > > >> >
> > > > > > > > >> > 2015-03-20 14:17:31,183 INFO
> > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total
> > tasks
> > > =
> > > > 1
> > > > > > > > >> unassigned
> > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > 2F{name_of_node_I_took_down}
> > > > > > > > >> >
> > > > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > took_
> > > > > > > > >>
> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > > > > in_progress
> > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done
> > = 0
> > > > > > error =
> > > > > > > > 0}
> > > > > > > > >> >
> > > > > > > > >> > 2015-03-20 14:17:36,184 INFO
> > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total
> > tasks
> > > =
> > > > 1
> > > > > > > > >> unassigned
> > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > 2F{name_of_node_I_took_down}
> > > > > > > > >> >
> > > > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > took_
> > > > > > > > >>
> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > > > > in_progress
> > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done
> > = 0
> > > > > > error =
> > > > > > > > 0}
> > > > > > > > >> >
> > > > > > > > >> > 2015-03-20 14:17:42,185 INFO
> > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total
> > tasks
> > > =
> > > > 1
> > > > > > > > >> unassigned
> > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > 2F{name_of_node_I_took_down}
> > > > > > > > >> >
> > > > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > took_
> > > > > > > > >>
> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > > > > in_progress
> > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done
> > = 0
> > > > > > error =
> > > > > > > > 0}
> > > > > > > > >> >
> > > > > > > > >> > 2015-03-20 14:17:48,184 INFO
> > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total
> > tasks
> > > =
> > > > 1
> > > > > > > > >> unassigned
> > > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> > 2F{name_of_node_I_took_down}
> > > > > > > > >> >
> > > > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> > took_
> > > > > > > > >>
> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > > > > in_progress
> > > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done
> > = 0
> > > > > > error =
> > > > > > > > 0}
> > > > > > > > >> > In the meantime, In hbase...out log I got this:
> > > > > > > > >> >
> > > > > > > > >> > ==> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out <==
> > > > > > > > >> >
> > > > > > > > >> > java.io.IOException: Call to
> > > > > > > > >> >
> {name_of_node_I_took_down}/{ip_of_local_interface_I_took_
> > dow
> > > > > > > n}:60020
> > > > > > > > >> > failed on local exception:
> > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> > CallTimeoutException:
> > > > > Call
> > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
> > > > > > > > >> t.java:1532)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:
> > > > > 1502)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> >
> > > > > > > > >> >
> org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(
> > Rpc
> > > > > > > > >> Client.java:1684)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> >
> > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> > BlockingRpcChannelImpl
> > > > > > > ementati
> > > > > > > > >> on.callBlockingMethod(RpcClient.java:1737)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> >
> > > > > > > > >> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$
> > > > > > > AdminService$
> > > > > > > > >> BlockingStub.getRegionInfo(AdminProtos.java:20806)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> >
> > > > > > > > >> > org.apache.hadoop.hbase.client.HBaseAdmin.
> > getCompactionState
> > > > > > > > >> (HBaseAdmin.java:2524)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> >
> > > > > > > > >> > org.apache.hadoop.hbase.generated.master.table_jsp._
> > jspServi
> > > > > > > > >> ce(table_jsp.java:167)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.
> > > > > > > java:98)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
> > > > > > > > >> .java:511)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> >
> > > > > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> > doFilte
> > > > > > > > >> r(ServletHandler.java:1221)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> >
> > > > > > > > >> > org.apache.hadoop.http.lib.StaticUserWebFilter$
> > StaticUserFil
> > > > > > > > >> ter.doFilter(StaticUserWebFilter.java:109)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> >
> > > > > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> > doFilte
> > > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> >
> > > > > > > > >> > org.apache.hadoop.http.HttpServer$QuotingInputFilter.
> > doFilte
> > > > > > > > >> r(HttpServer.java:1081)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> >
> > > > > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> > doFilte
> > > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > >
> > > > org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> >
> > > > > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> > doFilte
> > > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
> > > > > > > > >> er.java:399)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> > org.mortbay.jetty.security.SecurityHandler.handle(
> > SecurityHa
> > > > > > > > >> ndler.java:216)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
> > > > > > > > >> er.java:182)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
> > > > > > > > >> er.java:766)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
> > > > > > > > >> java:450)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> >
> > > > > > > > >> > org.mortbay.jetty.handler.ContextHandlerCollection.
> > handle(Co
> > > > > > > > >> ntextHandlerCollection.java:230)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
> > > > > > > > >> er.java:152)
> > > > > > > > >> >
> > > > > > > > >> > at org.mortbay.jetty.Server.handle(Server.java:326)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
> > > > > > > > >> n.java:542)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> >
> > > > > > > > >> > org.mortbay.jetty.HttpConnection$RequestHandler.
> > headerComple
> > > > > > > > >> te(HttpConnection.java:928)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> >
> > > > > > > > >> > org.mortbay.io.nio.SelectChannelEndPoint.run(
> > SelectChannelEn
> > > > > > > > >> dPoint.java:410)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> >
> > > > > > > > >> > org.mortbay.thread.QueuedThreadPool$PoolThread.run(
> > > > > > > > >> QueuedThreadPool.java:582)
> > > > > > > > >> >
> > > > > > > > >> > Caused by: org.apache.hadoop.hbase.ipc.RpcClient$
> > > > > > > CallTimeoutException:
> > > > > > > > >> Call
> > > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> >
> > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.
> > cleanupCall
> > > > > > > > >> s(RpcClient.java:1234)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > > > > > >> >
> > > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.
> > readRespons
> > > > > > > > >> e(RpcClient.java:1171)
> > > > > > > > >> >
> > > > > > > > >> > at
> > > > org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
> > > > > > > > >> nt.java:751)
> > > > > > > > >> > Beside this same issue, please note that first message
> was
> > > at
> > > > > > > > 2015-03-20
> > > > > > > > >> > 14:17:26,015. And then (we got to the point when it
> > started
> > > > > > > > transition):
> > > > > > > > >> >
> > > > > > > > >> > 2015-03-20 14:32:35,059 INFO
> > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > > > > > > > >> > /hbase/splitWAL/WALs%2F
> > > > > > > > >> >
> > > > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > > > > > >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
> > 14268
> > > > > > > 60404905
> > > > > > > > >> > entered state: DONE
> > > > {fqdn_of_new_live_node},60020,1426859445623
> > > > > > > > >> >
> > > > > > > > >> > 2015-03-20 14:32:35,109 INFO
> > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: Done
> > > splitting
> > > > > > > > >> > /hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > > > > >> >
> > > > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}
> > > > > > > > >> > %252C60020%252C1426860403261.1426860404905
> > > > > > > > >> >
> > > > > > > > >> > 2015-03-20 14:32:35,190 INFO
> > > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: finished
> > > > > splitting
> > > > > > > > >> (more
> > > > > > > > >> > than or equal to) 9 bytes in 1 log files in
> > > > > > > > >> >
> > > > > > > > >> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_
> > of_
> > > > > > > > >> node_I_took_down},60020,1426860403261-splitting]
> > > > > > > > >> > in 909083ms
> > > > > > > > >> >
> > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > org.apache.hadoop.hbase.master
> > > > > > > > >> .RegionStates:
> > > > > > > > >> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c
> > state=OPEN,
> > > > > > > > >> > ts=1426860639088,
> > > > > > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > >> to
> > > > > > > > >> > {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE,
> > > > > ts=1426861955191,
> > > > > > > > >> server=
> > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > >> >
> > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > org.apache.hadoop.hbase.master
> > > > > > > > >> .RegionStates:
> > > > > > > > >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
> > > > > > > > >> {name_of_node_I_took_down}
> > > > > > > > >> > ,60020,1426860403261
> > > > > > > > >> >
> > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > org.apache.hadoop.hbase.master
> > > > > > > > >> .RegionStates:
> > > > > > > > >> > Transitioned {25ab6e7b42e36ddaa723d71be5954543
> > state=OPEN,
> > > > > > > > >> > ts=1426860641783,
> > > > > > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > >> to
> > > > > > > > >> > {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE,
> > > > > ts=1426861955191,
> > > > > > > > >> server=
> > > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > > > > > >> >
> > > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> > org.apache.hadoop.hbase.master
> > > > > > > > >> .RegionStates:
> > > > > > > > >> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
> > > > > > > > >> {name_of_node_I_took_down}
> > > > > > > > >> > ,60020,1426860403261
> > > > > > > > >> > At this point, note that it finished SplitLogManager
> task
> > at
> > > > > > > 14:32:35
> > > > > > > > >> and
> > > > > > > > >> > started transitioning just after that. So this is 15
> > minutes
> > > > > that
> > > > > > > I'm
> > > > > > > > >> > talking about.
> > > > > > > > >> >
> > > > > > > > >> > What am I missing?
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <
> > > > > > nkeywal@gmail.com>
> > > > > > > > >> wrote:
> > > > > > > > >> >
> > > > > > > > >> > > You've changed the value of hbase.zookeeper.timeout to
> > 15
> > > > > > > minutes? A
> > > > > > > > >> very
> > > > > > > > >> > > reasonable target is 1 minute before relocating the
> > > regions.
> > > > > > > That's
> > > > > > > > >> the
> > > > > > > > >> > > default iirc. You can push it to 20s, but then
> > > > > > > gc-stopping-the-world
> > > > > > > > >> > > becomes more of an issue. 15 minutes is really a lot.
> > The
> > > > hdfs
> > > > > > > stale
> > > > > > > > >> mode
> > > > > > > > >> > > must always be used, with a lower timeout than the
> hbase
> > > > one.
> > > > > > > > >> > >
> > > > > > > > >> > > Client side there should be nothing to do (excepted
> for
> > > > > advanced
> > > > > > > > >> stuff);
> > > > > > > > >> > at
> > > > > > > > >> > > each retry the client checks the location of the
> > regions.
> > > If
> > > > > you
> > > > > > > > lower
> > > > > > > > >> > the
> > > > > > > > >> > > number of retry the client will fail sooner, but
> usually
> > > you
> > > > > > don't
> > > > > > > > >> want
> > > > > > > > >> > the
> > > > > > > > >> > > client to fail, you want the servers to reallocate
> > > quickly.
> > > > > > > > >> > >
> > > > > > > > >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <
> > > > > > > > dejan.menges@gmail.com
> > > > > > > > >> >
> > > > > > > > >> > > wrote:
> > > > > > > > >> > >
> > > > > > > > >> > > > Hi,
> > > > > > > > >> > > >
> > > > > > > > >> > > > Sorry for little bit late update, but managed to
> > narrow
> > > it
> > > > > > > little
> > > > > > > > >> bit
> > > > > > > > >> > > down.
> > > > > > > > >> > > >
> > > > > > > > >> > > > We didn't update yet, as we are using Hortonworks
> > > > > distribution
> > > > > > > > right
> > > > > > > > >> > now,
> > > > > > > > >> > > > and even if we update we will get 0.98.4. However,
> > looks
> > > > > that
> > > > > > > > issue
> > > > > > > > >> > here
> > > > > > > > >> > > > was in our use case and configuration (still looking
> > > into
> > > > > it).
> > > > > > > > >> > > >
> > > > > > > > >> > > > Basically, initially I saw that when one server goes
> > > down,
> > > > > we
> > > > > > > > start
> > > > > > > > >> > > having
> > > > > > > > >> > > > performance issues in general, but it managed to be
> on
> > > our
> > > > > > > client
> > > > > > > > >> side,
> > > > > > > > >> > > due
> > > > > > > > >> > > > to caching, and clients were trying to reconnect to
> > > nodes
> > > > > that
> > > > > > > > were
> > > > > > > > >> > > offline
> > > > > > > > >> > > > and later trying to get regions from those nodes
> too.
> > > This
> > > > > is
> > > > > > > > >> basically
> > > > > > > > >> > > why
> > > > > > > > >> > > > on server side I didn't manage to see anything in
> logs
> > > > that
> > > > > > > would
> > > > > > > > >> be at
> > > > > > > > >> > > > least little bit interesting and point me into
> desired
> > > > > > > direction.
> > > > > > > > >> > > >
> > > > > > > > >> > > > Another question that popped up to me is - in case
> > > server
> > > > is
> > > > > > > down
> > > > > > > > >> (and
> > > > > > > > >> > > with
> > > > > > > > >> > > > it DataNode and HRegionServer it was hosting) -
> what's
> > > > > optimal
> > > > > > > > time
> > > > > > > > >> to
> > > > > > > > >> > > set
> > > > > > > > >> > > > for HMaster to consider server dead reassign regions
> > > > > somewhere
> > > > > > > > >> else, as
> > > > > > > > >> > > > this is another performance bottleneck we hit during
> > > > > inability
> > > > > > > to
> > > > > > > > >> > access
> > > > > > > > >> > > > regions? In our case it's configured to 15 minutes,
> > and
> > > > > simple
> > > > > > > > logic
> > > > > > > > >> > > tells
> > > > > > > > >> > > > me if you want it earlier then configure lower
> number
> > of
> > > > > > > retries,
> > > > > > > > >> but
> > > > > > > > >> > > issue
> > > > > > > > >> > > > is as always in details, so not sure if anyone knows
> > > some
> > > > > > better
> > > > > > > > >> math
> > > > > > > > >> > for
> > > > > > > > >> > > > this?
> > > > > > > > >> > > >
> > > > > > > > >> > > > And last question - is it possible to manually force
> > > HBase
> > > > > to
> > > > > > > > >> reassign
> > > > > > > > >> > > > regions? In this case, while HMaster is retrying to
> > > > contact
> > > > > > node
> > > > > > > > >> that's
> > > > > > > > >> > > > dead, it's impossible to force it using 'balancer'
> > > > command.
> > > > > > > > >> > > >
> > > > > > > > >> > > > Thanks a lot!
> > > > > > > > >> > > >
> > > > > > > > >> > > > Dejan
> > > > > > > > >> > > >
> > > > > > > > >> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <
> > > > > > > > >> dejan.menges@gmail.com>
> > > > > > > > >> > > > wrote:
> > > > > > > > >> > > >
> > > > > > > > >> > > > > Hi,
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > To be very honest - there's no particular reason
> why
> > > we
> > > > > > stick
> > > > > > > to
> > > > > > > > >> this
> > > > > > > > >> > > > one,
> > > > > > > > >> > > > > beside just lack of time currently to go through
> > > upgrade
> > > > > > > > process,
> > > > > > > > >> but
> > > > > > > > >> > > > looks
> > > > > > > > >> > > > > to me that's going to be next step.
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > Had a crazy day, didn't have time to go through
> all
> > > logs
> > > > > > > again,
> > > > > > > > >> plus
> > > > > > > > >> > > one
> > > > > > > > >> > > > > of the machines (last one where we had this issue)
> > is
> > > > > fully
> > > > > > > > >> > > reprovisioned
> > > > > > > > >> > > > > yesterday so I don't have logs from there anymore.
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > Beside upgrading,  what I will talk about today,
> can
> > > you
> > > > > > just
> > > > > > > > >> point
> > > > > > > > >> > me
> > > > > > > > >> > > to
> > > > > > > > >> > > > > the specific RPC issue in 0.98.0? Thing is that we
> > > have
> > > > > some
> > > > > > > > >> strange
> > > > > > > > >> > > > > moments with RPC in this case, and just want to
> see
> > if
> > > > > > that's
> > > > > > > > the
> > > > > > > > >> > same
> > > > > > > > >> > > > > thing (and we were even suspecting to RPC).
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > Thanks a lot!
> > > > > > > > >> > > > > Dejan
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew Purtell <
> > > > > > > > >> apurtell@apache.org
> > > > > > > > >> > >
> > > > > > > > >> > > > > wrote:
> > > > > > > > >> > > > >
> > > > > > > > >> > > > >> Is there a particular reason why you are using
> > HBase
> > > > > > 0.98.0?
> > > > > > > > The
> > > > > > > > >> > > latest
> > > > > > > > >> > > > >> 0.98 release is 0.98.11. There's a known
> > performance
> > > > > issue
> > > > > > > with
> > > > > > > > >> > 0.98.0
> > > > > > > > >> > > > >> pertaining to RPC that was fixed in later
> releases,
> > > you
> > > > > > > should
> > > > > > > > >> move
> > > > > > > > >> > up
> > > > > > > > >> > > > >> from
> > > > > > > > >> > > > >> 0.98.0. In addition hundreds of improvements and
> > bug
> > > > > fixes
> > > > > > > have
> > > > > > > > >> gone
> > > > > > > > >> > > > into
> > > > > > > > >> > > > >> the ten releases since 0.98.0.
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <
> > > > > > > > >> > dejan.menges@gmail.com
> > > > > > > > >> > > >
> > > > > > > > >> > > > >> wrote:
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> > Hi All,
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > We have a strange issue with HBase performance
> > > > (overall
> > > > > > > > cluster
> > > > > > > > >> > > > >> > performance) in case one of datanodes in the
> > > cluster
> > > > > > > > >> unexpectedly
> > > > > > > > >> > > goes
> > > > > > > > >> > > > >> > down.
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > So scenario is like follows:
> > > > > > > > >> > > > >> > - Cluster works fine, it's stable.
> > > > > > > > >> > > > >> > - One DataNode unexpectedly goes down (PSU
> issue,
> > > > > network
> > > > > > > > >> issue,
> > > > > > > > >> > > > >> anything)
> > > > > > > > >> > > > >> > - Whole HBase cluster goes down (performance
> > > becomes
> > > > so
> > > > > > bad
> > > > > > > > >> that
> > > > > > > > >> > we
> > > > > > > > >> > > > >> have to
> > > > > > > > >> > > > >> > restart all RegionServers to get it back to
> > life).
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > Most funny and latest issue that happened was
> > that
> > > we
> > > > > > added
> > > > > > > > new
> > > > > > > > >> > node
> > > > > > > > >> > > > to
> > > > > > > > >> > > > >> the
> > > > > > > > >> > > > >> > cluster (having 8 x 4T SATA disks) and we left
> > just
> > > > > > > DataNode
> > > > > > > > >> > running
> > > > > > > > >> > > > on
> > > > > > > > >> > > > >> it
> > > > > > > > >> > > > >> > to give it couple of days to get some data. At
> > some
> > > > > point
> > > > > > > in
> > > > > > > > >> time,
> > > > > > > > >> > > due
> > > > > > > > >> > > > >> to
> > > > > > > > >> > > > >> > hardware issue, server rebooted (twice during
> > three
> > > > > > hours)
> > > > > > > in
> > > > > > > > >> > moment
> > > > > > > > >> > > > >> when
> > > > > > > > >> > > > >> > it had maybe 5% of data it would have in a
> couple
> > > of
> > > > > > days.
> > > > > > > > >> Nothing
> > > > > > > > >> > > > else
> > > > > > > > >> > > > >> > beside DataNode was running, and once it went
> > down,
> > > > it
> > > > > > > > affected
> > > > > > > > >> > > > literary
> > > > > > > > >> > > > >> > everything, and restarting RegionServers in the
> > end
> > > > > fixed
> > > > > > > it.
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >> > We are using HBase 0.98.0 with Hadoop 2.4.0
> > > > > > > > >> > > > >> >
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> --
> > > > > > > > >> > > > >> Best regards,
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >>    - Andy
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >> Problems worthy of attack prove their worth by
> > > hitting
> > > > > > back.
> > > > > > > -
> > > > > > > > >> Piet
> > > > > > > > >> > > Hein
> > > > > > > > >> > > > >> (via Tom White)
> > > > > > > > >> > > > >>
> > > > > > > > >> > > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > >
> > > > > > > > >> > >
> > > > > > > > >> >
> > > > > > > > >>
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Strange issue when DataNode goes down

Posted by Dejan Menges <de...@gmail.com>.
Interesting discussion I also found, gives me some more light on what
Nicolas is talking about - https://issues.apache.org/jira/browse/HDFS-3703

On Mon, Mar 23, 2015 at 3:53 PM Bryan Beaudreault <bb...@hubspot.com>
wrote:

> So it is safe to set hbase.lease.recovery.timeout lower if you also
> set heartbeat.recheck.interval lower (lowering that 10.5 min dead node
> timer)?  Or is it recommended to not touch either of those?
>
> Reading the above with interest, thanks for digging in here guys.
>
> On Mon, Mar 23, 2015 at 10:13 AM, Nicolas Liochon <nk...@gmail.com>
> wrote:
>
> > If the node is actually down it's fine. But the node may not be that down
> > (CAP theorem here); and then it's looking for trouble.
> > HDFS, by default declare a node as dead after 10:30. 15 minutes is an
> extra
> > security. It seems your hdfs settings are different (or there is a
> bug...).
> > There should be some info in the hdfs logs.
> >
> > On Mon, Mar 23, 2015 at 3:05 PM, Dejan Menges <de...@gmail.com>
> > wrote:
> >
> > > Will take a look.
> > >
> > > Actually, if node is down (someone unplugged network cable, it just
> died,
> > > whatever) what's chance it's going to become live so write can
> continue?
> > On
> > > the other side, HBase is not starting recovery trying to contact node
> > which
> > > is not there anymore, and even elected as dead on Namenode side
> (another
> > > thing I didn't understood quite good).
> > >
> > > So what I was expecting is that as soon as Namenode decided node is
> dead,
> > > that it would be enough for RegionServer to stop trying to recover from
> > the
> > > dead node, but it wasn't the case. Also, this whole MTTR article in
> HBase
> > > book doesn't work at all with this parameter set to it's default value
> > (15
> > > minutes).
> > >
> > > So I'm still struggling to figure out what's drawback exactly on this?
> > >
> > > On Mon, Mar 23, 2015 at 2:50 PM Nicolas Liochon <nk...@gmail.com>
> > wrote:
> > >
> > > > Thanks for the explanation. There is an issue if you modify this
> > setting
> > > > however.
> > > > hbase tries to recover the lease (i.e. be sure that nobody is
> writing)
> > > > If you change hbase.lease.recovery.timeout hbase will start the
> > recovery
> > > > (i.e. start to read) even if it's not sure that nobody's writing.
> That
> > > > means there is a dataloss risk.
> > > > Basically, you must not see this warning: WARN
> > > > org.apache.hadoop.hbase.util.FSHDFSUtils: Cannot recoverLease after
> > > trying
> > > > for[]
> > > >
> > > > The recoverLease must succeed. The fact that it does not after X
> tries
> > is
> > > > strange.
> > > > There may be a mistmatch between the hbase parameters and the hdfs
> > ones.
> > > > You may need to have a look at the comments in FSHDFSUtils.java
> > > >
> > > >
> > > >
> > > >
> > > > On Mon, Mar 23, 2015 at 2:15 PM, Dejan Menges <
> dejan.menges@gmail.com>
> > > > wrote:
> > > >
> > > > > I found the issue and fixed it, and will try to explain here what
> was
> > > > > exactly in our case in case someone else finds this interesting
> too.
> > > > >
> > > > > So initially, we had (couple of times) some network and hardware
> > issues
> > > > in
> > > > > our datacenters. When one server would die (literary die, we had
> some
> > > > issue
> > > > > with PSUs) we saw issues with overall cluster performance on HBase
> > > side.
> > > > As
> > > > > cluster is quite big and live, it was also quite hard to figure out
> > > exact
> > > > > root cause and how to fix the stuff we wanted to fix.
> > > > >
> > > > > So I set up another cluster, four nodes (with DataNode and
> > > RegionServer)
> > > > > and two other nodes with HMaster and Namenode in HA, using same
> stuff
> > > we
> > > > > use on production. We pumped some data into it, and I was able to
> > > > reproduce
> > > > > same issue last week on it. What I tried to do is to cut one server
> > > (shut
> > > > > down it's interface) when all is good with cluster, when we have
> > load,
> > > > and
> > > > > see what's going to happen.
> > > > >
> > > > > On Friday, after Nicolas mentioned, I started taking a look in
> HBase
> > > logs
> > > > > on the node which was mentioned in HMaster log as the one taking
> over
> > > > > regions for the dead server. Basically what I was able to observe
> was
> > > 15
> > > > > minutes time (+- couple of seconds, what was also interesting, and
> > will
> > > > got
> > > > > later to that) between HMaster figures out that one of it's
> > > RegionServers
> > > > > is dead, and the time one of the mentioned nodes starts taking over
> > > those
> > > > > regions and they start appearing in HMaster's Web UI.
> > > > >
> > > > > I then set up everything like mentioned here
> > > > > http://hbase.apache.org/book.html#mttr - but still had exactly the
> > > same
> > > > > issues. Went over (again and again) all currently configured stuff,
> > but
> > > > > still had the same issue.
> > > > >
> > > > > Then I started looking into HDFS. Opened NameNode UI, saw all is
> > good,
> > > > took
> > > > > one node down, was also looking RegionServer logs in the same time
> -
> > > and
> > > > I
> > > > > also see that it took ~15 minutes for Namenode to elect dead node
> as
> > > > dead.
> > > > > Somehow in the same moment regions started getting back to life. I
> > > > remember
> > > > > in some older versions dfs timeout checks and check retries. Looked
> > > into
> > > > > defaults for our Hadoop version -
> > > > >
> > > > > http://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/
> > > > hadoop-hdfs/hdfs-default.xml
> > > > > - and saw there that there's no recheck parameter anymore. Strange,
> > as
> > > on
> > > > > StackOverflow I found post from month ago, for newer version than
> we
> > > use
> > > > > (we use 2.4.1, guy was using 2.6 - dfs.namenode.heartbeat.
> > > > recheck-interval)
> > > > > I set it to 10 seconds as he mentioned, having checks every three
> > > seconds
> > > > > (default) and got DataNode elected as dead in ~45 seconds, as he
> > > > mentioned.
> > > > > Not sure why this parameter is not documented, but obviously it
> > works.
> > > > >
> > > > > Then figured out it still didn't fix our HBase failover issue. I
> was
> > > > > looking into HBase book again and again, and then saw this part:
> > > > >
> > > > > "How much time we allow elapse between calls to recover lease.
> Should
> > > be
> > > > > larger than the dfs timeout."
> > > > >
> > > > > This was description for hbase.lease.recovery.dfs.timeout. Wasn't
> > sure
> > > > from
> > > > > the comment what of all timeouts that's possible to set in
> > Hadoop/HBase
> > > > and
> > > > > that has something to do with DFS is this all about. But picked
> > > > > hbase.lease.recovery.timeout, and that was the catch.
> > > > >
> > > > > Initially, by default, hbase.lease.recovery.timeout is set to 15
> > > minutes.
> > > > > Not sure why, and wasn't able to find yet why, but getting this
> down
> > to
> > > > one
> > > > > minute (what's more than OK for us) I was able to get rid of our
> > issue.
> > > > Not
> > > > > also sure why this is not mentioned in MTTR section in HBase book,
> as
> > > > > obviously MTTR doesn't work at all with this default timeout, at
> > least
> > > it
> > > > > doesn't work the way we expected it to work.
> > > > >
> > > > > So thanks again for everyone being spammed with this, and specially
> > > > thanks
> > > > > to Nicolas pointing me to the right direction.
> > > > >
> > > > > On Mon, Mar 23, 2015 at 1:37 PM Nicolas Liochon <nkeywal@gmail.com
> >
> > > > wrote:
> > > > >
> > > > > > the attachments are rejected by the mailing list, can you put
> then
> > on
> > > > > > pastebin?
> > > > > >
> > > > > > stale is mandatory (so it's good), but the issue here is just
> > before.
> > > > The
> > > > > > region server needs to read the file. In order to be sure that
> > there
> > > is
> > > > > no
> > > > > > data loss, it needs to "recover the lease", that means ensuring
> > that
> > > > > nobody
> > > > > > is writing the file. The regionserver calls the namenode to do
> this
> > > > > > recoverLease. So there should be some info in the namenode logs.
> > You
> > > > have
> > > > > > HDFS-4721 on your hdfs? The hbase book details (more or less...)
> > this
> > > > > > recoverLease stuff.
> > > > > >
> > > > > >
> > > > > > On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <
> > > dejan.menges@gmail.com
> > > > >
> > > > > > wrote:
> > > > > >
> > > > > > > And also, just checked - dfs.namenode.avoid.read.stale.
> datanode
> > and
> > > > > > > dfs.namenode.avoid.write.stale.datanode
> > > > > > > are both true, and dfs.namenode.stale.datanode.interval is
> set to
> > > > > > default
> > > > > > > 30000.
> > > > > > >
> > > > > > > On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <
> > > > dejan.menges@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hi Nicolas,
> > > > > > > >
> > > > > > > > Please find log attached.
> > > > > > > >
> > > > > > > > As I see it now more clearly, it was trying to recover HDFS
> > WALs
> > > > from
> > > > > > > node
> > > > > > > > that's dead:
> > > > > > > >
> > > > > > > > 2015-03-23 08:53:44,381 WARN
> > > > > org.apache.hadoop.hbase.util.FSHDFSUtils:
> > > > > > > > Cannot recoverLease after trying for 900000ms
> > > > > > > > (hbase.lease.recovery.timeout); continuing, but may be
> > > > DATALOSS!!!;
> > > > > > > > attempt=40 on
> > > > > > > >
> > > > > > > file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intent
> > > > > > ionally_get_down_by_getting_network_down},60020,1426862900
> > > > > > 506-splitting/{node_i_intentionally_get_down_by_gett
> > > > > > ing_network_down}%2C60020%2C1426862900506.1427096924508
> > > > > > > > after 908210ms
> > > > > > > >
> > > > > > > > And as you can see from the log, it tried 40 times, what took
> > it
> > > > > > exactly
> > > > > > > > 15 minutes.
> > > > > > > >
> > > > > > > > There's probably some parameter to tune to cut it of from 40
> > > times
> > > > /
> > > > > 15
> > > > > > > > minutes to something more useful, as for 15 minutes we don't
> > have
> > > > our
> > > > > > > > regions available, and HDFS have however replication factor
> 3.
> > > > > > > >
> > > > > > > > Googling, if I figure out what's this I will post it here.
> Will
> > > > also
> > > > > > > > appreciate if someone knows how to cut this down.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Dejan
> > > > > > > >
> > > > > > > > On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <
> > > nkeywal@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > >
> > > > > > > >> The split is done by the region servers (the master
> > > coordinates).
> > > > Is
> > > > > > > there
> > > > > > > >> some interesting stuff in their logs?
> > > > > > > >>
> > > > > > > >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <
> > > > > dejan.menges@gmail.com
> > > > > > >
> > > > > > > >> wrote:
> > > > > > > >>
> > > > > > > >> > With client issue was that it was retrying connecting to
> the
> > > > same
> > > > > > > region
> > > > > > > >> > servers even when they were reassigned. Lowering it down
> > > helped
> > > > in
> > > > > > > this
> > > > > > > >> > specific use case, but yes, we still want servers to
> > > reallocate
> > > > > > > quickly.
> > > > > > > >> >
> > > > > > > >> > What got me here:
> > > > > > > >> >
> > > > > > > >> > http://hbase.apache.org/book.html#mttr
> > > > > > > >> >
> > > > > > > >> > I basically set configuration exactly the same way as it's
> > > > > explained
> > > > > > > >> here.
> > > > > > > >> > *zookeeper.session.timeout* is (and was before) 60000 (one
> > > > > minute).
> > > > > > > >> >
> > > > > > > >> > So basically what happens is: - simulating network issues
> we
> > > had
> > > > > > > >> recently).
> > > > > > > >> > - After short time I see in HBase that my RegionServer is
> > > dead,
> > > > > and
> > > > > > as
> > > > > > > >> > total number of regions I see previous total minus number
> of
> > > > > regions
> > > > > > > >> that
> > > > > > > >> > were hosted on the node hosting RegionServer that just
> > > > > > 'disappeared'.
> > > > > > > >> > - In this point I want my clus
> > > > > > > >> >
> > > > > > > >> > - I have test cluster consisting of four nodes, every node
> > > being
> > > > > > > >> DataNode
> > > > > > > >> > and RegionServer.
> > > > > > > >> > - I simulate network partition on one (connect to it
> through
> > > > > console
> > > > > > > and
> > > > > > > >> > take network interface downter to recover as soon as
> > possible,
> > > > to
> > > > > > > start
> > > > > > > >> > serving missing regions.
> > > > > > > >> > - First thing I see in HMaster logs are:
> > > > > > > >> >
> > > > > > > >> > 2015-03-20 14:17:26,015 INFO
> > > > > > > >> > org.apache.hadoop.hbase.zookeeper.RegionServerTracker:
> > > > > RegionServer
> > > > > > > >> > ephemeral node deleted, processing expiration
> > > > > > > >> > [{name_of_node_I_took_down},60020,1426860403261]
> > > > > > > >> >
> > > > > > > >> > 2015-03-20 14:17:26,067 INFO
> > > > > > > >> >
> > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler:
> > > > > > > Splitting
> > > > > > > >> > logs for {name_of_node_I_took_down},60020,1426860403261
> > before
> > > > > > > >> assignment.
> > > > > > > >> >
> > > > > > > >> > 2015-03-20 14:17:26,105 INFO
> > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: dead
> > splitlog
> > > > > > workers
> > > > > > > [
> > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261]
> > > > > > > >> >
> > > > > > > >> > 2015-03-20 14:17:26,107 INFO
> > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: started
> > > > splitting
> > > > > 1
> > > > > > > >> logs in
> > > > > > > >> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_
> I_
> > > > > > took_down}
> > > > > > > >> > ,60020,1426860403261-splitting]
> > > > > > > >> >
> > > > > > > >> > 2015-03-20 14:17:26,150 INFO
> > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > > > > > > >> > /hbase/splitWAL/WALs%2F
> > > > > > > >> >
> > > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > > > > >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
> 14268
> > > > > > 60404905
> > > > > > > >> > acquired by {fqdn_of_another_live_hnode},
> 60020,1426859445623
> > > > > > > >> >
> > > > > > > >> > 2015-03-20 14:17:26,182 INFO
> > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total
> tasks
> > =
> > > 1
> > > > > > > >> unassigned
> > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> 2F{name_of_node_I_took_down}
> > > > > > > >> >
> > > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> took_
> > > > > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > > > in_progress
> > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done
> = 0
> > > > > error =
> > > > > > > 0}
> > > > > > > >> >
> > > > > > > >> > 2015-03-20 14:17:31,183 INFO
> > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total
> tasks
> > =
> > > 1
> > > > > > > >> unassigned
> > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> 2F{name_of_node_I_took_down}
> > > > > > > >> >
> > > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> took_
> > > > > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > > > in_progress
> > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done
> = 0
> > > > > error =
> > > > > > > 0}
> > > > > > > >> >
> > > > > > > >> > 2015-03-20 14:17:36,184 INFO
> > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total
> tasks
> > =
> > > 1
> > > > > > > >> unassigned
> > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> 2F{name_of_node_I_took_down}
> > > > > > > >> >
> > > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> took_
> > > > > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > > > in_progress
> > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done
> = 0
> > > > > error =
> > > > > > > 0}
> > > > > > > >> >
> > > > > > > >> > 2015-03-20 14:17:42,185 INFO
> > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total
> tasks
> > =
> > > 1
> > > > > > > >> unassigned
> > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> 2F{name_of_node_I_took_down}
> > > > > > > >> >
> > > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> took_
> > > > > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > > > in_progress
> > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done
> = 0
> > > > > error =
> > > > > > > 0}
> > > > > > > >> >
> > > > > > > >> > 2015-03-20 14:17:48,184 INFO
> > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total
> tasks
> > =
> > > 1
> > > > > > > >> unassigned
> > > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%
> 2F{name_of_node_I_took_down}
> > > > > > > >> >
> > > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> took_
> > > > > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > > > in_progress
> > > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done
> = 0
> > > > > error =
> > > > > > > 0}
> > > > > > > >> > In the meantime, In hbase...out log I got this:
> > > > > > > >> >
> > > > > > > >> > ==> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out <==
> > > > > > > >> >
> > > > > > > >> > java.io.IOException: Call to
> > > > > > > >> > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_
> dow
> > > > > > n}:60020
> > > > > > > >> > failed on local exception:
> > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> CallTimeoutException:
> > > > Call
> > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > > > > >> >
> > > > > > > >> > at
> > > org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
> > > > > > > >> t.java:1532)
> > > > > > > >> >
> > > > > > > >> > at
> > org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:
> > > > 1502)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> >
> > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(
> Rpc
> > > > > > > >> Client.java:1684)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> >
> > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$
> BlockingRpcChannelImpl
> > > > > > ementati
> > > > > > > >> on.callBlockingMethod(RpcClient.java:1737)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> >
> > > > > > > >> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$
> > > > > > AdminService$
> > > > > > > >> BlockingStub.getRegionInfo(AdminProtos.java:20806)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> >
> > > > > > > >> > org.apache.hadoop.hbase.client.HBaseAdmin.
> getCompactionState
> > > > > > > >> (HBaseAdmin.java:2524)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> >
> > > > > > > >> > org.apache.hadoop.hbase.generated.master.table_jsp._
> jspServi
> > > > > > > >> ce(table_jsp.java:167)
> > > > > > > >> >
> > > > > > > >> > at
> > org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.
> > > > > > java:98)
> > > > > > > >> >
> > > > > > > >> > at
> > > javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
> > > > > > > >> >
> > > > > > > >> > at
> > > org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
> > > > > > > >> .java:511)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> >
> > > > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> doFilte
> > > > > > > >> r(ServletHandler.java:1221)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> >
> > > > > > > >> > org.apache.hadoop.http.lib.StaticUserWebFilter$
> StaticUserFil
> > > > > > > >> ter.doFilter(StaticUserWebFilter.java:109)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> >
> > > > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> doFilte
> > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> >
> > > > > > > >> > org.apache.hadoop.http.HttpServer$QuotingInputFilter.
> doFilte
> > > > > > > >> r(HttpServer.java:1081)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> >
> > > > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> doFilte
> > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > >
> > > org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> >
> > > > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> doFilte
> > > > > > > >> r(ServletHandler.java:1212)
> > > > > > > >> >
> > > > > > > >> > at
> > > org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
> > > > > > > >> er.java:399)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> > org.mortbay.jetty.security.SecurityHandler.handle(
> SecurityHa
> > > > > > > >> ndler.java:216)
> > > > > > > >> >
> > > > > > > >> > at
> > > org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
> > > > > > > >> er.java:182)
> > > > > > > >> >
> > > > > > > >> > at
> > > org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
> > > > > > > >> er.java:766)
> > > > > > > >> >
> > > > > > > >> > at
> > > org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
> > > > > > > >> java:450)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> >
> > > > > > > >> > org.mortbay.jetty.handler.ContextHandlerCollection.
> handle(Co
> > > > > > > >> ntextHandlerCollection.java:230)
> > > > > > > >> >
> > > > > > > >> > at
> > > org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
> > > > > > > >> er.java:152)
> > > > > > > >> >
> > > > > > > >> > at org.mortbay.jetty.Server.handle(Server.java:326)
> > > > > > > >> >
> > > > > > > >> > at
> > > org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
> > > > > > > >> n.java:542)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> >
> > > > > > > >> > org.mortbay.jetty.HttpConnection$RequestHandler.
> headerComple
> > > > > > > >> te(HttpConnection.java:928)
> > > > > > > >> >
> > > > > > > >> > at
> > org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> >
> > > > > > > >> > org.mortbay.io.nio.SelectChannelEndPoint.run(
> SelectChannelEn
> > > > > > > >> dPoint.java:410)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> >
> > > > > > > >> > org.mortbay.thread.QueuedThreadPool$PoolThread.run(
> > > > > > > >> QueuedThreadPool.java:582)
> > > > > > > >> >
> > > > > > > >> > Caused by: org.apache.hadoop.hbase.ipc.RpcClient$
> > > > > > CallTimeoutException:
> > > > > > > >> Call
> > > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> >
> > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.
> cleanupCall
> > > > > > > >> s(RpcClient.java:1234)
> > > > > > > >> >
> > > > > > > >> > at
> > > > > > > >> >
> > > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.
> readRespons
> > > > > > > >> e(RpcClient.java:1171)
> > > > > > > >> >
> > > > > > > >> > at
> > > org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
> > > > > > > >> nt.java:751)
> > > > > > > >> > Beside this same issue, please note that first message was
> > at
> > > > > > > 2015-03-20
> > > > > > > >> > 14:17:26,015. And then (we got to the point when it
> started
> > > > > > > transition):
> > > > > > > >> >
> > > > > > > >> > 2015-03-20 14:32:35,059 INFO
> > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > > > > > > >> > /hbase/splitWAL/WALs%2F
> > > > > > > >> >
> > > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > > > > >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.
> 14268
> > > > > > 60404905
> > > > > > > >> > entered state: DONE
> > > {fqdn_of_new_live_node},60020,1426859445623
> > > > > > > >> >
> > > > > > > >> > 2015-03-20 14:32:35,109 INFO
> > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: Done
> > splitting
> > > > > > > >> > /hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > > > >> >
> > > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}
> > > > > > > >> > %252C60020%252C1426860403261.1426860404905
> > > > > > > >> >
> > > > > > > >> > 2015-03-20 14:32:35,190 INFO
> > > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: finished
> > > > splitting
> > > > > > > >> (more
> > > > > > > >> > than or equal to) 9 bytes in 1 log files in
> > > > > > > >> >
> > > > > > > >> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_
> of_
> > > > > > > >> node_I_took_down},60020,1426860403261-splitting]
> > > > > > > >> > in 909083ms
> > > > > > > >> >
> > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> org.apache.hadoop.hbase.master
> > > > > > > >> .RegionStates:
> > > > > > > >> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c
> state=OPEN,
> > > > > > > >> > ts=1426860639088,
> > > > > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > > > > >> to
> > > > > > > >> > {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE,
> > > > ts=1426861955191,
> > > > > > > >> server=
> > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > > > > >> >
> > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> org.apache.hadoop.hbase.master
> > > > > > > >> .RegionStates:
> > > > > > > >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
> > > > > > > >> {name_of_node_I_took_down}
> > > > > > > >> > ,60020,1426860403261
> > > > > > > >> >
> > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> org.apache.hadoop.hbase.master
> > > > > > > >> .RegionStates:
> > > > > > > >> > Transitioned {25ab6e7b42e36ddaa723d71be5954543
> state=OPEN,
> > > > > > > >> > ts=1426860641783,
> > > > > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > > > > >> to
> > > > > > > >> > {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE,
> > > > ts=1426861955191,
> > > > > > > >> server=
> > > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > > > > >> >
> > > > > > > >> > 2015-03-20 14:32:35,191 INFO
> org.apache.hadoop.hbase.master
> > > > > > > >> .RegionStates:
> > > > > > > >> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
> > > > > > > >> {name_of_node_I_took_down}
> > > > > > > >> > ,60020,1426860403261
> > > > > > > >> > At this point, note that it finished SplitLogManager task
> at
> > > > > > 14:32:35
> > > > > > > >> and
> > > > > > > >> > started transitioning just after that. So this is 15
> minutes
> > > > that
> > > > > > I'm
> > > > > > > >> > talking about.
> > > > > > > >> >
> > > > > > > >> > What am I missing?
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <
> > > > > nkeywal@gmail.com>
> > > > > > > >> wrote:
> > > > > > > >> >
> > > > > > > >> > > You've changed the value of hbase.zookeeper.timeout to
> 15
> > > > > > minutes? A
> > > > > > > >> very
> > > > > > > >> > > reasonable target is 1 minute before relocating the
> > regions.
> > > > > > That's
> > > > > > > >> the
> > > > > > > >> > > default iirc. You can push it to 20s, but then
> > > > > > gc-stopping-the-world
> > > > > > > >> > > becomes more of an issue. 15 minutes is really a lot.
> The
> > > hdfs
> > > > > > stale
> > > > > > > >> mode
> > > > > > > >> > > must always be used, with a lower timeout than the hbase
> > > one.
> > > > > > > >> > >
> > > > > > > >> > > Client side there should be nothing to do (excepted for
> > > > advanced
> > > > > > > >> stuff);
> > > > > > > >> > at
> > > > > > > >> > > each retry the client checks the location of the
> regions.
> > If
> > > > you
> > > > > > > lower
> > > > > > > >> > the
> > > > > > > >> > > number of retry the client will fail sooner, but usually
> > you
> > > > > don't
> > > > > > > >> want
> > > > > > > >> > the
> > > > > > > >> > > client to fail, you want the servers to reallocate
> > quickly.
> > > > > > > >> > >
> > > > > > > >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <
> > > > > > > dejan.menges@gmail.com
> > > > > > > >> >
> > > > > > > >> > > wrote:
> > > > > > > >> > >
> > > > > > > >> > > > Hi,
> > > > > > > >> > > >
> > > > > > > >> > > > Sorry for little bit late update, but managed to
> narrow
> > it
> > > > > > little
> > > > > > > >> bit
> > > > > > > >> > > down.
> > > > > > > >> > > >
> > > > > > > >> > > > We didn't update yet, as we are using Hortonworks
> > > > distribution
> > > > > > > right
> > > > > > > >> > now,
> > > > > > > >> > > > and even if we update we will get 0.98.4. However,
> looks
> > > > that
> > > > > > > issue
> > > > > > > >> > here
> > > > > > > >> > > > was in our use case and configuration (still looking
> > into
> > > > it).
> > > > > > > >> > > >
> > > > > > > >> > > > Basically, initially I saw that when one server goes
> > down,
> > > > we
> > > > > > > start
> > > > > > > >> > > having
> > > > > > > >> > > > performance issues in general, but it managed to be on
> > our
> > > > > > client
> > > > > > > >> side,
> > > > > > > >> > > due
> > > > > > > >> > > > to caching, and clients were trying to reconnect to
> > nodes
> > > > that
> > > > > > > were
> > > > > > > >> > > offline
> > > > > > > >> > > > and later trying to get regions from those nodes too.
> > This
> > > > is
> > > > > > > >> basically
> > > > > > > >> > > why
> > > > > > > >> > > > on server side I didn't manage to see anything in logs
> > > that
> > > > > > would
> > > > > > > >> be at
> > > > > > > >> > > > least little bit interesting and point me into desired
> > > > > > direction.
> > > > > > > >> > > >
> > > > > > > >> > > > Another question that popped up to me is - in case
> > server
> > > is
> > > > > > down
> > > > > > > >> (and
> > > > > > > >> > > with
> > > > > > > >> > > > it DataNode and HRegionServer it was hosting) - what's
> > > > optimal
> > > > > > > time
> > > > > > > >> to
> > > > > > > >> > > set
> > > > > > > >> > > > for HMaster to consider server dead reassign regions
> > > > somewhere
> > > > > > > >> else, as
> > > > > > > >> > > > this is another performance bottleneck we hit during
> > > > inability
> > > > > > to
> > > > > > > >> > access
> > > > > > > >> > > > regions? In our case it's configured to 15 minutes,
> and
> > > > simple
> > > > > > > logic
> > > > > > > >> > > tells
> > > > > > > >> > > > me if you want it earlier then configure lower number
> of
> > > > > > retries,
> > > > > > > >> but
> > > > > > > >> > > issue
> > > > > > > >> > > > is as always in details, so not sure if anyone knows
> > some
> > > > > better
> > > > > > > >> math
> > > > > > > >> > for
> > > > > > > >> > > > this?
> > > > > > > >> > > >
> > > > > > > >> > > > And last question - is it possible to manually force
> > HBase
> > > > to
> > > > > > > >> reassign
> > > > > > > >> > > > regions? In this case, while HMaster is retrying to
> > > contact
> > > > > node
> > > > > > > >> that's
> > > > > > > >> > > > dead, it's impossible to force it using 'balancer'
> > > command.
> > > > > > > >> > > >
> > > > > > > >> > > > Thanks a lot!
> > > > > > > >> > > >
> > > > > > > >> > > > Dejan
> > > > > > > >> > > >
> > > > > > > >> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <
> > > > > > > >> dejan.menges@gmail.com>
> > > > > > > >> > > > wrote:
> > > > > > > >> > > >
> > > > > > > >> > > > > Hi,
> > > > > > > >> > > > >
> > > > > > > >> > > > > To be very honest - there's no particular reason why
> > we
> > > > > stick
> > > > > > to
> > > > > > > >> this
> > > > > > > >> > > > one,
> > > > > > > >> > > > > beside just lack of time currently to go through
> > upgrade
> > > > > > > process,
> > > > > > > >> but
> > > > > > > >> > > > looks
> > > > > > > >> > > > > to me that's going to be next step.
> > > > > > > >> > > > >
> > > > > > > >> > > > > Had a crazy day, didn't have time to go through all
> > logs
> > > > > > again,
> > > > > > > >> plus
> > > > > > > >> > > one
> > > > > > > >> > > > > of the machines (last one where we had this issue)
> is
> > > > fully
> > > > > > > >> > > reprovisioned
> > > > > > > >> > > > > yesterday so I don't have logs from there anymore.
> > > > > > > >> > > > >
> > > > > > > >> > > > > Beside upgrading,  what I will talk about today, can
> > you
> > > > > just
> > > > > > > >> point
> > > > > > > >> > me
> > > > > > > >> > > to
> > > > > > > >> > > > > the specific RPC issue in 0.98.0? Thing is that we
> > have
> > > > some
> > > > > > > >> strange
> > > > > > > >> > > > > moments with RPC in this case, and just want to see
> if
> > > > > that's
> > > > > > > the
> > > > > > > >> > same
> > > > > > > >> > > > > thing (and we were even suspecting to RPC).
> > > > > > > >> > > > >
> > > > > > > >> > > > > Thanks a lot!
> > > > > > > >> > > > > Dejan
> > > > > > > >> > > > >
> > > > > > > >> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew Purtell <
> > > > > > > >> apurtell@apache.org
> > > > > > > >> > >
> > > > > > > >> > > > > wrote:
> > > > > > > >> > > > >
> > > > > > > >> > > > >> Is there a particular reason why you are using
> HBase
> > > > > 0.98.0?
> > > > > > > The
> > > > > > > >> > > latest
> > > > > > > >> > > > >> 0.98 release is 0.98.11. There's a known
> performance
> > > > issue
> > > > > > with
> > > > > > > >> > 0.98.0
> > > > > > > >> > > > >> pertaining to RPC that was fixed in later releases,
> > you
> > > > > > should
> > > > > > > >> move
> > > > > > > >> > up
> > > > > > > >> > > > >> from
> > > > > > > >> > > > >> 0.98.0. In addition hundreds of improvements and
> bug
> > > > fixes
> > > > > > have
> > > > > > > >> gone
> > > > > > > >> > > > into
> > > > > > > >> > > > >> the ten releases since 0.98.0.
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <
> > > > > > > >> > dejan.menges@gmail.com
> > > > > > > >> > > >
> > > > > > > >> > > > >> wrote:
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> > Hi All,
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > We have a strange issue with HBase performance
> > > (overall
> > > > > > > cluster
> > > > > > > >> > > > >> > performance) in case one of datanodes in the
> > cluster
> > > > > > > >> unexpectedly
> > > > > > > >> > > goes
> > > > > > > >> > > > >> > down.
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > So scenario is like follows:
> > > > > > > >> > > > >> > - Cluster works fine, it's stable.
> > > > > > > >> > > > >> > - One DataNode unexpectedly goes down (PSU issue,
> > > > network
> > > > > > > >> issue,
> > > > > > > >> > > > >> anything)
> > > > > > > >> > > > >> > - Whole HBase cluster goes down (performance
> > becomes
> > > so
> > > > > bad
> > > > > > > >> that
> > > > > > > >> > we
> > > > > > > >> > > > >> have to
> > > > > > > >> > > > >> > restart all RegionServers to get it back to
> life).
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > Most funny and latest issue that happened was
> that
> > we
> > > > > added
> > > > > > > new
> > > > > > > >> > node
> > > > > > > >> > > > to
> > > > > > > >> > > > >> the
> > > > > > > >> > > > >> > cluster (having 8 x 4T SATA disks) and we left
> just
> > > > > > DataNode
> > > > > > > >> > running
> > > > > > > >> > > > on
> > > > > > > >> > > > >> it
> > > > > > > >> > > > >> > to give it couple of days to get some data. At
> some
> > > > point
> > > > > > in
> > > > > > > >> time,
> > > > > > > >> > > due
> > > > > > > >> > > > >> to
> > > > > > > >> > > > >> > hardware issue, server rebooted (twice during
> three
> > > > > hours)
> > > > > > in
> > > > > > > >> > moment
> > > > > > > >> > > > >> when
> > > > > > > >> > > > >> > it had maybe 5% of data it would have in a couple
> > of
> > > > > days.
> > > > > > > >> Nothing
> > > > > > > >> > > > else
> > > > > > > >> > > > >> > beside DataNode was running, and once it went
> down,
> > > it
> > > > > > > affected
> > > > > > > >> > > > literary
> > > > > > > >> > > > >> > everything, and restarting RegionServers in the
> end
> > > > fixed
> > > > > > it.
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >> > We are using HBase 0.98.0 with Hadoop 2.4.0
> > > > > > > >> > > > >> >
> > > > > > > >> > > > >>
> > > > > > > >> > > > >>
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> --
> > > > > > > >> > > > >> Best regards,
> > > > > > > >> > > > >>
> > > > > > > >> > > > >>    - Andy
> > > > > > > >> > > > >>
> > > > > > > >> > > > >> Problems worthy of attack prove their worth by
> > hitting
> > > > > back.
> > > > > > -
> > > > > > > >> Piet
> > > > > > > >> > > Hein
> > > > > > > >> > > > >> (via Tom White)
> > > > > > > >> > > > >>
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Strange issue when DataNode goes down

Posted by Bryan Beaudreault <bb...@hubspot.com>.
So it is safe to set hbase.lease.recovery.timeout lower if you also
set heartbeat.recheck.interval lower (lowering that 10.5 min dead node
timer)?  Or is it recommended to not touch either of those?

Reading the above with interest, thanks for digging in here guys.

On Mon, Mar 23, 2015 at 10:13 AM, Nicolas Liochon <nk...@gmail.com> wrote:

> If the node is actually down it's fine. But the node may not be that down
> (CAP theorem here); and then it's looking for trouble.
> HDFS, by default declare a node as dead after 10:30. 15 minutes is an extra
> security. It seems your hdfs settings are different (or there is a bug...).
> There should be some info in the hdfs logs.
>
> On Mon, Mar 23, 2015 at 3:05 PM, Dejan Menges <de...@gmail.com>
> wrote:
>
> > Will take a look.
> >
> > Actually, if node is down (someone unplugged network cable, it just died,
> > whatever) what's chance it's going to become live so write can continue?
> On
> > the other side, HBase is not starting recovery trying to contact node
> which
> > is not there anymore, and even elected as dead on Namenode side (another
> > thing I didn't understood quite good).
> >
> > So what I was expecting is that as soon as Namenode decided node is dead,
> > that it would be enough for RegionServer to stop trying to recover from
> the
> > dead node, but it wasn't the case. Also, this whole MTTR article in HBase
> > book doesn't work at all with this parameter set to it's default value
> (15
> > minutes).
> >
> > So I'm still struggling to figure out what's drawback exactly on this?
> >
> > On Mon, Mar 23, 2015 at 2:50 PM Nicolas Liochon <nk...@gmail.com>
> wrote:
> >
> > > Thanks for the explanation. There is an issue if you modify this
> setting
> > > however.
> > > hbase tries to recover the lease (i.e. be sure that nobody is writing)
> > > If you change hbase.lease.recovery.timeout hbase will start the
> recovery
> > > (i.e. start to read) even if it's not sure that nobody's writing. That
> > > means there is a dataloss risk.
> > > Basically, you must not see this warning: WARN
> > > org.apache.hadoop.hbase.util.FSHDFSUtils: Cannot recoverLease after
> > trying
> > > for[]
> > >
> > > The recoverLease must succeed. The fact that it does not after X tries
> is
> > > strange.
> > > There may be a mistmatch between the hbase parameters and the hdfs
> ones.
> > > You may need to have a look at the comments in FSHDFSUtils.java
> > >
> > >
> > >
> > >
> > > On Mon, Mar 23, 2015 at 2:15 PM, Dejan Menges <de...@gmail.com>
> > > wrote:
> > >
> > > > I found the issue and fixed it, and will try to explain here what was
> > > > exactly in our case in case someone else finds this interesting too.
> > > >
> > > > So initially, we had (couple of times) some network and hardware
> issues
> > > in
> > > > our datacenters. When one server would die (literary die, we had some
> > > issue
> > > > with PSUs) we saw issues with overall cluster performance on HBase
> > side.
> > > As
> > > > cluster is quite big and live, it was also quite hard to figure out
> > exact
> > > > root cause and how to fix the stuff we wanted to fix.
> > > >
> > > > So I set up another cluster, four nodes (with DataNode and
> > RegionServer)
> > > > and two other nodes with HMaster and Namenode in HA, using same stuff
> > we
> > > > use on production. We pumped some data into it, and I was able to
> > > reproduce
> > > > same issue last week on it. What I tried to do is to cut one server
> > (shut
> > > > down it's interface) when all is good with cluster, when we have
> load,
> > > and
> > > > see what's going to happen.
> > > >
> > > > On Friday, after Nicolas mentioned, I started taking a look in HBase
> > logs
> > > > on the node which was mentioned in HMaster log as the one taking over
> > > > regions for the dead server. Basically what I was able to observe was
> > 15
> > > > minutes time (+- couple of seconds, what was also interesting, and
> will
> > > got
> > > > later to that) between HMaster figures out that one of it's
> > RegionServers
> > > > is dead, and the time one of the mentioned nodes starts taking over
> > those
> > > > regions and they start appearing in HMaster's Web UI.
> > > >
> > > > I then set up everything like mentioned here
> > > > http://hbase.apache.org/book.html#mttr - but still had exactly the
> > same
> > > > issues. Went over (again and again) all currently configured stuff,
> but
> > > > still had the same issue.
> > > >
> > > > Then I started looking into HDFS. Opened NameNode UI, saw all is
> good,
> > > took
> > > > one node down, was also looking RegionServer logs in the same time -
> > and
> > > I
> > > > also see that it took ~15 minutes for Namenode to elect dead node as
> > > dead.
> > > > Somehow in the same moment regions started getting back to life. I
> > > remember
> > > > in some older versions dfs timeout checks and check retries. Looked
> > into
> > > > defaults for our Hadoop version -
> > > >
> > > > http://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/
> > > hadoop-hdfs/hdfs-default.xml
> > > > - and saw there that there's no recheck parameter anymore. Strange,
> as
> > on
> > > > StackOverflow I found post from month ago, for newer version than we
> > use
> > > > (we use 2.4.1, guy was using 2.6 - dfs.namenode.heartbeat.
> > > recheck-interval)
> > > > I set it to 10 seconds as he mentioned, having checks every three
> > seconds
> > > > (default) and got DataNode elected as dead in ~45 seconds, as he
> > > mentioned.
> > > > Not sure why this parameter is not documented, but obviously it
> works.
> > > >
> > > > Then figured out it still didn't fix our HBase failover issue. I was
> > > > looking into HBase book again and again, and then saw this part:
> > > >
> > > > "How much time we allow elapse between calls to recover lease. Should
> > be
> > > > larger than the dfs timeout."
> > > >
> > > > This was description for hbase.lease.recovery.dfs.timeout. Wasn't
> sure
> > > from
> > > > the comment what of all timeouts that's possible to set in
> Hadoop/HBase
> > > and
> > > > that has something to do with DFS is this all about. But picked
> > > > hbase.lease.recovery.timeout, and that was the catch.
> > > >
> > > > Initially, by default, hbase.lease.recovery.timeout is set to 15
> > minutes.
> > > > Not sure why, and wasn't able to find yet why, but getting this down
> to
> > > one
> > > > minute (what's more than OK for us) I was able to get rid of our
> issue.
> > > Not
> > > > also sure why this is not mentioned in MTTR section in HBase book, as
> > > > obviously MTTR doesn't work at all with this default timeout, at
> least
> > it
> > > > doesn't work the way we expected it to work.
> > > >
> > > > So thanks again for everyone being spammed with this, and specially
> > > thanks
> > > > to Nicolas pointing me to the right direction.
> > > >
> > > > On Mon, Mar 23, 2015 at 1:37 PM Nicolas Liochon <nk...@gmail.com>
> > > wrote:
> > > >
> > > > > the attachments are rejected by the mailing list, can you put then
> on
> > > > > pastebin?
> > > > >
> > > > > stale is mandatory (so it's good), but the issue here is just
> before.
> > > The
> > > > > region server needs to read the file. In order to be sure that
> there
> > is
> > > > no
> > > > > data loss, it needs to "recover the lease", that means ensuring
> that
> > > > nobody
> > > > > is writing the file. The regionserver calls the namenode to do this
> > > > > recoverLease. So there should be some info in the namenode logs.
> You
> > > have
> > > > > HDFS-4721 on your hdfs? The hbase book details (more or less...)
> this
> > > > > recoverLease stuff.
> > > > >
> > > > >
> > > > > On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <
> > dejan.menges@gmail.com
> > > >
> > > > > wrote:
> > > > >
> > > > > > And also, just checked - dfs.namenode.avoid.read.stale.datanode
> and
> > > > > > dfs.namenode.avoid.write.stale.datanode
> > > > > > are both true, and dfs.namenode.stale.datanode.interval is set to
> > > > > default
> > > > > > 30000.
> > > > > >
> > > > > > On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <
> > > dejan.menges@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Nicolas,
> > > > > > >
> > > > > > > Please find log attached.
> > > > > > >
> > > > > > > As I see it now more clearly, it was trying to recover HDFS
> WALs
> > > from
> > > > > > node
> > > > > > > that's dead:
> > > > > > >
> > > > > > > 2015-03-23 08:53:44,381 WARN
> > > > org.apache.hadoop.hbase.util.FSHDFSUtils:
> > > > > > > Cannot recoverLease after trying for 900000ms
> > > > > > > (hbase.lease.recovery.timeout); continuing, but may be
> > > DATALOSS!!!;
> > > > > > > attempt=40 on
> > > > > > >
> > > > > > file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intent
> > > > > ionally_get_down_by_getting_network_down},60020,1426862900
> > > > > 506-splitting/{node_i_intentionally_get_down_by_gett
> > > > > ing_network_down}%2C60020%2C1426862900506.1427096924508
> > > > > > > after 908210ms
> > > > > > >
> > > > > > > And as you can see from the log, it tried 40 times, what took
> it
> > > > > exactly
> > > > > > > 15 minutes.
> > > > > > >
> > > > > > > There's probably some parameter to tune to cut it of from 40
> > times
> > > /
> > > > 15
> > > > > > > minutes to something more useful, as for 15 minutes we don't
> have
> > > our
> > > > > > > regions available, and HDFS have however replication factor 3.
> > > > > > >
> > > > > > > Googling, if I figure out what's this I will post it here. Will
> > > also
> > > > > > > appreciate if someone knows how to cut this down.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Dejan
> > > > > > >
> > > > > > > On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <
> > nkeywal@gmail.com
> > > >
> > > > > > wrote:
> > > > > > >
> > > > > > >> The split is done by the region servers (the master
> > coordinates).
> > > Is
> > > > > > there
> > > > > > >> some interesting stuff in their logs?
> > > > > > >>
> > > > > > >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <
> > > > dejan.menges@gmail.com
> > > > > >
> > > > > > >> wrote:
> > > > > > >>
> > > > > > >> > With client issue was that it was retrying connecting to the
> > > same
> > > > > > region
> > > > > > >> > servers even when they were reassigned. Lowering it down
> > helped
> > > in
> > > > > > this
> > > > > > >> > specific use case, but yes, we still want servers to
> > reallocate
> > > > > > quickly.
> > > > > > >> >
> > > > > > >> > What got me here:
> > > > > > >> >
> > > > > > >> > http://hbase.apache.org/book.html#mttr
> > > > > > >> >
> > > > > > >> > I basically set configuration exactly the same way as it's
> > > > explained
> > > > > > >> here.
> > > > > > >> > *zookeeper.session.timeout* is (and was before) 60000 (one
> > > > minute).
> > > > > > >> >
> > > > > > >> > So basically what happens is: - simulating network issues we
> > had
> > > > > > >> recently).
> > > > > > >> > - After short time I see in HBase that my RegionServer is
> > dead,
> > > > and
> > > > > as
> > > > > > >> > total number of regions I see previous total minus number of
> > > > regions
> > > > > > >> that
> > > > > > >> > were hosted on the node hosting RegionServer that just
> > > > > 'disappeared'.
> > > > > > >> > - In this point I want my clus
> > > > > > >> >
> > > > > > >> > - I have test cluster consisting of four nodes, every node
> > being
> > > > > > >> DataNode
> > > > > > >> > and RegionServer.
> > > > > > >> > - I simulate network partition on one (connect to it through
> > > > console
> > > > > > and
> > > > > > >> > take network interface downter to recover as soon as
> possible,
> > > to
> > > > > > start
> > > > > > >> > serving missing regions.
> > > > > > >> > - First thing I see in HMaster logs are:
> > > > > > >> >
> > > > > > >> > 2015-03-20 14:17:26,015 INFO
> > > > > > >> > org.apache.hadoop.hbase.zookeeper.RegionServerTracker:
> > > > RegionServer
> > > > > > >> > ephemeral node deleted, processing expiration
> > > > > > >> > [{name_of_node_I_took_down},60020,1426860403261]
> > > > > > >> >
> > > > > > >> > 2015-03-20 14:17:26,067 INFO
> > > > > > >> >
> org.apache.hadoop.hbase.master.handler.ServerShutdownHandler:
> > > > > > Splitting
> > > > > > >> > logs for {name_of_node_I_took_down},60020,1426860403261
> before
> > > > > > >> assignment.
> > > > > > >> >
> > > > > > >> > 2015-03-20 14:17:26,105 INFO
> > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: dead
> splitlog
> > > > > workers
> > > > > > [
> > > > > > >> > {name_of_node_I_took_down},60020,1426860403261]
> > > > > > >> >
> > > > > > >> > 2015-03-20 14:17:26,107 INFO
> > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: started
> > > splitting
> > > > 1
> > > > > > >> logs in
> > > > > > >> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_I_
> > > > > took_down}
> > > > > > >> > ,60020,1426860403261-splitting]
> > > > > > >> >
> > > > > > >> > 2015-03-20 14:17:26,150 INFO
> > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > > > > > >> > /hbase/splitWAL/WALs%2F
> > > > > > >> >
> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > > > >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.14268
> > > > > 60404905
> > > > > > >> > acquired by {fqdn_of_another_live_hnode},60020,1426859445623
> > > > > > >> >
> > > > > > >> > 2015-03-20 14:17:26,182 INFO
> > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks
> =
> > 1
> > > > > > >> unassigned
> > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > > >> >
> > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > > in_progress
> > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> > > > error =
> > > > > > 0}
> > > > > > >> >
> > > > > > >> > 2015-03-20 14:17:31,183 INFO
> > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks
> =
> > 1
> > > > > > >> unassigned
> > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > > >> >
> > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > > in_progress
> > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> > > > error =
> > > > > > 0}
> > > > > > >> >
> > > > > > >> > 2015-03-20 14:17:36,184 INFO
> > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks
> =
> > 1
> > > > > > >> unassigned
> > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > > >> >
> > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > > in_progress
> > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> > > > error =
> > > > > > 0}
> > > > > > >> >
> > > > > > >> > 2015-03-20 14:17:42,185 INFO
> > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks
> =
> > 1
> > > > > > >> unassigned
> > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > > >> >
> > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > > in_progress
> > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> > > > error =
> > > > > > 0}
> > > > > > >> >
> > > > > > >> > 2015-03-20 14:17:48,184 INFO
> > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks
> =
> > 1
> > > > > > >> unassigned
> > > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > > >> >
> > > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > > in_progress
> > > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> > > > error =
> > > > > > 0}
> > > > > > >> > In the meantime, In hbase...out log I got this:
> > > > > > >> >
> > > > > > >> > ==> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out <==
> > > > > > >> >
> > > > > > >> > java.io.IOException: Call to
> > > > > > >> > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_dow
> > > > > n}:60020
> > > > > > >> > failed on local exception:
> > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$CallTimeoutException:
> > > Call
> > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > > > >> >
> > > > > > >> > at
> > org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
> > > > > > >> t.java:1532)
> > > > > > >> >
> > > > > > >> > at
> org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:
> > > 1502)
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> >
> > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(Rpc
> > > > > > >> Client.java:1684)
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> >
> > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$BlockingRpcChannelImpl
> > > > > ementati
> > > > > > >> on.callBlockingMethod(RpcClient.java:1737)
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> >
> > > > > > >> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$
> > > > > AdminService$
> > > > > > >> BlockingStub.getRegionInfo(AdminProtos.java:20806)
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> >
> > > > > > >> > org.apache.hadoop.hbase.client.HBaseAdmin.getCompactionState
> > > > > > >> (HBaseAdmin.java:2524)
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> >
> > > > > > >> > org.apache.hadoop.hbase.generated.master.table_jsp._jspServi
> > > > > > >> ce(table_jsp.java:167)
> > > > > > >> >
> > > > > > >> > at
> org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.
> > > > > java:98)
> > > > > > >> >
> > > > > > >> > at
> > javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
> > > > > > >> >
> > > > > > >> > at
> > org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
> > > > > > >> .java:511)
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> >
> > > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > > > > > >> r(ServletHandler.java:1221)
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> >
> > > > > > >> > org.apache.hadoop.http.lib.StaticUserWebFilter$StaticUserFil
> > > > > > >> ter.doFilter(StaticUserWebFilter.java:109)
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> >
> > > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > > > > > >> r(ServletHandler.java:1212)
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> >
> > > > > > >> > org.apache.hadoop.http.HttpServer$QuotingInputFilter.doFilte
> > > > > > >> r(HttpServer.java:1081)
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> >
> > > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > > > > > >> r(ServletHandler.java:1212)
> > > > > > >> >
> > > > > > >> > at
> > > > > >
> > org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> >
> > > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > > > > > >> r(ServletHandler.java:1212)
> > > > > > >> >
> > > > > > >> > at
> > org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
> > > > > > >> er.java:399)
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> > org.mortbay.jetty.security.SecurityHandler.handle(SecurityHa
> > > > > > >> ndler.java:216)
> > > > > > >> >
> > > > > > >> > at
> > org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
> > > > > > >> er.java:182)
> > > > > > >> >
> > > > > > >> > at
> > org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
> > > > > > >> er.java:766)
> > > > > > >> >
> > > > > > >> > at
> > org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
> > > > > > >> java:450)
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> >
> > > > > > >> > org.mortbay.jetty.handler.ContextHandlerCollection.handle(Co
> > > > > > >> ntextHandlerCollection.java:230)
> > > > > > >> >
> > > > > > >> > at
> > org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
> > > > > > >> er.java:152)
> > > > > > >> >
> > > > > > >> > at org.mortbay.jetty.Server.handle(Server.java:326)
> > > > > > >> >
> > > > > > >> > at
> > org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
> > > > > > >> n.java:542)
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> >
> > > > > > >> > org.mortbay.jetty.HttpConnection$RequestHandler.headerComple
> > > > > > >> te(HttpConnection.java:928)
> > > > > > >> >
> > > > > > >> > at
> org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
> > > > > > >> >
> > > > > > >> > at
> > > > org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
> > > > > > >> >
> > > > > > >> > at
> > > > org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> >
> > > > > > >> > org.mortbay.io.nio.SelectChannelEndPoint.run(SelectChannelEn
> > > > > > >> dPoint.java:410)
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> >
> > > > > > >> > org.mortbay.thread.QueuedThreadPool$PoolThread.run(
> > > > > > >> QueuedThreadPool.java:582)
> > > > > > >> >
> > > > > > >> > Caused by: org.apache.hadoop.hbase.ipc.RpcClient$
> > > > > CallTimeoutException:
> > > > > > >> Call
> > > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> >
> > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.cleanupCall
> > > > > > >> s(RpcClient.java:1234)
> > > > > > >> >
> > > > > > >> > at
> > > > > > >> >
> > > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.readRespons
> > > > > > >> e(RpcClient.java:1171)
> > > > > > >> >
> > > > > > >> > at
> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
> > > > > > >> nt.java:751)
> > > > > > >> > Beside this same issue, please note that first message was
> at
> > > > > > 2015-03-20
> > > > > > >> > 14:17:26,015. And then (we got to the point when it started
> > > > > > transition):
> > > > > > >> >
> > > > > > >> > 2015-03-20 14:32:35,059 INFO
> > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > > > > > >> > /hbase/splitWAL/WALs%2F
> > > > > > >> >
> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > > > >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.14268
> > > > > 60404905
> > > > > > >> > entered state: DONE
> > {fqdn_of_new_live_node},60020,1426859445623
> > > > > > >> >
> > > > > > >> > 2015-03-20 14:32:35,109 INFO
> > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: Done
> splitting
> > > > > > >> > /hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > > >> >
> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}
> > > > > > >> > %252C60020%252C1426860403261.1426860404905
> > > > > > >> >
> > > > > > >> > 2015-03-20 14:32:35,190 INFO
> > > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: finished
> > > splitting
> > > > > > >> (more
> > > > > > >> > than or equal to) 9 bytes in 1 log files in
> > > > > > >> >
> > > > > > >> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_of_
> > > > > > >> node_I_took_down},60020,1426860403261-splitting]
> > > > > > >> > in 909083ms
> > > > > > >> >
> > > > > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > > > > > >> .RegionStates:
> > > > > > >> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c state=OPEN,
> > > > > > >> > ts=1426860639088,
> > > > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > > > >> to
> > > > > > >> > {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE,
> > > ts=1426861955191,
> > > > > > >> server=
> > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > > > >> >
> > > > > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > > > > > >> .RegionStates:
> > > > > > >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
> > > > > > >> {name_of_node_I_took_down}
> > > > > > >> > ,60020,1426860403261
> > > > > > >> >
> > > > > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > > > > > >> .RegionStates:
> > > > > > >> > Transitioned {25ab6e7b42e36ddaa723d71be5954543 state=OPEN,
> > > > > > >> > ts=1426860641783,
> > > > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > > > >> to
> > > > > > >> > {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE,
> > > ts=1426861955191,
> > > > > > >> server=
> > > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > > > >> >
> > > > > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > > > > > >> .RegionStates:
> > > > > > >> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
> > > > > > >> {name_of_node_I_took_down}
> > > > > > >> > ,60020,1426860403261
> > > > > > >> > At this point, note that it finished SplitLogManager task at
> > > > > 14:32:35
> > > > > > >> and
> > > > > > >> > started transitioning just after that. So this is 15 minutes
> > > that
> > > > > I'm
> > > > > > >> > talking about.
> > > > > > >> >
> > > > > > >> > What am I missing?
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <
> > > > nkeywal@gmail.com>
> > > > > > >> wrote:
> > > > > > >> >
> > > > > > >> > > You've changed the value of hbase.zookeeper.timeout to 15
> > > > > minutes? A
> > > > > > >> very
> > > > > > >> > > reasonable target is 1 minute before relocating the
> regions.
> > > > > That's
> > > > > > >> the
> > > > > > >> > > default iirc. You can push it to 20s, but then
> > > > > gc-stopping-the-world
> > > > > > >> > > becomes more of an issue. 15 minutes is really a lot. The
> > hdfs
> > > > > stale
> > > > > > >> mode
> > > > > > >> > > must always be used, with a lower timeout than the hbase
> > one.
> > > > > > >> > >
> > > > > > >> > > Client side there should be nothing to do (excepted for
> > > advanced
> > > > > > >> stuff);
> > > > > > >> > at
> > > > > > >> > > each retry the client checks the location of the regions.
> If
> > > you
> > > > > > lower
> > > > > > >> > the
> > > > > > >> > > number of retry the client will fail sooner, but usually
> you
> > > > don't
> > > > > > >> want
> > > > > > >> > the
> > > > > > >> > > client to fail, you want the servers to reallocate
> quickly.
> > > > > > >> > >
> > > > > > >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <
> > > > > > dejan.menges@gmail.com
> > > > > > >> >
> > > > > > >> > > wrote:
> > > > > > >> > >
> > > > > > >> > > > Hi,
> > > > > > >> > > >
> > > > > > >> > > > Sorry for little bit late update, but managed to narrow
> it
> > > > > little
> > > > > > >> bit
> > > > > > >> > > down.
> > > > > > >> > > >
> > > > > > >> > > > We didn't update yet, as we are using Hortonworks
> > > distribution
> > > > > > right
> > > > > > >> > now,
> > > > > > >> > > > and even if we update we will get 0.98.4. However, looks
> > > that
> > > > > > issue
> > > > > > >> > here
> > > > > > >> > > > was in our use case and configuration (still looking
> into
> > > it).
> > > > > > >> > > >
> > > > > > >> > > > Basically, initially I saw that when one server goes
> down,
> > > we
> > > > > > start
> > > > > > >> > > having
> > > > > > >> > > > performance issues in general, but it managed to be on
> our
> > > > > client
> > > > > > >> side,
> > > > > > >> > > due
> > > > > > >> > > > to caching, and clients were trying to reconnect to
> nodes
> > > that
> > > > > > were
> > > > > > >> > > offline
> > > > > > >> > > > and later trying to get regions from those nodes too.
> This
> > > is
> > > > > > >> basically
> > > > > > >> > > why
> > > > > > >> > > > on server side I didn't manage to see anything in logs
> > that
> > > > > would
> > > > > > >> be at
> > > > > > >> > > > least little bit interesting and point me into desired
> > > > > direction.
> > > > > > >> > > >
> > > > > > >> > > > Another question that popped up to me is - in case
> server
> > is
> > > > > down
> > > > > > >> (and
> > > > > > >> > > with
> > > > > > >> > > > it DataNode and HRegionServer it was hosting) - what's
> > > optimal
> > > > > > time
> > > > > > >> to
> > > > > > >> > > set
> > > > > > >> > > > for HMaster to consider server dead reassign regions
> > > somewhere
> > > > > > >> else, as
> > > > > > >> > > > this is another performance bottleneck we hit during
> > > inability
> > > > > to
> > > > > > >> > access
> > > > > > >> > > > regions? In our case it's configured to 15 minutes, and
> > > simple
> > > > > > logic
> > > > > > >> > > tells
> > > > > > >> > > > me if you want it earlier then configure lower number of
> > > > > retries,
> > > > > > >> but
> > > > > > >> > > issue
> > > > > > >> > > > is as always in details, so not sure if anyone knows
> some
> > > > better
> > > > > > >> math
> > > > > > >> > for
> > > > > > >> > > > this?
> > > > > > >> > > >
> > > > > > >> > > > And last question - is it possible to manually force
> HBase
> > > to
> > > > > > >> reassign
> > > > > > >> > > > regions? In this case, while HMaster is retrying to
> > contact
> > > > node
> > > > > > >> that's
> > > > > > >> > > > dead, it's impossible to force it using 'balancer'
> > command.
> > > > > > >> > > >
> > > > > > >> > > > Thanks a lot!
> > > > > > >> > > >
> > > > > > >> > > > Dejan
> > > > > > >> > > >
> > > > > > >> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <
> > > > > > >> dejan.menges@gmail.com>
> > > > > > >> > > > wrote:
> > > > > > >> > > >
> > > > > > >> > > > > Hi,
> > > > > > >> > > > >
> > > > > > >> > > > > To be very honest - there's no particular reason why
> we
> > > > stick
> > > > > to
> > > > > > >> this
> > > > > > >> > > > one,
> > > > > > >> > > > > beside just lack of time currently to go through
> upgrade
> > > > > > process,
> > > > > > >> but
> > > > > > >> > > > looks
> > > > > > >> > > > > to me that's going to be next step.
> > > > > > >> > > > >
> > > > > > >> > > > > Had a crazy day, didn't have time to go through all
> logs
> > > > > again,
> > > > > > >> plus
> > > > > > >> > > one
> > > > > > >> > > > > of the machines (last one where we had this issue) is
> > > fully
> > > > > > >> > > reprovisioned
> > > > > > >> > > > > yesterday so I don't have logs from there anymore.
> > > > > > >> > > > >
> > > > > > >> > > > > Beside upgrading,  what I will talk about today, can
> you
> > > > just
> > > > > > >> point
> > > > > > >> > me
> > > > > > >> > > to
> > > > > > >> > > > > the specific RPC issue in 0.98.0? Thing is that we
> have
> > > some
> > > > > > >> strange
> > > > > > >> > > > > moments with RPC in this case, and just want to see if
> > > > that's
> > > > > > the
> > > > > > >> > same
> > > > > > >> > > > > thing (and we were even suspecting to RPC).
> > > > > > >> > > > >
> > > > > > >> > > > > Thanks a lot!
> > > > > > >> > > > > Dejan
> > > > > > >> > > > >
> > > > > > >> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew Purtell <
> > > > > > >> apurtell@apache.org
> > > > > > >> > >
> > > > > > >> > > > > wrote:
> > > > > > >> > > > >
> > > > > > >> > > > >> Is there a particular reason why you are using HBase
> > > > 0.98.0?
> > > > > > The
> > > > > > >> > > latest
> > > > > > >> > > > >> 0.98 release is 0.98.11. There's a known performance
> > > issue
> > > > > with
> > > > > > >> > 0.98.0
> > > > > > >> > > > >> pertaining to RPC that was fixed in later releases,
> you
> > > > > should
> > > > > > >> move
> > > > > > >> > up
> > > > > > >> > > > >> from
> > > > > > >> > > > >> 0.98.0. In addition hundreds of improvements and bug
> > > fixes
> > > > > have
> > > > > > >> gone
> > > > > > >> > > > into
> > > > > > >> > > > >> the ten releases since 0.98.0.
> > > > > > >> > > > >>
> > > > > > >> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <
> > > > > > >> > dejan.menges@gmail.com
> > > > > > >> > > >
> > > > > > >> > > > >> wrote:
> > > > > > >> > > > >>
> > > > > > >> > > > >> > Hi All,
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > We have a strange issue with HBase performance
> > (overall
> > > > > > cluster
> > > > > > >> > > > >> > performance) in case one of datanodes in the
> cluster
> > > > > > >> unexpectedly
> > > > > > >> > > goes
> > > > > > >> > > > >> > down.
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > So scenario is like follows:
> > > > > > >> > > > >> > - Cluster works fine, it's stable.
> > > > > > >> > > > >> > - One DataNode unexpectedly goes down (PSU issue,
> > > network
> > > > > > >> issue,
> > > > > > >> > > > >> anything)
> > > > > > >> > > > >> > - Whole HBase cluster goes down (performance
> becomes
> > so
> > > > bad
> > > > > > >> that
> > > > > > >> > we
> > > > > > >> > > > >> have to
> > > > > > >> > > > >> > restart all RegionServers to get it back to life).
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > Most funny and latest issue that happened was that
> we
> > > > added
> > > > > > new
> > > > > > >> > node
> > > > > > >> > > > to
> > > > > > >> > > > >> the
> > > > > > >> > > > >> > cluster (having 8 x 4T SATA disks) and we left just
> > > > > DataNode
> > > > > > >> > running
> > > > > > >> > > > on
> > > > > > >> > > > >> it
> > > > > > >> > > > >> > to give it couple of days to get some data. At some
> > > point
> > > > > in
> > > > > > >> time,
> > > > > > >> > > due
> > > > > > >> > > > >> to
> > > > > > >> > > > >> > hardware issue, server rebooted (twice during three
> > > > hours)
> > > > > in
> > > > > > >> > moment
> > > > > > >> > > > >> when
> > > > > > >> > > > >> > it had maybe 5% of data it would have in a couple
> of
> > > > days.
> > > > > > >> Nothing
> > > > > > >> > > > else
> > > > > > >> > > > >> > beside DataNode was running, and once it went down,
> > it
> > > > > > affected
> > > > > > >> > > > literary
> > > > > > >> > > > >> > everything, and restarting RegionServers in the end
> > > fixed
> > > > > it.
> > > > > > >> > > > >> >
> > > > > > >> > > > >> > We are using HBase 0.98.0 with Hadoop 2.4.0
> > > > > > >> > > > >> >
> > > > > > >> > > > >>
> > > > > > >> > > > >>
> > > > > > >> > > > >>
> > > > > > >> > > > >> --
> > > > > > >> > > > >> Best regards,
> > > > > > >> > > > >>
> > > > > > >> > > > >>    - Andy
> > > > > > >> > > > >>
> > > > > > >> > > > >> Problems worthy of attack prove their worth by
> hitting
> > > > back.
> > > > > -
> > > > > > >> Piet
> > > > > > >> > > Hein
> > > > > > >> > > > >> (via Tom White)
> > > > > > >> > > > >>
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Strange issue when DataNode goes down

Posted by Nicolas Liochon <nk...@gmail.com>.
If the node is actually down it's fine. But the node may not be that down
(CAP theorem here); and then it's looking for trouble.
HDFS, by default declare a node as dead after 10:30. 15 minutes is an extra
security. It seems your hdfs settings are different (or there is a bug...).
There should be some info in the hdfs logs.

On Mon, Mar 23, 2015 at 3:05 PM, Dejan Menges <de...@gmail.com>
wrote:

> Will take a look.
>
> Actually, if node is down (someone unplugged network cable, it just died,
> whatever) what's chance it's going to become live so write can continue? On
> the other side, HBase is not starting recovery trying to contact node which
> is not there anymore, and even elected as dead on Namenode side (another
> thing I didn't understood quite good).
>
> So what I was expecting is that as soon as Namenode decided node is dead,
> that it would be enough for RegionServer to stop trying to recover from the
> dead node, but it wasn't the case. Also, this whole MTTR article in HBase
> book doesn't work at all with this parameter set to it's default value (15
> minutes).
>
> So I'm still struggling to figure out what's drawback exactly on this?
>
> On Mon, Mar 23, 2015 at 2:50 PM Nicolas Liochon <nk...@gmail.com> wrote:
>
> > Thanks for the explanation. There is an issue if you modify this setting
> > however.
> > hbase tries to recover the lease (i.e. be sure that nobody is writing)
> > If you change hbase.lease.recovery.timeout hbase will start the recovery
> > (i.e. start to read) even if it's not sure that nobody's writing. That
> > means there is a dataloss risk.
> > Basically, you must not see this warning: WARN
> > org.apache.hadoop.hbase.util.FSHDFSUtils: Cannot recoverLease after
> trying
> > for[]
> >
> > The recoverLease must succeed. The fact that it does not after X tries is
> > strange.
> > There may be a mistmatch between the hbase parameters and the hdfs ones.
> > You may need to have a look at the comments in FSHDFSUtils.java
> >
> >
> >
> >
> > On Mon, Mar 23, 2015 at 2:15 PM, Dejan Menges <de...@gmail.com>
> > wrote:
> >
> > > I found the issue and fixed it, and will try to explain here what was
> > > exactly in our case in case someone else finds this interesting too.
> > >
> > > So initially, we had (couple of times) some network and hardware issues
> > in
> > > our datacenters. When one server would die (literary die, we had some
> > issue
> > > with PSUs) we saw issues with overall cluster performance on HBase
> side.
> > As
> > > cluster is quite big and live, it was also quite hard to figure out
> exact
> > > root cause and how to fix the stuff we wanted to fix.
> > >
> > > So I set up another cluster, four nodes (with DataNode and
> RegionServer)
> > > and two other nodes with HMaster and Namenode in HA, using same stuff
> we
> > > use on production. We pumped some data into it, and I was able to
> > reproduce
> > > same issue last week on it. What I tried to do is to cut one server
> (shut
> > > down it's interface) when all is good with cluster, when we have load,
> > and
> > > see what's going to happen.
> > >
> > > On Friday, after Nicolas mentioned, I started taking a look in HBase
> logs
> > > on the node which was mentioned in HMaster log as the one taking over
> > > regions for the dead server. Basically what I was able to observe was
> 15
> > > minutes time (+- couple of seconds, what was also interesting, and will
> > got
> > > later to that) between HMaster figures out that one of it's
> RegionServers
> > > is dead, and the time one of the mentioned nodes starts taking over
> those
> > > regions and they start appearing in HMaster's Web UI.
> > >
> > > I then set up everything like mentioned here
> > > http://hbase.apache.org/book.html#mttr - but still had exactly the
> same
> > > issues. Went over (again and again) all currently configured stuff, but
> > > still had the same issue.
> > >
> > > Then I started looking into HDFS. Opened NameNode UI, saw all is good,
> > took
> > > one node down, was also looking RegionServer logs in the same time -
> and
> > I
> > > also see that it took ~15 minutes for Namenode to elect dead node as
> > dead.
> > > Somehow in the same moment regions started getting back to life. I
> > remember
> > > in some older versions dfs timeout checks and check retries. Looked
> into
> > > defaults for our Hadoop version -
> > >
> > > http://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/
> > hadoop-hdfs/hdfs-default.xml
> > > - and saw there that there's no recheck parameter anymore. Strange, as
> on
> > > StackOverflow I found post from month ago, for newer version than we
> use
> > > (we use 2.4.1, guy was using 2.6 - dfs.namenode.heartbeat.
> > recheck-interval)
> > > I set it to 10 seconds as he mentioned, having checks every three
> seconds
> > > (default) and got DataNode elected as dead in ~45 seconds, as he
> > mentioned.
> > > Not sure why this parameter is not documented, but obviously it works.
> > >
> > > Then figured out it still didn't fix our HBase failover issue. I was
> > > looking into HBase book again and again, and then saw this part:
> > >
> > > "How much time we allow elapse between calls to recover lease. Should
> be
> > > larger than the dfs timeout."
> > >
> > > This was description for hbase.lease.recovery.dfs.timeout. Wasn't sure
> > from
> > > the comment what of all timeouts that's possible to set in Hadoop/HBase
> > and
> > > that has something to do with DFS is this all about. But picked
> > > hbase.lease.recovery.timeout, and that was the catch.
> > >
> > > Initially, by default, hbase.lease.recovery.timeout is set to 15
> minutes.
> > > Not sure why, and wasn't able to find yet why, but getting this down to
> > one
> > > minute (what's more than OK for us) I was able to get rid of our issue.
> > Not
> > > also sure why this is not mentioned in MTTR section in HBase book, as
> > > obviously MTTR doesn't work at all with this default timeout, at least
> it
> > > doesn't work the way we expected it to work.
> > >
> > > So thanks again for everyone being spammed with this, and specially
> > thanks
> > > to Nicolas pointing me to the right direction.
> > >
> > > On Mon, Mar 23, 2015 at 1:37 PM Nicolas Liochon <nk...@gmail.com>
> > wrote:
> > >
> > > > the attachments are rejected by the mailing list, can you put then on
> > > > pastebin?
> > > >
> > > > stale is mandatory (so it's good), but the issue here is just before.
> > The
> > > > region server needs to read the file. In order to be sure that there
> is
> > > no
> > > > data loss, it needs to "recover the lease", that means ensuring that
> > > nobody
> > > > is writing the file. The regionserver calls the namenode to do this
> > > > recoverLease. So there should be some info in the namenode logs. You
> > have
> > > > HDFS-4721 on your hdfs? The hbase book details (more or less...) this
> > > > recoverLease stuff.
> > > >
> > > >
> > > > On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <
> dejan.menges@gmail.com
> > >
> > > > wrote:
> > > >
> > > > > And also, just checked - dfs.namenode.avoid.read.stale.datanode and
> > > > > dfs.namenode.avoid.write.stale.datanode
> > > > > are both true, and dfs.namenode.stale.datanode.interval is set to
> > > > default
> > > > > 30000.
> > > > >
> > > > > On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <
> > dejan.menges@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi Nicolas,
> > > > > >
> > > > > > Please find log attached.
> > > > > >
> > > > > > As I see it now more clearly, it was trying to recover HDFS WALs
> > from
> > > > > node
> > > > > > that's dead:
> > > > > >
> > > > > > 2015-03-23 08:53:44,381 WARN
> > > org.apache.hadoop.hbase.util.FSHDFSUtils:
> > > > > > Cannot recoverLease after trying for 900000ms
> > > > > > (hbase.lease.recovery.timeout); continuing, but may be
> > DATALOSS!!!;
> > > > > > attempt=40 on
> > > > > >
> > > > > file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intent
> > > > ionally_get_down_by_getting_network_down},60020,1426862900
> > > > 506-splitting/{node_i_intentionally_get_down_by_gett
> > > > ing_network_down}%2C60020%2C1426862900506.1427096924508
> > > > > > after 908210ms
> > > > > >
> > > > > > And as you can see from the log, it tried 40 times, what took it
> > > > exactly
> > > > > > 15 minutes.
> > > > > >
> > > > > > There's probably some parameter to tune to cut it of from 40
> times
> > /
> > > 15
> > > > > > minutes to something more useful, as for 15 minutes we don't have
> > our
> > > > > > regions available, and HDFS have however replication factor 3.
> > > > > >
> > > > > > Googling, if I figure out what's this I will post it here. Will
> > also
> > > > > > appreciate if someone knows how to cut this down.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Dejan
> > > > > >
> > > > > > On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <
> nkeywal@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > >> The split is done by the region servers (the master
> coordinates).
> > Is
> > > > > there
> > > > > >> some interesting stuff in their logs?
> > > > > >>
> > > > > >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <
> > > dejan.menges@gmail.com
> > > > >
> > > > > >> wrote:
> > > > > >>
> > > > > >> > With client issue was that it was retrying connecting to the
> > same
> > > > > region
> > > > > >> > servers even when they were reassigned. Lowering it down
> helped
> > in
> > > > > this
> > > > > >> > specific use case, but yes, we still want servers to
> reallocate
> > > > > quickly.
> > > > > >> >
> > > > > >> > What got me here:
> > > > > >> >
> > > > > >> > http://hbase.apache.org/book.html#mttr
> > > > > >> >
> > > > > >> > I basically set configuration exactly the same way as it's
> > > explained
> > > > > >> here.
> > > > > >> > *zookeeper.session.timeout* is (and was before) 60000 (one
> > > minute).
> > > > > >> >
> > > > > >> > So basically what happens is: - simulating network issues we
> had
> > > > > >> recently).
> > > > > >> > - After short time I see in HBase that my RegionServer is
> dead,
> > > and
> > > > as
> > > > > >> > total number of regions I see previous total minus number of
> > > regions
> > > > > >> that
> > > > > >> > were hosted on the node hosting RegionServer that just
> > > > 'disappeared'.
> > > > > >> > - In this point I want my clus
> > > > > >> >
> > > > > >> > - I have test cluster consisting of four nodes, every node
> being
> > > > > >> DataNode
> > > > > >> > and RegionServer.
> > > > > >> > - I simulate network partition on one (connect to it through
> > > console
> > > > > and
> > > > > >> > take network interface downter to recover as soon as possible,
> > to
> > > > > start
> > > > > >> > serving missing regions.
> > > > > >> > - First thing I see in HMaster logs are:
> > > > > >> >
> > > > > >> > 2015-03-20 14:17:26,015 INFO
> > > > > >> > org.apache.hadoop.hbase.zookeeper.RegionServerTracker:
> > > RegionServer
> > > > > >> > ephemeral node deleted, processing expiration
> > > > > >> > [{name_of_node_I_took_down},60020,1426860403261]
> > > > > >> >
> > > > > >> > 2015-03-20 14:17:26,067 INFO
> > > > > >> > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler:
> > > > > Splitting
> > > > > >> > logs for {name_of_node_I_took_down},60020,1426860403261 before
> > > > > >> assignment.
> > > > > >> >
> > > > > >> > 2015-03-20 14:17:26,105 INFO
> > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: dead splitlog
> > > > workers
> > > > > [
> > > > > >> > {name_of_node_I_took_down},60020,1426860403261]
> > > > > >> >
> > > > > >> > 2015-03-20 14:17:26,107 INFO
> > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: started
> > splitting
> > > 1
> > > > > >> logs in
> > > > > >> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_I_
> > > > took_down}
> > > > > >> > ,60020,1426860403261-splitting]
> > > > > >> >
> > > > > >> > 2015-03-20 14:17:26,150 INFO
> > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > > > > >> > /hbase/splitWAL/WALs%2F
> > > > > >> >
> {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > > >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.14268
> > > > 60404905
> > > > > >> > acquired by {fqdn_of_another_live_hnode},60020,1426859445623
> > > > > >> >
> > > > > >> > 2015-03-20 14:17:26,182 INFO
> > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks =
> 1
> > > > > >> unassigned
> > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > >> >
> > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > in_progress
> > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> > > error =
> > > > > 0}
> > > > > >> >
> > > > > >> > 2015-03-20 14:17:31,183 INFO
> > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks =
> 1
> > > > > >> unassigned
> > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > >> >
> > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > in_progress
> > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> > > error =
> > > > > 0}
> > > > > >> >
> > > > > >> > 2015-03-20 14:17:36,184 INFO
> > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks =
> 1
> > > > > >> unassigned
> > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > >> >
> > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > in_progress
> > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> > > error =
> > > > > 0}
> > > > > >> >
> > > > > >> > 2015-03-20 14:17:42,185 INFO
> > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks =
> 1
> > > > > >> unassigned
> > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > >> >
> > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > in_progress
> > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> > > error =
> > > > > 0}
> > > > > >> >
> > > > > >> > 2015-03-20 14:17:48,184 INFO
> > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks =
> 1
> > > > > >> unassigned
> > > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > >> >
> > > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > > in_progress
> > > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> > > error =
> > > > > 0}
> > > > > >> > In the meantime, In hbase...out log I got this:
> > > > > >> >
> > > > > >> > ==> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out <==
> > > > > >> >
> > > > > >> > java.io.IOException: Call to
> > > > > >> > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_dow
> > > > n}:60020
> > > > > >> > failed on local exception:
> > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$CallTimeoutException:
> > Call
> > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > > >> >
> > > > > >> > at
> org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
> > > > > >> t.java:1532)
> > > > > >> >
> > > > > >> > at org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:
> > 1502)
> > > > > >> >
> > > > > >> > at
> > > > > >> >
> > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(Rpc
> > > > > >> Client.java:1684)
> > > > > >> >
> > > > > >> > at
> > > > > >> >
> > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$BlockingRpcChannelImpl
> > > > ementati
> > > > > >> on.callBlockingMethod(RpcClient.java:1737)
> > > > > >> >
> > > > > >> > at
> > > > > >> >
> > > > > >> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$
> > > > AdminService$
> > > > > >> BlockingStub.getRegionInfo(AdminProtos.java:20806)
> > > > > >> >
> > > > > >> > at
> > > > > >> >
> > > > > >> > org.apache.hadoop.hbase.client.HBaseAdmin.getCompactionState
> > > > > >> (HBaseAdmin.java:2524)
> > > > > >> >
> > > > > >> > at
> > > > > >> >
> > > > > >> > org.apache.hadoop.hbase.generated.master.table_jsp._jspServi
> > > > > >> ce(table_jsp.java:167)
> > > > > >> >
> > > > > >> > at org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.
> > > > java:98)
> > > > > >> >
> > > > > >> > at
> javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
> > > > > >> >
> > > > > >> > at
> org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
> > > > > >> .java:511)
> > > > > >> >
> > > > > >> > at
> > > > > >> >
> > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > > > > >> r(ServletHandler.java:1221)
> > > > > >> >
> > > > > >> > at
> > > > > >> >
> > > > > >> > org.apache.hadoop.http.lib.StaticUserWebFilter$StaticUserFil
> > > > > >> ter.doFilter(StaticUserWebFilter.java:109)
> > > > > >> >
> > > > > >> > at
> > > > > >> >
> > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > > > > >> r(ServletHandler.java:1212)
> > > > > >> >
> > > > > >> > at
> > > > > >> >
> > > > > >> > org.apache.hadoop.http.HttpServer$QuotingInputFilter.doFilte
> > > > > >> r(HttpServer.java:1081)
> > > > > >> >
> > > > > >> > at
> > > > > >> >
> > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > > > > >> r(ServletHandler.java:1212)
> > > > > >> >
> > > > > >> > at
> > > > >
> org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
> > > > > >> >
> > > > > >> > at
> > > > > >> >
> > > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > > > > >> r(ServletHandler.java:1212)
> > > > > >> >
> > > > > >> > at
> org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
> > > > > >> er.java:399)
> > > > > >> >
> > > > > >> > at
> > > > > >> > org.mortbay.jetty.security.SecurityHandler.handle(SecurityHa
> > > > > >> ndler.java:216)
> > > > > >> >
> > > > > >> > at
> org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
> > > > > >> er.java:182)
> > > > > >> >
> > > > > >> > at
> org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
> > > > > >> er.java:766)
> > > > > >> >
> > > > > >> > at
> org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
> > > > > >> java:450)
> > > > > >> >
> > > > > >> > at
> > > > > >> >
> > > > > >> > org.mortbay.jetty.handler.ContextHandlerCollection.handle(Co
> > > > > >> ntextHandlerCollection.java:230)
> > > > > >> >
> > > > > >> > at
> org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
> > > > > >> er.java:152)
> > > > > >> >
> > > > > >> > at org.mortbay.jetty.Server.handle(Server.java:326)
> > > > > >> >
> > > > > >> > at
> org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
> > > > > >> n.java:542)
> > > > > >> >
> > > > > >> > at
> > > > > >> >
> > > > > >> > org.mortbay.jetty.HttpConnection$RequestHandler.headerComple
> > > > > >> te(HttpConnection.java:928)
> > > > > >> >
> > > > > >> > at org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
> > > > > >> >
> > > > > >> > at
> > > org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
> > > > > >> >
> > > > > >> > at
> > > org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
> > > > > >> >
> > > > > >> > at
> > > > > >> >
> > > > > >> > org.mortbay.io.nio.SelectChannelEndPoint.run(SelectChannelEn
> > > > > >> dPoint.java:410)
> > > > > >> >
> > > > > >> > at
> > > > > >> >
> > > > > >> > org.mortbay.thread.QueuedThreadPool$PoolThread.run(
> > > > > >> QueuedThreadPool.java:582)
> > > > > >> >
> > > > > >> > Caused by: org.apache.hadoop.hbase.ipc.RpcClient$
> > > > CallTimeoutException:
> > > > > >> Call
> > > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > > >> >
> > > > > >> > at
> > > > > >> >
> > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.cleanupCall
> > > > > >> s(RpcClient.java:1234)
> > > > > >> >
> > > > > >> > at
> > > > > >> >
> > > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.readRespons
> > > > > >> e(RpcClient.java:1171)
> > > > > >> >
> > > > > >> > at
> org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
> > > > > >> nt.java:751)
> > > > > >> > Beside this same issue, please note that first message was at
> > > > > 2015-03-20
> > > > > >> > 14:17:26,015. And then (we got to the point when it started
> > > > > transition):
> > > > > >> >
> > > > > >> > 2015-03-20 14:32:35,059 INFO
> > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > > > > >> > /hbase/splitWAL/WALs%2F
> > > > > >> >
> {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > > >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.14268
> > > > 60404905
> > > > > >> > entered state: DONE
> {fqdn_of_new_live_node},60020,1426859445623
> > > > > >> >
> > > > > >> > 2015-03-20 14:32:35,109 INFO
> > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: Done splitting
> > > > > >> > /hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > > >> >
> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}
> > > > > >> > %252C60020%252C1426860403261.1426860404905
> > > > > >> >
> > > > > >> > 2015-03-20 14:32:35,190 INFO
> > > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: finished
> > splitting
> > > > > >> (more
> > > > > >> > than or equal to) 9 bytes in 1 log files in
> > > > > >> >
> > > > > >> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_of_
> > > > > >> node_I_took_down},60020,1426860403261-splitting]
> > > > > >> > in 909083ms
> > > > > >> >
> > > > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > > > > >> .RegionStates:
> > > > > >> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c state=OPEN,
> > > > > >> > ts=1426860639088,
> > > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > > >> to
> > > > > >> > {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE,
> > ts=1426861955191,
> > > > > >> server=
> > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > > >> >
> > > > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > > > > >> .RegionStates:
> > > > > >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
> > > > > >> {name_of_node_I_took_down}
> > > > > >> > ,60020,1426860403261
> > > > > >> >
> > > > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > > > > >> .RegionStates:
> > > > > >> > Transitioned {25ab6e7b42e36ddaa723d71be5954543 state=OPEN,
> > > > > >> > ts=1426860641783,
> > > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > > >> to
> > > > > >> > {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE,
> > ts=1426861955191,
> > > > > >> server=
> > > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > > >> >
> > > > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > > > > >> .RegionStates:
> > > > > >> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
> > > > > >> {name_of_node_I_took_down}
> > > > > >> > ,60020,1426860403261
> > > > > >> > At this point, note that it finished SplitLogManager task at
> > > > 14:32:35
> > > > > >> and
> > > > > >> > started transitioning just after that. So this is 15 minutes
> > that
> > > > I'm
> > > > > >> > talking about.
> > > > > >> >
> > > > > >> > What am I missing?
> > > > > >> >
> > > > > >> >
> > > > > >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <
> > > nkeywal@gmail.com>
> > > > > >> wrote:
> > > > > >> >
> > > > > >> > > You've changed the value of hbase.zookeeper.timeout to 15
> > > > minutes? A
> > > > > >> very
> > > > > >> > > reasonable target is 1 minute before relocating the regions.
> > > > That's
> > > > > >> the
> > > > > >> > > default iirc. You can push it to 20s, but then
> > > > gc-stopping-the-world
> > > > > >> > > becomes more of an issue. 15 minutes is really a lot. The
> hdfs
> > > > stale
> > > > > >> mode
> > > > > >> > > must always be used, with a lower timeout than the hbase
> one.
> > > > > >> > >
> > > > > >> > > Client side there should be nothing to do (excepted for
> > advanced
> > > > > >> stuff);
> > > > > >> > at
> > > > > >> > > each retry the client checks the location of the regions. If
> > you
> > > > > lower
> > > > > >> > the
> > > > > >> > > number of retry the client will fail sooner, but usually you
> > > don't
> > > > > >> want
> > > > > >> > the
> > > > > >> > > client to fail, you want the servers to reallocate quickly.
> > > > > >> > >
> > > > > >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <
> > > > > dejan.menges@gmail.com
> > > > > >> >
> > > > > >> > > wrote:
> > > > > >> > >
> > > > > >> > > > Hi,
> > > > > >> > > >
> > > > > >> > > > Sorry for little bit late update, but managed to narrow it
> > > > little
> > > > > >> bit
> > > > > >> > > down.
> > > > > >> > > >
> > > > > >> > > > We didn't update yet, as we are using Hortonworks
> > distribution
> > > > > right
> > > > > >> > now,
> > > > > >> > > > and even if we update we will get 0.98.4. However, looks
> > that
> > > > > issue
> > > > > >> > here
> > > > > >> > > > was in our use case and configuration (still looking into
> > it).
> > > > > >> > > >
> > > > > >> > > > Basically, initially I saw that when one server goes down,
> > we
> > > > > start
> > > > > >> > > having
> > > > > >> > > > performance issues in general, but it managed to be on our
> > > > client
> > > > > >> side,
> > > > > >> > > due
> > > > > >> > > > to caching, and clients were trying to reconnect to nodes
> > that
> > > > > were
> > > > > >> > > offline
> > > > > >> > > > and later trying to get regions from those nodes too. This
> > is
> > > > > >> basically
> > > > > >> > > why
> > > > > >> > > > on server side I didn't manage to see anything in logs
> that
> > > > would
> > > > > >> be at
> > > > > >> > > > least little bit interesting and point me into desired
> > > > direction.
> > > > > >> > > >
> > > > > >> > > > Another question that popped up to me is - in case server
> is
> > > > down
> > > > > >> (and
> > > > > >> > > with
> > > > > >> > > > it DataNode and HRegionServer it was hosting) - what's
> > optimal
> > > > > time
> > > > > >> to
> > > > > >> > > set
> > > > > >> > > > for HMaster to consider server dead reassign regions
> > somewhere
> > > > > >> else, as
> > > > > >> > > > this is another performance bottleneck we hit during
> > inability
> > > > to
> > > > > >> > access
> > > > > >> > > > regions? In our case it's configured to 15 minutes, and
> > simple
> > > > > logic
> > > > > >> > > tells
> > > > > >> > > > me if you want it earlier then configure lower number of
> > > > retries,
> > > > > >> but
> > > > > >> > > issue
> > > > > >> > > > is as always in details, so not sure if anyone knows some
> > > better
> > > > > >> math
> > > > > >> > for
> > > > > >> > > > this?
> > > > > >> > > >
> > > > > >> > > > And last question - is it possible to manually force HBase
> > to
> > > > > >> reassign
> > > > > >> > > > regions? In this case, while HMaster is retrying to
> contact
> > > node
> > > > > >> that's
> > > > > >> > > > dead, it's impossible to force it using 'balancer'
> command.
> > > > > >> > > >
> > > > > >> > > > Thanks a lot!
> > > > > >> > > >
> > > > > >> > > > Dejan
> > > > > >> > > >
> > > > > >> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <
> > > > > >> dejan.menges@gmail.com>
> > > > > >> > > > wrote:
> > > > > >> > > >
> > > > > >> > > > > Hi,
> > > > > >> > > > >
> > > > > >> > > > > To be very honest - there's no particular reason why we
> > > stick
> > > > to
> > > > > >> this
> > > > > >> > > > one,
> > > > > >> > > > > beside just lack of time currently to go through upgrade
> > > > > process,
> > > > > >> but
> > > > > >> > > > looks
> > > > > >> > > > > to me that's going to be next step.
> > > > > >> > > > >
> > > > > >> > > > > Had a crazy day, didn't have time to go through all logs
> > > > again,
> > > > > >> plus
> > > > > >> > > one
> > > > > >> > > > > of the machines (last one where we had this issue) is
> > fully
> > > > > >> > > reprovisioned
> > > > > >> > > > > yesterday so I don't have logs from there anymore.
> > > > > >> > > > >
> > > > > >> > > > > Beside upgrading,  what I will talk about today, can you
> > > just
> > > > > >> point
> > > > > >> > me
> > > > > >> > > to
> > > > > >> > > > > the specific RPC issue in 0.98.0? Thing is that we have
> > some
> > > > > >> strange
> > > > > >> > > > > moments with RPC in this case, and just want to see if
> > > that's
> > > > > the
> > > > > >> > same
> > > > > >> > > > > thing (and we were even suspecting to RPC).
> > > > > >> > > > >
> > > > > >> > > > > Thanks a lot!
> > > > > >> > > > > Dejan
> > > > > >> > > > >
> > > > > >> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew Purtell <
> > > > > >> apurtell@apache.org
> > > > > >> > >
> > > > > >> > > > > wrote:
> > > > > >> > > > >
> > > > > >> > > > >> Is there a particular reason why you are using HBase
> > > 0.98.0?
> > > > > The
> > > > > >> > > latest
> > > > > >> > > > >> 0.98 release is 0.98.11. There's a known performance
> > issue
> > > > with
> > > > > >> > 0.98.0
> > > > > >> > > > >> pertaining to RPC that was fixed in later releases, you
> > > > should
> > > > > >> move
> > > > > >> > up
> > > > > >> > > > >> from
> > > > > >> > > > >> 0.98.0. In addition hundreds of improvements and bug
> > fixes
> > > > have
> > > > > >> gone
> > > > > >> > > > into
> > > > > >> > > > >> the ten releases since 0.98.0.
> > > > > >> > > > >>
> > > > > >> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <
> > > > > >> > dejan.menges@gmail.com
> > > > > >> > > >
> > > > > >> > > > >> wrote:
> > > > > >> > > > >>
> > > > > >> > > > >> > Hi All,
> > > > > >> > > > >> >
> > > > > >> > > > >> > We have a strange issue with HBase performance
> (overall
> > > > > cluster
> > > > > >> > > > >> > performance) in case one of datanodes in the cluster
> > > > > >> unexpectedly
> > > > > >> > > goes
> > > > > >> > > > >> > down.
> > > > > >> > > > >> >
> > > > > >> > > > >> > So scenario is like follows:
> > > > > >> > > > >> > - Cluster works fine, it's stable.
> > > > > >> > > > >> > - One DataNode unexpectedly goes down (PSU issue,
> > network
> > > > > >> issue,
> > > > > >> > > > >> anything)
> > > > > >> > > > >> > - Whole HBase cluster goes down (performance becomes
> so
> > > bad
> > > > > >> that
> > > > > >> > we
> > > > > >> > > > >> have to
> > > > > >> > > > >> > restart all RegionServers to get it back to life).
> > > > > >> > > > >> >
> > > > > >> > > > >> > Most funny and latest issue that happened was that we
> > > added
> > > > > new
> > > > > >> > node
> > > > > >> > > > to
> > > > > >> > > > >> the
> > > > > >> > > > >> > cluster (having 8 x 4T SATA disks) and we left just
> > > > DataNode
> > > > > >> > running
> > > > > >> > > > on
> > > > > >> > > > >> it
> > > > > >> > > > >> > to give it couple of days to get some data. At some
> > point
> > > > in
> > > > > >> time,
> > > > > >> > > due
> > > > > >> > > > >> to
> > > > > >> > > > >> > hardware issue, server rebooted (twice during three
> > > hours)
> > > > in
> > > > > >> > moment
> > > > > >> > > > >> when
> > > > > >> > > > >> > it had maybe 5% of data it would have in a couple of
> > > days.
> > > > > >> Nothing
> > > > > >> > > > else
> > > > > >> > > > >> > beside DataNode was running, and once it went down,
> it
> > > > > affected
> > > > > >> > > > literary
> > > > > >> > > > >> > everything, and restarting RegionServers in the end
> > fixed
> > > > it.
> > > > > >> > > > >> >
> > > > > >> > > > >> > We are using HBase 0.98.0 with Hadoop 2.4.0
> > > > > >> > > > >> >
> > > > > >> > > > >>
> > > > > >> > > > >>
> > > > > >> > > > >>
> > > > > >> > > > >> --
> > > > > >> > > > >> Best regards,
> > > > > >> > > > >>
> > > > > >> > > > >>    - Andy
> > > > > >> > > > >>
> > > > > >> > > > >> Problems worthy of attack prove their worth by hitting
> > > back.
> > > > -
> > > > > >> Piet
> > > > > >> > > Hein
> > > > > >> > > > >> (via Tom White)
> > > > > >> > > > >>
> > > > > >> > > > >
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Strange issue when DataNode goes down

Posted by Dejan Menges <de...@gmail.com>.
Will take a look.

Actually, if node is down (someone unplugged network cable, it just died,
whatever) what's chance it's going to become live so write can continue? On
the other side, HBase is not starting recovery trying to contact node which
is not there anymore, and even elected as dead on Namenode side (another
thing I didn't understood quite good).

So what I was expecting is that as soon as Namenode decided node is dead,
that it would be enough for RegionServer to stop trying to recover from the
dead node, but it wasn't the case. Also, this whole MTTR article in HBase
book doesn't work at all with this parameter set to it's default value (15
minutes).

So I'm still struggling to figure out what's drawback exactly on this?

On Mon, Mar 23, 2015 at 2:50 PM Nicolas Liochon <nk...@gmail.com> wrote:

> Thanks for the explanation. There is an issue if you modify this setting
> however.
> hbase tries to recover the lease (i.e. be sure that nobody is writing)
> If you change hbase.lease.recovery.timeout hbase will start the recovery
> (i.e. start to read) even if it's not sure that nobody's writing. That
> means there is a dataloss risk.
> Basically, you must not see this warning: WARN
> org.apache.hadoop.hbase.util.FSHDFSUtils: Cannot recoverLease after trying
> for[]
>
> The recoverLease must succeed. The fact that it does not after X tries is
> strange.
> There may be a mistmatch between the hbase parameters and the hdfs ones.
> You may need to have a look at the comments in FSHDFSUtils.java
>
>
>
>
> On Mon, Mar 23, 2015 at 2:15 PM, Dejan Menges <de...@gmail.com>
> wrote:
>
> > I found the issue and fixed it, and will try to explain here what was
> > exactly in our case in case someone else finds this interesting too.
> >
> > So initially, we had (couple of times) some network and hardware issues
> in
> > our datacenters. When one server would die (literary die, we had some
> issue
> > with PSUs) we saw issues with overall cluster performance on HBase side.
> As
> > cluster is quite big and live, it was also quite hard to figure out exact
> > root cause and how to fix the stuff we wanted to fix.
> >
> > So I set up another cluster, four nodes (with DataNode and RegionServer)
> > and two other nodes with HMaster and Namenode in HA, using same stuff we
> > use on production. We pumped some data into it, and I was able to
> reproduce
> > same issue last week on it. What I tried to do is to cut one server (shut
> > down it's interface) when all is good with cluster, when we have load,
> and
> > see what's going to happen.
> >
> > On Friday, after Nicolas mentioned, I started taking a look in HBase logs
> > on the node which was mentioned in HMaster log as the one taking over
> > regions for the dead server. Basically what I was able to observe was 15
> > minutes time (+- couple of seconds, what was also interesting, and will
> got
> > later to that) between HMaster figures out that one of it's RegionServers
> > is dead, and the time one of the mentioned nodes starts taking over those
> > regions and they start appearing in HMaster's Web UI.
> >
> > I then set up everything like mentioned here
> > http://hbase.apache.org/book.html#mttr - but still had exactly the same
> > issues. Went over (again and again) all currently configured stuff, but
> > still had the same issue.
> >
> > Then I started looking into HDFS. Opened NameNode UI, saw all is good,
> took
> > one node down, was also looking RegionServer logs in the same time - and
> I
> > also see that it took ~15 minutes for Namenode to elect dead node as
> dead.
> > Somehow in the same moment regions started getting back to life. I
> remember
> > in some older versions dfs timeout checks and check retries. Looked into
> > defaults for our Hadoop version -
> >
> > http://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/
> hadoop-hdfs/hdfs-default.xml
> > - and saw there that there's no recheck parameter anymore. Strange, as on
> > StackOverflow I found post from month ago, for newer version than we use
> > (we use 2.4.1, guy was using 2.6 - dfs.namenode.heartbeat.
> recheck-interval)
> > I set it to 10 seconds as he mentioned, having checks every three seconds
> > (default) and got DataNode elected as dead in ~45 seconds, as he
> mentioned.
> > Not sure why this parameter is not documented, but obviously it works.
> >
> > Then figured out it still didn't fix our HBase failover issue. I was
> > looking into HBase book again and again, and then saw this part:
> >
> > "How much time we allow elapse between calls to recover lease. Should be
> > larger than the dfs timeout."
> >
> > This was description for hbase.lease.recovery.dfs.timeout. Wasn't sure
> from
> > the comment what of all timeouts that's possible to set in Hadoop/HBase
> and
> > that has something to do with DFS is this all about. But picked
> > hbase.lease.recovery.timeout, and that was the catch.
> >
> > Initially, by default, hbase.lease.recovery.timeout is set to 15 minutes.
> > Not sure why, and wasn't able to find yet why, but getting this down to
> one
> > minute (what's more than OK for us) I was able to get rid of our issue.
> Not
> > also sure why this is not mentioned in MTTR section in HBase book, as
> > obviously MTTR doesn't work at all with this default timeout, at least it
> > doesn't work the way we expected it to work.
> >
> > So thanks again for everyone being spammed with this, and specially
> thanks
> > to Nicolas pointing me to the right direction.
> >
> > On Mon, Mar 23, 2015 at 1:37 PM Nicolas Liochon <nk...@gmail.com>
> wrote:
> >
> > > the attachments are rejected by the mailing list, can you put then on
> > > pastebin?
> > >
> > > stale is mandatory (so it's good), but the issue here is just before.
> The
> > > region server needs to read the file. In order to be sure that there is
> > no
> > > data loss, it needs to "recover the lease", that means ensuring that
> > nobody
> > > is writing the file. The regionserver calls the namenode to do this
> > > recoverLease. So there should be some info in the namenode logs. You
> have
> > > HDFS-4721 on your hdfs? The hbase book details (more or less...) this
> > > recoverLease stuff.
> > >
> > >
> > > On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <dejan.menges@gmail.com
> >
> > > wrote:
> > >
> > > > And also, just checked - dfs.namenode.avoid.read.stale.datanode and
> > > > dfs.namenode.avoid.write.stale.datanode
> > > > are both true, and dfs.namenode.stale.datanode.interval is set to
> > > default
> > > > 30000.
> > > >
> > > > On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <
> dejan.menges@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Nicolas,
> > > > >
> > > > > Please find log attached.
> > > > >
> > > > > As I see it now more clearly, it was trying to recover HDFS WALs
> from
> > > > node
> > > > > that's dead:
> > > > >
> > > > > 2015-03-23 08:53:44,381 WARN
> > org.apache.hadoop.hbase.util.FSHDFSUtils:
> > > > > Cannot recoverLease after trying for 900000ms
> > > > > (hbase.lease.recovery.timeout); continuing, but may be
> DATALOSS!!!;
> > > > > attempt=40 on
> > > > >
> > > > file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intent
> > > ionally_get_down_by_getting_network_down},60020,1426862900
> > > 506-splitting/{node_i_intentionally_get_down_by_gett
> > > ing_network_down}%2C60020%2C1426862900506.1427096924508
> > > > > after 908210ms
> > > > >
> > > > > And as you can see from the log, it tried 40 times, what took it
> > > exactly
> > > > > 15 minutes.
> > > > >
> > > > > There's probably some parameter to tune to cut it of from 40 times
> /
> > 15
> > > > > minutes to something more useful, as for 15 minutes we don't have
> our
> > > > > regions available, and HDFS have however replication factor 3.
> > > > >
> > > > > Googling, if I figure out what's this I will post it here. Will
> also
> > > > > appreciate if someone knows how to cut this down.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Dejan
> > > > >
> > > > > On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <nkeywal@gmail.com
> >
> > > > wrote:
> > > > >
> > > > >> The split is done by the region servers (the master coordinates).
> Is
> > > > there
> > > > >> some interesting stuff in their logs?
> > > > >>
> > > > >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <
> > dejan.menges@gmail.com
> > > >
> > > > >> wrote:
> > > > >>
> > > > >> > With client issue was that it was retrying connecting to the
> same
> > > > region
> > > > >> > servers even when they were reassigned. Lowering it down helped
> in
> > > > this
> > > > >> > specific use case, but yes, we still want servers to reallocate
> > > > quickly.
> > > > >> >
> > > > >> > What got me here:
> > > > >> >
> > > > >> > http://hbase.apache.org/book.html#mttr
> > > > >> >
> > > > >> > I basically set configuration exactly the same way as it's
> > explained
> > > > >> here.
> > > > >> > *zookeeper.session.timeout* is (and was before) 60000 (one
> > minute).
> > > > >> >
> > > > >> > So basically what happens is: - simulating network issues we had
> > > > >> recently).
> > > > >> > - After short time I see in HBase that my RegionServer is dead,
> > and
> > > as
> > > > >> > total number of regions I see previous total minus number of
> > regions
> > > > >> that
> > > > >> > were hosted on the node hosting RegionServer that just
> > > 'disappeared'.
> > > > >> > - In this point I want my clus
> > > > >> >
> > > > >> > - I have test cluster consisting of four nodes, every node being
> > > > >> DataNode
> > > > >> > and RegionServer.
> > > > >> > - I simulate network partition on one (connect to it through
> > console
> > > > and
> > > > >> > take network interface downter to recover as soon as possible,
> to
> > > > start
> > > > >> > serving missing regions.
> > > > >> > - First thing I see in HMaster logs are:
> > > > >> >
> > > > >> > 2015-03-20 14:17:26,015 INFO
> > > > >> > org.apache.hadoop.hbase.zookeeper.RegionServerTracker:
> > RegionServer
> > > > >> > ephemeral node deleted, processing expiration
> > > > >> > [{name_of_node_I_took_down},60020,1426860403261]
> > > > >> >
> > > > >> > 2015-03-20 14:17:26,067 INFO
> > > > >> > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler:
> > > > Splitting
> > > > >> > logs for {name_of_node_I_took_down},60020,1426860403261 before
> > > > >> assignment.
> > > > >> >
> > > > >> > 2015-03-20 14:17:26,105 INFO
> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: dead splitlog
> > > workers
> > > > [
> > > > >> > {name_of_node_I_took_down},60020,1426860403261]
> > > > >> >
> > > > >> > 2015-03-20 14:17:26,107 INFO
> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: started
> splitting
> > 1
> > > > >> logs in
> > > > >> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_I_
> > > took_down}
> > > > >> > ,60020,1426860403261-splitting]
> > > > >> >
> > > > >> > 2015-03-20 14:17:26,150 INFO
> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > > > >> > /hbase/splitWAL/WALs%2F
> > > > >> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.14268
> > > 60404905
> > > > >> > acquired by {fqdn_of_another_live_hnode},60020,1426859445623
> > > > >> >
> > > > >> > 2015-03-20 14:17:26,182 INFO
> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> > > > >> unassigned
> > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > >> >
> > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > in_progress
> > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> > error =
> > > > 0}
> > > > >> >
> > > > >> > 2015-03-20 14:17:31,183 INFO
> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> > > > >> unassigned
> > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > >> >
> > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > in_progress
> > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> > error =
> > > > 0}
> > > > >> >
> > > > >> > 2015-03-20 14:17:36,184 INFO
> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> > > > >> unassigned
> > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > >> >
> > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > in_progress
> > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> > error =
> > > > 0}
> > > > >> >
> > > > >> > 2015-03-20 14:17:42,185 INFO
> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> > > > >> unassigned
> > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > >> >
> > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > in_progress
> > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> > error =
> > > > 0}
> > > > >> >
> > > > >> > 2015-03-20 14:17:48,184 INFO
> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> > > > >> unassigned
> > > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > >> >
> > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > > in_progress
> > > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> > error =
> > > > 0}
> > > > >> > In the meantime, In hbase...out log I got this:
> > > > >> >
> > > > >> > ==> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out <==
> > > > >> >
> > > > >> > java.io.IOException: Call to
> > > > >> > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_dow
> > > n}:60020
> > > > >> > failed on local exception:
> > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$CallTimeoutException:
> Call
> > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > >> >
> > > > >> > at org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
> > > > >> t.java:1532)
> > > > >> >
> > > > >> > at org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:
> 1502)
> > > > >> >
> > > > >> > at
> > > > >> >
> > > > >> > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(Rpc
> > > > >> Client.java:1684)
> > > > >> >
> > > > >> > at
> > > > >> >
> > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$BlockingRpcChannelImpl
> > > ementati
> > > > >> on.callBlockingMethod(RpcClient.java:1737)
> > > > >> >
> > > > >> > at
> > > > >> >
> > > > >> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$
> > > AdminService$
> > > > >> BlockingStub.getRegionInfo(AdminProtos.java:20806)
> > > > >> >
> > > > >> > at
> > > > >> >
> > > > >> > org.apache.hadoop.hbase.client.HBaseAdmin.getCompactionState
> > > > >> (HBaseAdmin.java:2524)
> > > > >> >
> > > > >> > at
> > > > >> >
> > > > >> > org.apache.hadoop.hbase.generated.master.table_jsp._jspServi
> > > > >> ce(table_jsp.java:167)
> > > > >> >
> > > > >> > at org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.
> > > java:98)
> > > > >> >
> > > > >> > at javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
> > > > >> >
> > > > >> > at org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
> > > > >> .java:511)
> > > > >> >
> > > > >> > at
> > > > >> >
> > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > > > >> r(ServletHandler.java:1221)
> > > > >> >
> > > > >> > at
> > > > >> >
> > > > >> > org.apache.hadoop.http.lib.StaticUserWebFilter$StaticUserFil
> > > > >> ter.doFilter(StaticUserWebFilter.java:109)
> > > > >> >
> > > > >> > at
> > > > >> >
> > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > > > >> r(ServletHandler.java:1212)
> > > > >> >
> > > > >> > at
> > > > >> >
> > > > >> > org.apache.hadoop.http.HttpServer$QuotingInputFilter.doFilte
> > > > >> r(HttpServer.java:1081)
> > > > >> >
> > > > >> > at
> > > > >> >
> > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > > > >> r(ServletHandler.java:1212)
> > > > >> >
> > > > >> > at
> > > > org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
> > > > >> >
> > > > >> > at
> > > > >> >
> > > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > > > >> r(ServletHandler.java:1212)
> > > > >> >
> > > > >> > at org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
> > > > >> er.java:399)
> > > > >> >
> > > > >> > at
> > > > >> > org.mortbay.jetty.security.SecurityHandler.handle(SecurityHa
> > > > >> ndler.java:216)
> > > > >> >
> > > > >> > at org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
> > > > >> er.java:182)
> > > > >> >
> > > > >> > at org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
> > > > >> er.java:766)
> > > > >> >
> > > > >> > at org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
> > > > >> java:450)
> > > > >> >
> > > > >> > at
> > > > >> >
> > > > >> > org.mortbay.jetty.handler.ContextHandlerCollection.handle(Co
> > > > >> ntextHandlerCollection.java:230)
> > > > >> >
> > > > >> > at org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
> > > > >> er.java:152)
> > > > >> >
> > > > >> > at org.mortbay.jetty.Server.handle(Server.java:326)
> > > > >> >
> > > > >> > at org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
> > > > >> n.java:542)
> > > > >> >
> > > > >> > at
> > > > >> >
> > > > >> > org.mortbay.jetty.HttpConnection$RequestHandler.headerComple
> > > > >> te(HttpConnection.java:928)
> > > > >> >
> > > > >> > at org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
> > > > >> >
> > > > >> > at
> > org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
> > > > >> >
> > > > >> > at
> > org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
> > > > >> >
> > > > >> > at
> > > > >> >
> > > > >> > org.mortbay.io.nio.SelectChannelEndPoint.run(SelectChannelEn
> > > > >> dPoint.java:410)
> > > > >> >
> > > > >> > at
> > > > >> >
> > > > >> > org.mortbay.thread.QueuedThreadPool$PoolThread.run(
> > > > >> QueuedThreadPool.java:582)
> > > > >> >
> > > > >> > Caused by: org.apache.hadoop.hbase.ipc.RpcClient$
> > > CallTimeoutException:
> > > > >> Call
> > > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > > >> >
> > > > >> > at
> > > > >> >
> > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.cleanupCall
> > > > >> s(RpcClient.java:1234)
> > > > >> >
> > > > >> > at
> > > > >> >
> > > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.readRespons
> > > > >> e(RpcClient.java:1171)
> > > > >> >
> > > > >> > at org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
> > > > >> nt.java:751)
> > > > >> > Beside this same issue, please note that first message was at
> > > > 2015-03-20
> > > > >> > 14:17:26,015. And then (we got to the point when it started
> > > > transition):
> > > > >> >
> > > > >> > 2015-03-20 14:32:35,059 INFO
> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > > > >> > /hbase/splitWAL/WALs%2F
> > > > >> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > > >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.14268
> > > 60404905
> > > > >> > entered state: DONE {fqdn_of_new_live_node},60020,1426859445623
> > > > >> >
> > > > >> > 2015-03-20 14:32:35,109 INFO
> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: Done splitting
> > > > >> > /hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}
> > > > >> > %252C60020%252C1426860403261.1426860404905
> > > > >> >
> > > > >> > 2015-03-20 14:32:35,190 INFO
> > > > >> > org.apache.hadoop.hbase.master.SplitLogManager: finished
> splitting
> > > > >> (more
> > > > >> > than or equal to) 9 bytes in 1 log files in
> > > > >> >
> > > > >> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_of_
> > > > >> node_I_took_down},60020,1426860403261-splitting]
> > > > >> > in 909083ms
> > > > >> >
> > > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > > > >> .RegionStates:
> > > > >> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c state=OPEN,
> > > > >> > ts=1426860639088,
> > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > >> to
> > > > >> > {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE,
> ts=1426861955191,
> > > > >> server=
> > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > >> >
> > > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > > > >> .RegionStates:
> > > > >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
> > > > >> {name_of_node_I_took_down}
> > > > >> > ,60020,1426860403261
> > > > >> >
> > > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > > > >> .RegionStates:
> > > > >> > Transitioned {25ab6e7b42e36ddaa723d71be5954543 state=OPEN,
> > > > >> > ts=1426860641783,
> > > > server={name_of_node_I_took_down},60020,1426860403261}
> > > > >> to
> > > > >> > {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE,
> ts=1426861955191,
> > > > >> server=
> > > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > > >> >
> > > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > > > >> .RegionStates:
> > > > >> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
> > > > >> {name_of_node_I_took_down}
> > > > >> > ,60020,1426860403261
> > > > >> > At this point, note that it finished SplitLogManager task at
> > > 14:32:35
> > > > >> and
> > > > >> > started transitioning just after that. So this is 15 minutes
> that
> > > I'm
> > > > >> > talking about.
> > > > >> >
> > > > >> > What am I missing?
> > > > >> >
> > > > >> >
> > > > >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <
> > nkeywal@gmail.com>
> > > > >> wrote:
> > > > >> >
> > > > >> > > You've changed the value of hbase.zookeeper.timeout to 15
> > > minutes? A
> > > > >> very
> > > > >> > > reasonable target is 1 minute before relocating the regions.
> > > That's
> > > > >> the
> > > > >> > > default iirc. You can push it to 20s, but then
> > > gc-stopping-the-world
> > > > >> > > becomes more of an issue. 15 minutes is really a lot. The hdfs
> > > stale
> > > > >> mode
> > > > >> > > must always be used, with a lower timeout than the hbase one.
> > > > >> > >
> > > > >> > > Client side there should be nothing to do (excepted for
> advanced
> > > > >> stuff);
> > > > >> > at
> > > > >> > > each retry the client checks the location of the regions. If
> you
> > > > lower
> > > > >> > the
> > > > >> > > number of retry the client will fail sooner, but usually you
> > don't
> > > > >> want
> > > > >> > the
> > > > >> > > client to fail, you want the servers to reallocate quickly.
> > > > >> > >
> > > > >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <
> > > > dejan.menges@gmail.com
> > > > >> >
> > > > >> > > wrote:
> > > > >> > >
> > > > >> > > > Hi,
> > > > >> > > >
> > > > >> > > > Sorry for little bit late update, but managed to narrow it
> > > little
> > > > >> bit
> > > > >> > > down.
> > > > >> > > >
> > > > >> > > > We didn't update yet, as we are using Hortonworks
> distribution
> > > > right
> > > > >> > now,
> > > > >> > > > and even if we update we will get 0.98.4. However, looks
> that
> > > > issue
> > > > >> > here
> > > > >> > > > was in our use case and configuration (still looking into
> it).
> > > > >> > > >
> > > > >> > > > Basically, initially I saw that when one server goes down,
> we
> > > > start
> > > > >> > > having
> > > > >> > > > performance issues in general, but it managed to be on our
> > > client
> > > > >> side,
> > > > >> > > due
> > > > >> > > > to caching, and clients were trying to reconnect to nodes
> that
> > > > were
> > > > >> > > offline
> > > > >> > > > and later trying to get regions from those nodes too. This
> is
> > > > >> basically
> > > > >> > > why
> > > > >> > > > on server side I didn't manage to see anything in logs that
> > > would
> > > > >> be at
> > > > >> > > > least little bit interesting and point me into desired
> > > direction.
> > > > >> > > >
> > > > >> > > > Another question that popped up to me is - in case server is
> > > down
> > > > >> (and
> > > > >> > > with
> > > > >> > > > it DataNode and HRegionServer it was hosting) - what's
> optimal
> > > > time
> > > > >> to
> > > > >> > > set
> > > > >> > > > for HMaster to consider server dead reassign regions
> somewhere
> > > > >> else, as
> > > > >> > > > this is another performance bottleneck we hit during
> inability
> > > to
> > > > >> > access
> > > > >> > > > regions? In our case it's configured to 15 minutes, and
> simple
> > > > logic
> > > > >> > > tells
> > > > >> > > > me if you want it earlier then configure lower number of
> > > retries,
> > > > >> but
> > > > >> > > issue
> > > > >> > > > is as always in details, so not sure if anyone knows some
> > better
> > > > >> math
> > > > >> > for
> > > > >> > > > this?
> > > > >> > > >
> > > > >> > > > And last question - is it possible to manually force HBase
> to
> > > > >> reassign
> > > > >> > > > regions? In this case, while HMaster is retrying to contact
> > node
> > > > >> that's
> > > > >> > > > dead, it's impossible to force it using 'balancer' command.
> > > > >> > > >
> > > > >> > > > Thanks a lot!
> > > > >> > > >
> > > > >> > > > Dejan
> > > > >> > > >
> > > > >> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <
> > > > >> dejan.menges@gmail.com>
> > > > >> > > > wrote:
> > > > >> > > >
> > > > >> > > > > Hi,
> > > > >> > > > >
> > > > >> > > > > To be very honest - there's no particular reason why we
> > stick
> > > to
> > > > >> this
> > > > >> > > > one,
> > > > >> > > > > beside just lack of time currently to go through upgrade
> > > > process,
> > > > >> but
> > > > >> > > > looks
> > > > >> > > > > to me that's going to be next step.
> > > > >> > > > >
> > > > >> > > > > Had a crazy day, didn't have time to go through all logs
> > > again,
> > > > >> plus
> > > > >> > > one
> > > > >> > > > > of the machines (last one where we had this issue) is
> fully
> > > > >> > > reprovisioned
> > > > >> > > > > yesterday so I don't have logs from there anymore.
> > > > >> > > > >
> > > > >> > > > > Beside upgrading,  what I will talk about today, can you
> > just
> > > > >> point
> > > > >> > me
> > > > >> > > to
> > > > >> > > > > the specific RPC issue in 0.98.0? Thing is that we have
> some
> > > > >> strange
> > > > >> > > > > moments with RPC in this case, and just want to see if
> > that's
> > > > the
> > > > >> > same
> > > > >> > > > > thing (and we were even suspecting to RPC).
> > > > >> > > > >
> > > > >> > > > > Thanks a lot!
> > > > >> > > > > Dejan
> > > > >> > > > >
> > > > >> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew Purtell <
> > > > >> apurtell@apache.org
> > > > >> > >
> > > > >> > > > > wrote:
> > > > >> > > > >
> > > > >> > > > >> Is there a particular reason why you are using HBase
> > 0.98.0?
> > > > The
> > > > >> > > latest
> > > > >> > > > >> 0.98 release is 0.98.11. There's a known performance
> issue
> > > with
> > > > >> > 0.98.0
> > > > >> > > > >> pertaining to RPC that was fixed in later releases, you
> > > should
> > > > >> move
> > > > >> > up
> > > > >> > > > >> from
> > > > >> > > > >> 0.98.0. In addition hundreds of improvements and bug
> fixes
> > > have
> > > > >> gone
> > > > >> > > > into
> > > > >> > > > >> the ten releases since 0.98.0.
> > > > >> > > > >>
> > > > >> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <
> > > > >> > dejan.menges@gmail.com
> > > > >> > > >
> > > > >> > > > >> wrote:
> > > > >> > > > >>
> > > > >> > > > >> > Hi All,
> > > > >> > > > >> >
> > > > >> > > > >> > We have a strange issue with HBase performance (overall
> > > > cluster
> > > > >> > > > >> > performance) in case one of datanodes in the cluster
> > > > >> unexpectedly
> > > > >> > > goes
> > > > >> > > > >> > down.
> > > > >> > > > >> >
> > > > >> > > > >> > So scenario is like follows:
> > > > >> > > > >> > - Cluster works fine, it's stable.
> > > > >> > > > >> > - One DataNode unexpectedly goes down (PSU issue,
> network
> > > > >> issue,
> > > > >> > > > >> anything)
> > > > >> > > > >> > - Whole HBase cluster goes down (performance becomes so
> > bad
> > > > >> that
> > > > >> > we
> > > > >> > > > >> have to
> > > > >> > > > >> > restart all RegionServers to get it back to life).
> > > > >> > > > >> >
> > > > >> > > > >> > Most funny and latest issue that happened was that we
> > added
> > > > new
> > > > >> > node
> > > > >> > > > to
> > > > >> > > > >> the
> > > > >> > > > >> > cluster (having 8 x 4T SATA disks) and we left just
> > > DataNode
> > > > >> > running
> > > > >> > > > on
> > > > >> > > > >> it
> > > > >> > > > >> > to give it couple of days to get some data. At some
> point
> > > in
> > > > >> time,
> > > > >> > > due
> > > > >> > > > >> to
> > > > >> > > > >> > hardware issue, server rebooted (twice during three
> > hours)
> > > in
> > > > >> > moment
> > > > >> > > > >> when
> > > > >> > > > >> > it had maybe 5% of data it would have in a couple of
> > days.
> > > > >> Nothing
> > > > >> > > > else
> > > > >> > > > >> > beside DataNode was running, and once it went down, it
> > > > affected
> > > > >> > > > literary
> > > > >> > > > >> > everything, and restarting RegionServers in the end
> fixed
> > > it.
> > > > >> > > > >> >
> > > > >> > > > >> > We are using HBase 0.98.0 with Hadoop 2.4.0
> > > > >> > > > >> >
> > > > >> > > > >>
> > > > >> > > > >>
> > > > >> > > > >>
> > > > >> > > > >> --
> > > > >> > > > >> Best regards,
> > > > >> > > > >>
> > > > >> > > > >>    - Andy
> > > > >> > > > >>
> > > > >> > > > >> Problems worthy of attack prove their worth by hitting
> > back.
> > > -
> > > > >> Piet
> > > > >> > > Hein
> > > > >> > > > >> (via Tom White)
> > > > >> > > > >>
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > >
> > >
> >
>

Re: Strange issue when DataNode goes down

Posted by Nicolas Liochon <nk...@gmail.com>.
Thanks for the explanation. There is an issue if you modify this setting
however.
hbase tries to recover the lease (i.e. be sure that nobody is writing)
If you change hbase.lease.recovery.timeout hbase will start the recovery
(i.e. start to read) even if it's not sure that nobody's writing. That
means there is a dataloss risk.
Basically, you must not see this warning: WARN
org.apache.hadoop.hbase.util.FSHDFSUtils: Cannot recoverLease after trying
for[]

The recoverLease must succeed. The fact that it does not after X tries is
strange.
There may be a mistmatch between the hbase parameters and the hdfs ones.
You may need to have a look at the comments in FSHDFSUtils.java




On Mon, Mar 23, 2015 at 2:15 PM, Dejan Menges <de...@gmail.com>
wrote:

> I found the issue and fixed it, and will try to explain here what was
> exactly in our case in case someone else finds this interesting too.
>
> So initially, we had (couple of times) some network and hardware issues in
> our datacenters. When one server would die (literary die, we had some issue
> with PSUs) we saw issues with overall cluster performance on HBase side. As
> cluster is quite big and live, it was also quite hard to figure out exact
> root cause and how to fix the stuff we wanted to fix.
>
> So I set up another cluster, four nodes (with DataNode and RegionServer)
> and two other nodes with HMaster and Namenode in HA, using same stuff we
> use on production. We pumped some data into it, and I was able to reproduce
> same issue last week on it. What I tried to do is to cut one server (shut
> down it's interface) when all is good with cluster, when we have load, and
> see what's going to happen.
>
> On Friday, after Nicolas mentioned, I started taking a look in HBase logs
> on the node which was mentioned in HMaster log as the one taking over
> regions for the dead server. Basically what I was able to observe was 15
> minutes time (+- couple of seconds, what was also interesting, and will got
> later to that) between HMaster figures out that one of it's RegionServers
> is dead, and the time one of the mentioned nodes starts taking over those
> regions and they start appearing in HMaster's Web UI.
>
> I then set up everything like mentioned here
> http://hbase.apache.org/book.html#mttr - but still had exactly the same
> issues. Went over (again and again) all currently configured stuff, but
> still had the same issue.
>
> Then I started looking into HDFS. Opened NameNode UI, saw all is good, took
> one node down, was also looking RegionServer logs in the same time - and I
> also see that it took ~15 minutes for Namenode to elect dead node as dead.
> Somehow in the same moment regions started getting back to life. I remember
> in some older versions dfs timeout checks and check retries. Looked into
> defaults for our Hadoop version -
>
> http://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/hadoop-hdfs/hdfs-default.xml
> - and saw there that there's no recheck parameter anymore. Strange, as on
> StackOverflow I found post from month ago, for newer version than we use
> (we use 2.4.1, guy was using 2.6 - dfs.namenode.heartbeat.recheck-interval)
> I set it to 10 seconds as he mentioned, having checks every three seconds
> (default) and got DataNode elected as dead in ~45 seconds, as he mentioned.
> Not sure why this parameter is not documented, but obviously it works.
>
> Then figured out it still didn't fix our HBase failover issue. I was
> looking into HBase book again and again, and then saw this part:
>
> "How much time we allow elapse between calls to recover lease. Should be
> larger than the dfs timeout."
>
> This was description for hbase.lease.recovery.dfs.timeout. Wasn't sure from
> the comment what of all timeouts that's possible to set in Hadoop/HBase and
> that has something to do with DFS is this all about. But picked
> hbase.lease.recovery.timeout, and that was the catch.
>
> Initially, by default, hbase.lease.recovery.timeout is set to 15 minutes.
> Not sure why, and wasn't able to find yet why, but getting this down to one
> minute (what's more than OK for us) I was able to get rid of our issue. Not
> also sure why this is not mentioned in MTTR section in HBase book, as
> obviously MTTR doesn't work at all with this default timeout, at least it
> doesn't work the way we expected it to work.
>
> So thanks again for everyone being spammed with this, and specially thanks
> to Nicolas pointing me to the right direction.
>
> On Mon, Mar 23, 2015 at 1:37 PM Nicolas Liochon <nk...@gmail.com> wrote:
>
> > the attachments are rejected by the mailing list, can you put then on
> > pastebin?
> >
> > stale is mandatory (so it's good), but the issue here is just before. The
> > region server needs to read the file. In order to be sure that there is
> no
> > data loss, it needs to "recover the lease", that means ensuring that
> nobody
> > is writing the file. The regionserver calls the namenode to do this
> > recoverLease. So there should be some info in the namenode logs. You have
> > HDFS-4721 on your hdfs? The hbase book details (more or less...) this
> > recoverLease stuff.
> >
> >
> > On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <de...@gmail.com>
> > wrote:
> >
> > > And also, just checked - dfs.namenode.avoid.read.stale.datanode and
> > > dfs.namenode.avoid.write.stale.datanode
> > > are both true, and dfs.namenode.stale.datanode.interval is set to
> > default
> > > 30000.
> > >
> > > On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <de...@gmail.com>
> > > wrote:
> > >
> > > > Hi Nicolas,
> > > >
> > > > Please find log attached.
> > > >
> > > > As I see it now more clearly, it was trying to recover HDFS WALs from
> > > node
> > > > that's dead:
> > > >
> > > > 2015-03-23 08:53:44,381 WARN
> org.apache.hadoop.hbase.util.FSHDFSUtils:
> > > > Cannot recoverLease after trying for 900000ms
> > > > (hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!;
> > > > attempt=40 on
> > > >
> > > file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intent
> > ionally_get_down_by_getting_network_down},60020,1426862900
> > 506-splitting/{node_i_intentionally_get_down_by_gett
> > ing_network_down}%2C60020%2C1426862900506.1427096924508
> > > > after 908210ms
> > > >
> > > > And as you can see from the log, it tried 40 times, what took it
> > exactly
> > > > 15 minutes.
> > > >
> > > > There's probably some parameter to tune to cut it of from 40 times /
> 15
> > > > minutes to something more useful, as for 15 minutes we don't have our
> > > > regions available, and HDFS have however replication factor 3.
> > > >
> > > > Googling, if I figure out what's this I will post it here. Will also
> > > > appreciate if someone knows how to cut this down.
> > > >
> > > > Thanks,
> > > >
> > > > Dejan
> > > >
> > > > On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <nk...@gmail.com>
> > > wrote:
> > > >
> > > >> The split is done by the region servers (the master coordinates). Is
> > > there
> > > >> some interesting stuff in their logs?
> > > >>
> > > >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <
> dejan.menges@gmail.com
> > >
> > > >> wrote:
> > > >>
> > > >> > With client issue was that it was retrying connecting to the same
> > > region
> > > >> > servers even when they were reassigned. Lowering it down helped in
> > > this
> > > >> > specific use case, but yes, we still want servers to reallocate
> > > quickly.
> > > >> >
> > > >> > What got me here:
> > > >> >
> > > >> > http://hbase.apache.org/book.html#mttr
> > > >> >
> > > >> > I basically set configuration exactly the same way as it's
> explained
> > > >> here.
> > > >> > *zookeeper.session.timeout* is (and was before) 60000 (one
> minute).
> > > >> >
> > > >> > So basically what happens is: - simulating network issues we had
> > > >> recently).
> > > >> > - After short time I see in HBase that my RegionServer is dead,
> and
> > as
> > > >> > total number of regions I see previous total minus number of
> regions
> > > >> that
> > > >> > were hosted on the node hosting RegionServer that just
> > 'disappeared'.
> > > >> > - In this point I want my clus
> > > >> >
> > > >> > - I have test cluster consisting of four nodes, every node being
> > > >> DataNode
> > > >> > and RegionServer.
> > > >> > - I simulate network partition on one (connect to it through
> console
> > > and
> > > >> > take network interface downter to recover as soon as possible, to
> > > start
> > > >> > serving missing regions.
> > > >> > - First thing I see in HMaster logs are:
> > > >> >
> > > >> > 2015-03-20 14:17:26,015 INFO
> > > >> > org.apache.hadoop.hbase.zookeeper.RegionServerTracker:
> RegionServer
> > > >> > ephemeral node deleted, processing expiration
> > > >> > [{name_of_node_I_took_down},60020,1426860403261]
> > > >> >
> > > >> > 2015-03-20 14:17:26,067 INFO
> > > >> > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler:
> > > Splitting
> > > >> > logs for {name_of_node_I_took_down},60020,1426860403261 before
> > > >> assignment.
> > > >> >
> > > >> > 2015-03-20 14:17:26,105 INFO
> > > >> > org.apache.hadoop.hbase.master.SplitLogManager: dead splitlog
> > workers
> > > [
> > > >> > {name_of_node_I_took_down},60020,1426860403261]
> > > >> >
> > > >> > 2015-03-20 14:17:26,107 INFO
> > > >> > org.apache.hadoop.hbase.master.SplitLogManager: started splitting
> 1
> > > >> logs in
> > > >> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_I_
> > took_down}
> > > >> > ,60020,1426860403261-splitting]
> > > >> >
> > > >> > 2015-03-20 14:17:26,150 INFO
> > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > > >> > /hbase/splitWAL/WALs%2F
> > > >> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.14268
> > 60404905
> > > >> > acquired by {fqdn_of_another_live_hnode},60020,1426859445623
> > > >> >
> > > >> > 2015-03-20 14:17:26,182 INFO
> > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> > > >> unassigned
> > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > >> >
> > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > in_progress
> > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> error =
> > > 0}
> > > >> >
> > > >> > 2015-03-20 14:17:31,183 INFO
> > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> > > >> unassigned
> > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > >> >
> > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > in_progress
> > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> error =
> > > 0}
> > > >> >
> > > >> > 2015-03-20 14:17:36,184 INFO
> > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> > > >> unassigned
> > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > >> >
> > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > in_progress
> > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> error =
> > > 0}
> > > >> >
> > > >> > 2015-03-20 14:17:42,185 INFO
> > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> > > >> unassigned
> > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > >> >
> > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > in_progress
> > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> error =
> > > 0}
> > > >> >
> > > >> > 2015-03-20 14:17:48,184 INFO
> > > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> > > >> unassigned
> > > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > >> >
> > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> > in_progress
> > > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0
> error =
> > > 0}
> > > >> > In the meantime, In hbase...out log I got this:
> > > >> >
> > > >> > ==> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out <==
> > > >> >
> > > >> > java.io.IOException: Call to
> > > >> > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_dow
> > n}:60020
> > > >> > failed on local exception:
> > > >> > org.apache.hadoop.hbase.ipc.RpcClient$CallTimeoutException: Call
> > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > >> >
> > > >> > at org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
> > > >> t.java:1532)
> > > >> >
> > > >> > at org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:1502)
> > > >> >
> > > >> > at
> > > >> >
> > > >> > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(Rpc
> > > >> Client.java:1684)
> > > >> >
> > > >> > at
> > > >> >
> > > >> > org.apache.hadoop.hbase.ipc.RpcClient$BlockingRpcChannelImpl
> > ementati
> > > >> on.callBlockingMethod(RpcClient.java:1737)
> > > >> >
> > > >> > at
> > > >> >
> > > >> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$
> > AdminService$
> > > >> BlockingStub.getRegionInfo(AdminProtos.java:20806)
> > > >> >
> > > >> > at
> > > >> >
> > > >> > org.apache.hadoop.hbase.client.HBaseAdmin.getCompactionState
> > > >> (HBaseAdmin.java:2524)
> > > >> >
> > > >> > at
> > > >> >
> > > >> > org.apache.hadoop.hbase.generated.master.table_jsp._jspServi
> > > >> ce(table_jsp.java:167)
> > > >> >
> > > >> > at org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.
> > java:98)
> > > >> >
> > > >> > at javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
> > > >> >
> > > >> > at org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
> > > >> .java:511)
> > > >> >
> > > >> > at
> > > >> >
> > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > > >> r(ServletHandler.java:1221)
> > > >> >
> > > >> > at
> > > >> >
> > > >> > org.apache.hadoop.http.lib.StaticUserWebFilter$StaticUserFil
> > > >> ter.doFilter(StaticUserWebFilter.java:109)
> > > >> >
> > > >> > at
> > > >> >
> > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > > >> r(ServletHandler.java:1212)
> > > >> >
> > > >> > at
> > > >> >
> > > >> > org.apache.hadoop.http.HttpServer$QuotingInputFilter.doFilte
> > > >> r(HttpServer.java:1081)
> > > >> >
> > > >> > at
> > > >> >
> > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > > >> r(ServletHandler.java:1212)
> > > >> >
> > > >> > at
> > > org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
> > > >> >
> > > >> > at
> > > >> >
> > > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > > >> r(ServletHandler.java:1212)
> > > >> >
> > > >> > at org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
> > > >> er.java:399)
> > > >> >
> > > >> > at
> > > >> > org.mortbay.jetty.security.SecurityHandler.handle(SecurityHa
> > > >> ndler.java:216)
> > > >> >
> > > >> > at org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
> > > >> er.java:182)
> > > >> >
> > > >> > at org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
> > > >> er.java:766)
> > > >> >
> > > >> > at org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
> > > >> java:450)
> > > >> >
> > > >> > at
> > > >> >
> > > >> > org.mortbay.jetty.handler.ContextHandlerCollection.handle(Co
> > > >> ntextHandlerCollection.java:230)
> > > >> >
> > > >> > at org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
> > > >> er.java:152)
> > > >> >
> > > >> > at org.mortbay.jetty.Server.handle(Server.java:326)
> > > >> >
> > > >> > at org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
> > > >> n.java:542)
> > > >> >
> > > >> > at
> > > >> >
> > > >> > org.mortbay.jetty.HttpConnection$RequestHandler.headerComple
> > > >> te(HttpConnection.java:928)
> > > >> >
> > > >> > at org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
> > > >> >
> > > >> > at
> org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
> > > >> >
> > > >> > at
> org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
> > > >> >
> > > >> > at
> > > >> >
> > > >> > org.mortbay.io.nio.SelectChannelEndPoint.run(SelectChannelEn
> > > >> dPoint.java:410)
> > > >> >
> > > >> > at
> > > >> >
> > > >> > org.mortbay.thread.QueuedThreadPool$PoolThread.run(
> > > >> QueuedThreadPool.java:582)
> > > >> >
> > > >> > Caused by: org.apache.hadoop.hbase.ipc.RpcClient$
> > CallTimeoutException:
> > > >> Call
> > > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > > >> >
> > > >> > at
> > > >> >
> > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.cleanupCall
> > > >> s(RpcClient.java:1234)
> > > >> >
> > > >> > at
> > > >> >
> > > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.readRespons
> > > >> e(RpcClient.java:1171)
> > > >> >
> > > >> > at org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
> > > >> nt.java:751)
> > > >> > Beside this same issue, please note that first message was at
> > > 2015-03-20
> > > >> > 14:17:26,015. And then (we got to the point when it started
> > > transition):
> > > >> >
> > > >> > 2015-03-20 14:32:35,059 INFO
> > > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > > >> > /hbase/splitWAL/WALs%2F
> > > >> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > > >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.14268
> > 60404905
> > > >> > entered state: DONE {fqdn_of_new_live_node},60020,1426859445623
> > > >> >
> > > >> > 2015-03-20 14:32:35,109 INFO
> > > >> > org.apache.hadoop.hbase.master.SplitLogManager: Done splitting
> > > >> > /hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}
> > > >> > %252C60020%252C1426860403261.1426860404905
> > > >> >
> > > >> > 2015-03-20 14:32:35,190 INFO
> > > >> > org.apache.hadoop.hbase.master.SplitLogManager: finished splitting
> > > >> (more
> > > >> > than or equal to) 9 bytes in 1 log files in
> > > >> >
> > > >> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_of_
> > > >> node_I_took_down},60020,1426860403261-splitting]
> > > >> > in 909083ms
> > > >> >
> > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > > >> .RegionStates:
> > > >> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c state=OPEN,
> > > >> > ts=1426860639088,
> > > server={name_of_node_I_took_down},60020,1426860403261}
> > > >> to
> > > >> > {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE, ts=1426861955191,
> > > >> server=
> > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > >> >
> > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > > >> .RegionStates:
> > > >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
> > > >> {name_of_node_I_took_down}
> > > >> > ,60020,1426860403261
> > > >> >
> > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > > >> .RegionStates:
> > > >> > Transitioned {25ab6e7b42e36ddaa723d71be5954543 state=OPEN,
> > > >> > ts=1426860641783,
> > > server={name_of_node_I_took_down},60020,1426860403261}
> > > >> to
> > > >> > {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE, ts=1426861955191,
> > > >> server=
> > > >> > {name_of_node_I_took_down},60020,1426860403261}
> > > >> >
> > > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > > >> .RegionStates:
> > > >> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
> > > >> {name_of_node_I_took_down}
> > > >> > ,60020,1426860403261
> > > >> > At this point, note that it finished SplitLogManager task at
> > 14:32:35
> > > >> and
> > > >> > started transitioning just after that. So this is 15 minutes that
> > I'm
> > > >> > talking about.
> > > >> >
> > > >> > What am I missing?
> > > >> >
> > > >> >
> > > >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <
> nkeywal@gmail.com>
> > > >> wrote:
> > > >> >
> > > >> > > You've changed the value of hbase.zookeeper.timeout to 15
> > minutes? A
> > > >> very
> > > >> > > reasonable target is 1 minute before relocating the regions.
> > That's
> > > >> the
> > > >> > > default iirc. You can push it to 20s, but then
> > gc-stopping-the-world
> > > >> > > becomes more of an issue. 15 minutes is really a lot. The hdfs
> > stale
> > > >> mode
> > > >> > > must always be used, with a lower timeout than the hbase one.
> > > >> > >
> > > >> > > Client side there should be nothing to do (excepted for advanced
> > > >> stuff);
> > > >> > at
> > > >> > > each retry the client checks the location of the regions. If you
> > > lower
> > > >> > the
> > > >> > > number of retry the client will fail sooner, but usually you
> don't
> > > >> want
> > > >> > the
> > > >> > > client to fail, you want the servers to reallocate quickly.
> > > >> > >
> > > >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <
> > > dejan.menges@gmail.com
> > > >> >
> > > >> > > wrote:
> > > >> > >
> > > >> > > > Hi,
> > > >> > > >
> > > >> > > > Sorry for little bit late update, but managed to narrow it
> > little
> > > >> bit
> > > >> > > down.
> > > >> > > >
> > > >> > > > We didn't update yet, as we are using Hortonworks distribution
> > > right
> > > >> > now,
> > > >> > > > and even if we update we will get 0.98.4. However, looks that
> > > issue
> > > >> > here
> > > >> > > > was in our use case and configuration (still looking into it).
> > > >> > > >
> > > >> > > > Basically, initially I saw that when one server goes down, we
> > > start
> > > >> > > having
> > > >> > > > performance issues in general, but it managed to be on our
> > client
> > > >> side,
> > > >> > > due
> > > >> > > > to caching, and clients were trying to reconnect to nodes that
> > > were
> > > >> > > offline
> > > >> > > > and later trying to get regions from those nodes too. This is
> > > >> basically
> > > >> > > why
> > > >> > > > on server side I didn't manage to see anything in logs that
> > would
> > > >> be at
> > > >> > > > least little bit interesting and point me into desired
> > direction.
> > > >> > > >
> > > >> > > > Another question that popped up to me is - in case server is
> > down
> > > >> (and
> > > >> > > with
> > > >> > > > it DataNode and HRegionServer it was hosting) - what's optimal
> > > time
> > > >> to
> > > >> > > set
> > > >> > > > for HMaster to consider server dead reassign regions somewhere
> > > >> else, as
> > > >> > > > this is another performance bottleneck we hit during inability
> > to
> > > >> > access
> > > >> > > > regions? In our case it's configured to 15 minutes, and simple
> > > logic
> > > >> > > tells
> > > >> > > > me if you want it earlier then configure lower number of
> > retries,
> > > >> but
> > > >> > > issue
> > > >> > > > is as always in details, so not sure if anyone knows some
> better
> > > >> math
> > > >> > for
> > > >> > > > this?
> > > >> > > >
> > > >> > > > And last question - is it possible to manually force HBase to
> > > >> reassign
> > > >> > > > regions? In this case, while HMaster is retrying to contact
> node
> > > >> that's
> > > >> > > > dead, it's impossible to force it using 'balancer' command.
> > > >> > > >
> > > >> > > > Thanks a lot!
> > > >> > > >
> > > >> > > > Dejan
> > > >> > > >
> > > >> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <
> > > >> dejan.menges@gmail.com>
> > > >> > > > wrote:
> > > >> > > >
> > > >> > > > > Hi,
> > > >> > > > >
> > > >> > > > > To be very honest - there's no particular reason why we
> stick
> > to
> > > >> this
> > > >> > > > one,
> > > >> > > > > beside just lack of time currently to go through upgrade
> > > process,
> > > >> but
> > > >> > > > looks
> > > >> > > > > to me that's going to be next step.
> > > >> > > > >
> > > >> > > > > Had a crazy day, didn't have time to go through all logs
> > again,
> > > >> plus
> > > >> > > one
> > > >> > > > > of the machines (last one where we had this issue) is fully
> > > >> > > reprovisioned
> > > >> > > > > yesterday so I don't have logs from there anymore.
> > > >> > > > >
> > > >> > > > > Beside upgrading,  what I will talk about today, can you
> just
> > > >> point
> > > >> > me
> > > >> > > to
> > > >> > > > > the specific RPC issue in 0.98.0? Thing is that we have some
> > > >> strange
> > > >> > > > > moments with RPC in this case, and just want to see if
> that's
> > > the
> > > >> > same
> > > >> > > > > thing (and we were even suspecting to RPC).
> > > >> > > > >
> > > >> > > > > Thanks a lot!
> > > >> > > > > Dejan
> > > >> > > > >
> > > >> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew Purtell <
> > > >> apurtell@apache.org
> > > >> > >
> > > >> > > > > wrote:
> > > >> > > > >
> > > >> > > > >> Is there a particular reason why you are using HBase
> 0.98.0?
> > > The
> > > >> > > latest
> > > >> > > > >> 0.98 release is 0.98.11. There's a known performance issue
> > with
> > > >> > 0.98.0
> > > >> > > > >> pertaining to RPC that was fixed in later releases, you
> > should
> > > >> move
> > > >> > up
> > > >> > > > >> from
> > > >> > > > >> 0.98.0. In addition hundreds of improvements and bug fixes
> > have
> > > >> gone
> > > >> > > > into
> > > >> > > > >> the ten releases since 0.98.0.
> > > >> > > > >>
> > > >> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <
> > > >> > dejan.menges@gmail.com
> > > >> > > >
> > > >> > > > >> wrote:
> > > >> > > > >>
> > > >> > > > >> > Hi All,
> > > >> > > > >> >
> > > >> > > > >> > We have a strange issue with HBase performance (overall
> > > cluster
> > > >> > > > >> > performance) in case one of datanodes in the cluster
> > > >> unexpectedly
> > > >> > > goes
> > > >> > > > >> > down.
> > > >> > > > >> >
> > > >> > > > >> > So scenario is like follows:
> > > >> > > > >> > - Cluster works fine, it's stable.
> > > >> > > > >> > - One DataNode unexpectedly goes down (PSU issue, network
> > > >> issue,
> > > >> > > > >> anything)
> > > >> > > > >> > - Whole HBase cluster goes down (performance becomes so
> bad
> > > >> that
> > > >> > we
> > > >> > > > >> have to
> > > >> > > > >> > restart all RegionServers to get it back to life).
> > > >> > > > >> >
> > > >> > > > >> > Most funny and latest issue that happened was that we
> added
> > > new
> > > >> > node
> > > >> > > > to
> > > >> > > > >> the
> > > >> > > > >> > cluster (having 8 x 4T SATA disks) and we left just
> > DataNode
> > > >> > running
> > > >> > > > on
> > > >> > > > >> it
> > > >> > > > >> > to give it couple of days to get some data. At some point
> > in
> > > >> time,
> > > >> > > due
> > > >> > > > >> to
> > > >> > > > >> > hardware issue, server rebooted (twice during three
> hours)
> > in
> > > >> > moment
> > > >> > > > >> when
> > > >> > > > >> > it had maybe 5% of data it would have in a couple of
> days.
> > > >> Nothing
> > > >> > > > else
> > > >> > > > >> > beside DataNode was running, and once it went down, it
> > > affected
> > > >> > > > literary
> > > >> > > > >> > everything, and restarting RegionServers in the end fixed
> > it.
> > > >> > > > >> >
> > > >> > > > >> > We are using HBase 0.98.0 with Hadoop 2.4.0
> > > >> > > > >> >
> > > >> > > > >>
> > > >> > > > >>
> > > >> > > > >>
> > > >> > > > >> --
> > > >> > > > >> Best regards,
> > > >> > > > >>
> > > >> > > > >>    - Andy
> > > >> > > > >>
> > > >> > > > >> Problems worthy of attack prove their worth by hitting
> back.
> > -
> > > >> Piet
> > > >> > > Hein
> > > >> > > > >> (via Tom White)
> > > >> > > > >>
> > > >> > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > >
> >
>

Re: Strange issue when DataNode goes down

Posted by Dejan Menges <de...@gmail.com>.
I found the issue and fixed it, and will try to explain here what was
exactly in our case in case someone else finds this interesting too.

So initially, we had (couple of times) some network and hardware issues in
our datacenters. When one server would die (literary die, we had some issue
with PSUs) we saw issues with overall cluster performance on HBase side. As
cluster is quite big and live, it was also quite hard to figure out exact
root cause and how to fix the stuff we wanted to fix.

So I set up another cluster, four nodes (with DataNode and RegionServer)
and two other nodes with HMaster and Namenode in HA, using same stuff we
use on production. We pumped some data into it, and I was able to reproduce
same issue last week on it. What I tried to do is to cut one server (shut
down it's interface) when all is good with cluster, when we have load, and
see what's going to happen.

On Friday, after Nicolas mentioned, I started taking a look in HBase logs
on the node which was mentioned in HMaster log as the one taking over
regions for the dead server. Basically what I was able to observe was 15
minutes time (+- couple of seconds, what was also interesting, and will got
later to that) between HMaster figures out that one of it's RegionServers
is dead, and the time one of the mentioned nodes starts taking over those
regions and they start appearing in HMaster's Web UI.

I then set up everything like mentioned here
http://hbase.apache.org/book.html#mttr - but still had exactly the same
issues. Went over (again and again) all currently configured stuff, but
still had the same issue.

Then I started looking into HDFS. Opened NameNode UI, saw all is good, took
one node down, was also looking RegionServer logs in the same time - and I
also see that it took ~15 minutes for Namenode to elect dead node as dead.
Somehow in the same moment regions started getting back to life. I remember
in some older versions dfs timeout checks and check retries. Looked into
defaults for our Hadoop version -
http://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/hadoop-hdfs/hdfs-default.xml
- and saw there that there's no recheck parameter anymore. Strange, as on
StackOverflow I found post from month ago, for newer version than we use
(we use 2.4.1, guy was using 2.6 - dfs.namenode.heartbeat.recheck-interval)
I set it to 10 seconds as he mentioned, having checks every three seconds
(default) and got DataNode elected as dead in ~45 seconds, as he mentioned.
Not sure why this parameter is not documented, but obviously it works.

Then figured out it still didn't fix our HBase failover issue. I was
looking into HBase book again and again, and then saw this part:

"How much time we allow elapse between calls to recover lease. Should be
larger than the dfs timeout."

This was description for hbase.lease.recovery.dfs.timeout. Wasn't sure from
the comment what of all timeouts that's possible to set in Hadoop/HBase and
that has something to do with DFS is this all about. But picked
hbase.lease.recovery.timeout, and that was the catch.

Initially, by default, hbase.lease.recovery.timeout is set to 15 minutes.
Not sure why, and wasn't able to find yet why, but getting this down to one
minute (what's more than OK for us) I was able to get rid of our issue. Not
also sure why this is not mentioned in MTTR section in HBase book, as
obviously MTTR doesn't work at all with this default timeout, at least it
doesn't work the way we expected it to work.

So thanks again for everyone being spammed with this, and specially thanks
to Nicolas pointing me to the right direction.

On Mon, Mar 23, 2015 at 1:37 PM Nicolas Liochon <nk...@gmail.com> wrote:

> the attachments are rejected by the mailing list, can you put then on
> pastebin?
>
> stale is mandatory (so it's good), but the issue here is just before. The
> region server needs to read the file. In order to be sure that there is no
> data loss, it needs to "recover the lease", that means ensuring that nobody
> is writing the file. The regionserver calls the namenode to do this
> recoverLease. So there should be some info in the namenode logs. You have
> HDFS-4721 on your hdfs? The hbase book details (more or less...) this
> recoverLease stuff.
>
>
> On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <de...@gmail.com>
> wrote:
>
> > And also, just checked - dfs.namenode.avoid.read.stale.datanode and
> > dfs.namenode.avoid.write.stale.datanode
> > are both true, and dfs.namenode.stale.datanode.interval is set to
> default
> > 30000.
> >
> > On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <de...@gmail.com>
> > wrote:
> >
> > > Hi Nicolas,
> > >
> > > Please find log attached.
> > >
> > > As I see it now more clearly, it was trying to recover HDFS WALs from
> > node
> > > that's dead:
> > >
> > > 2015-03-23 08:53:44,381 WARN org.apache.hadoop.hbase.util.FSHDFSUtils:
> > > Cannot recoverLease after trying for 900000ms
> > > (hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!;
> > > attempt=40 on
> > >
> > file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intent
> ionally_get_down_by_getting_network_down},60020,1426862900
> 506-splitting/{node_i_intentionally_get_down_by_gett
> ing_network_down}%2C60020%2C1426862900506.1427096924508
> > > after 908210ms
> > >
> > > And as you can see from the log, it tried 40 times, what took it
> exactly
> > > 15 minutes.
> > >
> > > There's probably some parameter to tune to cut it of from 40 times / 15
> > > minutes to something more useful, as for 15 minutes we don't have our
> > > regions available, and HDFS have however replication factor 3.
> > >
> > > Googling, if I figure out what's this I will post it here. Will also
> > > appreciate if someone knows how to cut this down.
> > >
> > > Thanks,
> > >
> > > Dejan
> > >
> > > On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <nk...@gmail.com>
> > wrote:
> > >
> > >> The split is done by the region servers (the master coordinates). Is
> > there
> > >> some interesting stuff in their logs?
> > >>
> > >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <dejan.menges@gmail.com
> >
> > >> wrote:
> > >>
> > >> > With client issue was that it was retrying connecting to the same
> > region
> > >> > servers even when they were reassigned. Lowering it down helped in
> > this
> > >> > specific use case, but yes, we still want servers to reallocate
> > quickly.
> > >> >
> > >> > What got me here:
> > >> >
> > >> > http://hbase.apache.org/book.html#mttr
> > >> >
> > >> > I basically set configuration exactly the same way as it's explained
> > >> here.
> > >> > *zookeeper.session.timeout* is (and was before) 60000 (one minute).
> > >> >
> > >> > So basically what happens is: - simulating network issues we had
> > >> recently).
> > >> > - After short time I see in HBase that my RegionServer is dead, and
> as
> > >> > total number of regions I see previous total minus number of regions
> > >> that
> > >> > were hosted on the node hosting RegionServer that just
> 'disappeared'.
> > >> > - In this point I want my clus
> > >> >
> > >> > - I have test cluster consisting of four nodes, every node being
> > >> DataNode
> > >> > and RegionServer.
> > >> > - I simulate network partition on one (connect to it through console
> > and
> > >> > take network interface downter to recover as soon as possible, to
> > start
> > >> > serving missing regions.
> > >> > - First thing I see in HMaster logs are:
> > >> >
> > >> > 2015-03-20 14:17:26,015 INFO
> > >> > org.apache.hadoop.hbase.zookeeper.RegionServerTracker: RegionServer
> > >> > ephemeral node deleted, processing expiration
> > >> > [{name_of_node_I_took_down},60020,1426860403261]
> > >> >
> > >> > 2015-03-20 14:17:26,067 INFO
> > >> > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler:
> > Splitting
> > >> > logs for {name_of_node_I_took_down},60020,1426860403261 before
> > >> assignment.
> > >> >
> > >> > 2015-03-20 14:17:26,105 INFO
> > >> > org.apache.hadoop.hbase.master.SplitLogManager: dead splitlog
> workers
> > [
> > >> > {name_of_node_I_took_down},60020,1426860403261]
> > >> >
> > >> > 2015-03-20 14:17:26,107 INFO
> > >> > org.apache.hadoop.hbase.master.SplitLogManager: started splitting 1
> > >> logs in
> > >> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_I_
> took_down}
> > >> > ,60020,1426860403261-splitting]
> > >> >
> > >> > 2015-03-20 14:17:26,150 INFO
> > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > >> > /hbase/splitWAL/WALs%2F
> > >> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.14268
> 60404905
> > >> > acquired by {fqdn_of_another_live_hnode},60020,1426859445623
> > >> >
> > >> > 2015-03-20 14:17:26,182 INFO
> > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> > >> unassigned
> > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > >> >
> > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> in_progress
> > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error =
> > 0}
> > >> >
> > >> > 2015-03-20 14:17:31,183 INFO
> > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> > >> unassigned
> > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > >> >
> > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> in_progress
> > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error =
> > 0}
> > >> >
> > >> > 2015-03-20 14:17:36,184 INFO
> > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> > >> unassigned
> > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > >> >
> > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> in_progress
> > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error =
> > 0}
> > >> >
> > >> > 2015-03-20 14:17:42,185 INFO
> > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> > >> unassigned
> > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > >> >
> > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> in_progress
> > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error =
> > 0}
> > >> >
> > >> > 2015-03-20 14:17:48,184 INFO
> > >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> > >> unassigned
> > >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > >> >
> > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> > >> down}%252C60020%252C1426860403261.1426860404905=last_update
> > >> > = 1426861046182 last_version = 2 cur_worker_name =
> > >> > {fqdn_of_another_live_node},60020,1426859445623 status =
> in_progress
> > >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error =
> > 0}
> > >> > In the meantime, In hbase...out log I got this:
> > >> >
> > >> > ==> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out <==
> > >> >
> > >> > java.io.IOException: Call to
> > >> > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_dow
> n}:60020
> > >> > failed on local exception:
> > >> > org.apache.hadoop.hbase.ipc.RpcClient$CallTimeoutException: Call
> > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > >> >
> > >> > at org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
> > >> t.java:1532)
> > >> >
> > >> > at org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:1502)
> > >> >
> > >> > at
> > >> >
> > >> > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(Rpc
> > >> Client.java:1684)
> > >> >
> > >> > at
> > >> >
> > >> > org.apache.hadoop.hbase.ipc.RpcClient$BlockingRpcChannelImpl
> ementati
> > >> on.callBlockingMethod(RpcClient.java:1737)
> > >> >
> > >> > at
> > >> >
> > >> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$
> AdminService$
> > >> BlockingStub.getRegionInfo(AdminProtos.java:20806)
> > >> >
> > >> > at
> > >> >
> > >> > org.apache.hadoop.hbase.client.HBaseAdmin.getCompactionState
> > >> (HBaseAdmin.java:2524)
> > >> >
> > >> > at
> > >> >
> > >> > org.apache.hadoop.hbase.generated.master.table_jsp._jspServi
> > >> ce(table_jsp.java:167)
> > >> >
> > >> > at org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.
> java:98)
> > >> >
> > >> > at javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
> > >> >
> > >> > at org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
> > >> .java:511)
> > >> >
> > >> > at
> > >> >
> > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > >> r(ServletHandler.java:1221)
> > >> >
> > >> > at
> > >> >
> > >> > org.apache.hadoop.http.lib.StaticUserWebFilter$StaticUserFil
> > >> ter.doFilter(StaticUserWebFilter.java:109)
> > >> >
> > >> > at
> > >> >
> > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > >> r(ServletHandler.java:1212)
> > >> >
> > >> > at
> > >> >
> > >> > org.apache.hadoop.http.HttpServer$QuotingInputFilter.doFilte
> > >> r(HttpServer.java:1081)
> > >> >
> > >> > at
> > >> >
> > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > >> r(ServletHandler.java:1212)
> > >> >
> > >> > at
> > org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
> > >> >
> > >> > at
> > >> >
> > >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> > >> r(ServletHandler.java:1212)
> > >> >
> > >> > at org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
> > >> er.java:399)
> > >> >
> > >> > at
> > >> > org.mortbay.jetty.security.SecurityHandler.handle(SecurityHa
> > >> ndler.java:216)
> > >> >
> > >> > at org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
> > >> er.java:182)
> > >> >
> > >> > at org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
> > >> er.java:766)
> > >> >
> > >> > at org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
> > >> java:450)
> > >> >
> > >> > at
> > >> >
> > >> > org.mortbay.jetty.handler.ContextHandlerCollection.handle(Co
> > >> ntextHandlerCollection.java:230)
> > >> >
> > >> > at org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
> > >> er.java:152)
> > >> >
> > >> > at org.mortbay.jetty.Server.handle(Server.java:326)
> > >> >
> > >> > at org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
> > >> n.java:542)
> > >> >
> > >> > at
> > >> >
> > >> > org.mortbay.jetty.HttpConnection$RequestHandler.headerComple
> > >> te(HttpConnection.java:928)
> > >> >
> > >> > at org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
> > >> >
> > >> > at org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
> > >> >
> > >> > at org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
> > >> >
> > >> > at
> > >> >
> > >> > org.mortbay.io.nio.SelectChannelEndPoint.run(SelectChannelEn
> > >> dPoint.java:410)
> > >> >
> > >> > at
> > >> >
> > >> > org.mortbay.thread.QueuedThreadPool$PoolThread.run(
> > >> QueuedThreadPool.java:582)
> > >> >
> > >> > Caused by: org.apache.hadoop.hbase.ipc.RpcClient$
> CallTimeoutException:
> > >> Call
> > >> > id=93152, waitTime=60044, rpcTimeout=60000
> > >> >
> > >> > at
> > >> >
> > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.cleanupCall
> > >> s(RpcClient.java:1234)
> > >> >
> > >> > at
> > >> >
> > >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.readRespons
> > >> e(RpcClient.java:1171)
> > >> >
> > >> > at org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
> > >> nt.java:751)
> > >> > Beside this same issue, please note that first message was at
> > 2015-03-20
> > >> > 14:17:26,015. And then (we got to the point when it started
> > transition):
> > >> >
> > >> > 2015-03-20 14:32:35,059 INFO
> > >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > >> > /hbase/splitWAL/WALs%2F
> > >> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.14268
> 60404905
> > >> > entered state: DONE {fqdn_of_new_live_node},60020,1426859445623
> > >> >
> > >> > 2015-03-20 14:32:35,109 INFO
> > >> > org.apache.hadoop.hbase.master.SplitLogManager: Done splitting
> > >> > /hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}
> > >> > %252C60020%252C1426860403261.1426860404905
> > >> >
> > >> > 2015-03-20 14:32:35,190 INFO
> > >> > org.apache.hadoop.hbase.master.SplitLogManager: finished splitting
> > >> (more
> > >> > than or equal to) 9 bytes in 1 log files in
> > >> >
> > >> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_of_
> > >> node_I_took_down},60020,1426860403261-splitting]
> > >> > in 909083ms
> > >> >
> > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > >> .RegionStates:
> > >> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c state=OPEN,
> > >> > ts=1426860639088,
> > server={name_of_node_I_took_down},60020,1426860403261}
> > >> to
> > >> > {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE, ts=1426861955191,
> > >> server=
> > >> > {name_of_node_I_took_down},60020,1426860403261}
> > >> >
> > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > >> .RegionStates:
> > >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
> > >> {name_of_node_I_took_down}
> > >> > ,60020,1426860403261
> > >> >
> > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > >> .RegionStates:
> > >> > Transitioned {25ab6e7b42e36ddaa723d71be5954543 state=OPEN,
> > >> > ts=1426860641783,
> > server={name_of_node_I_took_down},60020,1426860403261}
> > >> to
> > >> > {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE, ts=1426861955191,
> > >> server=
> > >> > {name_of_node_I_took_down},60020,1426860403261}
> > >> >
> > >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> > >> .RegionStates:
> > >> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
> > >> {name_of_node_I_took_down}
> > >> > ,60020,1426860403261
> > >> > At this point, note that it finished SplitLogManager task at
> 14:32:35
> > >> and
> > >> > started transitioning just after that. So this is 15 minutes that
> I'm
> > >> > talking about.
> > >> >
> > >> > What am I missing?
> > >> >
> > >> >
> > >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <nk...@gmail.com>
> > >> wrote:
> > >> >
> > >> > > You've changed the value of hbase.zookeeper.timeout to 15
> minutes? A
> > >> very
> > >> > > reasonable target is 1 minute before relocating the regions.
> That's
> > >> the
> > >> > > default iirc. You can push it to 20s, but then
> gc-stopping-the-world
> > >> > > becomes more of an issue. 15 minutes is really a lot. The hdfs
> stale
> > >> mode
> > >> > > must always be used, with a lower timeout than the hbase one.
> > >> > >
> > >> > > Client side there should be nothing to do (excepted for advanced
> > >> stuff);
> > >> > at
> > >> > > each retry the client checks the location of the regions. If you
> > lower
> > >> > the
> > >> > > number of retry the client will fail sooner, but usually you don't
> > >> want
> > >> > the
> > >> > > client to fail, you want the servers to reallocate quickly.
> > >> > >
> > >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <
> > dejan.menges@gmail.com
> > >> >
> > >> > > wrote:
> > >> > >
> > >> > > > Hi,
> > >> > > >
> > >> > > > Sorry for little bit late update, but managed to narrow it
> little
> > >> bit
> > >> > > down.
> > >> > > >
> > >> > > > We didn't update yet, as we are using Hortonworks distribution
> > right
> > >> > now,
> > >> > > > and even if we update we will get 0.98.4. However, looks that
> > issue
> > >> > here
> > >> > > > was in our use case and configuration (still looking into it).
> > >> > > >
> > >> > > > Basically, initially I saw that when one server goes down, we
> > start
> > >> > > having
> > >> > > > performance issues in general, but it managed to be on our
> client
> > >> side,
> > >> > > due
> > >> > > > to caching, and clients were trying to reconnect to nodes that
> > were
> > >> > > offline
> > >> > > > and later trying to get regions from those nodes too. This is
> > >> basically
> > >> > > why
> > >> > > > on server side I didn't manage to see anything in logs that
> would
> > >> be at
> > >> > > > least little bit interesting and point me into desired
> direction.
> > >> > > >
> > >> > > > Another question that popped up to me is - in case server is
> down
> > >> (and
> > >> > > with
> > >> > > > it DataNode and HRegionServer it was hosting) - what's optimal
> > time
> > >> to
> > >> > > set
> > >> > > > for HMaster to consider server dead reassign regions somewhere
> > >> else, as
> > >> > > > this is another performance bottleneck we hit during inability
> to
> > >> > access
> > >> > > > regions? In our case it's configured to 15 minutes, and simple
> > logic
> > >> > > tells
> > >> > > > me if you want it earlier then configure lower number of
> retries,
> > >> but
> > >> > > issue
> > >> > > > is as always in details, so not sure if anyone knows some better
> > >> math
> > >> > for
> > >> > > > this?
> > >> > > >
> > >> > > > And last question - is it possible to manually force HBase to
> > >> reassign
> > >> > > > regions? In this case, while HMaster is retrying to contact node
> > >> that's
> > >> > > > dead, it's impossible to force it using 'balancer' command.
> > >> > > >
> > >> > > > Thanks a lot!
> > >> > > >
> > >> > > > Dejan
> > >> > > >
> > >> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <
> > >> dejan.menges@gmail.com>
> > >> > > > wrote:
> > >> > > >
> > >> > > > > Hi,
> > >> > > > >
> > >> > > > > To be very honest - there's no particular reason why we stick
> to
> > >> this
> > >> > > > one,
> > >> > > > > beside just lack of time currently to go through upgrade
> > process,
> > >> but
> > >> > > > looks
> > >> > > > > to me that's going to be next step.
> > >> > > > >
> > >> > > > > Had a crazy day, didn't have time to go through all logs
> again,
> > >> plus
> > >> > > one
> > >> > > > > of the machines (last one where we had this issue) is fully
> > >> > > reprovisioned
> > >> > > > > yesterday so I don't have logs from there anymore.
> > >> > > > >
> > >> > > > > Beside upgrading,  what I will talk about today, can you just
> > >> point
> > >> > me
> > >> > > to
> > >> > > > > the specific RPC issue in 0.98.0? Thing is that we have some
> > >> strange
> > >> > > > > moments with RPC in this case, and just want to see if that's
> > the
> > >> > same
> > >> > > > > thing (and we were even suspecting to RPC).
> > >> > > > >
> > >> > > > > Thanks a lot!
> > >> > > > > Dejan
> > >> > > > >
> > >> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew Purtell <
> > >> apurtell@apache.org
> > >> > >
> > >> > > > > wrote:
> > >> > > > >
> > >> > > > >> Is there a particular reason why you are using HBase 0.98.0?
> > The
> > >> > > latest
> > >> > > > >> 0.98 release is 0.98.11. There's a known performance issue
> with
> > >> > 0.98.0
> > >> > > > >> pertaining to RPC that was fixed in later releases, you
> should
> > >> move
> > >> > up
> > >> > > > >> from
> > >> > > > >> 0.98.0. In addition hundreds of improvements and bug fixes
> have
> > >> gone
> > >> > > > into
> > >> > > > >> the ten releases since 0.98.0.
> > >> > > > >>
> > >> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <
> > >> > dejan.menges@gmail.com
> > >> > > >
> > >> > > > >> wrote:
> > >> > > > >>
> > >> > > > >> > Hi All,
> > >> > > > >> >
> > >> > > > >> > We have a strange issue with HBase performance (overall
> > cluster
> > >> > > > >> > performance) in case one of datanodes in the cluster
> > >> unexpectedly
> > >> > > goes
> > >> > > > >> > down.
> > >> > > > >> >
> > >> > > > >> > So scenario is like follows:
> > >> > > > >> > - Cluster works fine, it's stable.
> > >> > > > >> > - One DataNode unexpectedly goes down (PSU issue, network
> > >> issue,
> > >> > > > >> anything)
> > >> > > > >> > - Whole HBase cluster goes down (performance becomes so bad
> > >> that
> > >> > we
> > >> > > > >> have to
> > >> > > > >> > restart all RegionServers to get it back to life).
> > >> > > > >> >
> > >> > > > >> > Most funny and latest issue that happened was that we added
> > new
> > >> > node
> > >> > > > to
> > >> > > > >> the
> > >> > > > >> > cluster (having 8 x 4T SATA disks) and we left just
> DataNode
> > >> > running
> > >> > > > on
> > >> > > > >> it
> > >> > > > >> > to give it couple of days to get some data. At some point
> in
> > >> time,
> > >> > > due
> > >> > > > >> to
> > >> > > > >> > hardware issue, server rebooted (twice during three hours)
> in
> > >> > moment
> > >> > > > >> when
> > >> > > > >> > it had maybe 5% of data it would have in a couple of days.
> > >> Nothing
> > >> > > > else
> > >> > > > >> > beside DataNode was running, and once it went down, it
> > affected
> > >> > > > literary
> > >> > > > >> > everything, and restarting RegionServers in the end fixed
> it.
> > >> > > > >> >
> > >> > > > >> > We are using HBase 0.98.0 with Hadoop 2.4.0
> > >> > > > >> >
> > >> > > > >>
> > >> > > > >>
> > >> > > > >>
> > >> > > > >> --
> > >> > > > >> Best regards,
> > >> > > > >>
> > >> > > > >>    - Andy
> > >> > > > >>
> > >> > > > >> Problems worthy of attack prove their worth by hitting back.
> -
> > >> Piet
> > >> > > Hein
> > >> > > > >> (via Tom White)
> > >> > > > >>
> > >> > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> >
>

Re: Strange issue when DataNode goes down

Posted by Nicolas Liochon <nk...@gmail.com>.
the attachments are rejected by the mailing list, can you put then on
pastebin?

stale is mandatory (so it's good), but the issue here is just before. The
region server needs to read the file. In order to be sure that there is no
data loss, it needs to "recover the lease", that means ensuring that nobody
is writing the file. The regionserver calls the namenode to do this
recoverLease. So there should be some info in the namenode logs. You have
HDFS-4721 on your hdfs? The hbase book details (more or less...) this
recoverLease stuff.


On Mon, Mar 23, 2015 at 10:33 AM, Dejan Menges <de...@gmail.com>
wrote:

> And also, just checked - dfs.namenode.avoid.read.stale.datanode and
> dfs.namenode.avoid.write.stale.datanode
> are both true, and dfs.namenode.stale.datanode.interval is set to default
> 30000.
>
> On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <de...@gmail.com>
> wrote:
>
> > Hi Nicolas,
> >
> > Please find log attached.
> >
> > As I see it now more clearly, it was trying to recover HDFS WALs from
> node
> > that's dead:
> >
> > 2015-03-23 08:53:44,381 WARN org.apache.hadoop.hbase.util.FSHDFSUtils:
> > Cannot recoverLease after trying for 900000ms
> > (hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!;
> > attempt=40 on
> >
> file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intentionally_get_down_by_getting_network_down},60020,1426862900506-splitting/{node_i_intentionally_get_down_by_getting_network_down}%2C60020%2C1426862900506.1427096924508
> > after 908210ms
> >
> > And as you can see from the log, it tried 40 times, what took it exactly
> > 15 minutes.
> >
> > There's probably some parameter to tune to cut it of from 40 times / 15
> > minutes to something more useful, as for 15 minutes we don't have our
> > regions available, and HDFS have however replication factor 3.
> >
> > Googling, if I figure out what's this I will post it here. Will also
> > appreciate if someone knows how to cut this down.
> >
> > Thanks,
> >
> > Dejan
> >
> > On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <nk...@gmail.com>
> wrote:
> >
> >> The split is done by the region servers (the master coordinates). Is
> there
> >> some interesting stuff in their logs?
> >>
> >> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <de...@gmail.com>
> >> wrote:
> >>
> >> > With client issue was that it was retrying connecting to the same
> region
> >> > servers even when they were reassigned. Lowering it down helped in
> this
> >> > specific use case, but yes, we still want servers to reallocate
> quickly.
> >> >
> >> > What got me here:
> >> >
> >> > http://hbase.apache.org/book.html#mttr
> >> >
> >> > I basically set configuration exactly the same way as it's explained
> >> here.
> >> > *zookeeper.session.timeout* is (and was before) 60000 (one minute).
> >> >
> >> > So basically what happens is: - simulating network issues we had
> >> recently).
> >> > - After short time I see in HBase that my RegionServer is dead, and as
> >> > total number of regions I see previous total minus number of regions
> >> that
> >> > were hosted on the node hosting RegionServer that just 'disappeared'.
> >> > - In this point I want my clus
> >> >
> >> > - I have test cluster consisting of four nodes, every node being
> >> DataNode
> >> > and RegionServer.
> >> > - I simulate network partition on one (connect to it through console
> and
> >> > take network interface downter to recover as soon as possible, to
> start
> >> > serving missing regions.
> >> > - First thing I see in HMaster logs are:
> >> >
> >> > 2015-03-20 14:17:26,015 INFO
> >> > org.apache.hadoop.hbase.zookeeper.RegionServerTracker: RegionServer
> >> > ephemeral node deleted, processing expiration
> >> > [{name_of_node_I_took_down},60020,1426860403261]
> >> >
> >> > 2015-03-20 14:17:26,067 INFO
> >> > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler:
> Splitting
> >> > logs for {name_of_node_I_took_down},60020,1426860403261 before
> >> assignment.
> >> >
> >> > 2015-03-20 14:17:26,105 INFO
> >> > org.apache.hadoop.hbase.master.SplitLogManager: dead splitlog workers
> [
> >> > {name_of_node_I_took_down},60020,1426860403261]
> >> >
> >> > 2015-03-20 14:17:26,107 INFO
> >> > org.apache.hadoop.hbase.master.SplitLogManager: started splitting 1
> >> logs in
> >> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_I_took_down}
> >> > ,60020,1426860403261-splitting]
> >> >
> >> > 2015-03-20 14:17:26,150 INFO
> >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> >> > /hbase/splitWAL/WALs%2F
> >> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905
> >> > acquired by {fqdn_of_another_live_hnode},60020,1426859445623
> >> >
> >> > 2015-03-20 14:17:26,182 INFO
> >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> >> unassigned
> >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> >> >
> >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> >> down}%252C60020%252C1426860403261.1426860404905=last_update
> >> > = 1426861046182 last_version = 2 cur_worker_name =
> >> > {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
> >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error =
> 0}
> >> >
> >> > 2015-03-20 14:17:31,183 INFO
> >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> >> unassigned
> >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> >> >
> >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> >> down}%252C60020%252C1426860403261.1426860404905=last_update
> >> > = 1426861046182 last_version = 2 cur_worker_name =
> >> > {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
> >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error =
> 0}
> >> >
> >> > 2015-03-20 14:17:36,184 INFO
> >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> >> unassigned
> >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> >> >
> >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> >> down}%252C60020%252C1426860403261.1426860404905=last_update
> >> > = 1426861046182 last_version = 2 cur_worker_name =
> >> > {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
> >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error =
> 0}
> >> >
> >> > 2015-03-20 14:17:42,185 INFO
> >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> >> unassigned
> >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> >> >
> >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> >> down}%252C60020%252C1426860403261.1426860404905=last_update
> >> > = 1426861046182 last_version = 2 cur_worker_name =
> >> > {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
> >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error =
> 0}
> >> >
> >> > 2015-03-20 14:17:48,184 INFO
> >> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> >> unassigned
> >> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> >> >
> >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
> >> down}%252C60020%252C1426860403261.1426860404905=last_update
> >> > = 1426861046182 last_version = 2 cur_worker_name =
> >> > {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
> >> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error =
> 0}
> >> > In the meantime, In hbase...out log I got this:
> >> >
> >> > ==> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out <==
> >> >
> >> > java.io.IOException: Call to
> >> > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_down}:60020
> >> > failed on local exception:
> >> > org.apache.hadoop.hbase.ipc.RpcClient$CallTimeoutException: Call
> >> > id=93152, waitTime=60044, rpcTimeout=60000
> >> >
> >> > at org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
> >> t.java:1532)
> >> >
> >> > at org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:1502)
> >> >
> >> > at
> >> >
> >> > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(Rpc
> >> Client.java:1684)
> >> >
> >> > at
> >> >
> >> > org.apache.hadoop.hbase.ipc.RpcClient$BlockingRpcChannelImplementati
> >> on.callBlockingMethod(RpcClient.java:1737)
> >> >
> >> > at
> >> >
> >> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$AdminService$
> >> BlockingStub.getRegionInfo(AdminProtos.java:20806)
> >> >
> >> > at
> >> >
> >> > org.apache.hadoop.hbase.client.HBaseAdmin.getCompactionState
> >> (HBaseAdmin.java:2524)
> >> >
> >> > at
> >> >
> >> > org.apache.hadoop.hbase.generated.master.table_jsp._jspServi
> >> ce(table_jsp.java:167)
> >> >
> >> > at org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.java:98)
> >> >
> >> > at javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
> >> >
> >> > at org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
> >> .java:511)
> >> >
> >> > at
> >> >
> >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> >> r(ServletHandler.java:1221)
> >> >
> >> > at
> >> >
> >> > org.apache.hadoop.http.lib.StaticUserWebFilter$StaticUserFil
> >> ter.doFilter(StaticUserWebFilter.java:109)
> >> >
> >> > at
> >> >
> >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> >> r(ServletHandler.java:1212)
> >> >
> >> > at
> >> >
> >> > org.apache.hadoop.http.HttpServer$QuotingInputFilter.doFilte
> >> r(HttpServer.java:1081)
> >> >
> >> > at
> >> >
> >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> >> r(ServletHandler.java:1212)
> >> >
> >> > at
> org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
> >> >
> >> > at
> >> >
> >> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
> >> r(ServletHandler.java:1212)
> >> >
> >> > at org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
> >> er.java:399)
> >> >
> >> > at
> >> > org.mortbay.jetty.security.SecurityHandler.handle(SecurityHa
> >> ndler.java:216)
> >> >
> >> > at org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
> >> er.java:182)
> >> >
> >> > at org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
> >> er.java:766)
> >> >
> >> > at org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
> >> java:450)
> >> >
> >> > at
> >> >
> >> > org.mortbay.jetty.handler.ContextHandlerCollection.handle(Co
> >> ntextHandlerCollection.java:230)
> >> >
> >> > at org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
> >> er.java:152)
> >> >
> >> > at org.mortbay.jetty.Server.handle(Server.java:326)
> >> >
> >> > at org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
> >> n.java:542)
> >> >
> >> > at
> >> >
> >> > org.mortbay.jetty.HttpConnection$RequestHandler.headerComple
> >> te(HttpConnection.java:928)
> >> >
> >> > at org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
> >> >
> >> > at org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
> >> >
> >> > at org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
> >> >
> >> > at
> >> >
> >> > org.mortbay.io.nio.SelectChannelEndPoint.run(SelectChannelEn
> >> dPoint.java:410)
> >> >
> >> > at
> >> >
> >> > org.mortbay.thread.QueuedThreadPool$PoolThread.run(
> >> QueuedThreadPool.java:582)
> >> >
> >> > Caused by: org.apache.hadoop.hbase.ipc.RpcClient$CallTimeoutException:
> >> Call
> >> > id=93152, waitTime=60044, rpcTimeout=60000
> >> >
> >> > at
> >> >
> >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.cleanupCall
> >> s(RpcClient.java:1234)
> >> >
> >> > at
> >> >
> >> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.readRespons
> >> e(RpcClient.java:1171)
> >> >
> >> > at org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
> >> nt.java:751)
> >> > Beside this same issue, please note that first message was at
> 2015-03-20
> >> > 14:17:26,015. And then (we got to the point when it started
> transition):
> >> >
> >> > 2015-03-20 14:32:35,059 INFO
> >> > org.apache.hadoop.hbase.master.SplitLogManager: task
> >> > /hbase/splitWAL/WALs%2F
> >> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> >> > {name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905
> >> > entered state: DONE {fqdn_of_new_live_node},60020,1426859445623
> >> >
> >> > 2015-03-20 14:32:35,109 INFO
> >> > org.apache.hadoop.hbase.master.SplitLogManager: Done splitting
> >> > /hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> >> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}
> >> > %252C60020%252C1426860403261.1426860404905
> >> >
> >> > 2015-03-20 14:32:35,190 INFO
> >> > org.apache.hadoop.hbase.master.SplitLogManager: finished splitting
> >> (more
> >> > than or equal to) 9 bytes in 1 log files in
> >> >
> >> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_of_
> >> node_I_took_down},60020,1426860403261-splitting]
> >> > in 909083ms
> >> >
> >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> >> .RegionStates:
> >> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c state=OPEN,
> >> > ts=1426860639088,
> server={name_of_node_I_took_down},60020,1426860403261}
> >> to
> >> > {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE, ts=1426861955191,
> >> server=
> >> > {name_of_node_I_took_down},60020,1426860403261}
> >> >
> >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> >> .RegionStates:
> >> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
> >> {name_of_node_I_took_down}
> >> > ,60020,1426860403261
> >> >
> >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> >> .RegionStates:
> >> > Transitioned {25ab6e7b42e36ddaa723d71be5954543 state=OPEN,
> >> > ts=1426860641783,
> server={name_of_node_I_took_down},60020,1426860403261}
> >> to
> >> > {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE, ts=1426861955191,
> >> server=
> >> > {name_of_node_I_took_down},60020,1426860403261}
> >> >
> >> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
> >> .RegionStates:
> >> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
> >> {name_of_node_I_took_down}
> >> > ,60020,1426860403261
> >> > At this point, note that it finished SplitLogManager task at 14:32:35
> >> and
> >> > started transitioning just after that. So this is 15 minutes that I'm
> >> > talking about.
> >> >
> >> > What am I missing?
> >> >
> >> >
> >> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <nk...@gmail.com>
> >> wrote:
> >> >
> >> > > You've changed the value of hbase.zookeeper.timeout to 15 minutes? A
> >> very
> >> > > reasonable target is 1 minute before relocating the regions. That's
> >> the
> >> > > default iirc. You can push it to 20s, but then gc-stopping-the-world
> >> > > becomes more of an issue. 15 minutes is really a lot. The hdfs stale
> >> mode
> >> > > must always be used, with a lower timeout than the hbase one.
> >> > >
> >> > > Client side there should be nothing to do (excepted for advanced
> >> stuff);
> >> > at
> >> > > each retry the client checks the location of the regions. If you
> lower
> >> > the
> >> > > number of retry the client will fail sooner, but usually you don't
> >> want
> >> > the
> >> > > client to fail, you want the servers to reallocate quickly.
> >> > >
> >> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <
> dejan.menges@gmail.com
> >> >
> >> > > wrote:
> >> > >
> >> > > > Hi,
> >> > > >
> >> > > > Sorry for little bit late update, but managed to narrow it little
> >> bit
> >> > > down.
> >> > > >
> >> > > > We didn't update yet, as we are using Hortonworks distribution
> right
> >> > now,
> >> > > > and even if we update we will get 0.98.4. However, looks that
> issue
> >> > here
> >> > > > was in our use case and configuration (still looking into it).
> >> > > >
> >> > > > Basically, initially I saw that when one server goes down, we
> start
> >> > > having
> >> > > > performance issues in general, but it managed to be on our client
> >> side,
> >> > > due
> >> > > > to caching, and clients were trying to reconnect to nodes that
> were
> >> > > offline
> >> > > > and later trying to get regions from those nodes too. This is
> >> basically
> >> > > why
> >> > > > on server side I didn't manage to see anything in logs that would
> >> be at
> >> > > > least little bit interesting and point me into desired direction.
> >> > > >
> >> > > > Another question that popped up to me is - in case server is down
> >> (and
> >> > > with
> >> > > > it DataNode and HRegionServer it was hosting) - what's optimal
> time
> >> to
> >> > > set
> >> > > > for HMaster to consider server dead reassign regions somewhere
> >> else, as
> >> > > > this is another performance bottleneck we hit during inability to
> >> > access
> >> > > > regions? In our case it's configured to 15 minutes, and simple
> logic
> >> > > tells
> >> > > > me if you want it earlier then configure lower number of retries,
> >> but
> >> > > issue
> >> > > > is as always in details, so not sure if anyone knows some better
> >> math
> >> > for
> >> > > > this?
> >> > > >
> >> > > > And last question - is it possible to manually force HBase to
> >> reassign
> >> > > > regions? In this case, while HMaster is retrying to contact node
> >> that's
> >> > > > dead, it's impossible to force it using 'balancer' command.
> >> > > >
> >> > > > Thanks a lot!
> >> > > >
> >> > > > Dejan
> >> > > >
> >> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <
> >> dejan.menges@gmail.com>
> >> > > > wrote:
> >> > > >
> >> > > > > Hi,
> >> > > > >
> >> > > > > To be very honest - there's no particular reason why we stick to
> >> this
> >> > > > one,
> >> > > > > beside just lack of time currently to go through upgrade
> process,
> >> but
> >> > > > looks
> >> > > > > to me that's going to be next step.
> >> > > > >
> >> > > > > Had a crazy day, didn't have time to go through all logs again,
> >> plus
> >> > > one
> >> > > > > of the machines (last one where we had this issue) is fully
> >> > > reprovisioned
> >> > > > > yesterday so I don't have logs from there anymore.
> >> > > > >
> >> > > > > Beside upgrading,  what I will talk about today, can you just
> >> point
> >> > me
> >> > > to
> >> > > > > the specific RPC issue in 0.98.0? Thing is that we have some
> >> strange
> >> > > > > moments with RPC in this case, and just want to see if that's
> the
> >> > same
> >> > > > > thing (and we were even suspecting to RPC).
> >> > > > >
> >> > > > > Thanks a lot!
> >> > > > > Dejan
> >> > > > >
> >> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew Purtell <
> >> apurtell@apache.org
> >> > >
> >> > > > > wrote:
> >> > > > >
> >> > > > >> Is there a particular reason why you are using HBase 0.98.0?
> The
> >> > > latest
> >> > > > >> 0.98 release is 0.98.11. There's a known performance issue with
> >> > 0.98.0
> >> > > > >> pertaining to RPC that was fixed in later releases, you should
> >> move
> >> > up
> >> > > > >> from
> >> > > > >> 0.98.0. In addition hundreds of improvements and bug fixes have
> >> gone
> >> > > > into
> >> > > > >> the ten releases since 0.98.0.
> >> > > > >>
> >> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <
> >> > dejan.menges@gmail.com
> >> > > >
> >> > > > >> wrote:
> >> > > > >>
> >> > > > >> > Hi All,
> >> > > > >> >
> >> > > > >> > We have a strange issue with HBase performance (overall
> cluster
> >> > > > >> > performance) in case one of datanodes in the cluster
> >> unexpectedly
> >> > > goes
> >> > > > >> > down.
> >> > > > >> >
> >> > > > >> > So scenario is like follows:
> >> > > > >> > - Cluster works fine, it's stable.
> >> > > > >> > - One DataNode unexpectedly goes down (PSU issue, network
> >> issue,
> >> > > > >> anything)
> >> > > > >> > - Whole HBase cluster goes down (performance becomes so bad
> >> that
> >> > we
> >> > > > >> have to
> >> > > > >> > restart all RegionServers to get it back to life).
> >> > > > >> >
> >> > > > >> > Most funny and latest issue that happened was that we added
> new
> >> > node
> >> > > > to
> >> > > > >> the
> >> > > > >> > cluster (having 8 x 4T SATA disks) and we left just DataNode
> >> > running
> >> > > > on
> >> > > > >> it
> >> > > > >> > to give it couple of days to get some data. At some point in
> >> time,
> >> > > due
> >> > > > >> to
> >> > > > >> > hardware issue, server rebooted (twice during three hours) in
> >> > moment
> >> > > > >> when
> >> > > > >> > it had maybe 5% of data it would have in a couple of days.
> >> Nothing
> >> > > > else
> >> > > > >> > beside DataNode was running, and once it went down, it
> affected
> >> > > > literary
> >> > > > >> > everything, and restarting RegionServers in the end fixed it.
> >> > > > >> >
> >> > > > >> > We are using HBase 0.98.0 with Hadoop 2.4.0
> >> > > > >> >
> >> > > > >>
> >> > > > >>
> >> > > > >>
> >> > > > >> --
> >> > > > >> Best regards,
> >> > > > >>
> >> > > > >>    - Andy
> >> > > > >>
> >> > > > >> Problems worthy of attack prove their worth by hitting back. -
> >> Piet
> >> > > Hein
> >> > > > >> (via Tom White)
> >> > > > >>
> >> > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
>

Re: Strange issue when DataNode goes down

Posted by Dejan Menges <de...@gmail.com>.
And also, just checked - dfs.namenode.avoid.read.stale.datanode and
dfs.namenode.avoid.write.stale.datanode
are both true, and dfs.namenode.stale.datanode.interval is set to default
30000.

On Mon, Mar 23, 2015 at 10:03 AM Dejan Menges <de...@gmail.com>
wrote:

> Hi Nicolas,
>
> Please find log attached.
>
> As I see it now more clearly, it was trying to recover HDFS WALs from node
> that's dead:
>
> 2015-03-23 08:53:44,381 WARN org.apache.hadoop.hbase.util.FSHDFSUtils:
> Cannot recoverLease after trying for 900000ms
> (hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!;
> attempt=40 on
> file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intentionally_get_down_by_getting_network_down},60020,1426862900506-splitting/{node_i_intentionally_get_down_by_getting_network_down}%2C60020%2C1426862900506.1427096924508
> after 908210ms
>
> And as you can see from the log, it tried 40 times, what took it exactly
> 15 minutes.
>
> There's probably some parameter to tune to cut it of from 40 times / 15
> minutes to something more useful, as for 15 minutes we don't have our
> regions available, and HDFS have however replication factor 3.
>
> Googling, if I figure out what's this I will post it here. Will also
> appreciate if someone knows how to cut this down.
>
> Thanks,
>
> Dejan
>
> On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <nk...@gmail.com> wrote:
>
>> The split is done by the region servers (the master coordinates). Is there
>> some interesting stuff in their logs?
>>
>> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <de...@gmail.com>
>> wrote:
>>
>> > With client issue was that it was retrying connecting to the same region
>> > servers even when they were reassigned. Lowering it down helped in this
>> > specific use case, but yes, we still want servers to reallocate quickly.
>> >
>> > What got me here:
>> >
>> > http://hbase.apache.org/book.html#mttr
>> >
>> > I basically set configuration exactly the same way as it's explained
>> here.
>> > *zookeeper.session.timeout* is (and was before) 60000 (one minute).
>> >
>> > So basically what happens is: - simulating network issues we had
>> recently).
>> > - After short time I see in HBase that my RegionServer is dead, and as
>> > total number of regions I see previous total minus number of regions
>> that
>> > were hosted on the node hosting RegionServer that just 'disappeared'.
>> > - In this point I want my clus
>> >
>> > - I have test cluster consisting of four nodes, every node being
>> DataNode
>> > and RegionServer.
>> > - I simulate network partition on one (connect to it through console and
>> > take network interface downter to recover as soon as possible, to start
>> > serving missing regions.
>> > - First thing I see in HMaster logs are:
>> >
>> > 2015-03-20 14:17:26,015 INFO
>> > org.apache.hadoop.hbase.zookeeper.RegionServerTracker: RegionServer
>> > ephemeral node deleted, processing expiration
>> > [{name_of_node_I_took_down},60020,1426860403261]
>> >
>> > 2015-03-20 14:17:26,067 INFO
>> > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler: Splitting
>> > logs for {name_of_node_I_took_down},60020,1426860403261 before
>> assignment.
>> >
>> > 2015-03-20 14:17:26,105 INFO
>> > org.apache.hadoop.hbase.master.SplitLogManager: dead splitlog workers [
>> > {name_of_node_I_took_down},60020,1426860403261]
>> >
>> > 2015-03-20 14:17:26,107 INFO
>> > org.apache.hadoop.hbase.master.SplitLogManager: started splitting 1
>> logs in
>> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_I_took_down}
>> > ,60020,1426860403261-splitting]
>> >
>> > 2015-03-20 14:17:26,150 INFO
>> > org.apache.hadoop.hbase.master.SplitLogManager: task
>> > /hbase/splitWAL/WALs%2F
>> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
>> > {name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905
>> > acquired by {fqdn_of_another_live_hnode},60020,1426859445623
>> >
>> > 2015-03-20 14:17:26,182 INFO
>> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
>> unassigned
>> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>> >
>> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
>> down}%252C60020%252C1426860403261.1426860404905=last_update
>> > = 1426861046182 last_version = 2 cur_worker_name =
>> > {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
>> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}
>> >
>> > 2015-03-20 14:17:31,183 INFO
>> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
>> unassigned
>> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>> >
>> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
>> down}%252C60020%252C1426860403261.1426860404905=last_update
>> > = 1426861046182 last_version = 2 cur_worker_name =
>> > {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
>> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}
>> >
>> > 2015-03-20 14:17:36,184 INFO
>> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
>> unassigned
>> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>> >
>> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
>> down}%252C60020%252C1426860403261.1426860404905=last_update
>> > = 1426861046182 last_version = 2 cur_worker_name =
>> > {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
>> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}
>> >
>> > 2015-03-20 14:17:42,185 INFO
>> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
>> unassigned
>> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>> >
>> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
>> down}%252C60020%252C1426860403261.1426860404905=last_update
>> > = 1426861046182 last_version = 2 cur_worker_name =
>> > {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
>> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}
>> >
>> > 2015-03-20 14:17:48,184 INFO
>> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
>> unassigned
>> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>> >
>> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_
>> down}%252C60020%252C1426860403261.1426860404905=last_update
>> > = 1426861046182 last_version = 2 cur_worker_name =
>> > {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
>> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}
>> > In the meantime, In hbase...out log I got this:
>> >
>> > ==> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out <==
>> >
>> > java.io.IOException: Call to
>> > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_down}:60020
>> > failed on local exception:
>> > org.apache.hadoop.hbase.ipc.RpcClient$CallTimeoutException: Call
>> > id=93152, waitTime=60044, rpcTimeout=60000
>> >
>> > at org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClien
>> t.java:1532)
>> >
>> > at org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:1502)
>> >
>> > at
>> >
>> > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(Rpc
>> Client.java:1684)
>> >
>> > at
>> >
>> > org.apache.hadoop.hbase.ipc.RpcClient$BlockingRpcChannelImplementati
>> on.callBlockingMethod(RpcClient.java:1737)
>> >
>> > at
>> >
>> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$AdminService$
>> BlockingStub.getRegionInfo(AdminProtos.java:20806)
>> >
>> > at
>> >
>> > org.apache.hadoop.hbase.client.HBaseAdmin.getCompactionState
>> (HBaseAdmin.java:2524)
>> >
>> > at
>> >
>> > org.apache.hadoop.hbase.generated.master.table_jsp._jspServi
>> ce(table_jsp.java:167)
>> >
>> > at org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.java:98)
>> >
>> > at javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
>> >
>> > at org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder
>> .java:511)
>> >
>> > at
>> >
>> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
>> r(ServletHandler.java:1221)
>> >
>> > at
>> >
>> > org.apache.hadoop.http.lib.StaticUserWebFilter$StaticUserFil
>> ter.doFilter(StaticUserWebFilter.java:109)
>> >
>> > at
>> >
>> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
>> r(ServletHandler.java:1212)
>> >
>> > at
>> >
>> > org.apache.hadoop.http.HttpServer$QuotingInputFilter.doFilte
>> r(HttpServer.java:1081)
>> >
>> > at
>> >
>> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
>> r(ServletHandler.java:1212)
>> >
>> > at org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
>> >
>> > at
>> >
>> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilte
>> r(ServletHandler.java:1212)
>> >
>> > at org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandl
>> er.java:399)
>> >
>> > at
>> > org.mortbay.jetty.security.SecurityHandler.handle(SecurityHa
>> ndler.java:216)
>> >
>> > at org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandl
>> er.java:182)
>> >
>> > at org.mortbay.jetty.handler.ContextHandler.handle(ContextHandl
>> er.java:766)
>> >
>> > at org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.
>> java:450)
>> >
>> > at
>> >
>> > org.mortbay.jetty.handler.ContextHandlerCollection.handle(Co
>> ntextHandlerCollection.java:230)
>> >
>> > at org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapp
>> er.java:152)
>> >
>> > at org.mortbay.jetty.Server.handle(Server.java:326)
>> >
>> > at org.mortbay.jetty.HttpConnection.handleRequest(HttpConnectio
>> n.java:542)
>> >
>> > at
>> >
>> > org.mortbay.jetty.HttpConnection$RequestHandler.headerComple
>> te(HttpConnection.java:928)
>> >
>> > at org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
>> >
>> > at org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
>> >
>> > at org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
>> >
>> > at
>> >
>> > org.mortbay.io.nio.SelectChannelEndPoint.run(SelectChannelEn
>> dPoint.java:410)
>> >
>> > at
>> >
>> > org.mortbay.thread.QueuedThreadPool$PoolThread.run(
>> QueuedThreadPool.java:582)
>> >
>> > Caused by: org.apache.hadoop.hbase.ipc.RpcClient$CallTimeoutException:
>> Call
>> > id=93152, waitTime=60044, rpcTimeout=60000
>> >
>> > at
>> >
>> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.cleanupCall
>> s(RpcClient.java:1234)
>> >
>> > at
>> >
>> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.readRespons
>> e(RpcClient.java:1171)
>> >
>> > at org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClie
>> nt.java:751)
>> > Beside this same issue, please note that first message was at 2015-03-20
>> > 14:17:26,015. And then (we got to the point when it started transition):
>> >
>> > 2015-03-20 14:32:35,059 INFO
>> > org.apache.hadoop.hbase.master.SplitLogManager: task
>> > /hbase/splitWAL/WALs%2F
>> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
>> > {name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905
>> > entered state: DONE {fqdn_of_new_live_node},60020,1426859445623
>> >
>> > 2015-03-20 14:32:35,109 INFO
>> > org.apache.hadoop.hbase.master.SplitLogManager: Done splitting
>> > /hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}
>> > %252C60020%252C1426860403261.1426860404905
>> >
>> > 2015-03-20 14:32:35,190 INFO
>> > org.apache.hadoop.hbase.master.SplitLogManager: finished splitting
>> (more
>> > than or equal to) 9 bytes in 1 log files in
>> >
>> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_of_
>> node_I_took_down},60020,1426860403261-splitting]
>> > in 909083ms
>> >
>> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
>> .RegionStates:
>> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c state=OPEN,
>> > ts=1426860639088, server={name_of_node_I_took_down},60020,1426860403261}
>> to
>> > {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE, ts=1426861955191,
>> server=
>> > {name_of_node_I_took_down},60020,1426860403261}
>> >
>> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
>> .RegionStates:
>> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
>> {name_of_node_I_took_down}
>> > ,60020,1426860403261
>> >
>> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
>> .RegionStates:
>> > Transitioned {25ab6e7b42e36ddaa723d71be5954543 state=OPEN,
>> > ts=1426860641783, server={name_of_node_I_took_down},60020,1426860403261}
>> to
>> > {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE, ts=1426861955191,
>> server=
>> > {name_of_node_I_took_down},60020,1426860403261}
>> >
>> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master
>> .RegionStates:
>> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
>> {name_of_node_I_took_down}
>> > ,60020,1426860403261
>> > At this point, note that it finished SplitLogManager task at 14:32:35
>> and
>> > started transitioning just after that. So this is 15 minutes that I'm
>> > talking about.
>> >
>> > What am I missing?
>> >
>> >
>> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <nk...@gmail.com>
>> wrote:
>> >
>> > > You've changed the value of hbase.zookeeper.timeout to 15 minutes? A
>> very
>> > > reasonable target is 1 minute before relocating the regions. That's
>> the
>> > > default iirc. You can push it to 20s, but then gc-stopping-the-world
>> > > becomes more of an issue. 15 minutes is really a lot. The hdfs stale
>> mode
>> > > must always be used, with a lower timeout than the hbase one.
>> > >
>> > > Client side there should be nothing to do (excepted for advanced
>> stuff);
>> > at
>> > > each retry the client checks the location of the regions. If you lower
>> > the
>> > > number of retry the client will fail sooner, but usually you don't
>> want
>> > the
>> > > client to fail, you want the servers to reallocate quickly.
>> > >
>> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <dejan.menges@gmail.com
>> >
>> > > wrote:
>> > >
>> > > > Hi,
>> > > >
>> > > > Sorry for little bit late update, but managed to narrow it little
>> bit
>> > > down.
>> > > >
>> > > > We didn't update yet, as we are using Hortonworks distribution right
>> > now,
>> > > > and even if we update we will get 0.98.4. However, looks that issue
>> > here
>> > > > was in our use case and configuration (still looking into it).
>> > > >
>> > > > Basically, initially I saw that when one server goes down, we start
>> > > having
>> > > > performance issues in general, but it managed to be on our client
>> side,
>> > > due
>> > > > to caching, and clients were trying to reconnect to nodes that were
>> > > offline
>> > > > and later trying to get regions from those nodes too. This is
>> basically
>> > > why
>> > > > on server side I didn't manage to see anything in logs that would
>> be at
>> > > > least little bit interesting and point me into desired direction.
>> > > >
>> > > > Another question that popped up to me is - in case server is down
>> (and
>> > > with
>> > > > it DataNode and HRegionServer it was hosting) - what's optimal time
>> to
>> > > set
>> > > > for HMaster to consider server dead reassign regions somewhere
>> else, as
>> > > > this is another performance bottleneck we hit during inability to
>> > access
>> > > > regions? In our case it's configured to 15 minutes, and simple logic
>> > > tells
>> > > > me if you want it earlier then configure lower number of retries,
>> but
>> > > issue
>> > > > is as always in details, so not sure if anyone knows some better
>> math
>> > for
>> > > > this?
>> > > >
>> > > > And last question - is it possible to manually force HBase to
>> reassign
>> > > > regions? In this case, while HMaster is retrying to contact node
>> that's
>> > > > dead, it's impossible to force it using 'balancer' command.
>> > > >
>> > > > Thanks a lot!
>> > > >
>> > > > Dejan
>> > > >
>> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <
>> dejan.menges@gmail.com>
>> > > > wrote:
>> > > >
>> > > > > Hi,
>> > > > >
>> > > > > To be very honest - there's no particular reason why we stick to
>> this
>> > > > one,
>> > > > > beside just lack of time currently to go through upgrade process,
>> but
>> > > > looks
>> > > > > to me that's going to be next step.
>> > > > >
>> > > > > Had a crazy day, didn't have time to go through all logs again,
>> plus
>> > > one
>> > > > > of the machines (last one where we had this issue) is fully
>> > > reprovisioned
>> > > > > yesterday so I don't have logs from there anymore.
>> > > > >
>> > > > > Beside upgrading,  what I will talk about today, can you just
>> point
>> > me
>> > > to
>> > > > > the specific RPC issue in 0.98.0? Thing is that we have some
>> strange
>> > > > > moments with RPC in this case, and just want to see if that's the
>> > same
>> > > > > thing (and we were even suspecting to RPC).
>> > > > >
>> > > > > Thanks a lot!
>> > > > > Dejan
>> > > > >
>> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew Purtell <
>> apurtell@apache.org
>> > >
>> > > > > wrote:
>> > > > >
>> > > > >> Is there a particular reason why you are using HBase 0.98.0? The
>> > > latest
>> > > > >> 0.98 release is 0.98.11. There's a known performance issue with
>> > 0.98.0
>> > > > >> pertaining to RPC that was fixed in later releases, you should
>> move
>> > up
>> > > > >> from
>> > > > >> 0.98.0. In addition hundreds of improvements and bug fixes have
>> gone
>> > > > into
>> > > > >> the ten releases since 0.98.0.
>> > > > >>
>> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <
>> > dejan.menges@gmail.com
>> > > >
>> > > > >> wrote:
>> > > > >>
>> > > > >> > Hi All,
>> > > > >> >
>> > > > >> > We have a strange issue with HBase performance (overall cluster
>> > > > >> > performance) in case one of datanodes in the cluster
>> unexpectedly
>> > > goes
>> > > > >> > down.
>> > > > >> >
>> > > > >> > So scenario is like follows:
>> > > > >> > - Cluster works fine, it's stable.
>> > > > >> > - One DataNode unexpectedly goes down (PSU issue, network
>> issue,
>> > > > >> anything)
>> > > > >> > - Whole HBase cluster goes down (performance becomes so bad
>> that
>> > we
>> > > > >> have to
>> > > > >> > restart all RegionServers to get it back to life).
>> > > > >> >
>> > > > >> > Most funny and latest issue that happened was that we added new
>> > node
>> > > > to
>> > > > >> the
>> > > > >> > cluster (having 8 x 4T SATA disks) and we left just DataNode
>> > running
>> > > > on
>> > > > >> it
>> > > > >> > to give it couple of days to get some data. At some point in
>> time,
>> > > due
>> > > > >> to
>> > > > >> > hardware issue, server rebooted (twice during three hours) in
>> > moment
>> > > > >> when
>> > > > >> > it had maybe 5% of data it would have in a couple of days.
>> Nothing
>> > > > else
>> > > > >> > beside DataNode was running, and once it went down, it affected
>> > > > literary
>> > > > >> > everything, and restarting RegionServers in the end fixed it.
>> > > > >> >
>> > > > >> > We are using HBase 0.98.0 with Hadoop 2.4.0
>> > > > >> >
>> > > > >>
>> > > > >>
>> > > > >>
>> > > > >> --
>> > > > >> Best regards,
>> > > > >>
>> > > > >>    - Andy
>> > > > >>
>> > > > >> Problems worthy of attack prove their worth by hitting back. -
>> Piet
>> > > Hein
>> > > > >> (via Tom White)
>> > > > >>
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>

Re: Strange issue when DataNode goes down

Posted by Dejan Menges <de...@gmail.com>.
Hi Nicolas,

Please find log attached.

As I see it now more clearly, it was trying to recover HDFS WALs from node
that's dead:

2015-03-23 08:53:44,381 WARN org.apache.hadoop.hbase.util.FSHDFSUtils:
Cannot recoverLease after trying for 900000ms
(hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!;
attempt=40 on
file=hdfs://{my_hmaster_node}:8020/hbase/WALs/{node_i_intentionally_get_down_by_getting_network_down},60020,1426862900506-splitting/{node_i_intentionally_get_down_by_getting_network_down}%2C60020%2C1426862900506.1427096924508
after 908210ms

And as you can see from the log, it tried 40 times, what took it exactly 15
minutes.

There's probably some parameter to tune to cut it of from 40 times / 15
minutes to something more useful, as for 15 minutes we don't have our
regions available, and HDFS have however replication factor 3.

Googling, if I figure out what's this I will post it here. Will also
appreciate if someone knows how to cut this down.

Thanks,

Dejan

On Fri, Mar 20, 2015 at 3:49 PM Nicolas Liochon <nk...@gmail.com> wrote:

> The split is done by the region servers (the master coordinates). Is there
> some interesting stuff in their logs?
>
> On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <de...@gmail.com>
> wrote:
>
> > With client issue was that it was retrying connecting to the same region
> > servers even when they were reassigned. Lowering it down helped in this
> > specific use case, but yes, we still want servers to reallocate quickly.
> >
> > What got me here:
> >
> > http://hbase.apache.org/book.html#mttr
> >
> > I basically set configuration exactly the same way as it's explained
> here.
> > *zookeeper.session.timeout* is (and was before) 60000 (one minute).
> >
> > So basically what happens is: - simulating network issues we had
> recently).
> > - After short time I see in HBase that my RegionServer is dead, and as
> > total number of regions I see previous total minus number of regions that
> > were hosted on the node hosting RegionServer that just 'disappeared'.
> > - In this point I want my clus
> >
> > - I have test cluster consisting of four nodes, every node being DataNode
> > and RegionServer.
> > - I simulate network partition on one (connect to it through console and
> > take network interface downter to recover as soon as possible, to start
> > serving missing regions.
> > - First thing I see in HMaster logs are:
> >
> > 2015-03-20 14:17:26,015 INFO
> > org.apache.hadoop.hbase.zookeeper.RegionServerTracker: RegionServer
> > ephemeral node deleted, processing expiration
> > [{name_of_node_I_took_down},60020,1426860403261]
> >
> > 2015-03-20 14:17:26,067 INFO
> > org.apache.hadoop.hbase.master.handler.ServerShutdownHandler: Splitting
> > logs for {name_of_node_I_took_down},60020,1426860403261 before
> assignment.
> >
> > 2015-03-20 14:17:26,105 INFO
> > org.apache.hadoop.hbase.master.SplitLogManager: dead splitlog workers [
> > {name_of_node_I_took_down},60020,1426860403261]
> >
> > 2015-03-20 14:17:26,107 INFO
> > org.apache.hadoop.hbase.master.SplitLogManager: started splitting 1
> logs in
> > [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_I_took_down}
> > ,60020,1426860403261-splitting]
> >
> > 2015-03-20 14:17:26,150 INFO
> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > /hbase/splitWAL/WALs%2F
> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > {name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905
> > acquired by {fqdn_of_another_live_hnode},60020,1426859445623
> >
> > 2015-03-20 14:17:26,182 INFO
> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> unassigned
> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> >
> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> took_down}%252C60020%252C1426860403261.1426860404905=last_update
> > = 1426861046182 last_version = 2 cur_worker_name =
> > {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}
> >
> > 2015-03-20 14:17:31,183 INFO
> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> unassigned
> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> >
> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> took_down}%252C60020%252C1426860403261.1426860404905=last_update
> > = 1426861046182 last_version = 2 cur_worker_name =
> > {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}
> >
> > 2015-03-20 14:17:36,184 INFO
> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> unassigned
> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> >
> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> took_down}%252C60020%252C1426860403261.1426860404905=last_update
> > = 1426861046182 last_version = 2 cur_worker_name =
> > {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}
> >
> > 2015-03-20 14:17:42,185 INFO
> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> unassigned
> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> >
> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> took_down}%252C60020%252C1426860403261.1426860404905=last_update
> > = 1426861046182 last_version = 2 cur_worker_name =
> > {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}
> >
> > 2015-03-20 14:17:48,184 INFO
> > org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1
> unassigned
> > = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> >
> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_
> took_down}%252C60020%252C1426860403261.1426860404905=last_update
> > = 1426861046182 last_version = 2 cur_worker_name =
> > {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
> > incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}
> > In the meantime, In hbase...out log I got this:
> >
> > ==> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out <==
> >
> > java.io.IOException: Call to
> > {name_of_node_I_took_down}/{ip_of_local_interface_I_took_down}:60020
> > failed on local exception:
> > org.apache.hadoop.hbase.ipc.RpcClient$CallTimeoutException: Call
> > id=93152, waitTime=60044, rpcTimeout=60000
> >
> > at org.apache.hadoop.hbase.ipc.RpcClient.wrapException(
> RpcClient.java:1532)
> >
> > at org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:1502)
> >
> > at
> >
> > org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(
> RpcClient.java:1684)
> >
> > at
> >
> > org.apache.hadoop.hbase.ipc.RpcClient$BlockingRpcChannelImplementati
> on.callBlockingMethod(RpcClient.java:1737)
> >
> > at
> >
> > org.apache.hadoop.hbase.protobuf.generated.AdminProtos$AdminService$
> BlockingStub.getRegionInfo(AdminProtos.java:20806)
> >
> > at
> >
> > org.apache.hadoop.hbase.client.HBaseAdmin.getCompactionState(HBaseAdmin.
> java:2524)
> >
> > at
> >
> > org.apache.hadoop.hbase.generated.master.table_jsp._
> jspService(table_jsp.java:167)
> >
> > at org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.java:98)
> >
> > at javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
> >
> > at org.mortbay.jetty.servlet.ServletHolder.handle(
> ServletHolder.java:511)
> >
> > at
> >
> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> doFilter(ServletHandler.java:1221)
> >
> > at
> >
> > org.apache.hadoop.http.lib.StaticUserWebFilter$
> StaticUserFilter.doFilter(StaticUserWebFilter.java:109)
> >
> > at
> >
> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> doFilter(ServletHandler.java:1212)
> >
> > at
> >
> > org.apache.hadoop.http.HttpServer$QuotingInputFilter.
> doFilter(HttpServer.java:1081)
> >
> > at
> >
> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> doFilter(ServletHandler.java:1212)
> >
> > at org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
> >
> > at
> >
> > org.mortbay.jetty.servlet.ServletHandler$CachedChain.
> doFilter(ServletHandler.java:1212)
> >
> > at org.mortbay.jetty.servlet.ServletHandler.handle(
> ServletHandler.java:399)
> >
> > at
> > org.mortbay.jetty.security.SecurityHandler.handle(
> SecurityHandler.java:216)
> >
> > at org.mortbay.jetty.servlet.SessionHandler.handle(
> SessionHandler.java:182)
> >
> > at org.mortbay.jetty.handler.ContextHandler.handle(
> ContextHandler.java:766)
> >
> > at org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.java:450)
> >
> > at
> >
> > org.mortbay.jetty.handler.ContextHandlerCollection.handle(
> ContextHandlerCollection.java:230)
> >
> > at org.mortbay.jetty.handler.HandlerWrapper.handle(
> HandlerWrapper.java:152)
> >
> > at org.mortbay.jetty.Server.handle(Server.java:326)
> >
> > at org.mortbay.jetty.HttpConnection.handleRequest(
> HttpConnection.java:542)
> >
> > at
> >
> > org.mortbay.jetty.HttpConnection$RequestHandler.
> headerComplete(HttpConnection.java:928)
> >
> > at org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
> >
> > at org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
> >
> > at org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
> >
> > at
> >
> > org.mortbay.io.nio.SelectChannelEndPoint.run(SelectChannelEndPoint.java:
> 410)
> >
> > at
> >
> > org.mortbay.thread.QueuedThreadPool$PoolThread.
> run(QueuedThreadPool.java:582)
> >
> > Caused by: org.apache.hadoop.hbase.ipc.RpcClient$CallTimeoutException:
> Call
> > id=93152, waitTime=60044, rpcTimeout=60000
> >
> > at
> >
> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.
> cleanupCalls(RpcClient.java:1234)
> >
> > at
> >
> > org.apache.hadoop.hbase.ipc.RpcClient$Connection.
> readResponse(RpcClient.java:1171)
> >
> > at org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(
> RpcClient.java:751)
> > Beside this same issue, please note that first message was at 2015-03-20
> > 14:17:26,015. And then (we got to the point when it started transition):
> >
> > 2015-03-20 14:32:35,059 INFO
> > org.apache.hadoop.hbase.master.SplitLogManager: task
> > /hbase/splitWAL/WALs%2F
> > {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> > {name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905
> > entered state: DONE {fqdn_of_new_live_node},60020,1426859445623
> >
> > 2015-03-20 14:32:35,109 INFO
> > org.apache.hadoop.hbase.master.SplitLogManager: Done splitting
> > /hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> > %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}
> > %252C60020%252C1426860403261.1426860404905
> >
> > 2015-03-20 14:32:35,190 INFO
> > org.apache.hadoop.hbase.master.SplitLogManager: finished splitting (more
> > than or equal to) 9 bytes in 1 log files in
> >
> > [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_
> of_node_I_took_down},60020,1426860403261-splitting]
> > in 909083ms
> >
> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.
> master.RegionStates:
> > Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c state=OPEN,
> > ts=1426860639088, server={name_of_node_I_took_down},60020,1426860403261}
> to
> > {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE, ts=1426861955191,
> server=
> > {name_of_node_I_took_down},60020,1426860403261}
> >
> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.
> master.RegionStates:
> > Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from
> {name_of_node_I_took_down}
> > ,60020,1426860403261
> >
> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.
> master.RegionStates:
> > Transitioned {25ab6e7b42e36ddaa723d71be5954543 state=OPEN,
> > ts=1426860641783, server={name_of_node_I_took_down},60020,1426860403261}
> to
> > {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE, ts=1426861955191,
> server=
> > {name_of_node_I_took_down},60020,1426860403261}
> >
> > 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.
> master.RegionStates:
> > Offlined 25ab6e7b42e36ddaa723d71be5954543 from
> {name_of_node_I_took_down}
> > ,60020,1426860403261
> > At this point, note that it finished SplitLogManager task at 14:32:35 and
> > started transitioning just after that. So this is 15 minutes that I'm
> > talking about.
> >
> > What am I missing?
> >
> >
> > On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <nk...@gmail.com>
> wrote:
> >
> > > You've changed the value of hbase.zookeeper.timeout to 15 minutes? A
> very
> > > reasonable target is 1 minute before relocating the regions. That's the
> > > default iirc. You can push it to 20s, but then gc-stopping-the-world
> > > becomes more of an issue. 15 minutes is really a lot. The hdfs stale
> mode
> > > must always be used, with a lower timeout than the hbase one.
> > >
> > > Client side there should be nothing to do (excepted for advanced
> stuff);
> > at
> > > each retry the client checks the location of the regions. If you lower
> > the
> > > number of retry the client will fail sooner, but usually you don't want
> > the
> > > client to fail, you want the servers to reallocate quickly.
> > >
> > > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <de...@gmail.com>
> > > wrote:
> > >
> > > > Hi,
> > > >
> > > > Sorry for little bit late update, but managed to narrow it little bit
> > > down.
> > > >
> > > > We didn't update yet, as we are using Hortonworks distribution right
> > now,
> > > > and even if we update we will get 0.98.4. However, looks that issue
> > here
> > > > was in our use case and configuration (still looking into it).
> > > >
> > > > Basically, initially I saw that when one server goes down, we start
> > > having
> > > > performance issues in general, but it managed to be on our client
> side,
> > > due
> > > > to caching, and clients were trying to reconnect to nodes that were
> > > offline
> > > > and later trying to get regions from those nodes too. This is
> basically
> > > why
> > > > on server side I didn't manage to see anything in logs that would be
> at
> > > > least little bit interesting and point me into desired direction.
> > > >
> > > > Another question that popped up to me is - in case server is down
> (and
> > > with
> > > > it DataNode and HRegionServer it was hosting) - what's optimal time
> to
> > > set
> > > > for HMaster to consider server dead reassign regions somewhere else,
> as
> > > > this is another performance bottleneck we hit during inability to
> > access
> > > > regions? In our case it's configured to 15 minutes, and simple logic
> > > tells
> > > > me if you want it earlier then configure lower number of retries, but
> > > issue
> > > > is as always in details, so not sure if anyone knows some better math
> > for
> > > > this?
> > > >
> > > > And last question - is it possible to manually force HBase to
> reassign
> > > > regions? In this case, while HMaster is retrying to contact node
> that's
> > > > dead, it's impossible to force it using 'balancer' command.
> > > >
> > > > Thanks a lot!
> > > >
> > > > Dejan
> > > >
> > > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <dejan.menges@gmail.com
> >
> > > > wrote:
> > > >
> > > > > Hi,
> > > > >
> > > > > To be very honest - there's no particular reason why we stick to
> this
> > > > one,
> > > > > beside just lack of time currently to go through upgrade process,
> but
> > > > looks
> > > > > to me that's going to be next step.
> > > > >
> > > > > Had a crazy day, didn't have time to go through all logs again,
> plus
> > > one
> > > > > of the machines (last one where we had this issue) is fully
> > > reprovisioned
> > > > > yesterday so I don't have logs from there anymore.
> > > > >
> > > > > Beside upgrading,  what I will talk about today, can you just point
> > me
> > > to
> > > > > the specific RPC issue in 0.98.0? Thing is that we have some
> strange
> > > > > moments with RPC in this case, and just want to see if that's the
> > same
> > > > > thing (and we were even suspecting to RPC).
> > > > >
> > > > > Thanks a lot!
> > > > > Dejan
> > > > >
> > > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew Purtell <
> apurtell@apache.org
> > >
> > > > > wrote:
> > > > >
> > > > >> Is there a particular reason why you are using HBase 0.98.0? The
> > > latest
> > > > >> 0.98 release is 0.98.11. There's a known performance issue with
> > 0.98.0
> > > > >> pertaining to RPC that was fixed in later releases, you should
> move
> > up
> > > > >> from
> > > > >> 0.98.0. In addition hundreds of improvements and bug fixes have
> gone
> > > > into
> > > > >> the ten releases since 0.98.0.
> > > > >>
> > > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <
> > dejan.menges@gmail.com
> > > >
> > > > >> wrote:
> > > > >>
> > > > >> > Hi All,
> > > > >> >
> > > > >> > We have a strange issue with HBase performance (overall cluster
> > > > >> > performance) in case one of datanodes in the cluster
> unexpectedly
> > > goes
> > > > >> > down.
> > > > >> >
> > > > >> > So scenario is like follows:
> > > > >> > - Cluster works fine, it's stable.
> > > > >> > - One DataNode unexpectedly goes down (PSU issue, network issue,
> > > > >> anything)
> > > > >> > - Whole HBase cluster goes down (performance becomes so bad that
> > we
> > > > >> have to
> > > > >> > restart all RegionServers to get it back to life).
> > > > >> >
> > > > >> > Most funny and latest issue that happened was that we added new
> > node
> > > > to
> > > > >> the
> > > > >> > cluster (having 8 x 4T SATA disks) and we left just DataNode
> > running
> > > > on
> > > > >> it
> > > > >> > to give it couple of days to get some data. At some point in
> time,
> > > due
> > > > >> to
> > > > >> > hardware issue, server rebooted (twice during three hours) in
> > moment
> > > > >> when
> > > > >> > it had maybe 5% of data it would have in a couple of days.
> Nothing
> > > > else
> > > > >> > beside DataNode was running, and once it went down, it affected
> > > > literary
> > > > >> > everything, and restarting RegionServers in the end fixed it.
> > > > >> >
> > > > >> > We are using HBase 0.98.0 with Hadoop 2.4.0
> > > > >> >
> > > > >>
> > > > >>
> > > > >>
> > > > >> --
> > > > >> Best regards,
> > > > >>
> > > > >>    - Andy
> > > > >>
> > > > >> Problems worthy of attack prove their worth by hitting back. -
> Piet
> > > Hein
> > > > >> (via Tom White)
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: Strange issue when DataNode goes down

Posted by Nicolas Liochon <nk...@gmail.com>.
The split is done by the region servers (the master coordinates). Is there
some interesting stuff in their logs?

On Fri, Mar 20, 2015 at 3:38 PM, Dejan Menges <de...@gmail.com>
wrote:

> With client issue was that it was retrying connecting to the same region
> servers even when they were reassigned. Lowering it down helped in this
> specific use case, but yes, we still want servers to reallocate quickly.
>
> What got me here:
>
> http://hbase.apache.org/book.html#mttr
>
> I basically set configuration exactly the same way as it's explained here.
> *zookeeper.session.timeout* is (and was before) 60000 (one minute).
>
> So basically what happens is: - simulating network issues we had recently).
> - After short time I see in HBase that my RegionServer is dead, and as
> total number of regions I see previous total minus number of regions that
> were hosted on the node hosting RegionServer that just 'disappeared'.
> - In this point I want my clus
>
> - I have test cluster consisting of four nodes, every node being DataNode
> and RegionServer.
> - I simulate network partition on one (connect to it through console and
> take network interface downter to recover as soon as possible, to start
> serving missing regions.
> - First thing I see in HMaster logs are:
>
> 2015-03-20 14:17:26,015 INFO
> org.apache.hadoop.hbase.zookeeper.RegionServerTracker: RegionServer
> ephemeral node deleted, processing expiration
> [{name_of_node_I_took_down},60020,1426860403261]
>
> 2015-03-20 14:17:26,067 INFO
> org.apache.hadoop.hbase.master.handler.ServerShutdownHandler: Splitting
> logs for {name_of_node_I_took_down},60020,1426860403261 before assignment.
>
> 2015-03-20 14:17:26,105 INFO
> org.apache.hadoop.hbase.master.SplitLogManager: dead splitlog workers [
> {name_of_node_I_took_down},60020,1426860403261]
>
> 2015-03-20 14:17:26,107 INFO
> org.apache.hadoop.hbase.master.SplitLogManager: started splitting 1 logs in
> [hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_I_took_down}
> ,60020,1426860403261-splitting]
>
> 2015-03-20 14:17:26,150 INFO
> org.apache.hadoop.hbase.master.SplitLogManager: task
> /hbase/splitWAL/WALs%2F
> {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> {name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905
> acquired by {fqdn_of_another_live_hnode},60020,1426859445623
>
> 2015-03-20 14:17:26,182 INFO
> org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1 unassigned
> = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>
> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905=last_update
> = 1426861046182 last_version = 2 cur_worker_name =
> {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
> incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}
>
> 2015-03-20 14:17:31,183 INFO
> org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1 unassigned
> = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>
> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905=last_update
> = 1426861046182 last_version = 2 cur_worker_name =
> {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
> incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}
>
> 2015-03-20 14:17:36,184 INFO
> org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1 unassigned
> = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>
> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905=last_update
> = 1426861046182 last_version = 2 cur_worker_name =
> {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
> incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}
>
> 2015-03-20 14:17:42,185 INFO
> org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1 unassigned
> = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>
> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905=last_update
> = 1426861046182 last_version = 2 cur_worker_name =
> {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
> incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}
>
> 2015-03-20 14:17:48,184 INFO
> org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1 unassigned
> = 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
>
> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905=last_update
> = 1426861046182 last_version = 2 cur_worker_name =
> {fqdn_of_another_live_node},60020,1426859445623 status = in_progress
> incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}
> In the meantime, In hbase...out log I got this:
>
> ==> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out <==
>
> java.io.IOException: Call to
> {name_of_node_I_took_down}/{ip_of_local_interface_I_took_down}:60020
> failed on local exception:
> org.apache.hadoop.hbase.ipc.RpcClient$CallTimeoutException: Call
> id=93152, waitTime=60044, rpcTimeout=60000
>
> at org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClient.java:1532)
>
> at org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:1502)
>
> at
>
> org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(RpcClient.java:1684)
>
> at
>
> org.apache.hadoop.hbase.ipc.RpcClient$BlockingRpcChannelImplementation.callBlockingMethod(RpcClient.java:1737)
>
> at
>
> org.apache.hadoop.hbase.protobuf.generated.AdminProtos$AdminService$BlockingStub.getRegionInfo(AdminProtos.java:20806)
>
> at
>
> org.apache.hadoop.hbase.client.HBaseAdmin.getCompactionState(HBaseAdmin.java:2524)
>
> at
>
> org.apache.hadoop.hbase.generated.master.table_jsp._jspService(table_jsp.java:167)
>
> at org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.java:98)
>
> at javax.servlet.http.HttpServlet.service(HttpServlet.java:770)
>
> at org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder.java:511)
>
> at
>
> org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1221)
>
> at
>
> org.apache.hadoop.http.lib.StaticUserWebFilter$StaticUserFilter.doFilter(StaticUserWebFilter.java:109)
>
> at
>
> org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1212)
>
> at
>
> org.apache.hadoop.http.HttpServer$QuotingInputFilter.doFilter(HttpServer.java:1081)
>
> at
>
> org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1212)
>
> at org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
>
> at
>
> org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1212)
>
> at org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandler.java:399)
>
> at
> org.mortbay.jetty.security.SecurityHandler.handle(SecurityHandler.java:216)
>
> at org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandler.java:182)
>
> at org.mortbay.jetty.handler.ContextHandler.handle(ContextHandler.java:766)
>
> at org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.java:450)
>
> at
>
> org.mortbay.jetty.handler.ContextHandlerCollection.handle(ContextHandlerCollection.java:230)
>
> at org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapper.java:152)
>
> at org.mortbay.jetty.Server.handle(Server.java:326)
>
> at org.mortbay.jetty.HttpConnection.handleRequest(HttpConnection.java:542)
>
> at
>
> org.mortbay.jetty.HttpConnection$RequestHandler.headerComplete(HttpConnection.java:928)
>
> at org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)
>
> at org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)
>
> at org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)
>
> at
>
> org.mortbay.io.nio.SelectChannelEndPoint.run(SelectChannelEndPoint.java:410)
>
> at
>
> org.mortbay.thread.QueuedThreadPool$PoolThread.run(QueuedThreadPool.java:582)
>
> Caused by: org.apache.hadoop.hbase.ipc.RpcClient$CallTimeoutException: Call
> id=93152, waitTime=60044, rpcTimeout=60000
>
> at
>
> org.apache.hadoop.hbase.ipc.RpcClient$Connection.cleanupCalls(RpcClient.java:1234)
>
> at
>
> org.apache.hadoop.hbase.ipc.RpcClient$Connection.readResponse(RpcClient.java:1171)
>
> at org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClient.java:751)
> Beside this same issue, please note that first message was at 2015-03-20
> 14:17:26,015. And then (we got to the point when it started transition):
>
> 2015-03-20 14:32:35,059 INFO
> org.apache.hadoop.hbase.master.SplitLogManager: task
> /hbase/splitWAL/WALs%2F
> {name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
> {name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905
> entered state: DONE {fqdn_of_new_live_node},60020,1426859445623
>
> 2015-03-20 14:32:35,109 INFO
> org.apache.hadoop.hbase.master.SplitLogManager: Done splitting
> /hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
> %2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}
> %252C60020%252C1426860403261.1426860404905
>
> 2015-03-20 14:32:35,190 INFO
> org.apache.hadoop.hbase.master.SplitLogManager: finished splitting (more
> than or equal to) 9 bytes in 1 log files in
>
> [hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_of_node_I_took_down},60020,1426860403261-splitting]
> in 909083ms
>
> 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master.RegionStates:
> Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c state=OPEN,
> ts=1426860639088, server={name_of_node_I_took_down},60020,1426860403261} to
> {0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE, ts=1426861955191, server=
> {name_of_node_I_took_down},60020,1426860403261}
>
> 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master.RegionStates:
> Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from {name_of_node_I_took_down}
> ,60020,1426860403261
>
> 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master.RegionStates:
> Transitioned {25ab6e7b42e36ddaa723d71be5954543 state=OPEN,
> ts=1426860641783, server={name_of_node_I_took_down},60020,1426860403261} to
> {25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE, ts=1426861955191, server=
> {name_of_node_I_took_down},60020,1426860403261}
>
> 2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master.RegionStates:
> Offlined 25ab6e7b42e36ddaa723d71be5954543 from {name_of_node_I_took_down}
> ,60020,1426860403261
> At this point, note that it finished SplitLogManager task at 14:32:35 and
> started transitioning just after that. So this is 15 minutes that I'm
> talking about.
>
> What am I missing?
>
>
> On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <nk...@gmail.com> wrote:
>
> > You've changed the value of hbase.zookeeper.timeout to 15 minutes? A very
> > reasonable target is 1 minute before relocating the regions. That's the
> > default iirc. You can push it to 20s, but then gc-stopping-the-world
> > becomes more of an issue. 15 minutes is really a lot. The hdfs stale mode
> > must always be used, with a lower timeout than the hbase one.
> >
> > Client side there should be nothing to do (excepted for advanced stuff);
> at
> > each retry the client checks the location of the regions. If you lower
> the
> > number of retry the client will fail sooner, but usually you don't want
> the
> > client to fail, you want the servers to reallocate quickly.
> >
> > On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <de...@gmail.com>
> > wrote:
> >
> > > Hi,
> > >
> > > Sorry for little bit late update, but managed to narrow it little bit
> > down.
> > >
> > > We didn't update yet, as we are using Hortonworks distribution right
> now,
> > > and even if we update we will get 0.98.4. However, looks that issue
> here
> > > was in our use case and configuration (still looking into it).
> > >
> > > Basically, initially I saw that when one server goes down, we start
> > having
> > > performance issues in general, but it managed to be on our client side,
> > due
> > > to caching, and clients were trying to reconnect to nodes that were
> > offline
> > > and later trying to get regions from those nodes too. This is basically
> > why
> > > on server side I didn't manage to see anything in logs that would be at
> > > least little bit interesting and point me into desired direction.
> > >
> > > Another question that popped up to me is - in case server is down (and
> > with
> > > it DataNode and HRegionServer it was hosting) - what's optimal time to
> > set
> > > for HMaster to consider server dead reassign regions somewhere else, as
> > > this is another performance bottleneck we hit during inability to
> access
> > > regions? In our case it's configured to 15 minutes, and simple logic
> > tells
> > > me if you want it earlier then configure lower number of retries, but
> > issue
> > > is as always in details, so not sure if anyone knows some better math
> for
> > > this?
> > >
> > > And last question - is it possible to manually force HBase to reassign
> > > regions? In this case, while HMaster is retrying to contact node that's
> > > dead, it's impossible to force it using 'balancer' command.
> > >
> > > Thanks a lot!
> > >
> > > Dejan
> > >
> > > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <de...@gmail.com>
> > > wrote:
> > >
> > > > Hi,
> > > >
> > > > To be very honest - there's no particular reason why we stick to this
> > > one,
> > > > beside just lack of time currently to go through upgrade process, but
> > > looks
> > > > to me that's going to be next step.
> > > >
> > > > Had a crazy day, didn't have time to go through all logs again, plus
> > one
> > > > of the machines (last one where we had this issue) is fully
> > reprovisioned
> > > > yesterday so I don't have logs from there anymore.
> > > >
> > > > Beside upgrading,  what I will talk about today, can you just point
> me
> > to
> > > > the specific RPC issue in 0.98.0? Thing is that we have some strange
> > > > moments with RPC in this case, and just want to see if that's the
> same
> > > > thing (and we were even suspecting to RPC).
> > > >
> > > > Thanks a lot!
> > > > Dejan
> > > >
> > > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew Purtell <apurtell@apache.org
> >
> > > > wrote:
> > > >
> > > >> Is there a particular reason why you are using HBase 0.98.0? The
> > latest
> > > >> 0.98 release is 0.98.11. There's a known performance issue with
> 0.98.0
> > > >> pertaining to RPC that was fixed in later releases, you should move
> up
> > > >> from
> > > >> 0.98.0. In addition hundreds of improvements and bug fixes have gone
> > > into
> > > >> the ten releases since 0.98.0.
> > > >>
> > > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <
> dejan.menges@gmail.com
> > >
> > > >> wrote:
> > > >>
> > > >> > Hi All,
> > > >> >
> > > >> > We have a strange issue with HBase performance (overall cluster
> > > >> > performance) in case one of datanodes in the cluster unexpectedly
> > goes
> > > >> > down.
> > > >> >
> > > >> > So scenario is like follows:
> > > >> > - Cluster works fine, it's stable.
> > > >> > - One DataNode unexpectedly goes down (PSU issue, network issue,
> > > >> anything)
> > > >> > - Whole HBase cluster goes down (performance becomes so bad that
> we
> > > >> have to
> > > >> > restart all RegionServers to get it back to life).
> > > >> >
> > > >> > Most funny and latest issue that happened was that we added new
> node
> > > to
> > > >> the
> > > >> > cluster (having 8 x 4T SATA disks) and we left just DataNode
> running
> > > on
> > > >> it
> > > >> > to give it couple of days to get some data. At some point in time,
> > due
> > > >> to
> > > >> > hardware issue, server rebooted (twice during three hours) in
> moment
> > > >> when
> > > >> > it had maybe 5% of data it would have in a couple of days. Nothing
> > > else
> > > >> > beside DataNode was running, and once it went down, it affected
> > > literary
> > > >> > everything, and restarting RegionServers in the end fixed it.
> > > >> >
> > > >> > We are using HBase 0.98.0 with Hadoop 2.4.0
> > > >> >
> > > >>
> > > >>
> > > >>
> > > >> --
> > > >> Best regards,
> > > >>
> > > >>    - Andy
> > > >>
> > > >> Problems worthy of attack prove their worth by hitting back. - Piet
> > Hein
> > > >> (via Tom White)
> > > >>
> > > >
> > > >
> > >
> >
>

Re: Strange issue when DataNode goes down

Posted by Dejan Menges <de...@gmail.com>.
With client issue was that it was retrying connecting to the same region
servers even when they were reassigned. Lowering it down helped in this
specific use case, but yes, we still want servers to reallocate quickly.

What got me here:

http://hbase.apache.org/book.html#mttr

I basically set configuration exactly the same way as it's explained here.
*zookeeper.session.timeout* is (and was before) 60000 (one minute).

So basically what happens is:

- I have test cluster consisting of four nodes, every node being DataNode
and RegionServer.
- I simulate network partition on one (connect to it through console and
take network interface down - simulating network issues we had recently).
- After short time I see in HBase that my RegionServer is dead, and as
total number of regions I see previous total minus number of regions that
were hosted on the node hosting RegionServer that just 'disappeared'.
- In this point I want my cluster to recover as soon as possible, to start
serving missing regions.
- First thing I see in HMaster logs are:

2015-03-20 14:17:26,015 INFO
org.apache.hadoop.hbase.zookeeper.RegionServerTracker: RegionServer
ephemeral node deleted, processing expiration
[{name_of_node_I_took_down},60020,1426860403261]

2015-03-20 14:17:26,067 INFO
org.apache.hadoop.hbase.master.handler.ServerShutdownHandler: Splitting
logs for {name_of_node_I_took_down},60020,1426860403261 before assignment.

2015-03-20 14:17:26,105 INFO
org.apache.hadoop.hbase.master.SplitLogManager: dead splitlog workers [
{name_of_node_I_took_down},60020,1426860403261]

2015-03-20 14:17:26,107 INFO
org.apache.hadoop.hbase.master.SplitLogManager: started splitting 1 logs in
[hdfs://{fqdn_of_hmaster}:8020/hbase/WALs/{name_of_node_I_took_down}
,60020,1426860403261-splitting]

2015-03-20 14:17:26,150 INFO
org.apache.hadoop.hbase.master.SplitLogManager: task /hbase/splitWAL/WALs%2F
{name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
{name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905
acquired by {fqdn_of_another_live_hnode},60020,1426859445623

2015-03-20 14:17:26,182 INFO
org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1 unassigned
= 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
%2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905=last_update
= 1426861046182 last_version = 2 cur_worker_name =
{fqdn_of_another_live_node},60020,1426859445623 status = in_progress
incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}

2015-03-20 14:17:31,183 INFO
org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1 unassigned
= 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
%2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905=last_update
= 1426861046182 last_version = 2 cur_worker_name =
{fqdn_of_another_live_node},60020,1426859445623 status = in_progress
incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}

2015-03-20 14:17:36,184 INFO
org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1 unassigned
= 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
%2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905=last_update
= 1426861046182 last_version = 2 cur_worker_name =
{fqdn_of_another_live_node},60020,1426859445623 status = in_progress
incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}

2015-03-20 14:17:42,185 INFO
org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1 unassigned
= 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
%2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905=last_update
= 1426861046182 last_version = 2 cur_worker_name =
{fqdn_of_another_live_node},60020,1426859445623 status = in_progress
incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}

2015-03-20 14:17:48,184 INFO
org.apache.hadoop.hbase.master.SplitLogManager: total tasks = 1 unassigned
= 0 tasks={/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
%2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905=last_update
= 1426861046182 last_version = 2 cur_worker_name =
{fqdn_of_another_live_node},60020,1426859445623 status = in_progress
incarnation = 0 resubmits = 0 batch = installed = 1 done = 0 error = 0}
In the meantime, In hbase...out log I got this:

==> hbase-hbase-master-{fqdn_of_my_hmaster_node}.out <==

java.io.IOException: Call to
{name_of_node_I_took_down}/{ip_of_local_interface_I_took_down}:60020
failed on local exception:
org.apache.hadoop.hbase.ipc.RpcClient$CallTimeoutException: Call
id=93152, waitTime=60044, rpcTimeout=60000

at org.apache.hadoop.hbase.ipc.RpcClient.wrapException(RpcClient.java:1532)

at org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:1502)

at
org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(RpcClient.java:1684)

at
org.apache.hadoop.hbase.ipc.RpcClient$BlockingRpcChannelImplementation.callBlockingMethod(RpcClient.java:1737)

at
org.apache.hadoop.hbase.protobuf.generated.AdminProtos$AdminService$BlockingStub.getRegionInfo(AdminProtos.java:20806)

at
org.apache.hadoop.hbase.client.HBaseAdmin.getCompactionState(HBaseAdmin.java:2524)

at
org.apache.hadoop.hbase.generated.master.table_jsp._jspService(table_jsp.java:167)

at org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.java:98)

at javax.servlet.http.HttpServlet.service(HttpServlet.java:770)

at org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder.java:511)

at
org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1221)

at
org.apache.hadoop.http.lib.StaticUserWebFilter$StaticUserFilter.doFilter(StaticUserWebFilter.java:109)

at
org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1212)

at
org.apache.hadoop.http.HttpServer$QuotingInputFilter.doFilter(HttpServer.java:1081)

at
org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1212)

at org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)

at
org.mortbay.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1212)

at org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandler.java:399)

at
org.mortbay.jetty.security.SecurityHandler.handle(SecurityHandler.java:216)

at org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandler.java:182)

at org.mortbay.jetty.handler.ContextHandler.handle(ContextHandler.java:766)

at org.mortbay.jetty.webapp.WebAppContext.handle(WebAppContext.java:450)

at
org.mortbay.jetty.handler.ContextHandlerCollection.handle(ContextHandlerCollection.java:230)

at org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapper.java:152)

at org.mortbay.jetty.Server.handle(Server.java:326)

at org.mortbay.jetty.HttpConnection.handleRequest(HttpConnection.java:542)

at
org.mortbay.jetty.HttpConnection$RequestHandler.headerComplete(HttpConnection.java:928)

at org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)

at org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)

at org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)

at
org.mortbay.io.nio.SelectChannelEndPoint.run(SelectChannelEndPoint.java:410)

at
org.mortbay.thread.QueuedThreadPool$PoolThread.run(QueuedThreadPool.java:582)

Caused by: org.apache.hadoop.hbase.ipc.RpcClient$CallTimeoutException: Call
id=93152, waitTime=60044, rpcTimeout=60000

at
org.apache.hadoop.hbase.ipc.RpcClient$Connection.cleanupCalls(RpcClient.java:1234)

at
org.apache.hadoop.hbase.ipc.RpcClient$Connection.readResponse(RpcClient.java:1171)

at org.apache.hadoop.hbase.ipc.RpcClient$Connection.run(RpcClient.java:751)
Beside this same issue, please note that first message was at 2015-03-20
14:17:26,015. And then (we got to the point when it started transition):

2015-03-20 14:32:35,059 INFO
org.apache.hadoop.hbase.master.SplitLogManager: task /hbase/splitWAL/WALs%2F
{name_of_node_I_took_down}%2C60020%2C1426860403261-splitting%2F
{name_of_node_I_took_down}%252C60020%252C1426860403261.1426860404905
entered state: DONE {fqdn_of_new_live_node},60020,1426859445623

2015-03-20 14:32:35,109 INFO
org.apache.hadoop.hbase.master.SplitLogManager: Done splitting
/hbase/splitWAL/WALs%2F{name_of_node_I_took_down}
%2C60020%2C1426860403261-splitting%2F{name_of_node_I_took_down}
%252C60020%252C1426860403261.1426860404905

2015-03-20 14:32:35,190 INFO
org.apache.hadoop.hbase.master.SplitLogManager: finished splitting (more
than or equal to) 9 bytes in 1 log files in
[hdfs://{fqdn_of_my_hmaster_node}:8020/hbase/WALs/{name_of_node_I_took_down},60020,1426860403261-splitting]
in 909083ms

2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master.RegionStates:
Transitioned {0e7cc87a4ef6c47a779186f5bf79a01c state=OPEN,
ts=1426860639088, server={name_of_node_I_took_down},60020,1426860403261} to
{0e7cc87a4ef6c47a779186f5bf79a01c state=OFFLINE, ts=1426861955191, server=
{name_of_node_I_took_down},60020,1426860403261}

2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master.RegionStates:
Offlined 0e7cc87a4ef6c47a779186f5bf79a01c from {name_of_node_I_took_down}
,60020,1426860403261

2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master.RegionStates:
Transitioned {25ab6e7b42e36ddaa723d71be5954543 state=OPEN,
ts=1426860641783, server={name_of_node_I_took_down},60020,1426860403261} to
{25ab6e7b42e36ddaa723d71be5954543 state=OFFLINE, ts=1426861955191, server=
{name_of_node_I_took_down},60020,1426860403261}

2015-03-20 14:32:35,191 INFO org.apache.hadoop.hbase.master.RegionStates:
Offlined 25ab6e7b42e36ddaa723d71be5954543 from {name_of_node_I_took_down}
,60020,1426860403261
At this point, note that it finished SplitLogManager task at 14:32:35 and
started transitioning just after that. So this is 15 minutes that I'm
talking about.

What am I missing?


On Fri, Mar 20, 2015 at 2:37 PM Nicolas Liochon <nk...@gmail.com> wrote:

> You've changed the value of hbase.zookeeper.timeout to 15 minutes? A very
> reasonable target is 1 minute before relocating the regions. That's the
> default iirc. You can push it to 20s, but then gc-stopping-the-world
> becomes more of an issue. 15 minutes is really a lot. The hdfs stale mode
> must always be used, with a lower timeout than the hbase one.
>
> Client side there should be nothing to do (excepted for advanced stuff); at
> each retry the client checks the location of the regions. If you lower the
> number of retry the client will fail sooner, but usually you don't want the
> client to fail, you want the servers to reallocate quickly.
>
> On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <de...@gmail.com>
> wrote:
>
> > Hi,
> >
> > Sorry for little bit late update, but managed to narrow it little bit
> down.
> >
> > We didn't update yet, as we are using Hortonworks distribution right now,
> > and even if we update we will get 0.98.4. However, looks that issue here
> > was in our use case and configuration (still looking into it).
> >
> > Basically, initially I saw that when one server goes down, we start
> having
> > performance issues in general, but it managed to be on our client side,
> due
> > to caching, and clients were trying to reconnect to nodes that were
> offline
> > and later trying to get regions from those nodes too. This is basically
> why
> > on server side I didn't manage to see anything in logs that would be at
> > least little bit interesting and point me into desired direction.
> >
> > Another question that popped up to me is - in case server is down (and
> with
> > it DataNode and HRegionServer it was hosting) - what's optimal time to
> set
> > for HMaster to consider server dead reassign regions somewhere else, as
> > this is another performance bottleneck we hit during inability to access
> > regions? In our case it's configured to 15 minutes, and simple logic
> tells
> > me if you want it earlier then configure lower number of retries, but
> issue
> > is as always in details, so not sure if anyone knows some better math for
> > this?
> >
> > And last question - is it possible to manually force HBase to reassign
> > regions? In this case, while HMaster is retrying to contact node that's
> > dead, it's impossible to force it using 'balancer' command.
> >
> > Thanks a lot!
> >
> > Dejan
> >
> > On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <de...@gmail.com>
> > wrote:
> >
> > > Hi,
> > >
> > > To be very honest - there's no particular reason why we stick to this
> > one,
> > > beside just lack of time currently to go through upgrade process, but
> > looks
> > > to me that's going to be next step.
> > >
> > > Had a crazy day, didn't have time to go through all logs again, plus
> one
> > > of the machines (last one where we had this issue) is fully
> reprovisioned
> > > yesterday so I don't have logs from there anymore.
> > >
> > > Beside upgrading,  what I will talk about today, can you just point me
> to
> > > the specific RPC issue in 0.98.0? Thing is that we have some strange
> > > moments with RPC in this case, and just want to see if that's the same
> > > thing (and we were even suspecting to RPC).
> > >
> > > Thanks a lot!
> > > Dejan
> > >
> > > On Mon, Mar 16, 2015 at 9:32 PM, Andrew Purtell <ap...@apache.org>
> > > wrote:
> > >
> > >> Is there a particular reason why you are using HBase 0.98.0? The
> latest
> > >> 0.98 release is 0.98.11. There's a known performance issue with 0.98.0
> > >> pertaining to RPC that was fixed in later releases, you should move up
> > >> from
> > >> 0.98.0. In addition hundreds of improvements and bug fixes have gone
> > into
> > >> the ten releases since 0.98.0.
> > >>
> > >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <dejan.menges@gmail.com
> >
> > >> wrote:
> > >>
> > >> > Hi All,
> > >> >
> > >> > We have a strange issue with HBase performance (overall cluster
> > >> > performance) in case one of datanodes in the cluster unexpectedly
> goes
> > >> > down.
> > >> >
> > >> > So scenario is like follows:
> > >> > - Cluster works fine, it's stable.
> > >> > - One DataNode unexpectedly goes down (PSU issue, network issue,
> > >> anything)
> > >> > - Whole HBase cluster goes down (performance becomes so bad that we
> > >> have to
> > >> > restart all RegionServers to get it back to life).
> > >> >
> > >> > Most funny and latest issue that happened was that we added new node
> > to
> > >> the
> > >> > cluster (having 8 x 4T SATA disks) and we left just DataNode running
> > on
> > >> it
> > >> > to give it couple of days to get some data. At some point in time,
> due
> > >> to
> > >> > hardware issue, server rebooted (twice during three hours) in moment
> > >> when
> > >> > it had maybe 5% of data it would have in a couple of days. Nothing
> > else
> > >> > beside DataNode was running, and once it went down, it affected
> > literary
> > >> > everything, and restarting RegionServers in the end fixed it.
> > >> >
> > >> > We are using HBase 0.98.0 with Hadoop 2.4.0
> > >> >
> > >>
> > >>
> > >>
> > >> --
> > >> Best regards,
> > >>
> > >>    - Andy
> > >>
> > >> Problems worthy of attack prove their worth by hitting back. - Piet
> Hein
> > >> (via Tom White)
> > >>
> > >
> > >
> >
>

Re: Strange issue when DataNode goes down

Posted by Nicolas Liochon <nk...@gmail.com>.
You've changed the value of hbase.zookeeper.timeout to 15 minutes? A very
reasonable target is 1 minute before relocating the regions. That's the
default iirc. You can push it to 20s, but then gc-stopping-the-world
becomes more of an issue. 15 minutes is really a lot. The hdfs stale mode
must always be used, with a lower timeout than the hbase one.

Client side there should be nothing to do (excepted for advanced stuff); at
each retry the client checks the location of the regions. If you lower the
number of retry the client will fail sooner, but usually you don't want the
client to fail, you want the servers to reallocate quickly.

On Fri, Mar 20, 2015 at 1:36 PM, Dejan Menges <de...@gmail.com>
wrote:

> Hi,
>
> Sorry for little bit late update, but managed to narrow it little bit down.
>
> We didn't update yet, as we are using Hortonworks distribution right now,
> and even if we update we will get 0.98.4. However, looks that issue here
> was in our use case and configuration (still looking into it).
>
> Basically, initially I saw that when one server goes down, we start having
> performance issues in general, but it managed to be on our client side, due
> to caching, and clients were trying to reconnect to nodes that were offline
> and later trying to get regions from those nodes too. This is basically why
> on server side I didn't manage to see anything in logs that would be at
> least little bit interesting and point me into desired direction.
>
> Another question that popped up to me is - in case server is down (and with
> it DataNode and HRegionServer it was hosting) - what's optimal time to set
> for HMaster to consider server dead reassign regions somewhere else, as
> this is another performance bottleneck we hit during inability to access
> regions? In our case it's configured to 15 minutes, and simple logic tells
> me if you want it earlier then configure lower number of retries, but issue
> is as always in details, so not sure if anyone knows some better math for
> this?
>
> And last question - is it possible to manually force HBase to reassign
> regions? In this case, while HMaster is retrying to contact node that's
> dead, it's impossible to force it using 'balancer' command.
>
> Thanks a lot!
>
> Dejan
>
> On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <de...@gmail.com>
> wrote:
>
> > Hi,
> >
> > To be very honest - there's no particular reason why we stick to this
> one,
> > beside just lack of time currently to go through upgrade process, but
> looks
> > to me that's going to be next step.
> >
> > Had a crazy day, didn't have time to go through all logs again, plus one
> > of the machines (last one where we had this issue) is fully reprovisioned
> > yesterday so I don't have logs from there anymore.
> >
> > Beside upgrading,  what I will talk about today, can you just point me to
> > the specific RPC issue in 0.98.0? Thing is that we have some strange
> > moments with RPC in this case, and just want to see if that's the same
> > thing (and we were even suspecting to RPC).
> >
> > Thanks a lot!
> > Dejan
> >
> > On Mon, Mar 16, 2015 at 9:32 PM, Andrew Purtell <ap...@apache.org>
> > wrote:
> >
> >> Is there a particular reason why you are using HBase 0.98.0? The latest
> >> 0.98 release is 0.98.11. There's a known performance issue with 0.98.0
> >> pertaining to RPC that was fixed in later releases, you should move up
> >> from
> >> 0.98.0. In addition hundreds of improvements and bug fixes have gone
> into
> >> the ten releases since 0.98.0.
> >>
> >> On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <de...@gmail.com>
> >> wrote:
> >>
> >> > Hi All,
> >> >
> >> > We have a strange issue with HBase performance (overall cluster
> >> > performance) in case one of datanodes in the cluster unexpectedly goes
> >> > down.
> >> >
> >> > So scenario is like follows:
> >> > - Cluster works fine, it's stable.
> >> > - One DataNode unexpectedly goes down (PSU issue, network issue,
> >> anything)
> >> > - Whole HBase cluster goes down (performance becomes so bad that we
> >> have to
> >> > restart all RegionServers to get it back to life).
> >> >
> >> > Most funny and latest issue that happened was that we added new node
> to
> >> the
> >> > cluster (having 8 x 4T SATA disks) and we left just DataNode running
> on
> >> it
> >> > to give it couple of days to get some data. At some point in time, due
> >> to
> >> > hardware issue, server rebooted (twice during three hours) in moment
> >> when
> >> > it had maybe 5% of data it would have in a couple of days. Nothing
> else
> >> > beside DataNode was running, and once it went down, it affected
> literary
> >> > everything, and restarting RegionServers in the end fixed it.
> >> >
> >> > We are using HBase 0.98.0 with Hadoop 2.4.0
> >> >
> >>
> >>
> >>
> >> --
> >> Best regards,
> >>
> >>    - Andy
> >>
> >> Problems worthy of attack prove their worth by hitting back. - Piet Hein
> >> (via Tom White)
> >>
> >
> >
>

Re: Strange issue when DataNode goes down

Posted by Dejan Menges <de...@gmail.com>.
Hi,

Sorry for little bit late update, but managed to narrow it little bit down.

We didn't update yet, as we are using Hortonworks distribution right now,
and even if we update we will get 0.98.4. However, looks that issue here
was in our use case and configuration (still looking into it).

Basically, initially I saw that when one server goes down, we start having
performance issues in general, but it managed to be on our client side, due
to caching, and clients were trying to reconnect to nodes that were offline
and later trying to get regions from those nodes too. This is basically why
on server side I didn't manage to see anything in logs that would be at
least little bit interesting and point me into desired direction.

Another question that popped up to me is - in case server is down (and with
it DataNode and HRegionServer it was hosting) - what's optimal time to set
for HMaster to consider server dead reassign regions somewhere else, as
this is another performance bottleneck we hit during inability to access
regions? In our case it's configured to 15 minutes, and simple logic tells
me if you want it earlier then configure lower number of retries, but issue
is as always in details, so not sure if anyone knows some better math for
this?

And last question - is it possible to manually force HBase to reassign
regions? In this case, while HMaster is retrying to contact node that's
dead, it's impossible to force it using 'balancer' command.

Thanks a lot!

Dejan

On Tue, Mar 17, 2015 at 9:37 AM Dejan Menges <de...@gmail.com> wrote:

> Hi,
>
> To be very honest - there's no particular reason why we stick to this one,
> beside just lack of time currently to go through upgrade process, but looks
> to me that's going to be next step.
>
> Had a crazy day, didn't have time to go through all logs again, plus one
> of the machines (last one where we had this issue) is fully reprovisioned
> yesterday so I don't have logs from there anymore.
>
> Beside upgrading,  what I will talk about today, can you just point me to
> the specific RPC issue in 0.98.0? Thing is that we have some strange
> moments with RPC in this case, and just want to see if that's the same
> thing (and we were even suspecting to RPC).
>
> Thanks a lot!
> Dejan
>
> On Mon, Mar 16, 2015 at 9:32 PM, Andrew Purtell <ap...@apache.org>
> wrote:
>
>> Is there a particular reason why you are using HBase 0.98.0? The latest
>> 0.98 release is 0.98.11. There's a known performance issue with 0.98.0
>> pertaining to RPC that was fixed in later releases, you should move up
>> from
>> 0.98.0. In addition hundreds of improvements and bug fixes have gone into
>> the ten releases since 0.98.0.
>>
>> On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <de...@gmail.com>
>> wrote:
>>
>> > Hi All,
>> >
>> > We have a strange issue with HBase performance (overall cluster
>> > performance) in case one of datanodes in the cluster unexpectedly goes
>> > down.
>> >
>> > So scenario is like follows:
>> > - Cluster works fine, it's stable.
>> > - One DataNode unexpectedly goes down (PSU issue, network issue,
>> anything)
>> > - Whole HBase cluster goes down (performance becomes so bad that we
>> have to
>> > restart all RegionServers to get it back to life).
>> >
>> > Most funny and latest issue that happened was that we added new node to
>> the
>> > cluster (having 8 x 4T SATA disks) and we left just DataNode running on
>> it
>> > to give it couple of days to get some data. At some point in time, due
>> to
>> > hardware issue, server rebooted (twice during three hours) in moment
>> when
>> > it had maybe 5% of data it would have in a couple of days. Nothing else
>> > beside DataNode was running, and once it went down, it affected literary
>> > everything, and restarting RegionServers in the end fixed it.
>> >
>> > We are using HBase 0.98.0 with Hadoop 2.4.0
>> >
>>
>>
>>
>> --
>> Best regards,
>>
>>    - Andy
>>
>> Problems worthy of attack prove their worth by hitting back. - Piet Hein
>> (via Tom White)
>>
>
>

Re: Strange issue when DataNode goes down

Posted by Dejan Menges <de...@gmail.com>.
Hi,

To be very honest - there's no particular reason why we stick to this one,
beside just lack of time currently to go through upgrade process, but looks
to me that's going to be next step.

Had a crazy day, didn't have time to go through all logs again, plus one of
the machines (last one where we had this issue) is fully reprovisioned
yesterday so I don't have logs from there anymore.

Beside upgrading,  what I will talk about today, can you just point me to
the specific RPC issue in 0.98.0? Thing is that we have some strange
moments with RPC in this case, and just want to see if that's the same
thing (and we were even suspecting to RPC).

Thanks a lot!
Dejan

On Mon, Mar 16, 2015 at 9:32 PM, Andrew Purtell <ap...@apache.org> wrote:

> Is there a particular reason why you are using HBase 0.98.0? The latest
> 0.98 release is 0.98.11. There's a known performance issue with 0.98.0
> pertaining to RPC that was fixed in later releases, you should move up from
> 0.98.0. In addition hundreds of improvements and bug fixes have gone into
> the ten releases since 0.98.0.
>
> On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <de...@gmail.com>
> wrote:
>
> > Hi All,
> >
> > We have a strange issue with HBase performance (overall cluster
> > performance) in case one of datanodes in the cluster unexpectedly goes
> > down.
> >
> > So scenario is like follows:
> > - Cluster works fine, it's stable.
> > - One DataNode unexpectedly goes down (PSU issue, network issue,
> anything)
> > - Whole HBase cluster goes down (performance becomes so bad that we have
> to
> > restart all RegionServers to get it back to life).
> >
> > Most funny and latest issue that happened was that we added new node to
> the
> > cluster (having 8 x 4T SATA disks) and we left just DataNode running on
> it
> > to give it couple of days to get some data. At some point in time, due to
> > hardware issue, server rebooted (twice during three hours) in moment when
> > it had maybe 5% of data it would have in a couple of days. Nothing else
> > beside DataNode was running, and once it went down, it affected literary
> > everything, and restarting RegionServers in the end fixed it.
> >
> > We are using HBase 0.98.0 with Hadoop 2.4.0
> >
>
>
>
> --
> Best regards,
>
>    - Andy
>
> Problems worthy of attack prove their worth by hitting back. - Piet Hein
> (via Tom White)
>

Re: Strange issue when DataNode goes down

Posted by Andrew Purtell <ap...@apache.org>.
Is there a particular reason why you are using HBase 0.98.0? The latest
0.98 release is 0.98.11. There's a known performance issue with 0.98.0
pertaining to RPC that was fixed in later releases, you should move up from
0.98.0. In addition hundreds of improvements and bug fixes have gone into
the ten releases since 0.98.0.

On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <de...@gmail.com>
wrote:

> Hi All,
>
> We have a strange issue with HBase performance (overall cluster
> performance) in case one of datanodes in the cluster unexpectedly goes
> down.
>
> So scenario is like follows:
> - Cluster works fine, it's stable.
> - One DataNode unexpectedly goes down (PSU issue, network issue, anything)
> - Whole HBase cluster goes down (performance becomes so bad that we have to
> restart all RegionServers to get it back to life).
>
> Most funny and latest issue that happened was that we added new node to the
> cluster (having 8 x 4T SATA disks) and we left just DataNode running on it
> to give it couple of days to get some data. At some point in time, due to
> hardware issue, server rebooted (twice during three hours) in moment when
> it had maybe 5% of data it would have in a couple of days. Nothing else
> beside DataNode was running, and once it went down, it affected literary
> everything, and restarting RegionServers in the end fixed it.
>
> We are using HBase 0.98.0 with Hadoop 2.4.0
>



-- 
Best regards,

   - Andy

Problems worthy of attack prove their worth by hitting back. - Piet Hein
(via Tom White)

Re: Strange issue when DataNode goes down

Posted by Ted Yu <yu...@gmail.com>.
Have you examined region server logs (for the servers with bas performance)
to see if there was some clue ?

Taking a few jstack's may also help reveal something.

BTW 0.98.11 has been released. You may want to consider upgrading.

Cheers

On Mon, Mar 16, 2015 at 6:40 AM, Dejan Menges <de...@gmail.com>
wrote:

> Hi All,
>
> We have a strange issue with HBase performance (overall cluster
> performance) in case one of datanodes in the cluster unexpectedly goes
> down.
>
> So scenario is like follows:
> - Cluster works fine, it's stable.
> - One DataNode unexpectedly goes down (PSU issue, network issue, anything)
> - Whole HBase cluster goes down (performance becomes so bad that we have to
> restart all RegionServers to get it back to life).
>
> Most funny and latest issue that happened was that we added new node to the
> cluster (having 8 x 4T SATA disks) and we left just DataNode running on it
> to give it couple of days to get some data. At some point in time, due to
> hardware issue, server rebooted (twice during three hours) in moment when
> it had maybe 5% of data it would have in a couple of days. Nothing else
> beside DataNode was running, and once it went down, it affected literary
> everything, and restarting RegionServers in the end fixed it.
>
> We are using HBase 0.98.0 with Hadoop 2.4.0
>