[jira] [Commented] (HBASE-5635) If getTaskList() returns null splitlogWorker is down. It wont serve any requests.

2012-04-03 Thread Chinna Rao Lalam (Commented) (JIRA)

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

Chinna Rao Lalam commented on HBASE-5635:
-

Updated the patch with log message for retry and corrected the typo.

> If getTaskList() returns null splitlogWorker is down. It wont serve any 
> requests. 
> --
>
> Key: HBASE-5635
> URL: https://issues.apache.org/jira/browse/HBASE-5635
> Project: HBase
>  Issue Type: Bug
>  Components: wal
>Affects Versions: 0.92.1
>Reporter: Kristam Subba Swathi
> Attachments: HBASE-5635.1.patch, HBASE-5635.2.patch, HBASE-5635.patch
>
>
> During the hlog split operation if all the zookeepers are down ,then the 
> paths will be returned as null and the splitworker thread wil be exited
> Now this regionserver wil not be able to acquire any other tasks since the 
> splitworker thread is exited
> Please find the attached code for more details
> {code}
> private List getTaskList() {
> for (int i = 0; i < zkretries; i++) {
>   try {
> return (ZKUtil.listChildrenAndWatchForNewChildren(this.watcher,
> this.watcher.splitLogZNode));
>   } catch (KeeperException e) {
> LOG.warn("Could not get children of znode " +
> this.watcher.splitLogZNode, e);
> try {
>   Thread.sleep(1000);
> } catch (InterruptedException e1) {
>   LOG.warn("Interrupted while trying to get task list ...", e1);
>   Thread.currentThread().interrupt();
>   return null;
> }
>   }
> }
> {code}
> in the org.apache.hadoop.hbase.regionserver.SplitLogWorker 
>  

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (HBASE-5606) SplitLogManger async delete node hangs log splitting when ZK connection is lost

2012-04-03 Thread Chinna Rao Lalam (Commented) (JIRA)

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

Chinna Rao Lalam commented on HBASE-5606:
-

For me also patch looks clean.


> SplitLogManger async delete node hangs log splitting when ZK connection is 
> lost 
> 
>
> Key: HBASE-5606
> URL: https://issues.apache.org/jira/browse/HBASE-5606
> Project: HBase
>  Issue Type: Bug
>  Components: wal
>Affects Versions: 0.92.0
>Reporter: Gopinathan A
>Priority: Critical
> Fix For: 0.92.2
>
> Attachments: 
> 0001-HBASE-5606-SplitLogManger-async-delete-node-hangs-lo.patch, 
> 0001-HBASE-5606-SplitLogManger-async-delete-node-hangs-lo.patch
>
>
> 1. One rs died, the servershutdownhandler found it out and started the 
> distributed log splitting;
> 2. All tasks are failed due to ZK connection lost, so the all the tasks were 
> deleted asynchronously;
> 3. Servershutdownhandler retried the log splitting;
> 4. The asynchronously deletion in step 2 finally happened for new task
> 5. This made the SplitLogManger in hanging state.
> This leads to .META. region not assigened for long time
> {noformat}
> hbase-root-master-HOST-192-168-47-204.log.2012-03-14"(55413,79):2012-03-14 
> 19:28:47,932 DEBUG org.apache.hadoop.hbase.master.SplitLogManager: put up 
> splitlog task at znode 
> /hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020%2C1331720381665-splitting%2Flinux-114.site%252C60020%252C1331720381665.1331752316170
> hbase-root-master-HOST-192-168-47-204.log.2012-03-14"(89303,79):2012-03-14 
> 19:34:32,387 DEBUG org.apache.hadoop.hbase.master.SplitLogManager: put up 
> splitlog task at znode 
> /hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020%2C1331720381665-splitting%2Flinux-114.site%252C60020%252C1331720381665.1331752316170
> {noformat}
> {noformat}
> hbase-root-master-HOST-192-168-47-204.log.2012-03-14"(80417,99):2012-03-14 
> 19:34:31,196 DEBUG 
> org.apache.hadoop.hbase.master.SplitLogManager$DeleteAsyncCallback: deleted 
> /hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020%2C1331720381665-splitting%2Flinux-114.site%252C60020%252C1331720381665.1331752316170
> hbase-root-master-HOST-192-168-47-204.log.2012-03-14"(89456,99):2012-03-14 
> 19:34:32,497 DEBUG 
> org.apache.hadoop.hbase.master.SplitLogManager$DeleteAsyncCallback: deleted 
> /hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020%2C1331720381665-splitting%2Flinux-114.site%252C60020%252C1331720381665.1331752316170
> {noformat}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (HBASE-5635) If getTaskList() returns null splitlogWorker is down. It wont serve any requests.

2012-03-30 Thread Chinna Rao Lalam (Commented) (JIRA)

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

Chinna Rao Lalam commented on HBASE-5635:
-

Updated the patch with retry logic in getTaskList(). So if getTaskList() after 
retry if it returns null the splitLogWorker will be exited.

> If getTaskList() returns null splitlogWorker is down. It wont serve any 
> requests. 
> --
>
> Key: HBASE-5635
> URL: https://issues.apache.org/jira/browse/HBASE-5635
> Project: HBase
>  Issue Type: Bug
>  Components: wal
>Affects Versions: 0.92.1
>Reporter: Kristam Subba Swathi
> Attachments: HBASE-5635.1.patch, HBASE-5635.patch
>
>
> During the hlog split operation if all the zookeepers are down ,then the 
> paths will be returned as null and the splitworker thread wil be exited
> Now this regionserver wil not be able to acquire any other tasks since the 
> splitworker thread is exited
> Please find the attached code for more details
> {code}
> private List getTaskList() {
> for (int i = 0; i < zkretries; i++) {
>   try {
> return (ZKUtil.listChildrenAndWatchForNewChildren(this.watcher,
> this.watcher.splitLogZNode));
>   } catch (KeeperException e) {
> LOG.warn("Could not get children of znode " +
> this.watcher.splitLogZNode, e);
> try {
>   Thread.sleep(1000);
> } catch (InterruptedException e1) {
>   LOG.warn("Interrupted while trying to get task list ...", e1);
>   Thread.currentThread().interrupt();
>   return null;
> }
>   }
> }
> {code}
> in the org.apache.hadoop.hbase.regionserver.SplitLogWorker 
>  

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (HBASE-5635) If getTaskList() returns null splitlogWorker is down. It wont serve any requests.

2012-03-27 Thread Chinna Rao Lalam (Commented) (JIRA)

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

Chinna Rao Lalam commented on HBASE-5635:
-

In SplitLogWorker.taskLoop() if getTaskList() returns null it is comming out 
form taskLoop(). Here when ever getTaskList() return null this thread should 
wait and when ever it will be notified it will continue the work. So 
splitLogWorker thread wont be exited
 
Attached the patch with this change.

> If getTaskList() returns null splitlogWorker is down. It wont serve any 
> requests. 
> --
>
> Key: HBASE-5635
> URL: https://issues.apache.org/jira/browse/HBASE-5635
> Project: HBase
>  Issue Type: Bug
>  Components: wal
>Affects Versions: 0.92.1
>Reporter: Kristam Subba Swathi
> Attachments: HBASE-5635.patch
>
>
> During the hlog split operation if all the zookeepers are down ,then the 
> paths will be returned as null and the splitworker thread wil be exited
> Now this regionserver wil not be able to acquire any other tasks since the 
> splitworker thread is exited
> Please find the attached code for more details
> --
> private List getTaskList() {
> for (int i = 0; i < zkretries; i++) {
>   try {
> return (ZKUtil.listChildrenAndWatchForNewChildren(this.watcher,
> this.watcher.splitLogZNode));
>   } catch (KeeperException e) {
> LOG.warn("Could not get children of znode " +
> this.watcher.splitLogZNode, e);
> try {
>   Thread.sleep(1000);
> } catch (InterruptedException e1) {
>   LOG.warn("Interrupted while trying to get task list ...", e1);
>   Thread.currentThread().interrupt();
>   return null;
> }
>   }
> }
> in the org.apache.hadoop.hbase.regionserver.SplitLogWorker 
>  

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (HBASE-5606) SplitLogManger async delete node hangs log splitting when ZK connection is lost

2012-03-22 Thread Chinna Rao Lalam (Commented) (JIRA)

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

Chinna Rao Lalam commented on HBASE-5606:
-

@Prakash
Thanks prakash for the points

bq.in timeoutmonitor call getDataSetWatch() only if task has not already 
failed. (This is just an optimization and it can be done without any locking)

Here i am thinking call for getDataSetWatch() in timeoutmonitor should be 
sysnchronized because race condition may come in setDone() call and 
getDataSetWatch() in timeoutmonitor call.

bq.for this particular getDataSetWatch() call, store a IGNORE-ZK-ERROR flag in 
the zk async context. If a zk error happens silently then do nothing.

can u elaborate little more on this point. Normally if any error comes we are 
retrying this. Now by introducing this IGNORE-ZK-ERROR need to skip the retry, 
who will set this and when can this flag be true? when task is FAILURE it will 
be true? (if my understanding is not wrong). 


> SplitLogManger async delete node hangs log splitting when ZK connection is 
> lost 
> 
>
> Key: HBASE-5606
> URL: https://issues.apache.org/jira/browse/HBASE-5606
> Project: HBase
>  Issue Type: Bug
>  Components: wal
>Affects Versions: 0.92.0
>Reporter: Gopinathan A
>Priority: Critical
> Fix For: 0.92.2
>
> Attachments: 5606.txt
>
>
> 1. One rs died, the servershutdownhandler found it out and started the 
> distributed log splitting;
> 2. All tasks are failed due to ZK connection lost, so the all the tasks were 
> deleted asynchronously;
> 3. Servershutdownhandler retried the log splitting;
> 4. The asynchronously deletion in step 2 finally happened for new task
> 5. This made the SplitLogManger in hanging state.
> This leads to .META. region not assigened for long time
> {noformat}
> hbase-root-master-HOST-192-168-47-204.log.2012-03-14"(55413,79):2012-03-14 
> 19:28:47,932 DEBUG org.apache.hadoop.hbase.master.SplitLogManager: put up 
> splitlog task at znode 
> /hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020%2C1331720381665-splitting%2Flinux-114.site%252C60020%252C1331720381665.1331752316170
> hbase-root-master-HOST-192-168-47-204.log.2012-03-14"(89303,79):2012-03-14 
> 19:34:32,387 DEBUG org.apache.hadoop.hbase.master.SplitLogManager: put up 
> splitlog task at znode 
> /hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020%2C1331720381665-splitting%2Flinux-114.site%252C60020%252C1331720381665.1331752316170
> {noformat}
> {noformat}
> hbase-root-master-HOST-192-168-47-204.log.2012-03-14"(80417,99):2012-03-14 
> 19:34:31,196 DEBUG 
> org.apache.hadoop.hbase.master.SplitLogManager$DeleteAsyncCallback: deleted 
> /hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020%2C1331720381665-splitting%2Flinux-114.site%252C60020%252C1331720381665.1331752316170
> hbase-root-master-HOST-192-168-47-204.log.2012-03-14"(89456,99):2012-03-14 
> 19:34:32,497 DEBUG 
> org.apache.hadoop.hbase.master.SplitLogManager$DeleteAsyncCallback: deleted 
> /hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020%2C1331720381665-splitting%2Flinux-114.site%252C60020%252C1331720381665.1331752316170
> {noformat}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (HBASE-5606) SplitLogManger async delete node hangs log splitting when ZK connection is lost

2012-03-20 Thread Chinna Rao Lalam (Commented) (JIRA)

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

Chinna Rao Lalam commented on HBASE-5606:
-

I am trying to fix this any suggestions pls welcome..

> SplitLogManger async delete node hangs log splitting when ZK connection is 
> lost 
> 
>
> Key: HBASE-5606
> URL: https://issues.apache.org/jira/browse/HBASE-5606
> Project: HBase
>  Issue Type: Bug
>  Components: wal
>Affects Versions: 0.92.0
>Reporter: Gopinathan A
>Priority: Critical
> Fix For: 0.92.2
>
>
> 1. One rs died, the servershutdownhandler found it out and started the 
> distributed log splitting;
> 2. All tasks are failed due to ZK connection lost, so the all the tasks were 
> deleted asynchronously;
> 3. Servershutdownhandler retried the log splitting;
> 4. The asynchronously deletion in step 2 finally happened for new task
> 5. This made the SplitLogManger in hanging state.
> This leads to .META. region not assigened for long time
> {noformat}
> hbase-root-master-HOST-192-168-47-204.log.2012-03-14"(55413,79):2012-03-14 
> 19:28:47,932 DEBUG org.apache.hadoop.hbase.master.SplitLogManager: put up 
> splitlog task at znode 
> /hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020%2C1331720381665-splitting%2Flinux-114.site%252C60020%252C1331720381665.1331752316170
> hbase-root-master-HOST-192-168-47-204.log.2012-03-14"(89303,79):2012-03-14 
> 19:34:32,387 DEBUG org.apache.hadoop.hbase.master.SplitLogManager: put up 
> splitlog task at znode 
> /hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020%2C1331720381665-splitting%2Flinux-114.site%252C60020%252C1331720381665.1331752316170
> {noformat}
> {noformat}
> hbase-root-master-HOST-192-168-47-204.log.2012-03-14"(80417,99):2012-03-14 
> 19:34:31,196 DEBUG 
> org.apache.hadoop.hbase.master.SplitLogManager$DeleteAsyncCallback: deleted 
> /hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020%2C1331720381665-splitting%2Flinux-114.site%252C60020%252C1331720381665.1331752316170
> hbase-root-master-HOST-192-168-47-204.log.2012-03-14"(89456,99):2012-03-14 
> 19:34:32,497 DEBUG 
> org.apache.hadoop.hbase.master.SplitLogManager$DeleteAsyncCallback: deleted 
> /hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020%2C1331720381665-splitting%2Flinux-114.site%252C60020%252C1331720381665.1331752316170
> {noformat}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (HBASE-5606) SplitLogManger async delete node hangs log splitting when ZK connection is lost

2012-03-20 Thread Chinna Rao Lalam (Commented) (JIRA)

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

Chinna Rao Lalam commented on HBASE-5606:
-

This situation can come in 0.92 when

1)First time SplitLogManager installed one task after that it is not able to 
connect to Zookeeper(Because of CONNECTIONLOSS).

so the GetDataAsyncCallback will fail and it will retry which is register at 
the time of createNode() in installTask() or in TimeoutMonitor.

{noformat}
19:32:24,657 WARN 
org.apache.hadoop.hbase.master.SplitLogManager$GetDataAsyncCallback: getdata rc 
= CONNECTIONLOSS 
/hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020.1331752316170
 retry=0
{noformat}

2)When ever the GetDataAsyncCallback retry=0 it will call setDone() here it 
will increment batch.error and it will register one DeleteAsyncCallback.

3)So here installed != done so SplilogManger will throw exception and it will 
submit again.

4)"failed to set data watch" is happened 92 times so 92 DeleteAsyncCallback are 
registered and all 92 DeleteAsyncCallback will try till it success.

{noformat}
19:34:30,874 WARN org.apache.hadoop.hbase.master.SplitLogManager: failed to set 
data watch 
/hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020%2C1331720381665-splitting%2Flinux-114.site%252C60020%252C1331720381665.1331752316170
{noformat}

5) Because of Point:3 SplitLogManager will try to install the task but it found 
already installed task is FAILURE so it is waiting to change to DELETED

6)Once it got the Zookeer connection one of the DeleteAsyncCallback deleted the 
node  and it will notify the task which is waiting at Point:5

{noformat}
19:34:31,196 DEBUG 
org.apache.hadoop.hbase.master.SplitLogManager$DeleteAsyncCallback: deleted 
/hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020%2C1331720381665-splitting%2Flinux-114.site%252C60020%252C1331720381665.1331752316170
{noformat}

7) Point:5 after notified it will crete the node

{noformat}
19:34:32,387 DEBUG org.apache.hadoop.hbase.master.SplitLogManager: put up 
splitlog task at znode 
/hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020%2C1331720381665-splitting%2Flinux-114.site%252C60020%252C1331720381665.1331752316170
{noformat}

8) But already registered DeleteAsyncCallback's will execute and it will delete 
newly created node at Point:7

{noformat}
19:34:32,497 DEBUG 
org.apache.hadoop.hbase.master.SplitLogManager$DeleteAsyncCallback: deleted 
/hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020%2C1331720381665-splitting%2Flinux-114.site%252C60020%252C1331720381665.1331752316170
{noformat}

9) So because of the node is deleted and it removed form tasks map flow wont 
come to the piece of code to increment the batch.done or batch.error in 
setDone().
So waitTask() will be in infinite looping and it wont come out.

{noformat}
"MASTER_META_SERVER_OPERATIONS-HOST-192-168-47-204,6,1331719909985-1" 
prio=10 tid=0x40d7c000 nid=0x624b in Object.wait() [0x7ff090482000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
at java.lang.Object.wait(Native Method)
at 
org.apache.hadoop.hbase.master.SplitLogManager.waitTasks(SplitLogManager.java:316)
- locked <0x00078e6c4258> (a 
org.apache.hadoop.hbase.master.SplitLogManager$TaskBatch)
at 
org.apache.hadoop.hbase.master.SplitLogManager.splitLogDistributed(SplitLogManager.java:262)
{noformat}

> SplitLogManger async delete node hangs log splitting when ZK connection is 
> lost 
> 
>
> Key: HBASE-5606
> URL: https://issues.apache.org/jira/browse/HBASE-5606
> Project: HBase
>  Issue Type: Bug
>  Components: wal
>Affects Versions: 0.92.0
>Reporter: Gopinathan A
>Priority: Critical
> Fix For: 0.92.2
>
>
> 1. One rs died, the servershutdownhandler found it out and started the 
> distributed log splitting;
> 2. All tasks are failed due to ZK connection lost, so the all the tasks were 
> deleted asynchronously;
> 3. Servershutdownhandler retried the log splitting;
> 4. The asynchronously deletion in step 2 finally happened for new task
> 5. This made the SplitLogManger in hanging state.
> This leads to .META. region not assigened for long time
> {noformat}
> hbase-root-master-HOST-192-168-47-204.log.2012-03-14"(55413,79):2012-03-14 
> 19:28:47,932 DEBUG org.apache.hadoop.hbase.master.SplitLogManager: put up 
> splitlog task at znode 
> /hbase/splitlog/hdfs%3A%2F%2F192.168.47.205%3A9000%2Fhbase%2F.logs%2Flinux-114.site%2C60020%2C1331720381665-splitting%2Flinux-114.site%252C60020%252C1331720381665.133175