[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-29 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r496643970



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
##
@@ -619,9 +619,9 @@ private void finalizePipeline(PipelineID pipelineId) throws 
IOException {
* @throws IOException
*/
   protected void destroyPipeline(Pipeline pipeline) throws IOException {
-pipelineFactory.close(pipeline.getType(), pipeline);
 // remove the pipeline from the pipeline manager
 removePipeline(pipeline.getId());
+pipelineFactory.close(pipeline.getType(), pipeline);

Review comment:
   If did not change,  the order maybe: 
pipelineFactory.close(pipeline.getType(), pipeline) -> datanode close pipeline 
-> scm create new pipeline -> removePipeline(pipeline.getId()), then leader 
distribution will not be balance.
   For example, on server S1, S3, S3, there are 3 pipelines: P1, P2, P3 with 
leader: S1, S2, S3.
   S1 .. S2 .. S3
   P1 .. P2 .. P3
   
   If close P3, and order maybe: pipelineFactory.close(pipeline.getType(), P3) 
-> datanode close P3 -> scm create new pipeline P4 -> removePipeline(P3). When 
create new pipeline P4, because P3 has not been removed, P4 will choose S1 as 
the leader, finally S1 has 2 leaders, but S3 has no leader.
   S1 .. S2 .. S3
   P1 .. P2
   P4
   
   





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-29 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r496627193



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -121,18 +158,24 @@ public Pipeline create(ReplicationFactor factor) throws 
IOException {
   throw new IllegalStateException("Unknown factor: " + factor.name());
 }
 
+DatanodeDetails suggestedLeader = leaderChoosePolicy.chooseLeader(

Review comment:
   LeaderChoosePolicy is an interface, define member in interface is not 
common.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-29 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r496643970



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
##
@@ -619,9 +619,9 @@ private void finalizePipeline(PipelineID pipelineId) throws 
IOException {
* @throws IOException
*/
   protected void destroyPipeline(Pipeline pipeline) throws IOException {
-pipelineFactory.close(pipeline.getType(), pipeline);
 // remove the pipeline from the pipeline manager
 removePipeline(pipeline.getId());
+pipelineFactory.close(pipeline.getType(), pipeline);

Review comment:
   If did not change,  the order maybe: 
pipelineFactory.close(pipeline.getType(), pipeline) -> datanode close pipeline 
-> scm create new pipeline -> removePipeline(pipeline.getId()), then leader 
distribution will not be balance.
   For example, on server S1, S3, S3, there are 3 pipelines: P1, P2, P3 with 
leader: S1, S2, S3.
   S1 .. S2 .. S3
   P1 .. P2 .. P3
   
   If close P3, and order maybe: pipelineFactory.close(pipeline.getType(), P3) 
-> datanode close P3 -> scm create new pipeline P4 -> removePipeline(P3). When 
create new pipeline P4, because P3 has not been removed, P4 will choose S1 as 
the leader, finally S1 has 2 leaders, but S3 has no leader.
   S1 .. S2 .. S3
   P1 .. P2
   P4
   
   





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-29 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r496643970



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
##
@@ -619,9 +619,9 @@ private void finalizePipeline(PipelineID pipelineId) throws 
IOException {
* @throws IOException
*/
   protected void destroyPipeline(Pipeline pipeline) throws IOException {
-pipelineFactory.close(pipeline.getType(), pipeline);
 // remove the pipeline from the pipeline manager
 removePipeline(pipeline.getId());
+pipelineFactory.close(pipeline.getType(), pipeline);

Review comment:
   If did not change,  the order maybe: 
pipelineFactory.close(pipeline.getType(), pipeline) -> datanode close pipeline 
-> scm create new pipeline -> removePipeline(pipeline.getId()), then leader 
distribution will not be balance.
   For example, on server S1, S3, S3, there are 3 pipelines: P1, P2, P3 with 
leader: S1, S2, S3.
   S1 .. S2 .. S3
   P1 .. P2 .. P3
   
   If close P3, and order maybe: pipelineFactory.close(pipeline.getType(), P3) 
-> datanode close P3 -> scm create new pipeline P4 -> removePipeline(P3). When 
create new pipeline P4, because P3 has not been removed, P4 will choose S1 as 
the leader, then S1 has 2 leaders, but S3 has no leader.
   S1 .. S2 .. S3
   P1 .. P2
   P4
   
   





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-29 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r496628424



##
File path: 
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CreatePipelineCommand.java
##
@@ -48,16 +52,37 @@ public CreatePipelineCommand(final PipelineID pipelineID,
 this.factor = factor;
 this.type = type;
 this.nodelist = datanodeList;
+if (datanodeList.size() ==
+XceiverServerRatis.DEFAULT_PRIORITY_LIST.size()) {

Review comment:
   always new is a waste.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-29 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r496628148



##
File path: 
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
##
@@ -115,6 +116,9 @@
   private static final Logger LOG = LoggerFactory
   .getLogger(XceiverServerRatis.class);
   private static final AtomicLong CALL_ID_COUNTER = new AtomicLong();
+  public static final List DEFAULT_PRIORITY_LIST =

Review comment:
   why always create a new list ? The size of DEFAULT_PRIORITY_LIST is 3, 
most case the size of datanode list is 3, their size are equal.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-29 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r496627193



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -121,18 +158,24 @@ public Pipeline create(ReplicationFactor factor) throws 
IOException {
   throw new IllegalStateException("Unknown factor: " + factor.name());
 }
 
+DatanodeDetails suggestedLeader = leaderChoosePolicy.chooseLeader(

Review comment:
   LeaderChoosePolicy is an interface, define member in interface is not 
common.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-29 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r496624353



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManager.java
##
@@ -59,7 +59,8 @@ void addContainerToPipeline(PipelineID pipelineId, 
ContainerID containerID)
 pipelineStateMap.addContainerToPipeline(pipelineId, containerID);
   }
 
-  Pipeline getPipeline(PipelineID pipelineID) throws PipelineNotFoundException 
{
+  public Pipeline getPipeline(PipelineID pipelineID)

Review comment:
   This is necessary, otherwise we can not call `getPipeline` in 
`MinLeaderCountChoosePolicy`





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-22 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r492416925



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   > Plan weight for each of node as a leader when the cluster has 
thousands of nodes can be difficult.
   
   If each node has similar hardware, i.e. CPU, memory, we just plan weight as 
now, assign each node with same leader number, it is cheap and reasonable.
   
   I think the only case we need to consider is that some nodes' hardware is 
weaker than other nodes' obviously.  I think the weeker datanodes should engage 
in less pipeline than the stronger datanodes,  but ozone does not support this 
now.
   If we can support this, the maxum leader number of each datanode should be 
less or equal to ((1/3) * the pipeline number it engaged in),  and we select 
the datanode as the leader which has lowest value of (leader number / pipeline 
number it engaged in) in 3 datanodes,  this is also cheap.

##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   > Plan weight for each of node as a leader when the cluster has 
thousands of nodes can be difficult.
   
   If each node has similar hardware, i.e. CPU, memory, we just plan weight as 
now, assign each node with same leader number, it is cheap and reasonable.
   
   I think the only case we need to consider is that some nodes' hardware is 
weaker than other nodes' obviously.  I think the weeker datanodes should engage 
in less pipeline than the stronger datanodes,  but ozone does not support this 
now. If we can support this, the maxum leader number of each datanode should be 
less or equal to ((1/3) * the pipeline number it engaged in),  and we select 
the datanode as the leader which has lowest value of (leader number / pipeline 
number it engaged in) in 3 datanodes,  this is also cheap.

##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   > Plan weight for each of node as a leader when the cluster has 
thousands of nodes can be difficult.
   
   If each node has similar hardware, i.e. CPU, memory, we just plan weight as 
now, assign each node with same leader number, it is cheap and reasonable.
   
   I think the only case we need to consider is that some nodes' hardware is 
weaker than other nodes' obviously.  I think the weeker datanodes should engage 
in less pipeline than the stronger datanodes,  but ozone does not support this 
now. If we can support this, the maxum leader number of each datanode should be 
less or equal to ((1/3) * the pipeline number it engaged in),  and we select 
the datanode as the leader which has lowest value of (leader number / pipeline 
number it engaged in) in 3 datanodes,  this is also cheap.
   
   

##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/s

[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-22 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r492595294



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {
+suggestedLeaderCount.put(dn, suggestedLeaderCount.get(dn) + 1);
+  }
+} catch (PipelineNotFoundException e) {
+  LOG.debug("Pipeline not found in pipeline state manager : {}",
+  pipelineID, e);
+}
+  }
+}
+
+return suggestedLeaderCount;
+  }
+
+  private DatanodeDetails getSuggestedLeader(List dns) {
+Map suggestedLeaderCount =

Review comment:
   @bshashikant I agree. @xiaoyuyao What do you think of this suggestion?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-21 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r492416925



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   > Plan weight for each of node as a leader when the cluster has 
thousands of nodes can be difficult.
   
   If each node has similar hardware, i.e. CPU, memory, we just plan weight as 
now, assign each node with same leader number, it is cheap and reasonable.
   
   I think the only case we need to consider is that some nodes' hardware is 
weaker than other nodes' obviously.  I think the weeker datanodes should engage 
in less pipeline than the stronger datanodes,  but ozone does not support this 
now.
   If we can support this, the maxum leader number of each datanode should be 
less or equal to ((1/3) * the pipeline number it engaged in),  and we select 
the datanode as the leader which has lowest value of (leader number / pipeline 
number it engaged in) in 3 datanodes,  this is also cheap.

##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   > Plan weight for each of node as a leader when the cluster has 
thousands of nodes can be difficult.
   
   If each node has similar hardware, i.e. CPU, memory, we just plan weight as 
now, assign each node with same leader number, it is cheap and reasonable.
   
   I think the only case we need to consider is that some nodes' hardware is 
weaker than other nodes' obviously.  I think the weeker datanodes should engage 
in less pipeline than the stronger datanodes,  but ozone does not support this 
now. If we can support this, the maxum leader number of each datanode should be 
less or equal to ((1/3) * the pipeline number it engaged in),  and we select 
the datanode as the leader which has lowest value of (leader number / pipeline 
number it engaged in) in 3 datanodes,  this is also cheap.

##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   > Plan weight for each of node as a leader when the cluster has 
thousands of nodes can be difficult.
   
   If each node has similar hardware, i.e. CPU, memory, we just plan weight as 
now, assign each node with same leader number, it is cheap and reasonable.
   
   I think the only case we need to consider is that some nodes' hardware is 
weaker than other nodes' obviously.  I think the weeker datanodes should engage 
in less pipeline than the stronger datanodes,  but ozone does not support this 
now. If we can support this, the maxum leader number of each datanode should be 
less or equal to ((1/3) * the pipeline number it engaged in),  and we select 
the datanode as the leader which has lowest value of (leader number / pipeline 
number it engaged in) in 3 datanodes,  this is also cheap.
   
   

##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/s

[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-21 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r492416925



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   > Plan weight for each of node as a leader when the cluster has 
thousands of nodes can be difficult.
   
   If each node has similar hardware, i.e. CPU, memory, we just plan weight as 
now, assign each node with same leader number, it is cheap and reasonable.
   
   I think the only case we need to consider is that some nodes' hardware is 
weaker than other nodes' obviously.  I think the weeker datanodes should engage 
in less pipeline than the stronger datanodes,  but ozone does not support this 
now. If we can support this, the maxum leader number of each datanode should be 
less or equal to ((1/3) * the pipeline number it engaged in),  and we select 
the datanode as the leader which has lowest value of (leader number / pipeline 
number it engaged in) in 3 datanodes,  this is also cheap. We can change this 
if there is requirement in the future, but now it is enough to allocate the 
same leader number in each datanode.
   
   





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-21 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r492416925



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   > Plan weight for each of node as a leader when the cluster has 
thousands of nodes can be difficult.
   
   If each node has similar hardware, i.e. CPU, memory, we just plan weight as 
now, assign each node with same leader number, it is cheap and reasonable.
   
   I think the only case we need to consider is that some nodes' hardware is 
weaker than other nodes' obviously.  I think the weeker datanodes should engage 
in less pipeline than the stronger datanodes,  but ozone does not support this 
now. If we can support this, the maxum leader number of each datanode should be 
less or equal to ((1/3) * the pipeline number it engaged in),  and we select 
the datanode as the leader which has lowest value of (leader number / pipeline 
number it engaged in) in 3 datanodes,  this is also cheap.
   
   





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-21 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r492416925



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   > Plan weight for each of node as a leader when the cluster has 
thousands of nodes can be difficult.
   
   If each node has similar hardware, i.e. CPU, memory, we just plan weight as 
now, assign each node with same leader number, it is cheap and reasonable.
   
   I think the only case we need to consider is that some nodes' hardware is 
weaker than other nodes' obviously.  I think the weeker datanodes should engage 
in less pipeline than the stronger datanodes,  but ozone does not support this 
now. If we can support this, the maxum leader number of each datanode should be 
less or equal to ((1/3) * the pipeline number it engaged in),  and we select 
the datanode as the leader which has lowest value of (leader number / pipeline 
number it engaged in) in 3 datanodes,  this is also cheap.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-21 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r492416925



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   > Plan weight for each of node as a leader when the cluster has 
thousands of nodes can be difficult.
   
   If each node has similar hardware, i.e. CPU, memory, we just plan weight as 
now, assign each node with same leader number, it is cheap and reasonable.
   
   I think the only case we need to consider is that some nodes' hardware is 
weaker than other nodes' obviously.  I think the weeker datanodes should engage 
in less pipeline than the stronger datanodes,  but ozone does not support this 
now.
   If we can support this, the maxum leader number of each datanode should be 
less or equal to ((1/3) * the pipeline number it engaged in),  and we select 
the datanode as the leader which has lowest value of (leader number / pipeline 
number it engaged in) in 3 datanodes,  this is also cheap.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-17 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r490675374



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   @xiaoyuyao Good point, I also have thought this.
   
   > Any performance impact on the pipeline of forcing leader to be the 
original one.
   
   If there is performance problem, I can improve forcing leader change within 
1 second. I already know how to improve it, but has not implemented it.
   
   > Another situation I'm thinking of is writers on pipeline with slow 
leader(e.g., hardware slowness) may not be able to recover by leader change.
   
   We can find slow leader by some metric, decrease the priority of the slow 
leader, select one faster datanode and increase it's priority, so the faster 
datanode will grab the leadership from the slow leader.
   
   
   > In the case of S1 temporarily down, why don't we keep P1 leader on S3 and 
create P3 with leader on S1, this gives more flexibility for higher level to 
choose leader?
   
   I want the cluster leader distribution as we planned, if the plan is not 
appropriate, we can adjust the plan by change priority.
   
   If the leader distribution totally depends on hardware rather than plan, we 
maybe lost control of the leader distribution. Because the leaderId in scm was 
reported by datanode, it maybe a delayed leaderId. For example, datanode report:
   
   S1  ..  S2 ..   S3
   P1  ..  P2
   
   then P1's leader transfer to S3, but SCM has not received this report, SCM 
allocate P3's leader to S3, then
   
   S1 .. S2 .. S3
   P2 ..  P1
   ...P3
   
   It's not balance now.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-17 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r490675374



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   @xiaoyuyao Good point, I also have thought this.
   
   > Any performance impact on the pipeline of forcing leader to be the 
original one.
   
   If there is performance problem, I can improve forcing leader change within 
1 second. I already know how to improve it, but has not implemented it.
   
   > Another situation I'm thinking of is writers on pipeline with slow 
leader(e.g., hardware slowness) may not be able to recover by leader change.
   
   We can find slow leader by some metric, decrease the priority of the slow 
leader, select one faster datanode and increase it's priority, so the faster 
datanode will grab the leadership from the slow leader.
   
   
   > In the case of S1 temporarily down, why don't we keep P1 leader on S3 and 
create P3 with leader on S1, this gives more flexibility for higher level to 
choose leader?
   
   I want the cluster leader distribution as we planned, if the plan is not 
appropriate, we can adjust the plan by change priority.
   
   If the leader distribution totally depends on hardware rather than plan, we 
maybe lost control of the leader distribution. Because the leaderId in scm was 
reported by datanode, it maybe a delayed leaderId. For example, datanode report:
   
   S1  ..  S2 ..   S3
   P1  ..  P2
   
   then P1's leader transfer to S3, but SCM has not received this report, SCM 
allocate P3's leader to S3, then
   
   S1 .. S2 .. S3
   ...P2 ..  P1
   P3
   
   It's not balance now.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-17 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r490675374



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   @xiaoyuyao Good point, I also have thought this.
   
   > Any performance impact on the pipeline of forcing leader to be the 
original one.
   
   If there is performance problem, I can improve forcing leader change within 
1 second. I already know how to improve it, but has not implemented it.
   
   > Another situation I'm thinking of is writers on pipeline with slow 
leader(e.g., hardware slowness) may not be able to recover by leader change.
   
   We can find slow leader by some metric, decrease the priority of the slow 
leader, select one faster datanode and increase it's priority, so the faster 
datanode will grab the leadership from the slow leader.
   
   
   > In the case of S1 temporarily down, why don't we keep P1 leader on S3 and 
create P3 with leader on S1, this gives more flexibility for higher level to 
choose leader?
   
   I want the cluster leader distribution as we planned, if the plan is not 
appropriate, we can adjust the plan by change priority.
   
   If the leader distribution totally depends on hardware rather than plan, we 
maybe lost control of the leader distribution. Because the leaderId in scm was 
reported by datanode, it maybe a delayed leaderId. For example, datanode report:
   
   S1  ..  S2 ..   S3
   P1  ..  P2
   
   then P1's leader transfer to S3, but SCM has not received this report, SCM 
allocate P3's leader to S3, then
   
   S1 .. S2 .. S3
   ...P2 ..  P1
   ..P3
   
   It's not balance now.

##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   @xiaoyuyao Good point, I also have thought this.
   
   > Any performance impact on the pipeline of forcing leader to be the 
original one.
   
   If there is performance problem, I can improve forcing leader change within 
1 second. I already know how to improve it, but has not implemented it.
   
   > Another situation I'm thinking of is writers on pipeline with slow 
leader(e.g., hardware slowness) may not be able to recover by leader change.
   
   We can find slow leader by some metric, decrease the priority of the slow 
leader, select one faster datanode and increase it's priority, so the faster 
datanode will grab the leadership from the slow leader.
   
   
   > In the case of S1 temporarily down, why don't we keep P1 leader on S3 and 
create P3 with leader on S1, this gives more flexibility for higher level to 
choose leader?
   
   I want the cluster leader distribution as we planned, if the plan is not 
appropriate, we can adjust the plan by change priority.
   
   If the leader distribution totally depends on hardware rather than plan, we 
maybe lost control of the leader distribution. Because the leaderId in scm was 
reported by datanode, it maybe a delayed leaderId. For example, datanode report:
   
   S1  ..  S2 ..   S3
   P1  ..  P2
   
   then P1's leader transfer to S3, but SCM has not received this report, SCM 
allocate P3's leader to S3, then
   
   S1 .. S2 .. S3
   ...P2 ..  P1
   ...P3
   
   It's not balance now.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-17 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r490675374



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   @xiaoyuyao Good point, I also have thought this.
   
   > Any performance impact on the pipeline of forcing leader to be the 
original one.
   
   If there is performance problem, I can improve forcing leader change within 
1 second. I already know how to improve it, but has not implemented it.
   
   > Another situation I'm thinking of is writers on pipeline with slow 
leader(e.g., hardware slowness) may not be able to recover by leader change.
   
   We can find slow leader by some metric, decrease the priority of the slow 
leader, select one faster datanode and increase it's priority, so the faster 
datanode will grab the leadership from the slow leader.
   
   
   > In the case of S1 temporarily down, why don't we keep P1 leader on S3 and 
create P3 with leader on S1, this gives more flexibility for higher level to 
choose leader?
   
   I want the cluster leader distribution as we planned, if the plan is not 
appropriate, we can adjust the plan by change priority.
   
   If the leader distribution totally depends on hardware rather than plan, we 
maybe lost control of the leader distribution. Because the leaderId in scm was 
reported by datanode, it maybe a delayed leaderId. For example, datanode report:
   
   S1  S2  S3
   P1 P2
   
   then P1's leader transfer to S3, but SCM has not received this report, SCM 
allocate P3's leader to S3, then
   
   S1  S2  S3
   .P2   P1
   ..P3
   
   It's not balance now.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-17 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r490675374



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   @xiaoyuyao Good point, I also have thought this.
   
   > Any performance impact on the pipeline of forcing leader to be the 
original one.
   
   If there is performance problem, I can improve forcing leader change within 
1 second. I already know how to improve it, but has not implemented it.
   
   > Another situation I'm thinking of is writers on pipeline with slow 
leader(e.g., hardware slowness) may not be able to recover by leader change.
   
   We can find slow leader by some metric, decrease the priority of the slow 
leader, select one faster datanode and increase it's priority, so the faster 
datanode will grab the leadership from the slow leader.
   
   
   > In the case of S1 temporarily down, why don't we keep P1 leader on S3 and 
create P3 with leader on S1, this gives more flexibility for higher level to 
choose leader?
   
   I want the cluster leader distribution as we planned, if the plan is not 
appropriate, we can adjust the plan by change priority.
   
   If the leader distribution totally depends on hardware rather than plan, we 
maybe lost control of the leader distribution. Because the leaderId in scm was 
reported by datanode, it maybe a delayed leaderId. For example, datanode report:
   
   S1S2S3
   P1P2
   
   then P1's leader transfer to S3, but SCM has not received this report, SCM 
allocate P3's leader to S3, then
   
   S1  S2  S3
   .P2   P1
   ..P3
   
   It's not balance now.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-17 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r490675374



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   @xiaoyuyao Good point, I also have thought this.
   
   > Any performance impact on the pipeline of forcing leader to be the 
original one.
   
   If there is performance problem, I can improve forcing leader change within 
1 second. I already know how to improve it, but has not implemented it.
   
   > Another situation I'm thinking of is writers on pipeline with slow 
leader(e.g., hardware slowness) may not be able to recover by leader change.
   
   We can find slow leader by some metric, decrease the priority of the slow 
leader, select one faster datanode and increase it's priority, so the faster 
datanode will grab the leadership from the slow leader.
   
   
   > In the case of S1 temporarily down, why don't we keep P1 leader on S3 and 
create P3 with leader on S1, this gives more flexibility for higher level to 
choose leader?
   
   I want the cluster leader distribution as we planned, if the plan is not 
appropriate, we can adjust the plan by change priority.
   
   If the leader distribution totally depends on hardware rather than plan, we 
maybe lost control of the leader distribution. Because the leaderId in scm was 
reported by datanode, it maybe a delayed leaderId. For example, datanode report:
   
   S1  S2  S3
   P1 P2
   
   then P1's leader transfer to S3, but SCM has not received this report, SCM 
allocate P3's leader to S3, then
   
   S1  S2  S3
   .P2   P1
   ..P3.
   
   It's not balance now.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-17 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r490675374



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   @xiaoyuyao Good point, I also have thought this.
   
   > Any performance impact on the pipeline of forcing leader to be the 
original one.
   
   If there is performance problem, I can improve forcing leader change within 
1 second. I already know how to improve it, but has not implemented it.
   
   > Another situation I'm thinking of is writers on pipeline with slow 
leader(e.g., hardware slowness) may not be able to recover by leader change.
   
   We can find slow leader by some metric, decrease the priority of the slow 
leader, select one faster datanode and increase it's priority, so the faster 
datanode will grab the leadership from the slow leader.
   
   
   > In the case of S1 temporarily down, why don't we keep P1 leader on S3 and 
create P3 with leader on S1, this gives more flexibility for higher level to 
choose leader?
   
   I want the cluster leader distribution as we planned, if the plan is not 
appropriate, we can adjust the plan by change priority.
   
   If the leader distribution totally depends on hardware rather than plan, we 
maybe lost control of the leader distribution. Because the leaderId in scm was 
reported by datanode, it maybe a delayed leaderId. For example, datanode report:
   
   S1  S2  S3
   P1 P2
   
   then P1's leader transfer to S3, but SCM has not received this report, SCM 
allocate P3's leader to S3, then
   
   S1  S2  S3
   ..P2   P1
P3.
   
   It's not balance now.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-17 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r490675374



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   @xiaoyuyao Good point, I also have thought this.
   
   > Any performance impact on the pipeline of forcing leader to be the 
original one.
   
   If there is performance problem, I can improve forcing leader change within 
1 second. I already know how to improve it, but has not implemented it.
   
   > Another situation I'm thinking of is writers on pipeline with slow 
leader(e.g., hardware slowness) may not be able to recover by leader change.
   
   We can find slow leader by some metric, decrease the priority of the slow 
leader, select one faster datanode and increase it's priority, so the faster 
datanode will grab the leadership from the slow leader.
   
   
   > In the case of S1 temporarily down, why don't we keep P1 leader on S3 and 
create P3 with leader on S1, this gives more flexibility for higher level to 
choose leader?
   
   I want the cluster leader distribution as we planned, if the plan is not 
appropriate, we can adjust the plan by change priority.
   
   If the leader distribution totally depends on hardware rather than plan, we 
maybe lost control of the leader distribution. Because the leaderId in scm was 
reported by datanode, it maybe a delayed leaderId. For example, datanode report:
   
   S1  S2  S3
   P1 P2
   
   then P1's leader transfer to S3, but SCM has not received this report, SCM 
allocate P3's leader to S3, then
   
   S1  S2  S3
   .P2   P1
...P3.
   
   It's not balance now.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-17 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r490675374



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   @xiaoyuyao Good point, I also have thought this.
   
   > Any performance impact on the pipeline of forcing leader to be the 
original one.
   
   If there is performance problem, I can improve forcing leader change within 
1 second. I already know how to improve it, but has not implemented it.
   
   > Another situation I'm thinking of is writers on pipeline with slow 
leader(e.g., hardware slowness) may not be able to recover by leader change.
   
   We can find slow leader by some metric, decrease the priority of the slow 
leader, select one faster datanode and increase it's priority, so the faster 
datanode will grab the leadership from the slow leader.
   
   
   > In the case of S1 temporarily down, why don't we keep P1 leader on S3 and 
create P3 with leader on S1, this gives more flexibility for higher level to 
choose leader?
   
   I want the cluster leader distribution as we planned, if the plan is not 
appropriate, we can adjust the plan by change priority.
   
   If the leader distribution totally depends on hardware rather than plan, we 
maybe lost control of the leader distribution. Because the leaderId in scm was 
reported by datanode, it maybe a delayed leaderId. For example, datanode report:
   
   S1  S2  S3
   P1 P2
   
   then P1's leader transfer to S3, but SCM has not received this report, SCM 
allocate P3's leader to S3, then
   
   S1  S2  S3
P2   P1
   P3.
   
   It's not balance now.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-16 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r489833136



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   [RATIS-967](https://issues.apache.org/jira/browse/RATIS-967) guarantee 
the highpriority node act as leader when create pipeline. In a long running 
cluster, the leader maybe crash, then some follower will take the leadership, 
but when the old leader restart and catch up with current leader's log, the old 
leader can grab the leadership again by 
[RATIS-967](https://issues.apache.org/jira/browse/RATIS-967). 
   
   So let me suppose the following case, there are 3 servers: s1, s2, s3, there 
are 2 pipelines now,  the first pipeline's leader is s1, the second pipeline's 
leader is s2, both the 2 leaders are suggested leader with high priority. Then 
s1 crash, suppose s3 will take the first pipline's leader.  Then s1 restart, 
but has not grab leadership of the first pipeline. If we use getLeaderId() 
instead of getSuggestedLeaderId() to reflect the actual leader count, when we 
create the 3 third pipeline, we find the leader number on s1, s2, s3 is 0, 1, 
1, so we will select s1 as the suggest leader,  then s1 grab the leadership of 
the first pipeline by 
[RATIS-967](https://issues.apache.org/jira/browse/RATIS-967), so the leader 
number on s1, s2, s3 will be 2, 1, 0, it's not balance. 
   
   
   
   
   





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-16 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r489833136



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {

Review comment:
   [RATIS-967](https://issues.apache.org/jira/browse/RATIS-967) guarantee 
the highpriority node act as leader when create pipeline. In a long running 
cluster, the leader maybe crash, then some follower will take the leadership, 
but when the old leader restart and catch up with current leader's log, the old 
leader can grab the leadership again by 
[RATIS-967](https://issues.apache.org/jira/browse/RATIS-967). 
   
   So let me suppose the following case, there are 3 servers: s1, s2, s3, there 
are 2 pipelines now,  the first pipeline's leader is s1, the second pipeline's 
leader is s2, both the 2 leaders are suggested leader with high priority. Then 
s1 crash, suppose s3 will take the first pipline's leader.  Then s1 restart, 
but has not grab leadership of the first pipeline. If we use getLeaderId() 
instead of getSuggestedLeaderId() to reflect the actual leader count, when we 
create the 3 third pipeline, we find the leader number on s1, s2, s3 is 0, 1, 
1, so we will select s1 as the suggest leader,  then s1 grab the leadership of 
the first pipeline, so the leader number on s1, s2, s3 will be 2, 1, 0, it's 
not balance. 
   
   
   
   
   





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-16 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r489827467



##
File path: 
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java
##
@@ -102,7 +102,7 @@ public static void afterClass() {
 FileUtils.deleteQuietly(READ_TMP);
   }
 
-  @Test(timeout = 3)
+  @Test(timeout = 6)

Review comment:
   Yes, the highpriority node act as leader will increase the time of 
leader election,  but almost the increased time is within 10 seconds.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-16 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r489826760



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
##
@@ -98,8 +105,65 @@ private boolean exceedPipelineNumberLimit(ReplicationFactor 
factor) {
 return false;
   }
 
+  private Map getSuggestedLeaderCount(
+  List dns) {
+Map suggestedLeaderCount = new HashMap<>();
+for (DatanodeDetails dn : dns) {
+  suggestedLeaderCount.put(dn, 0);
+
+  Set pipelineIDSet = getNodeManager().getPipelines(dn);
+  for (PipelineID pipelineID : pipelineIDSet) {
+try {
+  Pipeline pipeline = 
getPipelineStateManager().getPipeline(pipelineID);
+  if (!pipeline.isClosed()
+  && dn.getUuid().equals(pipeline.getSuggestedLeaderId())) {
+suggestedLeaderCount.put(dn, suggestedLeaderCount.get(dn) + 1);
+  }
+} catch (PipelineNotFoundException e) {
+  LOG.debug("Pipeline not found in pipeline state manager : {}",
+  pipelineID, e);
+}
+  }
+}
+
+return suggestedLeaderCount;
+  }
+
+  private DatanodeDetails getSuggestedLeader(List dns) {
+Map suggestedLeaderCount =
+getSuggestedLeaderCount(dns);
+int minLeaderCount = Integer.MAX_VALUE;
+DatanodeDetails suggestedLeader = null;
+
+for (Map.Entry entry :
+suggestedLeaderCount.entrySet()) {
+  if (entry.getValue() < minLeaderCount) {
+minLeaderCount = entry.getValue();
+suggestedLeader = entry.getKey();
+  }
+}
+
+return suggestedLeader;
+  }
+
+  private List getPriorityList(
+  List dns, DatanodeDetails suggestedLeader) {
+List priorityList = new ArrayList<>();
+
+for (DatanodeDetails dn : dns) {
+  if (dn.getUuid().equals(suggestedLeader.getUuid())) {
+priorityList.add(HIGH_PRIORITY);

Review comment:
   Yes, [RATIS-967](https://issues.apache.org/jira/browse/RATIS-967) 
guarantee the highpriority node act as leader





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] runzhiwang commented on a change in pull request #1371: HDDS-2922. Balance ratis leader distribution in datanodes

2020-09-16 Thread GitBox


runzhiwang commented on a change in pull request #1371:
URL: https://github.com/apache/hadoop-ozone/pull/1371#discussion_r489826433



##
File path: 
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
##
@@ -711,10 +712,20 @@ private long 
calculatePipelineBytesWritten(HddsProtos.PipelineID pipelineID) {
 
   @Override
   public void addGroup(HddsProtos.PipelineID pipelineId,
-  Collection peers) throws IOException {
+  List peers) throws IOException {
+List priorityList =

Review comment:
   Updated





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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