You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by el...@apache.org on 2019/10/13 06:51:44 UTC
[hadoop-ozone] 03/04: HDDS-1868. Change leadeId definition to
ByteString instead of string.
This is an automated email from the ASF dual-hosted git repository.
elek pushed a commit to branch HDDS-1868
in repository https://gitbox.apache.org/repos/asf/hadoop-ozone.git
commit f48d825da5fb235e12c35280927f60c75b8bdd6d
Author: Siddharth Wagle <sw...@hortonworks.com>
AuthorDate: Wed Oct 9 14:23:26 2019 -0700
HDDS-1868. Change leadeId definition to ByteString instead of string.
---
.../apache/hadoop/hdds/scm/pipeline/Pipeline.java | 36 ++++++++++------------
hadoop-hdds/common/src/main/proto/hdds.proto | 13 ++++----
.../hdds/scm/pipeline/PipelineReportHandler.java | 3 +-
3 files changed, 24 insertions(+), 28 deletions(-)
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java
index d8c7267..2f1c8a3 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java
@@ -18,26 +18,25 @@
package org.apache.hadoop.hdds.scm.pipeline;
-import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
-import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
-import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
/**
* Represents a group of datanodes which store a container.
@@ -54,7 +53,7 @@ public final class Pipeline {
// nodes with ordered distance to client
private ThreadLocal<List<DatanodeDetails>> nodesInOrder = new ThreadLocal<>();
// Current reported Leader for the pipeline
- private RaftPeerId leaderId;
+ private ByteString leaderId = ByteString.EMPTY;
/**
* The immutable properties of pipeline object is used in
@@ -107,14 +106,14 @@ public final class Pipeline {
return state;
}
- public RaftPeerId getLeaderId() {
+ public ByteString getLeaderId() {
return leaderId;
}
/**
* Pipeline object, outside of letting leader id to be set, is immutable.
*/
- void setLeaderId(RaftPeerId leaderId) {
+ void setLeaderId(ByteString leaderId) {
this.leaderId = leaderId;
}
@@ -190,7 +189,7 @@ public final class Pipeline {
.setType(type)
.setFactor(factor)
.setState(PipelineState.getProtobuf(state))
- .setLeaderID(leaderId != null ? leaderId.toString() : "")
+ .setLeaderID(leaderId)
.addAllMembers(nodeStatus.keySet().stream()
.map(DatanodeDetails::getProtoBufMessage)
.collect(Collectors.toList()));
@@ -222,8 +221,7 @@ public final class Pipeline {
.setFactor(pipeline.getFactor())
.setType(pipeline.getType())
.setState(PipelineState.fromProtobuf(pipeline.getState()))
- .setLeaderId(StringUtils.isNotEmpty(pipeline.getLeaderID()) ?
- RaftPeerId.valueOf(pipeline.getLeaderID()) : null)
+ .setLeaderId(pipeline.getLeaderID())
.setNodes(pipeline.getMembersList().stream()
.map(DatanodeDetails::getFromProtoBuf).collect(Collectors.toList()))
.setNodesInOrder(pipeline.getMemberOrdersList())
@@ -292,7 +290,7 @@ public final class Pipeline {
private Map<DatanodeDetails, Long> nodeStatus = null;
private List<Integer> nodeOrder = null;
private List<DatanodeDetails> nodesInOrder = null;
- private RaftPeerId leaderId = null;
+ private ByteString leaderId = ByteString.EMPTY;
public Builder() {}
@@ -325,7 +323,7 @@ public final class Pipeline {
return this;
}
- public Builder setLeaderId(RaftPeerId leaderId1) {
+ public Builder setLeaderId(ByteString leaderId1) {
this.leaderId = leaderId1;
return this;
}
diff --git a/hadoop-hdds/common/src/main/proto/hdds.proto b/hadoop-hdds/common/src/main/proto/hdds.proto
index d2bb355..395481e 100644
--- a/hadoop-hdds/common/src/main/proto/hdds.proto
+++ b/hadoop-hdds/common/src/main/proto/hdds.proto
@@ -67,13 +67,12 @@ enum PipelineState {
}
message Pipeline {
- required string leaderID = 1;
- repeated DatanodeDetailsProto members = 2;
- // TODO: remove the state and leaderID from this class
- optional PipelineState state = 3 [default = PIPELINE_ALLOCATED];
- optional ReplicationType type = 4 [default = STAND_ALONE];
- optional ReplicationFactor factor = 5 [default = ONE];
- required PipelineID id = 6;
+ repeated DatanodeDetailsProto members = 1;
+ optional PipelineState state = 2 [default = PIPELINE_ALLOCATED];
+ optional ReplicationType type = 3 [default = STAND_ALONE];
+ optional ReplicationFactor factor = 4 [default = ONE];
+ required PipelineID id = 5;
+ optional bytes leaderID = 6;
repeated uint32 memberOrders = 7;
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java
index a9534ca..4c83ada 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java
@@ -113,8 +113,7 @@ public class PipelineReportHandler implements
reportedLeadersForPipeline.computeIfAbsent(pipelineID,
k -> new HashMap<>());
ids.put(dn.getUuid(), report.getLeaderID());
- pipeline.setLeaderId(
- RaftPeerId.valueOf(report.getLeaderID().toString()));
+ pipeline.setLeaderId(pipeline.getLeaderId());
}
if (pipeline.getPipelineState() == Pipeline.PipelineState.ALLOCATED) {
---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: hdfs-commits-help@hadoop.apache.org