[jira] [Comment Edited] (HDFS-12098) Ozone: Datanode is unable to register with scm if scm starts later

2017-07-16 Thread Weiwei Yang (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-12098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16089283#comment-16089283
 ] 

Weiwei Yang edited comment on HDFS-12098 at 7/17/17 4:01 AM:
-

Attached a test case patch to reproduce this issue. Please take a look at 
[^HDFS-12098-HDFS-7240.testcase.patch]. This patch simulates the scenario

# Start mini ozone cluster without starting scm
# Datanode is unable to register to scm
# Start scm, waiting for datanode to register
# Wait a while but datanode is still unable to successfully register to scm

if you apply this patch, it's gonna to fail. Some log from step 4 is 
interesting,

{noformat}
2017-07-17 11:46:02,451 [Datanode State Machine Thread - 0] INFO  ipc.Client 
(Client.java:handleConnectionFailure(933)) - Retrying connect to server: 
localhost/127.0.0.1:51183. Already tried 2 time(s); retry policy is 
RetryUpToMaximumCountWithFixedSleep(maxRetries=10, sleepTime=1000 MILLISECONDS)
2017-07-17 11:46:02,467 [Datanode State Machine Thread - 0] INFO  
endpoint.VersionEndpointTask (VersionEndpointTask.java:call(61))  - Version 
endpoint task (localhost/127.0.0.1:51183) transited to state REGISTER
2017-07-17 11:46:02,468 [Datanode State Machine Thread - 1] INFO  
endpoint.VersionEndpointTask (VersionEndpointTask.java:call(61))  - Version 
endpoint task (localhost/127.0.0.1:51183) transited to state HEARTBEAT
2017-07-17 11:46:02,469 [Datanode State Machine Thread - 2] INFO  
endpoint.VersionEndpointTask (VersionEndpointTask.java:call(61))  - Version 
endpoint task (localhost/127.0.0.1:51183) transited to state SHUTDOWN
2017-07-17 11:46:02,471 [Datanode State Machine Thread - 3] INFO  
endpoint.VersionEndpointTask (VersionEndpointTask.java:call(61))  - Version 
endpoint task (localhost/127.0.0.1:51183) transited to state SHUTDOWN
{noformat}

Instead of transiting to state {{HEARTBEAT}}, it transited to {{SHUTDOWN}}.

You might have noticed the patch changes some more code than just adding a 
test, that is because the reason I mentioned earlier. I also have added a 
method to check if a datanode is registered to scm so that we can check 
datanode state even scm is not started.

I have a patch to fix this also, if applied that patch, this test will pass. I 
am  ready to share that as well.

Thanks


was (Author: cheersyang):
Attached a test case patch to reproduce this issue. Please take a look at 
[^HDFS-12098-HDFS-7240.testcase.patch]. This patch simulates the scenario

# Start mini ozone cluster without starting scm
# Datanode is unable to register to scm
# Start scm, waiting for datanode to register
# Wait a while but datanode is still unable to successfully register to scm

Step 4 will print log

{noformat}
2017-07-17 11:46:02,451 [Datanode State Machine Thread - 0] INFO  ipc.Client 
(Client.java:handleConnectionFailure(933)) - Retrying connect to server: 
localhost/127.0.0.1:51183. Already tried 2 time(s); retry policy is 
RetryUpToMaximumCountWithFixedSleep(maxRetries=10, sleepTime=1000 MILLISECONDS)
2017-07-17 11:46:02,467 [Datanode State Machine Thread - 0] INFO  
endpoint.VersionEndpointTask (VersionEndpointTask.java:call(61))  - Version 
endpoint task (localhost/127.0.0.1:51183) transited to state REGISTER
2017-07-17 11:46:02,468 [Datanode State Machine Thread - 1] INFO  
endpoint.VersionEndpointTask (VersionEndpointTask.java:call(61))  - Version 
endpoint task (localhost/127.0.0.1:51183) transited to state HEARTBEAT
2017-07-17 11:46:02,469 [Datanode State Machine Thread - 2] INFO  
endpoint.VersionEndpointTask (VersionEndpointTask.java:call(61))  - Version 
endpoint task (localhost/127.0.0.1:51183) transited to state SHUTDOWN
2017-07-17 11:46:02,471 [Datanode State Machine Thread - 3] INFO  
endpoint.VersionEndpointTask (VersionEndpointTask.java:call(61))  - Version 
endpoint task (localhost/127.0.0.1:51183) transited to state SHUTDOWN
2017-07-17 11:46:03,457 [Datanode State Machine Thread - 0] INFO  
statemachine.DatanodeStateMachine 
(DatanodeStateMachine.java:lambda$startDaemon$0(272))  - Ozone container 
server started.
{noformat}

if you apply this patch, it's gonna to fail. You might have noticed the patch 
changes some more code than just adding a test, that is because the reason I 
mentioned earlier. I also have added a method to check if a datanode is 
registered to scm so that we can check datanode state even scm is not started.

I have a patch to fix this also, if applied that patch, this test will pass. I 
am  ready to share that as well.

Thanks

> Ozone: Datanode is unable to register with scm if scm starts later
> --
>
> Key: HDFS-12098
> URL: https://issues.apache.org/jira/browse/HDFS-12098
> Project: Hadoop HDFS
>  Issue Type: Sub-task
>  Components: datanode, ozone, scm
>

[jira] [Comment Edited] (HDFS-12098) Ozone: Datanode is unable to register with scm if scm starts later

2017-07-16 Thread Weiwei Yang (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-12098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16089283#comment-16089283
 ] 

Weiwei Yang edited comment on HDFS-12098 at 7/17/17 3:59 AM:
-

Attached a test case patch to reproduce this issue. Please take a look at 
[^HDFS-12098-HDFS-7240.testcase.patch]. This patch simulates the scenario

# Start mini ozone cluster without starting scm
# Datanode is unable to register to scm
# Start scm, waiting for datanode to register
# Wait a while but datanode is still unable to successfully register to scm

Step 4 will print log

{noformat}
2017-07-17 11:46:02,451 [Datanode State Machine Thread - 0] INFO  ipc.Client 
(Client.java:handleConnectionFailure(933)) - Retrying connect to server: 
localhost/127.0.0.1:51183. Already tried 2 time(s); retry policy is 
RetryUpToMaximumCountWithFixedSleep(maxRetries=10, sleepTime=1000 MILLISECONDS)
2017-07-17 11:46:02,467 [Datanode State Machine Thread - 0] INFO  
endpoint.VersionEndpointTask (VersionEndpointTask.java:call(61))  - Version 
endpoint task (localhost/127.0.0.1:51183) transited to state REGISTER
2017-07-17 11:46:02,468 [Datanode State Machine Thread - 1] INFO  
endpoint.VersionEndpointTask (VersionEndpointTask.java:call(61))  - Version 
endpoint task (localhost/127.0.0.1:51183) transited to state HEARTBEAT
2017-07-17 11:46:02,469 [Datanode State Machine Thread - 2] INFO  
endpoint.VersionEndpointTask (VersionEndpointTask.java:call(61))  - Version 
endpoint task (localhost/127.0.0.1:51183) transited to state SHUTDOWN
2017-07-17 11:46:02,471 [Datanode State Machine Thread - 3] INFO  
endpoint.VersionEndpointTask (VersionEndpointTask.java:call(61))  - Version 
endpoint task (localhost/127.0.0.1:51183) transited to state SHUTDOWN
2017-07-17 11:46:03,457 [Datanode State Machine Thread - 0] INFO  
statemachine.DatanodeStateMachine 
(DatanodeStateMachine.java:lambda$startDaemon$0(272))  - Ozone container 
server started.
{noformat}

if you apply this patch, it's gonna to fail. You might have noticed the patch 
changes some more code than just adding a test, that is because the reason I 
mentioned earlier. I also have added a method to check if a datanode is 
registered to scm so that we can check datanode state even scm is not started.

I have a patch to fix this also, if applied that patch, this test will pass. I 
am  ready to share that as well.

Thanks


was (Author: cheersyang):
Attached a test case patch to reproduce this issue. Please take a look at 
[^HDFS-12098-HDFS-7240.testcase.patch]. This patch simulates the scenario

# Start mini ozone cluster without starting scm
# Datanode is unable to register to scm
# Start scm, waiting for datanode to register
# Wait a while but datanode is still unable to successfully register to scm

if you apply this patch, it's gonna to fail. You might have noticed the patch 
changes some more code than just adding a test, that is because the reason I 
mentioned earlier. I also have added a method to check if a datanode is 
registered to scm so that we can check datanode state even scm is not started.

I have a patch to fix this also, if applied that patch, this test will pass. I 
am  ready to share that as well.

Thanks

> Ozone: Datanode is unable to register with scm if scm starts later
> --
>
> Key: HDFS-12098
> URL: https://issues.apache.org/jira/browse/HDFS-12098
> Project: Hadoop HDFS
>  Issue Type: Sub-task
>  Components: datanode, ozone, scm
>Reporter: Weiwei Yang
>Assignee: Weiwei Yang
>Priority: Critical
> Attachments: disabled-scm-test.patch, HDFS-12098-HDFS-7240.001.patch, 
> HDFS-12098-HDFS-7240.002.patch, HDFS-12098-HDFS-7240.testcase.patch, Screen 
> Shot 2017-07-11 at 4.58.08 PM.png, thread_dump.log
>
>
> Reproducing steps
> 1. Start namenode
> {{./bin/hdfs --daemon start namenode}}
> 2. Start datanode
> {{./bin/hdfs datanode}}
> will see following connection issues
> {noformat}
> 17/07/13 21:16:48 INFO ipc.Client: Retrying connect to server: 
> ozone1.fyre.ibm.com/172.16.165.133:9861. Already tried 0 time(s); retry 
> policy is RetryUpToMaximumCountWithFixedSleep(maxRetries=10, sleepTime=1 
> SECONDS)
> 17/07/13 21:16:49 INFO ipc.Client: Retrying connect to server: 
> ozone1.fyre.ibm.com/172.16.165.133:9861. Already tried 1 time(s); retry 
> policy is RetryUpToMaximumCountWithFixedSleep(maxRetries=10, sleepTime=1 
> SECONDS)
> 17/07/13 21:16:50 INFO ipc.Client: Retrying connect to server: 
> ozone1.fyre.ibm.com/172.16.165.133:9861. Already tried 2 time(s); retry 
> policy is RetryUpToMaximumCountWithFixedSleep(maxRetries=10, sleepTime=1 
> SECONDS)
> 17/07/13 21:16:51 INFO ipc.Client: Retrying connect to server: 
> ozone1.fyre.ibm.com/172.16.165.133:9861. Already tried 3 time(s); retry 
> 

[jira] [Comment Edited] (HDFS-12098) Ozone: Datanode is unable to register with scm if scm starts later

2017-07-16 Thread Weiwei Yang (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-12098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16089283#comment-16089283
 ] 

Weiwei Yang edited comment on HDFS-12098 at 7/17/17 3:58 AM:
-

Attached a test case patch to reproduce this issue. Please take a look at 
[^HDFS-12098-HDFS-7240.testcase.patch]. This patch simulates the scenario

# Start mini ozone cluster without starting scm
# Datanode is unable to register to scm
# Start scm, waiting for datanode to register
# Wait a while but datanode is still unable to successfully register to scm

if you apply this patch, it's gonna to fail. You might have noticed the patch 
changes some more code than just adding a test, that is because the reason I 
mentioned earlier. I also have added a method to check if a datanode is 
registered to scm so that we can check datanode state even scm is not started.

I have a patch to fix this also, if applied that patch, this test will pass. I 
am  ready to share that as well.

Thanks


was (Author: cheersyang):
Attached a test case patch to reproduce this issue. Please take a look at 
[^HDFS-12098-HDFS-7240.testcase.patch]. This patch simulates the scenario

# Start mini ozone cluster without starting scm
# Datanode is unable to register to scm
# Start scm, waiting for datanode to register
# Wait a while but datanode is still unable to successfully register to scm

if you apply this patch, it's gonna fail. You might have noticed the patch 
changes some more code than just adding a test, that is because the reason I 
mentioned earlier. I also have added a method to check if a datanode is 
registered to scm so that we can check datanode state even scm is not started.

I have a patch to fix this also, if applied that patch, this test will pass. I 
am  ready to share that as well.

Thanks

> Ozone: Datanode is unable to register with scm if scm starts later
> --
>
> Key: HDFS-12098
> URL: https://issues.apache.org/jira/browse/HDFS-12098
> Project: Hadoop HDFS
>  Issue Type: Sub-task
>  Components: datanode, ozone, scm
>Reporter: Weiwei Yang
>Assignee: Weiwei Yang
>Priority: Critical
> Attachments: disabled-scm-test.patch, HDFS-12098-HDFS-7240.001.patch, 
> HDFS-12098-HDFS-7240.002.patch, HDFS-12098-HDFS-7240.testcase.patch, Screen 
> Shot 2017-07-11 at 4.58.08 PM.png, thread_dump.log
>
>
> Reproducing steps
> 1. Start namenode
> {{./bin/hdfs --daemon start namenode}}
> 2. Start datanode
> {{./bin/hdfs datanode}}
> will see following connection issues
> {noformat}
> 17/07/13 21:16:48 INFO ipc.Client: Retrying connect to server: 
> ozone1.fyre.ibm.com/172.16.165.133:9861. Already tried 0 time(s); retry 
> policy is RetryUpToMaximumCountWithFixedSleep(maxRetries=10, sleepTime=1 
> SECONDS)
> 17/07/13 21:16:49 INFO ipc.Client: Retrying connect to server: 
> ozone1.fyre.ibm.com/172.16.165.133:9861. Already tried 1 time(s); retry 
> policy is RetryUpToMaximumCountWithFixedSleep(maxRetries=10, sleepTime=1 
> SECONDS)
> 17/07/13 21:16:50 INFO ipc.Client: Retrying connect to server: 
> ozone1.fyre.ibm.com/172.16.165.133:9861. Already tried 2 time(s); retry 
> policy is RetryUpToMaximumCountWithFixedSleep(maxRetries=10, sleepTime=1 
> SECONDS)
> 17/07/13 21:16:51 INFO ipc.Client: Retrying connect to server: 
> ozone1.fyre.ibm.com/172.16.165.133:9861. Already tried 3 time(s); retry 
> policy is RetryUpToMaximumCountWithFixedSleep(maxRetries=10, sleepTime=1 
> SECONDS)
> {noformat}
> this is expected because scm is not started yet
> 3. Start scm
> {{./bin/hdfs scm}}
> expecting datanode can register to this scm, expecting the log in scm
> {noformat}
> 17/07/13 21:22:30 INFO node.SCMNodeManager: Data node with ID: 
> af22862d-aafa-4941-9073-53224ae43e2c Registered.
> {noformat}
> but did *NOT* see this log. (_I debugged into the code and found the datanode 
> state was transited SHUTDOWN unexpectedly because the thread leaks, each of 
> those threads counted to set to next state and they all set to SHUTDOWN 
> state_)
> 4. Create a container from scm CLI
> {{./bin/hdfs scm -container -create -c 20170714c0}}
> this fails with following exception
> {noformat}
> Creating container : 20170714c0.
> Error executing 
> command:org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.ozone.scm.exceptions.SCMException):
>  Unable to create container while in chill mode
>   at 
> org.apache.hadoop.ozone.scm.container.ContainerMapping.allocateContainer(ContainerMapping.java:241)
>   at 
> org.apache.hadoop.ozone.scm.StorageContainerManager.allocateContainer(StorageContainerManager.java:392)
>   at 
> org.apache.hadoop.ozone.protocolPB.StorageContainerLocationProtocolServerSideTranslatorPB.allocateContainer(StorageContainerLocationProtocolServerSideTranslatorPB.java:73)
> {noformat}
> 

[jira] [Comment Edited] (HDFS-12098) Ozone: Datanode is unable to register with scm if scm starts later

2017-07-13 Thread Weiwei Yang (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-12098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16086658#comment-16086658
 ] 

Weiwei Yang edited comment on HDFS-12098 at 7/14/17 12:50 AM:
--

Hi [~anu]

bq. Looks like the main does call, SCM constructor ...

The main method you pasted is only called if scm is started, in bin/hdfs

{code}
scm)
  HADOOP_CLASSNAME='org.apache.hadoop.ozone.scm.StorageContainerManager'
  ...
{code}

if I don't start scm (like how I reproduce this issue in the description), it 
won't be called, and the port will not be bound. However, in MiniOzoneCluster, 
no matter scm is started or not, the constructor will be called. That's what I 
meant.

Thanks


was (Author: cheersyang):
Hi [~anu]

bq. Looks like the main does call, SCM constructor ...

The main method you pasted is only called if scm is started, in bin/hdfs

{code}
scm)
  HADOOP_CLASSNAME='org.apache.hadoop.ozone.scm.StorageContainerManager'
  ...
{code}

if I don't start scm (like how I reproduce this issue in the description), it 
won't be called, and the port will not be bound. That's what I meant.

Thanks

> Ozone: Datanode is unable to register with scm if scm starts later
> --
>
> Key: HDFS-12098
> URL: https://issues.apache.org/jira/browse/HDFS-12098
> Project: Hadoop HDFS
>  Issue Type: Sub-task
>  Components: datanode, ozone, scm
>Reporter: Weiwei Yang
>Assignee: Weiwei Yang
>Priority: Critical
> Attachments: disabled-scm-test.patch, HDFS-12098-HDFS-7240.001.patch, 
> HDFS-12098-HDFS-7240.002.patch, Screen Shot 2017-07-11 at 4.58.08 PM.png, 
> thread_dump.log
>
>
> Reproducing steps
> # Start datanode
> # Wait and see datanode state, it has connection issues, this is expected
> # Start SCM, expecting datanode could connect to the scm and the state 
> machine could transit to RUNNING. However in actual, its state transits to 
> SHUTDOWN, datanode enters chill mode.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Comment Edited] (HDFS-12098) Ozone: Datanode is unable to register with scm if scm starts later

2017-07-13 Thread Anu Engineer (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-12098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16086064#comment-16086064
 ] 

Anu Engineer edited comment on HDFS-12098 at 7/13/17 5:34 PM:
--

[~cheersyang] Sorry to be so dense. I am not sure I understand what this means.
bq. However, in a real cluster environment. Scm constructor will not be called, 
so the port will not be bound. 
Looks like the main does call, SCM constructor 
 
{code}
 /**
   * Main entry point for starting StorageContainerManager.
   *
   * @param argv arguments
   * @throws IOException if startup fails due to I/O error
   */
  public static void main(String[] argv) throws IOException {
StringUtils.startupShutdownMessage(StorageContainerManager.class,
argv, LOG);
try {
  StorageContainerManager scm = new StorageContainerManager(
  new OzoneConfiguration());
  scm.start();
  scm.join();
} catch (Throwable t) {
  LOG.error("Failed to start the StorageContainerManager.", t);
  terminate(1, t);
}
  }
{code}


was (Author: anu):
[~cheersyang] Sorry to be so dense. I am not sure I understand what this means 
well.
bq. However, in a real cluster environment. Scm constructor will not be called, 
so the port will not be bound. 
Looks like the main does call, SCM constructor 
 
{code}
 /**
   * Main entry point for starting StorageContainerManager.
   *
   * @param argv arguments
   * @throws IOException if startup fails due to I/O error
   */
  public static void main(String[] argv) throws IOException {
StringUtils.startupShutdownMessage(StorageContainerManager.class,
argv, LOG);
try {
  StorageContainerManager scm = new StorageContainerManager(
  new OzoneConfiguration());
  scm.start();
  scm.join();
} catch (Throwable t) {
  LOG.error("Failed to start the StorageContainerManager.", t);
  terminate(1, t);
}
  }
{code}

> Ozone: Datanode is unable to register with scm if scm starts later
> --
>
> Key: HDFS-12098
> URL: https://issues.apache.org/jira/browse/HDFS-12098
> Project: Hadoop HDFS
>  Issue Type: Sub-task
>  Components: datanode, ozone, scm
>Reporter: Weiwei Yang
>Assignee: Weiwei Yang
>Priority: Critical
> Attachments: disabled-scm-test.patch, HDFS-12098-HDFS-7240.001.patch, 
> HDFS-12098-HDFS-7240.002.patch, Screen Shot 2017-07-11 at 4.58.08 PM.png, 
> thread_dump.log
>
>
> Reproducing steps
> # Start datanode
> # Wait and see datanode state, it has connection issues, this is expected
> # Start SCM, expecting datanode could connect to the scm and the state 
> machine could transit to RUNNING. However in actual, its state transits to 
> SHUTDOWN, datanode enters chill mode.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Comment Edited] (HDFS-12098) Ozone: Datanode is unable to register with scm if scm starts later

2017-07-07 Thread Weiwei Yang (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-12098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16078213#comment-16078213
 ] 

Weiwei Yang edited comment on HDFS-12098 at 7/7/17 3:44 PM:


This is because datanode state machine leaks {{VersionEndpointTask}} thread. In 
the case scm is not yet started, more and more {{VersionEndpointTask}} threads 
keep retrying connection with scm,

{noformat}
INIT - RUNNING 
 \
GETVERSION
 new VersionEndpointTask submitted - retrying ...
   ... (HB interval)
 new VersionEndpointTask submitted - retrying ...
   ... (HB interval)
 new VersionEndpointTask submitted - retrying ...
   ...
{noformat}

the version endpoint tasks are launched in HB interval (5s on my env), so every 
5s there is a new task submitted; the retry policy for each getVersion call is 
10 * 1s = 10s, so every 10s a task can be finished. So every 10s there will be 
ONE thread leak.

Please see [^thread_dump.log], there are 20 VersionEndpointTask threads in 
WAITING state. And this number keeps increasing.

When scm is up, all pending tasks will be able to connect to scm and getVersion 
call returns, so each of them will count the state to next, since the state is 
shared in {{EndpointStateMachine}}, it increments more than 1 so when I review 
the state changes, it looks like below

{noformat}
REGISTER
HEARTBEAT
SHUTDOWN
SHUTDOWN
SHUTDOWN
... 
{noformat}

To fix this, instead of using a central ExecutorService carried in 
{{DatanodeStateMachine}}, instead we could init a fixed size of thread pool to 
execute end point tasks, and make sure the thread pool gets shutdown before 
entering next state (at end of await).


was (Author: cheersyang):
This is because datanode state machine leaks {{VersionEndpointTask}} thread. In 
the case scm is not yet started,
 more and more {{VersionEndpointTask}} threads keep retrying connection with 
scm,

{noformat}
INIT - RUNNING 
 \
GETVERSION
 new VersionEndpointTask submitted - retrying ...
   ... (HB interval)
 new VersionEndpointTask submitted - retrying ...
   ... (HB interval)
 new VersionEndpointTask submitted - retrying ...
   ...
{noformat}

the version endpoint tasks are launched in HB interval (5s on my env), so every 
5s there is a new task submitted; the retry policy for each getVersion call is 
10 * 1s = 10s, so every 10s a task can be finished. So every 10s there will be 
ONE thread leak.

Please see [^thread_dump.log], there are 20 VersionEndpointTask threads in 
WAITING state. And this number keeps increasing.

When scm is up, all pending tasks will be able to connect to scm and getVersion 
call returns, so each of them will count the state to next, since the state is 
shared in {{EndpointStateMachine}}, it increments more than 1 so when I review 
the state changes, it looks like below

{noformat}
REGISTER
HEARTBEAT
SHUTDOWN
SHUTDOWN
SHUTDOWN
... 
{noformat}

To fix this, instead of using a central ExecutorService carried in 
{{DatanodeStateMachine}}, instead we could init a fixed size of thread pool to 
execute end point tasks, and make sure the thread pool gets shutdown before 
entering next state (at end of await).

> Ozone: Datanode is unable to register with scm if scm starts later
> --
>
> Key: HDFS-12098
> URL: https://issues.apache.org/jira/browse/HDFS-12098
> Project: Hadoop HDFS
>  Issue Type: Sub-task
>  Components: datanode, ozone, scm
>Reporter: Weiwei Yang
>Assignee: Weiwei Yang
>Priority: Critical
> Attachments: HDFS-12098-HDFS-7240.001.patch, thread_dump.log
>
>
> Reproducing steps
> # Start datanode
> # Wait and see datanode state, it has connection issues, this is expected
> # Start SCM, expecting datanode could connect to the scm and the state 
> machine could transit to RUNNING. However in actual, its state transits to 
> SHUTDOWN, datanode enters chill mode.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Comment Edited] (HDFS-12098) Ozone: Datanode is unable to register with scm if scm starts later

2017-07-07 Thread Weiwei Yang (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-12098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16078213#comment-16078213
 ] 

Weiwei Yang edited comment on HDFS-12098 at 7/7/17 3:15 PM:


This is because datanode state machine leaks {{VersionEndpointTask}} thread. In 
the case scm is not yet started,
 more and more {{VersionEndpointTask}} threads keep retrying connection with 
scm,

{noformat}
INIT - RUNNING 
 \
GETVERSION
 new VersionEndpointTask submitted - retrying ...
   ... (HB interval)
 new VersionEndpointTask submitted - retrying ...
   ... (HB interval)
 new VersionEndpointTask submitted - retrying ...
   ...
{noformat}

the version endpoint tasks are launched in HB interval (5s on my env), so every 
5s there is a new task submitted; the retry policy for each getVersion call is 
10 * 1s = 10s, so every 10s a task can be finished. So every 10s there will be 
ONE thread leak.

Please see [^thread_dump.log], there are 20 VersionEndpointTask threads in 
WAITING state. And this number keeps increasing.

When scm is up, all pending tasks will be able to connect to scm and getVersion 
call returns, so each of them will count the state to next, since the state is 
shared in {{EndpointStateMachine}}, it increments more than 1 so when I review 
the state changes, it looks like below

{noformat}
REGISTER
HEARTBEAT
SHUTDOWN
SHUTDOWN
SHUTDOWN
... 
{noformat}

To fix this, instead of using a central ExecutorService carried in 
{{DatanodeStateMachine}}, instead we could init a fixed size of thread pool to 
execute end point tasks, and make sure the thread pool gets shutdown before 
entering next state (at end of await).


was (Author: cheersyang):
This is because datanode state machine leaks {{VersionEndpointTask}} thread. In 
the case scm is not yet started,
 more and more {{VersionEndpointTask}} threads keep retrying connection with 
scm,

{noformat}
INIT - RUNNING 
 \
GETVERSION
 new VersionEndpointTask submitted - retrying ...
   ... (HB interval)
 new VersionEndpointTask submitted - retrying ...
   ... (HB interval)
 new VersionEndpointTask submitted - retrying ...
   ...
{noformat}

the version endpoint tasks are launched in HB interval (5s on my env), so every 
5s there is a new task submitted; the retry policy for each getVersion call is 
10 * 1s = 10s, so every 10s a task can be finished. So every 10s there will be 
ONE thread leak.

When scm is up, all pending tasks will be able to connect to scm and getVersion 
call returns, so each of them will count the state to next, since the state is 
shared in {{EndpointStateMachine}}, it increments more than 1 so when I review 
the state changes, it looks like below

{noformat}
REGISTER
HEARTBEAT
SHUTDOWN
SHUTDOWN
SHUTDOWN
... 
{noformat}

> Ozone: Datanode is unable to register with scm if scm starts later
> --
>
> Key: HDFS-12098
> URL: https://issues.apache.org/jira/browse/HDFS-12098
> Project: Hadoop HDFS
>  Issue Type: Sub-task
>  Components: datanode, ozone, scm
>Reporter: Weiwei Yang
>Assignee: Weiwei Yang
>Priority: Critical
> Attachments: thread_dump.log
>
>
> Reproducing steps
> # Start datanode
> # Wait and see datanode state, it has connection issues, this is expected
> # Start SCM, expecting datanode could connect to the scm and the state 
> machine could transit to RUNNING. However in actual, its state transits to 
> SHUTDOWN, datanode enters chill mode.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Comment Edited] (HDFS-12098) Ozone: Datanode is unable to register with scm if scm starts later

2017-07-07 Thread Weiwei Yang (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-12098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16078213#comment-16078213
 ] 

Weiwei Yang edited comment on HDFS-12098 at 7/7/17 3:11 PM:


This is because datanode state machine leaks {{VersionEndpointTask}} thread. In 
the case scm is not yet started,
 more and more {{VersionEndpointTask}} threads keep retrying connection with 
scm,

{noformat}
INIT - RUNNING 
 \
GETVERSION
 new VersionEndpointTask submitted - retrying ...
   ... (HB interval)
 new VersionEndpointTask submitted - retrying ...
   ... (HB interval)
 new VersionEndpointTask submitted - retrying ...
   ...
{noformat}

the version endpoint tasks are launched in HB interval (5s on my env), so every 
5s there is a new task submitted; the retry policy for each getVersion call is 
10 * 1s = 10s, so every 10s a task can be finished. So every 10s there will be 
ONE thread leak.

When scm is up, all pending tasks will be able to connect to scm and getVersion 
call returns, so each of them will count the state to next, since the state is 
shared in {{EndpointStateMachine}}, it increments more than 1 so when I review 
the state changes, it looks like below

{noformat}
REGISTER
HEARTBEAT
SHUTDOWN
SHUTDOWN
SHUTDOWN
... 
{noformat}


was (Author: cheersyang):
This is because datanode state machine leaks {{VersionEndpointTask}} thread. In 
the case scm is not yet started,
 more and more {{VersionEndpointTask}} threads keep retrying connection with 
scm,

{noformat}
INIT - RUNNING 
 \
GETVERSION
   executor.execute(new VersionEndpointTask()) - retry on 
getVersion ...
   ... (HB interval)
   executor.execute(new VersionEndpointTask()) - retry on 
getVersion ...
   ... (HB interval)
   executor.execute(new VersionEndpointTask()) - retry on 
getVersion ...
   ...
{noformat}

the version endpoint tasks are launched in HB interval (5s on my env), so every 
5s there is a new task submitted; the retry policy for each getVersion call is 
10 * 1s = 10s, so every 10s a task can be finished. So every 10s there will be 
ONE thread leak.

When scm is up, all pending tasks will be able to connect to scm and getVersion 
call returns, so each of them will count the state to next, since the state is 
shared in {{EndpointStateMachine}}, it increments more than 1 so when I review 
the state changes, it looks like below

{noformat}
REGISTER
HEARTBEAT
SHUTDOWN
SHUTDOWN
SHUTDOWN
... 
{noformat}

> Ozone: Datanode is unable to register with scm if scm starts later
> --
>
> Key: HDFS-12098
> URL: https://issues.apache.org/jira/browse/HDFS-12098
> Project: Hadoop HDFS
>  Issue Type: Sub-task
>  Components: datanode, ozone, scm
>Reporter: Weiwei Yang
>Assignee: Weiwei Yang
>Priority: Critical
> Attachments: thread_dump.log
>
>
> Reproducing steps
> # Start datanode
> # Wait and see datanode state, it has connection issues, this is expected
> # Start SCM, expecting datanode could connect to the scm and the state 
> machine could transit to RUNNING. However in actual, its state transits to 
> SHUTDOWN, datanode enters chill mode.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org