Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -146,17 +146,6 @@ public DatanodeID getID() {
return id;
}

/**
* Returns the DataNode UUID.
*
* @return UUID of DataNode
*/
// TODO: Remove this in follow-up Jira (HDDS-12015)
@Deprecated
public UUID getUuid() {
return id.getUuid();
}

/**
* Returns the string representation of DataNode UUID.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ private DatanodeID(final UUID uuid) {
}

// Mainly used for JSON conversion
public String getID() {
public String getUuid() {
return toString();
}

Expand Down Expand Up @@ -121,11 +121,4 @@ private static HddsProtos.UUID toProto(final UUID id) {
.setLeastSigBits(id.getLeastSignificantBits())
.build();
}

// TODO: Remove this in follow-up Jira. (HDDS-12015)
// Exposing this temporarily to help with refactoring.
@Deprecated
public UUID getUuid() {
return uuid;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,8 @@
package org.apache.hadoop.hdds.scm.container;

import com.fasterxml.jackson.databind.annotation.JsonSerialize;
import java.util.UUID;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.DatanodeID;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.server.JsonUtils;

Expand All @@ -31,7 +31,7 @@ public final class ContainerReplicaInfo {
private long containerID;
private String state;
private DatanodeDetails datanodeDetails;
private UUID placeOfBirth;
private DatanodeID placeOfBirth;
private long sequenceId;
private long keyCount;
private long bytesUsed;
Expand All @@ -46,7 +46,7 @@ public static ContainerReplicaInfo fromProto(
.setState(proto.getState())
.setDatanodeDetails(DatanodeDetails
.getFromProtoBuf(proto.getDatanodeDetails()))
.setPlaceOfBirth(UUID.fromString(proto.getPlaceOfBirth()))
.setPlaceOfBirth(DatanodeID.fromUuidString(proto.getPlaceOfBirth()))
.setSequenceId(proto.getSequenceID())
.setKeyCount(proto.getKeyCount())
.setBytesUsed(proto.getBytesUsed())
Expand All @@ -71,7 +71,7 @@ public DatanodeDetails getDatanodeDetails() {
return datanodeDetails;
}

public UUID getPlaceOfBirth() {
public DatanodeID getPlaceOfBirth() {
return placeOfBirth;
}

Expand Down Expand Up @@ -117,7 +117,7 @@ public Builder setDatanodeDetails(DatanodeDetails datanodeDetails) {
return this;
}

public Builder setPlaceOfBirth(UUID placeOfBirth) {
public Builder setPlaceOfBirth(DatanodeID placeOfBirth) {
subject.placeOfBirth = placeOfBirth;
return this;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls;
import org.apache.hadoop.hdds.security.SecurityConfig;
import org.apache.hadoop.hdds.security.symmetric.SecretKeySignerClient;
Expand Down Expand Up @@ -119,7 +118,7 @@ public ContainerProtos.ContainerChecksumInfo getContainerChecksumInfo(long conta
public static Pipeline createSingleNodePipeline(DatanodeDetails dn) {
return Pipeline.newBuilder()
.setNodes(ImmutableList.of(dn))
.setId(PipelineID.valueOf(dn.getUuid()))
.setId(dn.getID().toPipelineID())
.setState(Pipeline.PipelineState.CLOSED)
.setReplicationConfig(StandaloneReplicationConfig.getInstance(
HddsProtos.ReplicationFactor.ONE)).build();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1066,7 +1066,7 @@ public DatanodeInfo getDatanodeInfo(DatanodeDetails dn) {
return nodeStateManager.getNode(dn);
} catch (NodeNotFoundException e) {
LOG.warn("Cannot retrieve DatanodeInfo, datanode {} not found.",
dn.getUuid());
dn.getID());
return null;
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -164,7 +164,7 @@ public static NodeReportProto getRandomNodeReport() {
public static NodeReportProto getRandomNodeReport(int numberOfStorageReport,
int numberOfMetadataStorageReport) {
DatanodeID nodeId = DatanodeID.randomID();
return getRandomNodeReport(nodeId, File.separator + nodeId.getID(),
return getRandomNodeReport(nodeId, File.separator + nodeId.getUuid(),
numberOfStorageReport, numberOfMetadataStorageReport);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ public void testNodeCanBeAddedAndRetrieved()
// Create a datanode, then add and retrieve it
DatanodeDetails dn = generateDatanode();
nsm.addNode(dn, UpgradeUtils.defaultLayoutVersionProto());
assertEquals(dn.getUuid(), nsm.getNode(dn).getUuid());
assertEquals(dn.getID(), nsm.getNode(dn).getID());
// Now get the status of the newly added node and it should be
// IN_SERVICE and HEALTHY
NodeStatus expectedState = NodeStatus.inServiceHealthy();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -122,12 +122,12 @@ public void testPipelineDatanodesIntersection(int nodeCount,
LOG.info("This pipeline: " + pipeline.getId().toString() +
" overlaps with previous pipeline: " + overlapPipeline.getId() +
". They share same set of datanodes as: " +
pipeline.getNodesInOrder().get(0).getUuid() + "/" +
pipeline.getNodesInOrder().get(1).getUuid() + "/" +
pipeline.getNodesInOrder().get(2).getUuid() + " and " +
overlapPipeline.getNodesInOrder().get(0).getUuid() + "/" +
overlapPipeline.getNodesInOrder().get(1).getUuid() + "/" +
overlapPipeline.getNodesInOrder().get(2).getUuid() +
pipeline.getNodesInOrder().get(0).getID() + "/" +
pipeline.getNodesInOrder().get(1).getID() + "/" +
pipeline.getNodesInOrder().get(2).getID() + " and " +
overlapPipeline.getNodesInOrder().get(0).getID() + "/" +
overlapPipeline.getNodesInOrder().get(1).getID() + "/" +
overlapPipeline.getNodesInOrder().get(2).getID() +
" is the same.");
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,14 +41,14 @@
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.UUID;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.conf.StorageUnit;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.DatanodeID;
import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
Expand Down Expand Up @@ -189,7 +189,7 @@ public void testChooseNodeBasedOnNetworkTopology() {
// nodeManager.getClusterNetworkTopologyMap(), anchor, excludedNodes);
assertThat(excludedNodes).doesNotContain(nextNode);
// next node should not be the same as anchor.
assertNotSame(anchor.getUuid(), nextNode.getUuid());
assertNotSame(anchor.getID(), nextNode.getID());
// next node should be on the same rack based on topology.
assertEquals(anchor.getNetworkLocation(), nextNode.getNetworkLocation());
}
Expand Down Expand Up @@ -388,15 +388,14 @@ private NetworkTopology createNetworkTopologyOnDifRacks() {

private DatanodeDetails overwriteLocationInNode(
DatanodeDetails datanode, Node node) {
DatanodeDetails result = DatanodeDetails.newBuilder()
.setUuid(datanode.getUuid())
return DatanodeDetails.newBuilder()
.setID(datanode.getID())
.setHostName(datanode.getHostName())
.setIpAddress(datanode.getIpAddress())
.addPort(datanode.getStandalonePort())
.addPort(datanode.getRatisPort())
.addPort(datanode.getRestPort())
.setNetworkLocation(node.getNetworkLocation()).build();
return result;
}

private List<DatanodeDetails> overWriteLocationInNodes(
Expand Down Expand Up @@ -427,7 +426,7 @@ public void testHeavyNodeShouldBeExcludedWithMinorityHeavy()
// NODES should be sufficient.
assertEquals(nodesRequired, pickedNodes1.size());
// make sure pipeline placement policy won't select duplicated NODES.
assertTrue(checkDuplicateNodesUUID(pickedNodes1));
assertTrue(checkDuplicateNodesID(pickedNodes1));

// majority of healthy NODES are heavily engaged in pipelines.
int majorityHeavy = healthyNodes.size() / 2 + 2;
Expand Down Expand Up @@ -608,11 +607,11 @@ private List<DatanodeDetails> setupSkewedRacks() {
return dns;
}

private boolean checkDuplicateNodesUUID(List<DatanodeDetails> nodes) {
HashSet<UUID> uuids = nodes.stream().
map(DatanodeDetails::getUuid).
private boolean checkDuplicateNodesID(List<DatanodeDetails> nodes) {
HashSet<DatanodeID> ids = nodes.stream().
map(DatanodeDetails::getID).
collect(Collectors.toCollection(HashSet::new));
return uuids.size() == nodes.size();
return ids.size() == nodes.size();
}

private void insertHeavyNodesIntoNodeManager(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -300,15 +300,15 @@ public List<DatanodeDetails.Port> getPorts() {
}

private static class NodeTopologyFull extends NodeTopologyDefault {
private String uuid;
private final String id;

NodeTopologyFull(DatanodeDetails node, String state) {
super(node, state);
uuid = node.getUuid().toString();
id = node.getID().toString();
}

public String getUuid() {
return uuid;
public String getId() {
return id;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -59,9 +59,9 @@ public class DecommissionStatusSubCommand extends ScmSubcommand {
@CommandLine.Mixin
private NodeSelectionMixin nodeSelectionMixin;

@CommandLine.Spec
@CommandLine.Spec
private CommandLine.Model.CommandSpec spec;

@Override
public void execute(ScmClient scmClient) throws IOException {
if (!nodeSelectionMixin.getHostname().isEmpty()) {
Expand Down Expand Up @@ -135,8 +135,7 @@ public void setErrorMessage(String errorMessage) {
}

private void printDetails(DatanodeDetails datanode) {
System.out.println();
System.out.println("Datanode: " + datanode.getUuid().toString() +
System.out.println("\nDatanode: " + datanode.getUuidString() +
" (" + datanode.getNetworkLocation() + "/" + datanode.getIpAddress()
+ "/" + datanode.getHostName() + ")");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -218,7 +218,7 @@ private void printDatanodeInfo(BasicDatanodeInfo dn) {
.append("No pipelines in cluster.")
.append(System.lineSeparator());
}
System.out.println("Datanode: " + datanode.getUuid().toString() +
System.out.println("Datanode: " + datanode.getUuidString() +
" (" + datanode.getNetworkLocation() + "/" + datanode.getIpAddress()
+ "/" + datanode.getHostName() + "/" + relatedPipelineNum +
" pipelines)");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ public void execute(ScmClient scmClient) throws IOException {
!Strings.isNullOrEmpty(exclusiveArguments.getIp()) ? exclusiveArguments.getIp()
: !Strings.isNullOrEmpty(exclusiveArguments.getHostname()) ? exclusiveArguments.getHostname()
: exclusiveArguments.address; //Fallback to deprecated --address for backward compatibility with older CLI.

List<HddsProtos.DatanodeUsageInfoProto> infoList;
if (count < 1) {
throw new IOException("Count must be an integer greater than 0.");
Expand Down Expand Up @@ -116,8 +116,8 @@ public void execute(ScmClient scmClient) throws IOException {
* @param info Information such as Capacity, SCMUsed etc.
*/
private void printInfo(DatanodeUsage info) {
System.out.printf("%-24s: %s %n", "UUID",
info.getDatanodeDetails().getUuid());
System.out.printf("%-24s: %s %n", "ID",
info.getDatanodeDetails().getUuidString());
System.out.printf("%-24s: %s %n", "IP Address",
info.getDatanodeDetails().getIpAddress());
System.out.printf("%-24s: %s %n", "Hostname",
Expand Down Expand Up @@ -161,7 +161,7 @@ private void printInfo(DatanodeUsage info) {
info.getFreeSpaceToSpare() + " B",
StringUtils.byteDesc(info.getFreeSpaceToSpare()));
System.out.printf("%-24s: %s (%s) %n", "Reserved",
info.getReserved() + " B",
info.getReserved() + " B",
StringUtils.byteDesc(info.getReserved()));
System.out.println();
}
Expand Down Expand Up @@ -230,7 +230,7 @@ private static class DatanodeUsage {
if (proto.hasFreeSpaceToSpare()) {
freeSpaceToSpare = proto.getFreeSpaceToSpare();
}
if (proto.hasReserved()) {
if (proto.hasReserved()) {
reserved = proto.getReserved();
}
}
Expand Down Expand Up @@ -329,7 +329,7 @@ public long getPipelineCount() {
return pipelineCount;
}

public long getReserved() {
public long getReserved() {
return reserved;
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -234,7 +234,7 @@ private void testReplicaIncludedInOutput(boolean includeIndex)

// Ensure each DN UUID is mentioned in the message:
for (DatanodeDetails dn : datanodes) {
Pattern uuidPattern = Pattern.compile(".*" + dn.getUuid().toString() + ".*",
Pattern uuidPattern = Pattern.compile(".*" + dn.getID().toString() + ".*",
Pattern.DOTALL);
assertThat(replica).matches(uuidPattern);
}
Expand Down Expand Up @@ -321,7 +321,7 @@ private void testJsonOutput() throws IOException {
assertTrue(json.matches("(?s).*replicas.*"));
for (DatanodeDetails dn : datanodes) {
Pattern pattern = Pattern.compile(
".*replicas.*" + dn.getUuid().toString() + ".*", Pattern.DOTALL);
".*replicas.*" + dn.getID().toString() + ".*", Pattern.DOTALL);
Matcher matcher = pattern.matcher(json);
assertTrue(matcher.matches());
}
Expand All @@ -340,7 +340,7 @@ private List<ContainerReplicaInfo> getReplicas(boolean includeIndex) {
.setContainerID(1)
.setBytesUsed(1234)
.setState("CLOSED")
.setPlaceOfBirth(dn.getUuid())
.setPlaceOfBirth(dn.getID())
.setDatanodeDetails(dn)
.setKeyCount(1)
.setSequenceId(1);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,7 +115,7 @@ public void testOutputDataFieldsAligning() throws IOException {

// then
String output = outContent.toString(CharEncoding.UTF_8);
assertThat(output).contains("UUID :");
assertThat(output).contains("ID :");
assertThat(output).contains("IP Address :");
assertThat(output).contains("Hostname :");
assertThat(output).contains("Ozone Capacity :");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.DatanodeID;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.OzoneClientConfig;
Expand Down Expand Up @@ -337,7 +338,7 @@ public void testSmallerThanChunkSize() throws IOException {
HddsProtos.DatanodeDetailsProto member =
blockList.getKeyLocations(0).getPipeline().getMembers(i);
MockDatanodeStorage mockDatanodeStorage =
storages.get(getMatchingStorage(storages, member.getUuid()));
storages.get(getMatchingStorage(storages, DatanodeID.fromProto(member.getId())));
dns.add(mockDatanodeStorage);
}
String firstBlockData = dns.get(0).getFullBlockData(new BlockID(
Expand Down Expand Up @@ -396,8 +397,8 @@ public void testPutBlockHasBlockGroupLen() throws IOException {
for (int i = 0; i < dataBlocks + parityBlocks; i++) {
MockDatanodeStorage mockDatanodeStorage = storages.get(
getMatchingStorage(storages,
blockList.getKeyLocations(0).getPipeline().getMembers(i)
.getUuid()));
DatanodeID.fromProto(blockList.getKeyLocations(0).getPipeline().getMembers(i)
.getId())));
final OzoneKeyDetails keyDetails = bucket.getKey(keyName);

ContainerProtos.BlockData block = mockDatanodeStorage.getBlock(
Expand All @@ -421,11 +422,11 @@ public void testPutBlockHasBlockGroupLen() throws IOException {
}

private static DatanodeDetails getMatchingStorage(
Map<DatanodeDetails, MockDatanodeStorage> storages, String uuid) {
Map<DatanodeDetails, MockDatanodeStorage> storages, DatanodeID id) {
Iterator<DatanodeDetails> iterator = storages.keySet().iterator();
while (iterator.hasNext()) {
DatanodeDetails dn = iterator.next();
if (dn.getUuid().toString().equals(uuid)) {
if (dn.getID().equals(id)) {
return dn;
}
}
Expand Down
Loading