From 84b56f86fa2d5ba5113a297734737c050817d77e Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Fri, 11 Mar 2022 15:30:17 -0800 Subject: [PATCH 01/29] HDDS-5916: DNs in pipeline raft group get stuck in infinite leader election in Kubernets env --- .../apache/hadoop/hdds/ratis/RatisHelper.java | 18 +++- .../states/datanode/InitDatanodeState.java | 2 +- .../hadoop/hdds/server/events/EventQueue.java | 21 +++- .../hadoop/hdds/scm/events/SCMEvents.java | 7 ++ .../apache/hadoop/hdds/scm/ha/SCMService.java | 1 + .../node/NodeIpOrHostnameUpdateHandler.java | 77 +++++++++++++++ .../hdds/scm/node/NodeStateManager.java | 20 ++++ .../hadoop/hdds/scm/node/SCMNodeManager.java | 97 ++++++++++++++----- .../hdds/scm/node/states/NodeStateMap.java | 25 +++++ .../pipeline/BackgroundPipelineCreator.java | 8 +- .../scm/server/StorageContainerManager.java | 6 ++ .../hdds/scm/node/TestNodeStateManager.java | 35 +++++++ .../hdds/scm/node/TestSCMNodeManager.java | 65 +++++++++++++ 13 files changed, 350 insertions(+), 32 deletions(-) create mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java index c1cd865036fb..c8c67af9eaa9 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hdds.StringUtils; 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.DatanodeDetails.Port; import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig; @@ -38,6 +39,7 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.security.x509.SecurityConfig; +import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.ratis.RaftConfigKeys; import org.apache.ratis.client.RaftClient; import org.apache.ratis.client.RaftClientConfigKeys; @@ -64,6 +66,8 @@ public final class RatisHelper { private static final Logger LOG = LoggerFactory.getLogger(RatisHelper.class); + private static final OzoneConfiguration conf = new OzoneConfiguration(); + // Prefix for Ratis Server GRPC and Ratis client conf. public static final String HDDS_DATANODE_RATIS_PREFIX_KEY = "hdds.ratis"; @@ -96,7 +100,13 @@ public static UUID toDatanodeId(RaftProtos.RaftPeerProto peerId) { } private static String toRaftPeerAddress(DatanodeDetails id, Port.Name port) { - return id.getIpAddress() + ":" + id.getPort(port).getValue(); + if (datanodeUseHostName()) { + LOG.debug("Datanode is using hostname for raft peer address"); + return id.getHostName() + ":" + id.getPort(port).getValue(); + } else { + LOG.debug("Datanode is using IP for raft peer address"); + return id.getIpAddress() + ":" + id.getPort(port).getValue(); + } } public static RaftPeerId toRaftPeerId(DatanodeDetails id) { @@ -323,6 +333,12 @@ public static Long getMinReplicatedIndex( .min(Long::compareTo).orElse(null); } + private static boolean datanodeUseHostName() { + return conf.getBoolean( + DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME, + DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT); + } + private static Class getClass(String name, Class xface) { try { diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java index ff53088a5db3..cd469620e1b5 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/datanode/InitDatanodeState.java @@ -125,7 +125,7 @@ private void persistContainerDatanodeDetails() { File idPath = new File(dataNodeIDPath); DatanodeDetails datanodeDetails = this.context.getParent() .getDatanodeDetails(); - if (datanodeDetails != null && !idPath.exists()) { + if (datanodeDetails != null) { try { ContainerUtils.writeDatanodeDetailsTo(datanodeDetails, idPath); } catch (IOException ex) { diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java index e3a18b74276f..155187999c90 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java @@ -26,6 +26,9 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import com.google.gson.ExclusionStrategy; +import com.google.gson.FieldAttributes; +import org.apache.hadoop.hdds.scm.net.NodeImpl; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Time; @@ -58,10 +61,26 @@ public class EventQueue implements EventPublisher, AutoCloseable { private boolean isRunning = true; - private static final Gson TRACING_SERIALIZER = new GsonBuilder().create(); + private static final Gson TRACING_SERIALIZER = new GsonBuilder() + .setExclusionStrategies(new DatanodeDetailsGsonExclusionStrategy()) + .create(); private boolean isSilent = false; + // The field parent in DatanodeDetails class has the circular reference which will result in Gson infinite recursive + // parsing. We need to exclude this field when generating json string for DatanodeDetails object + static class DatanodeDetailsGsonExclusionStrategy implements ExclusionStrategy { + @Override + public boolean shouldSkipField(FieldAttributes f) { + return f.getDeclaringClass() == NodeImpl.class && f.getName().equals("parent"); + } + + @Override + public boolean shouldSkipClass(Class aClass) { + return false; + } + } + /** * Add new handler to the event queue. *

diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java index 09381dfbec39..09dbe9af1db2 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java @@ -147,6 +147,13 @@ public final class SCMEvents { public static final TypedEvent NEW_NODE = new TypedEvent<>(DatanodeDetails.class, "New_Node"); + /** + * This event will be triggered whenever a datanode is registered with + * SCM with a different Ip or host name. + */ + public static final TypedEvent NODE_IP_OR_HOSTNAME_UPDATE = + new TypedEvent<>(DatanodeDetails.class, "Node_Ip_Or_Hostname_Update"); + /** * This event will be triggered whenever a datanode is moved from healthy to * stale state. diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMService.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMService.java index 4d7c435b1eff..4830e677cc37 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMService.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMService.java @@ -60,6 +60,7 @@ enum ServiceStatus { enum Event { PRE_CHECK_COMPLETED, NEW_NODE_HANDLER_TRIGGERED, + NODE_IP_OR_HOSTNAME_UPDATE_HANDLER_TRIGGERED, UNHEALTHY_TO_HEALTHY_NODE_HANDLER_TRIGGERED } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java new file mode 100644 index 000000000000..38ad202f2bd7 --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java @@ -0,0 +1,77 @@ +package org.apache.hadoop.hdds.scm.node; + +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.scm.ha.SCMService; +import org.apache.hadoop.hdds.scm.ha.SCMServiceManager; +import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; +import org.apache.hadoop.hdds.server.events.EventHandler; +import org.apache.hadoop.hdds.server.events.EventPublisher; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; + +public class NodeIpOrHostnameUpdateHandler implements EventHandler { + private static final Logger LOG = + LoggerFactory.getLogger(NodeIpOrHostnameUpdateHandler.class); + + private final PipelineManager pipelineManager; + private final NodeDecommissionManager decommissionManager; + private final SCMServiceManager serviceManager; + + public NodeIpOrHostnameUpdateHandler(PipelineManager pipelineManager, + NodeDecommissionManager decommissionManager, + SCMServiceManager serviceManager) { + this.pipelineManager = pipelineManager; + this.decommissionManager = decommissionManager; + this.serviceManager = serviceManager; + } + + @Override + public void onMessage(DatanodeDetails datanodeDetails, EventPublisher publisher) { + try { + closeStalePipelines(datanodeDetails); + serviceManager.notifyEventTriggered(SCMService.Event.NODE_IP_OR_HOSTNAME_UPDATE_HANDLER_TRIGGERED); + + if (datanodeDetails.getPersistedOpState() + != HddsProtos.NodeOperationalState.IN_SERVICE) { + decommissionManager.continueAdminForNode(datanodeDetails); + } + } catch (NodeNotFoundException e) { + // Should not happen, as the node has just registered to call this event + // handler. + LOG.warn("NodeNotFound when updating the node Ip or host name to the decommissionManager", + e); + } + } + + /** close the pipelines whose nodes' IPs are stale + * + * @param datanodeDetails new datanodeDetails + */ + private void closeStalePipelines(DatanodeDetails datanodeDetails) { + List pipelines = pipelineManager.getPipelines(); + List pipelinesWithStaleIpOrHostname = + pipelines.stream() + .filter(p -> p.getNodes().stream() + .anyMatch(n -> n.getUuid().equals(datanodeDetails.getUuid()) + && (!n.getIpAddress().equals(datanodeDetails.getIpAddress()) + || !n.getHostName().equals(datanodeDetails.getHostName())))) + .collect(Collectors.toList()); + LOG.info("Pipelines with stale IP or Host name: {}", pipelinesWithStaleIpOrHostname); + pipelinesWithStaleIpOrHostname.forEach(p -> { + try { + LOG.info("Closing pipeline: {}", p.getId()); + pipelineManager.closePipeline(p, false); + LOG.info("Closed pipeline: {}", p.getId()); + } catch (IOException e) { + LOG.error("Close pipeline failed: {}", p, e); + } + }); + } +} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java index ec8b84880cf5..c0fcc8f84809 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java @@ -389,6 +389,26 @@ public void updateLastKnownLayoutVersion(DatanodeDetails datanodeDetails, .updateLastKnownLayoutVersion(layoutInfo); } + /** + * Update node. + * + * @param datanodeDetails the datanode details + * @param layoutInfo the layoutInfo + * @throws NodeNotFoundException the node not found exception + */ + public void updateNode(DatanodeDetails datanodeDetails, LayoutVersionProto layoutInfo) + throws NodeNotFoundException { + DatanodeInfo datanodeInfo = nodeStateMap.getNodeInfo(datanodeDetails.getUuid()); + NodeStatus newNodeStatus = newNodeStatus(datanodeDetails, layoutInfo); + LOG.info("updating node {} from {} to {} with status {}", + datanodeDetails.getUuidString(), + datanodeInfo, + datanodeDetails, + newNodeStatus); + nodeStateMap.updateNode(datanodeDetails, newNodeStatus, layoutInfo); + updateLastKnownLayoutVersion(datanodeDetails, layoutInfo); + } + /** * Returns the current state of the node. * diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java index bef7ab272160..020b84ce02d9 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java @@ -359,33 +359,33 @@ public RegisteredCommand register( .build(); } - if (!isNodeRegistered(datanodeDetails)) { - InetAddress dnAddress = Server.getRemoteIp(); - if (dnAddress != null) { - // Mostly called inside an RPC, update ip and peer hostname - datanodeDetails.setHostName(dnAddress.getHostName()); - datanodeDetails.setIpAddress(dnAddress.getHostAddress()); - } - try { - String dnsName; - String networkLocation; - datanodeDetails.setNetworkName(datanodeDetails.getUuidString()); - if (useHostname) { - dnsName = datanodeDetails.getHostName(); - } else { - dnsName = datanodeDetails.getIpAddress(); - } - networkLocation = nodeResolve(dnsName); - if (networkLocation != null) { - datanodeDetails.setNetworkLocation(networkLocation); - } + InetAddress dnAddress = Server.getRemoteIp(); + if (dnAddress != null) { + // Mostly called inside an RPC, update ip + datanodeDetails.setIpAddress(dnAddress.getHostAddress()); + } + String dnsName; + String networkLocation; + datanodeDetails.setNetworkName(datanodeDetails.getUuidString()); + if (useHostname) { + dnsName = datanodeDetails.getHostName(); + } else { + dnsName = datanodeDetails.getIpAddress(); + } + networkLocation = nodeResolve(dnsName); + if (networkLocation != null) { + datanodeDetails.setNetworkLocation(networkLocation); + } + + if (!isNodeRegistered(datanodeDetails)) { + try{ clusterMap.add(datanodeDetails); nodeStateManager.addNode(datanodeDetails, layoutInfo); // Check that datanode in nodeStateManager has topology parent set DatanodeDetails dn = nodeStateManager.getNode(datanodeDetails); Preconditions.checkState(dn.getParent() != null); - addEntryTodnsToUuidMap(dnsName, datanodeDetails.getUuidString()); + addEntryToDnsToUuidMap(dnsName, datanodeDetails.getUuidString()); // Updating Node Report, as registration is successful processNodeReport(datanodeDetails, nodeReport); LOG.info("Registered Data node : {}", datanodeDetails); @@ -399,6 +399,40 @@ public RegisteredCommand register( LOG.error("Cannot find datanode {} from nodeStateManager", datanodeDetails.toString()); } + } else { + // Update datanode if it is registered but the ip or hostname changes + try { + final DatanodeInfo datanodeInfo = nodeStateManager.getNode(datanodeDetails); + if (!datanodeInfo.getIpAddress().equals(datanodeDetails.getIpAddress()) + || !datanodeInfo.getHostName().equals(datanodeDetails.getHostName())) { + LOG.info("Updating data node {} from {} to {}", + datanodeDetails.getUuidString(), + datanodeInfo, + datanodeDetails); + if (clusterMap.contains(datanodeInfo)) { + clusterMap.remove(datanodeInfo); + } + clusterMap.add(datanodeDetails); + + String oldDnsName; + if (useHostname) { + oldDnsName = datanodeInfo.getHostName(); + } else { + oldDnsName = datanodeInfo.getIpAddress(); + } + removeEntryFromDnsToUuidMap(oldDnsName); + addEntryToDnsToUuidMap(dnsName, datanodeDetails.getUuidString()); + + nodeStateManager.updateNode(datanodeDetails, layoutInfo); + DatanodeDetails dn = nodeStateManager.getNode(datanodeDetails); + Preconditions.checkState(dn.getParent() != null); + processNodeReport(datanodeDetails, nodeReport); + LOG.info("Updated Datanode to: {}", dn); + scmNodeEventPublisher.fireEvent(SCMEvents.NODE_IP_OR_HOSTNAME_UPDATE, dn); + } + } catch (NodeNotFoundException e) { + LOG.error("Cannot find datanode {} from nodeStateManager", datanodeDetails); + } } return RegisteredCommand.newBuilder().setErrorCode(ErrorCode.success) @@ -415,11 +449,9 @@ public RegisteredCommand register( * @param dnsName String representing the hostname or IP of the node * @param uuid String representing the UUID of the registered node. */ - @SuppressFBWarnings(value = "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION", - justification = "The method is synchronized and this is the only place " + - "dnsToUuidMap is modified") - private synchronized void addEntryTodnsToUuidMap( - String dnsName, String uuid) { + @SuppressFBWarnings(value = "AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION") + private synchronized void addEntryToDnsToUuidMap( + String dnsName, String uuid) { Set dnList = dnsToUuidMap.get(dnsName); if (dnList == null) { dnList = ConcurrentHashMap.newKeySet(); @@ -428,6 +460,19 @@ private synchronized void addEntryTodnsToUuidMap( dnList.add(uuid); } + private synchronized void removeEntryFromDnsToUuidMap(String dnsName) { + if (!dnsToUuidMap.contains(dnsName)) { + return; + } + Set dnSet = dnsToUuidMap.get(dnsName); + if (dnSet.contains(dnsName)) { + dnSet.remove(dnsName); + } + if (dnSet.isEmpty()) { + dnsToUuidMap.remove(dnsName); + } + } + /** * Send heartbeat to indicate the datanode is alive and doing well. * diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeStateMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeStateMap.java index 0d8580dde7c1..9531c9bda9f4 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeStateMap.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeStateMap.java @@ -91,6 +91,31 @@ public void addNode(DatanodeDetails datanodeDetails, NodeStatus nodeStatus, } } + /** + * Update a node in NodeStateMap. + * + * @param datanodeDetails DatanodeDetails + * @param nodeStatus initial NodeStatus + * @param layoutInfo initial LayoutVersionProto + * + */ + public void updateNode(DatanodeDetails datanodeDetails, NodeStatus nodeStatus, + LayoutVersionProto layoutInfo) + + throws NodeNotFoundException { + lock.writeLock().lock(); + try { + UUID id = datanodeDetails.getUuid(); + if (!nodeMap.containsKey(id)) { + throw new NodeNotFoundException("Node UUID: " + id); + } + nodeMap.put(id, new DatanodeInfo(datanodeDetails, nodeStatus, + layoutInfo)); + } finally { + lock.writeLock().unlock(); + } + } + /** * Updates the node health state. * diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java index 86a4061bfec3..50ffdbdee338 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java @@ -46,6 +46,7 @@ import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.STAND_ALONE; +import static org.apache.hadoop.hdds.scm.ha.SCMService.Event.NODE_IP_OR_HOSTNAME_UPDATE_HANDLER_TRIGGERED; import static org.apache.hadoop.hdds.scm.ha.SCMService.Event.UNHEALTHY_TO_HEALTHY_NODE_HANDLER_TRIGGERED; import static org.apache.hadoop.hdds.scm.ha.SCMService.Event.NEW_NODE_HANDLER_TRIGGERED; import static org.apache.hadoop.hdds.scm.ha.SCMService.Event.PRE_CHECK_COMPLETED; @@ -66,7 +67,7 @@ public class BackgroundPipelineCreator implements SCMService { * SCMService related variables. * 1) after leaving safe mode, BackgroundPipelineCreator needs to * wait for a while before really take effect. - * 2) NewNodeHandler, NonHealthyToHealthyNodeHandler, PreCheckComplete + * 2) NewNodeHandler, NodeIpOrHostnameUpdateHandler, NonHealthyToHealthyNodeHandler, PreCheckComplete * will trigger a one-shot run of BackgroundPipelineCreator, * no matter in safe mode or not. */ @@ -280,8 +281,9 @@ public void notifyEventTriggered(Event event) { return; } if (event == NEW_NODE_HANDLER_TRIGGERED - || event == UNHEALTHY_TO_HEALTHY_NODE_HANDLER_TRIGGERED - || event == PRE_CHECK_COMPLETED) { + || event == NODE_IP_OR_HOSTNAME_UPDATE_HANDLER_TRIGGERED + || event == UNHEALTHY_TO_HEALTHY_NODE_HANDLER_TRIGGERED + || event == PRE_CHECK_COMPLETED) { LOG.info("trigger a one-shot run on {}.", THREAD_NAME); serviceLock.lock(); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java index 05ac12cea0d7..151879149aa3 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java @@ -55,6 +55,7 @@ import org.apache.hadoop.hdds.scm.ha.SCMHAUtils; import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator; import org.apache.hadoop.hdds.scm.ScmInfo; +import org.apache.hadoop.hdds.scm.node.NodeIpOrHostnameUpdateHandler; import org.apache.hadoop.hdds.scm.server.upgrade.ScmHAUnfinalizedStateValidationAction; import org.apache.hadoop.hdds.scm.pipeline.WritableContainerFactory; import org.apache.hadoop.hdds.security.token.ContainerTokenGenerator; @@ -410,6 +411,8 @@ private void initializeEventHandlers() { NewNodeHandler newNodeHandler = new NewNodeHandler(pipelineManager, scmDecommissionManager, configuration, serviceManager); + NodeIpOrHostnameUpdateHandler nodeIpOrHostnameUpdateHandler = new NodeIpOrHostnameUpdateHandler(pipelineManager, + scmDecommissionManager, serviceManager); StaleNodeHandler staleNodeHandler = new StaleNodeHandler(scmNodeManager, pipelineManager, configuration); DeadNodeHandler deadNodeHandler = new DeadNodeHandler(scmNodeManager, @@ -471,6 +474,7 @@ private void initializeEventHandlers() { eventQueue.addHandler(SCMEvents.CONTAINER_ACTIONS, actionsHandler); eventQueue.addHandler(SCMEvents.CLOSE_CONTAINER, closeContainerHandler); eventQueue.addHandler(SCMEvents.NEW_NODE, newNodeHandler); + eventQueue.addHandler(SCMEvents.NODE_IP_OR_HOSTNAME_UPDATE, nodeIpOrHostnameUpdateHandler); eventQueue.addHandler(SCMEvents.STALE_NODE, staleNodeHandler); eventQueue.addHandler(SCMEvents.HEALTHY_READONLY_TO_HEALTHY_NODE, readOnlyHealthyToHealthyNodeHandler); @@ -1152,6 +1156,8 @@ private static InetSocketAddress getScmAddress(SCMHANodeDetails haDetails, Preconditions.checkNotNull(scmNodeInfoList, "scmNodeInfoList is null"); InetSocketAddress scmAddress = null; + LOG.info("!!!!!!!!!!!!!!!!! haDetails: {}", haDetails); + LOG.info("!!!!!!!!!!!!!!!!! haDetails.getLocalNodeDetails(): {}", haDetails.getLocalNodeDetails()); if (SCMHAUtils.getScmServiceId(conf) != null) { for (SCMNodeInfo scmNodeInfo : scmNodeInfoList) { if (haDetails.getLocalNodeDetails().getNodeId() != null diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeStateManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeStateManager.java index 7dfcbeb349f5..4c034205d3e1 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeStateManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeStateManager.java @@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; import org.apache.hadoop.hdds.scm.container.ContainerID; import org.apache.hadoop.hdds.upgrade.HDDSLayoutVersionManager; import org.apache.hadoop.hdds.utils.HddsServerUtil; @@ -306,6 +307,40 @@ public void testHealthEventsFiredWhenOpStateChanged() } } + @Test + public void testUpdateNode() throws NodeAlreadyExistsException, NodeNotFoundException { + UUID dnUuid = UUID.randomUUID(); + String ipAddress = "1.2.3.4"; + String hostName = "test-host"; + StorageContainerDatanodeProtocolProtos.LayoutVersionProto layoutVersionProto = + UpgradeUtils.toLayoutVersionProto(1, 2); + DatanodeDetails dn = DatanodeDetails.newBuilder() + .setUuid(dnUuid) + .setIpAddress(ipAddress) + .setHostName(hostName) + .setPersistedOpState(HddsProtos.NodeOperationalState.IN_MAINTENANCE) + .build(); + nsm.addNode(dn, layoutVersionProto); + + String newIpAddress = "2.3.4.5"; + String newHostName = "new-host"; + StorageContainerDatanodeProtocolProtos.LayoutVersionProto newLayoutVersionProto = + UpgradeUtils.defaultLayoutVersionProto(); + DatanodeDetails newDn = DatanodeDetails.newBuilder() + .setUuid(dnUuid) + .setIpAddress(newIpAddress) + .setHostName(newHostName) + .setPersistedOpState(HddsProtos.NodeOperationalState.IN_SERVICE) + .build(); + nsm.updateNode(newDn, newLayoutVersionProto); + + DatanodeInfo updatedDn = nsm.getNode(dn); + assertEquals(newIpAddress, updatedDn.getIpAddress()); + assertEquals(newHostName, updatedDn.getHostName()); + assertEquals(HddsProtos.NodeOperationalState.IN_SERVICE, updatedDn.getPersistedOpState()); + assertEquals(NodeStatus.inServiceHealthy(), updatedDn.getNodeStatus()); + } + private DatanodeDetails generateDatanode() { return DatanodeDetails.newBuilder().setUuid(UUID.randomUUID()).build(); } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java index c399229b53b4..573086982467 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java @@ -1862,4 +1862,69 @@ private void testGetNodesByAddress(boolean useHostname) } } + /** + * Test node register with updated IP and host name. + */ + @Test + public void testScmRegisterNodeWithUpdatedIpAndHostname() + throws IOException, InterruptedException, AuthenticationException { + OzoneConfiguration conf = getConf(); + conf.setTimeDuration(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL, 1000, + MILLISECONDS); + + // create table mapping file + String hostName = "host1"; + String ipAddress = "1.2.3.4"; + String mapFile = this.getClass().getClassLoader() + .getResource("nodegroup-mapping").getPath(); + conf.set(NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, + "org.apache.hadoop.net.TableMapping"); + conf.set(NET_TOPOLOGY_TABLE_MAPPING_FILE_KEY, mapFile); + conf.set(ScmConfigKeys.OZONE_SCM_NETWORK_TOPOLOGY_SCHEMA_FILE, + "network-topology-nodegroup.xml"); + + // use default IP address to resolve node + try (SCMNodeManager nodeManager = createNodeManager(conf)) { + String nodeUuid = UUID.randomUUID().toString(); + DatanodeDetails node = createDatanodeDetails( + nodeUuid, hostName, ipAddress, null); + nodeManager.register(node, null, null); + + // verify network topology cluster has all the registered nodes + Thread.sleep(2 * 1000); + NetworkTopology clusterMap = scm.getClusterMap(); + assertEquals(1, + nodeManager.getNodeCount(NodeStatus.inServiceHealthy())); + assertEquals(1, clusterMap.getNumOfLeafNode("")); + assertEquals(4, clusterMap.getMaxLevel()); + List nodeList = nodeManager.getAllNodes(); + assertEquals(1, nodeList.size()); + + DatanodeDetails returnedNode = nodeList.get(0); + assertEquals(ipAddress, returnedNode.getIpAddress()); + assertEquals(hostName, returnedNode.getHostName()); + Assert.assertTrue(returnedNode.getNetworkLocation().startsWith("/rack1/ng")); + Assert.assertTrue(returnedNode.getParent() != null); + + // test updating ip address and host name + String updatedIpAddress = "2.3.4.5"; + String updatedHostName = "host2"; + DatanodeDetails updatedNode = createDatanodeDetails( + nodeUuid, updatedHostName, updatedIpAddress, null); + nodeManager.register(updatedNode, null, null); + + assertEquals(1, + nodeManager.getNodeCount(NodeStatus.inServiceHealthy())); + assertEquals(1, clusterMap.getNumOfLeafNode("")); + assertEquals(4, clusterMap.getMaxLevel()); + List updatedNodeList = nodeManager.getAllNodes(); + assertEquals(1, updatedNodeList.size()); + + DatanodeDetails returnedUpdatedNode = updatedNodeList.get(0); + assertEquals(updatedIpAddress, returnedUpdatedNode.getIpAddress()); + assertEquals(updatedHostName, returnedUpdatedNode.getHostName()); + Assert.assertTrue(returnedUpdatedNode.getNetworkLocation().startsWith("/rack1/ng")); + Assert.assertTrue(returnedUpdatedNode.getParent() != null); + } + } } \ No newline at end of file From cb4adb98ad58fdd1708ac22825c0565bc1bf30e4 Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Mon, 14 Mar 2022 10:56:23 -0700 Subject: [PATCH 02/29] fix rat and checkstyle errors --- .../apache/hadoop/hdds/ratis/RatisHelper.java | 4 +- .../hadoop/hdds/server/events/EventQueue.java | 11 +- .../node/NodeIpOrHostnameUpdateHandler.java | 133 +++++++++++------- .../hdds/scm/node/NodeStateManager.java | 6 +- .../hadoop/hdds/scm/node/SCMNodeManager.java | 14 +- .../pipeline/BackgroundPipelineCreator.java | 3 +- .../scm/server/StorageContainerManager.java | 10 +- .../hdds/scm/node/TestNodeStateManager.java | 13 +- .../hdds/scm/node/TestSCMNodeManager.java | 6 +- 9 files changed, 122 insertions(+), 78 deletions(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java index c8c67af9eaa9..9a7a8e49c4fc 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java @@ -66,7 +66,7 @@ public final class RatisHelper { private static final Logger LOG = LoggerFactory.getLogger(RatisHelper.class); - private static final OzoneConfiguration conf = new OzoneConfiguration(); + private static final OzoneConfiguration CONF = new OzoneConfiguration(); // Prefix for Ratis Server GRPC and Ratis client conf. public static final String HDDS_DATANODE_RATIS_PREFIX_KEY = "hdds.ratis"; @@ -334,7 +334,7 @@ public static Long getMinReplicatedIndex( } private static boolean datanodeUseHostName() { - return conf.getBoolean( + return CONF.getBoolean( DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME, DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT); } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java index 155187999c90..a0b8ac95539b 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventQueue.java @@ -67,12 +67,15 @@ public class EventQueue implements EventPublisher, AutoCloseable { private boolean isSilent = false; - // The field parent in DatanodeDetails class has the circular reference which will result in Gson infinite recursive - // parsing. We need to exclude this field when generating json string for DatanodeDetails object - static class DatanodeDetailsGsonExclusionStrategy implements ExclusionStrategy { + // The field parent in DatanodeDetails class has the circular reference + // which will result in Gson infinite recursive parsing. We need to exclude + // this field when generating json string for DatanodeDetails object + static class DatanodeDetailsGsonExclusionStrategy + implements ExclusionStrategy { @Override public boolean shouldSkipField(FieldAttributes f) { - return f.getDeclaringClass() == NodeImpl.class && f.getName().equals("parent"); + return f.getDeclaringClass() == NodeImpl.class + && f.getName().equals("parent"); } @Override diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java index 38ad202f2bd7..b559035fc2ed 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java @@ -1,3 +1,19 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ package org.apache.hadoop.hdds.scm.node; import org.apache.hadoop.hdds.protocol.DatanodeDetails; @@ -16,62 +32,75 @@ import java.util.List; import java.util.stream.Collectors; -public class NodeIpOrHostnameUpdateHandler implements EventHandler { - private static final Logger LOG = - LoggerFactory.getLogger(NodeIpOrHostnameUpdateHandler.class); +/** + * Handles datanode ip or hostname change event. + */ +public class NodeIpOrHostnameUpdateHandler + implements EventHandler { + private static final Logger LOG = + LoggerFactory.getLogger(NodeIpOrHostnameUpdateHandler.class); - private final PipelineManager pipelineManager; - private final NodeDecommissionManager decommissionManager; - private final SCMServiceManager serviceManager; + private final PipelineManager pipelineManager; + private final NodeDecommissionManager decommissionManager; + private final SCMServiceManager serviceManager; - public NodeIpOrHostnameUpdateHandler(PipelineManager pipelineManager, - NodeDecommissionManager decommissionManager, - SCMServiceManager serviceManager) { - this.pipelineManager = pipelineManager; - this.decommissionManager = decommissionManager; - this.serviceManager = serviceManager; - } + public NodeIpOrHostnameUpdateHandler(PipelineManager pipelineManager, + NodeDecommissionManager decommissionManager, + SCMServiceManager serviceManager) { + this.pipelineManager = pipelineManager; + this.decommissionManager = decommissionManager; + this.serviceManager = serviceManager; + } - @Override - public void onMessage(DatanodeDetails datanodeDetails, EventPublisher publisher) { - try { - closeStalePipelines(datanodeDetails); - serviceManager.notifyEventTriggered(SCMService.Event.NODE_IP_OR_HOSTNAME_UPDATE_HANDLER_TRIGGERED); + @Override + public void onMessage(DatanodeDetails datanodeDetails, + EventPublisher publisher) { + try { + closeStalePipelines(datanodeDetails); + serviceManager.notifyEventTriggered(SCMService.Event + .NODE_IP_OR_HOSTNAME_UPDATE_HANDLER_TRIGGERED); - if (datanodeDetails.getPersistedOpState() - != HddsProtos.NodeOperationalState.IN_SERVICE) { - decommissionManager.continueAdminForNode(datanodeDetails); - } - } catch (NodeNotFoundException e) { - // Should not happen, as the node has just registered to call this event - // handler. - LOG.warn("NodeNotFound when updating the node Ip or host name to the decommissionManager", - e); - } + if (datanodeDetails.getPersistedOpState() + != HddsProtos.NodeOperationalState.IN_SERVICE) { + decommissionManager.continueAdminForNode(datanodeDetails); + } + } catch (NodeNotFoundException e) { + // Should not happen, as the node has just registered to call this event + // handler. + LOG.warn( + "NodeNotFound when updating the node Ip or host name to the " + + "decommissionManager", + e); } + } - /** close the pipelines whose nodes' IPs are stale - * - * @param datanodeDetails new datanodeDetails - */ - private void closeStalePipelines(DatanodeDetails datanodeDetails) { - List pipelines = pipelineManager.getPipelines(); - List pipelinesWithStaleIpOrHostname = - pipelines.stream() - .filter(p -> p.getNodes().stream() - .anyMatch(n -> n.getUuid().equals(datanodeDetails.getUuid()) - && (!n.getIpAddress().equals(datanodeDetails.getIpAddress()) - || !n.getHostName().equals(datanodeDetails.getHostName())))) - .collect(Collectors.toList()); - LOG.info("Pipelines with stale IP or Host name: {}", pipelinesWithStaleIpOrHostname); - pipelinesWithStaleIpOrHostname.forEach(p -> { - try { - LOG.info("Closing pipeline: {}", p.getId()); - pipelineManager.closePipeline(p, false); - LOG.info("Closed pipeline: {}", p.getId()); - } catch (IOException e) { - LOG.error("Close pipeline failed: {}", p, e); - } - }); - } + /** + * close the pipelines whose nodes' IPs are stale. + * + * @param datanodeDetails new datanodeDetails + */ + private void closeStalePipelines(DatanodeDetails datanodeDetails) { + List pipelines = pipelineManager.getPipelines(); + List pipelinesWithStaleIpOrHostname = + pipelines.stream() + .filter(p -> p.getNodes().stream() + .anyMatch(n -> n.getUuid() + .equals(datanodeDetails.getUuid()) + && (!n.getIpAddress() + .equals(datanodeDetails.getIpAddress()) + || !n.getHostName() + .equals(datanodeDetails.getHostName())))) + .collect(Collectors.toList()); + LOG.info("Pipelines with stale IP or Host name: {}", + pipelinesWithStaleIpOrHostname); + pipelinesWithStaleIpOrHostname.forEach(p -> { + try { + LOG.info("Closing pipeline: {}", p.getId()); + pipelineManager.closePipeline(p, false); + LOG.info("Closed pipeline: {}", p.getId()); + } catch (IOException e) { + LOG.error("Close pipeline failed: {}", p, e); + } + }); + } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java index c0fcc8f84809..5a4b0837293a 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java @@ -396,9 +396,11 @@ public void updateLastKnownLayoutVersion(DatanodeDetails datanodeDetails, * @param layoutInfo the layoutInfo * @throws NodeNotFoundException the node not found exception */ - public void updateNode(DatanodeDetails datanodeDetails, LayoutVersionProto layoutInfo) + public void updateNode(DatanodeDetails datanodeDetails, + LayoutVersionProto layoutInfo) throws NodeNotFoundException { - DatanodeInfo datanodeInfo = nodeStateMap.getNodeInfo(datanodeDetails.getUuid()); + DatanodeInfo datanodeInfo = + nodeStateMap.getNodeInfo(datanodeDetails.getUuid()); NodeStatus newNodeStatus = newNodeStatus(datanodeDetails, layoutInfo); LOG.info("updating node {} from {} to {} with status {}", datanodeDetails.getUuidString(), diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java index 020b84ce02d9..770abd32969c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java @@ -379,7 +379,7 @@ public RegisteredCommand register( } if (!isNodeRegistered(datanodeDetails)) { - try{ + try { clusterMap.add(datanodeDetails); nodeStateManager.addNode(datanodeDetails, layoutInfo); // Check that datanode in nodeStateManager has topology parent set @@ -402,9 +402,11 @@ public RegisteredCommand register( } else { // Update datanode if it is registered but the ip or hostname changes try { - final DatanodeInfo datanodeInfo = nodeStateManager.getNode(datanodeDetails); + final DatanodeInfo datanodeInfo = + nodeStateManager.getNode(datanodeDetails); if (!datanodeInfo.getIpAddress().equals(datanodeDetails.getIpAddress()) - || !datanodeInfo.getHostName().equals(datanodeDetails.getHostName())) { + || !datanodeInfo.getHostName() + .equals(datanodeDetails.getHostName())) { LOG.info("Updating data node {} from {} to {}", datanodeDetails.getUuidString(), datanodeInfo, @@ -428,10 +430,12 @@ public RegisteredCommand register( Preconditions.checkState(dn.getParent() != null); processNodeReport(datanodeDetails, nodeReport); LOG.info("Updated Datanode to: {}", dn); - scmNodeEventPublisher.fireEvent(SCMEvents.NODE_IP_OR_HOSTNAME_UPDATE, dn); + scmNodeEventPublisher + .fireEvent(SCMEvents.NODE_IP_OR_HOSTNAME_UPDATE, dn); } } catch (NodeNotFoundException e) { - LOG.error("Cannot find datanode {} from nodeStateManager", datanodeDetails); + LOG.error("Cannot find datanode {} from nodeStateManager", + datanodeDetails); } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java index 50ffdbdee338..9d2c925578e8 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java @@ -67,7 +67,8 @@ public class BackgroundPipelineCreator implements SCMService { * SCMService related variables. * 1) after leaving safe mode, BackgroundPipelineCreator needs to * wait for a while before really take effect. - * 2) NewNodeHandler, NodeIpOrHostnameUpdateHandler, NonHealthyToHealthyNodeHandler, PreCheckComplete + * 2) NewNodeHandler, NodeIpOrHostnameUpdateHandler, + * NonHealthyToHealthyNodeHandler, PreCheckComplete * will trigger a one-shot run of BackgroundPipelineCreator, * no matter in safe mode or not. */ diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java index 151879149aa3..cd6d7cdf5883 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java @@ -411,8 +411,9 @@ private void initializeEventHandlers() { NewNodeHandler newNodeHandler = new NewNodeHandler(pipelineManager, scmDecommissionManager, configuration, serviceManager); - NodeIpOrHostnameUpdateHandler nodeIpOrHostnameUpdateHandler = new NodeIpOrHostnameUpdateHandler(pipelineManager, - scmDecommissionManager, serviceManager); + NodeIpOrHostnameUpdateHandler nodeIpOrHostnameUpdateHandler = + new NodeIpOrHostnameUpdateHandler(pipelineManager, + scmDecommissionManager, serviceManager); StaleNodeHandler staleNodeHandler = new StaleNodeHandler(scmNodeManager, pipelineManager, configuration); DeadNodeHandler deadNodeHandler = new DeadNodeHandler(scmNodeManager, @@ -474,7 +475,8 @@ private void initializeEventHandlers() { eventQueue.addHandler(SCMEvents.CONTAINER_ACTIONS, actionsHandler); eventQueue.addHandler(SCMEvents.CLOSE_CONTAINER, closeContainerHandler); eventQueue.addHandler(SCMEvents.NEW_NODE, newNodeHandler); - eventQueue.addHandler(SCMEvents.NODE_IP_OR_HOSTNAME_UPDATE, nodeIpOrHostnameUpdateHandler); + eventQueue.addHandler(SCMEvents.NODE_IP_OR_HOSTNAME_UPDATE, + nodeIpOrHostnameUpdateHandler); eventQueue.addHandler(SCMEvents.STALE_NODE, staleNodeHandler); eventQueue.addHandler(SCMEvents.HEALTHY_READONLY_TO_HEALTHY_NODE, readOnlyHealthyToHealthyNodeHandler); @@ -1156,8 +1158,6 @@ private static InetSocketAddress getScmAddress(SCMHANodeDetails haDetails, Preconditions.checkNotNull(scmNodeInfoList, "scmNodeInfoList is null"); InetSocketAddress scmAddress = null; - LOG.info("!!!!!!!!!!!!!!!!! haDetails: {}", haDetails); - LOG.info("!!!!!!!!!!!!!!!!! haDetails.getLocalNodeDetails(): {}", haDetails.getLocalNodeDetails()); if (SCMHAUtils.getScmServiceId(conf) != null) { for (SCMNodeInfo scmNodeInfo : scmNodeInfoList) { if (haDetails.getLocalNodeDetails().getNodeId() != null diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeStateManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeStateManager.java index 4c034205d3e1..38bb1ce584d3 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeStateManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeStateManager.java @@ -308,11 +308,13 @@ public void testHealthEventsFiredWhenOpStateChanged() } @Test - public void testUpdateNode() throws NodeAlreadyExistsException, NodeNotFoundException { + public void testUpdateNode() throws NodeAlreadyExistsException, + NodeNotFoundException { UUID dnUuid = UUID.randomUUID(); String ipAddress = "1.2.3.4"; String hostName = "test-host"; - StorageContainerDatanodeProtocolProtos.LayoutVersionProto layoutVersionProto = + StorageContainerDatanodeProtocolProtos.LayoutVersionProto + layoutVersionProto = UpgradeUtils.toLayoutVersionProto(1, 2); DatanodeDetails dn = DatanodeDetails.newBuilder() .setUuid(dnUuid) @@ -324,8 +326,8 @@ public void testUpdateNode() throws NodeAlreadyExistsException, NodeNotFoundExce String newIpAddress = "2.3.4.5"; String newHostName = "new-host"; - StorageContainerDatanodeProtocolProtos.LayoutVersionProto newLayoutVersionProto = - UpgradeUtils.defaultLayoutVersionProto(); + StorageContainerDatanodeProtocolProtos.LayoutVersionProto + newLayoutVersionProto = UpgradeUtils.defaultLayoutVersionProto(); DatanodeDetails newDn = DatanodeDetails.newBuilder() .setUuid(dnUuid) .setIpAddress(newIpAddress) @@ -337,7 +339,8 @@ public void testUpdateNode() throws NodeAlreadyExistsException, NodeNotFoundExce DatanodeInfo updatedDn = nsm.getNode(dn); assertEquals(newIpAddress, updatedDn.getIpAddress()); assertEquals(newHostName, updatedDn.getHostName()); - assertEquals(HddsProtos.NodeOperationalState.IN_SERVICE, updatedDn.getPersistedOpState()); + assertEquals(HddsProtos.NodeOperationalState.IN_SERVICE, + updatedDn.getPersistedOpState()); assertEquals(NodeStatus.inServiceHealthy(), updatedDn.getNodeStatus()); } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java index 573086982467..006c5f0e815a 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java @@ -1903,7 +1903,8 @@ public void testScmRegisterNodeWithUpdatedIpAndHostname() DatanodeDetails returnedNode = nodeList.get(0); assertEquals(ipAddress, returnedNode.getIpAddress()); assertEquals(hostName, returnedNode.getHostName()); - Assert.assertTrue(returnedNode.getNetworkLocation().startsWith("/rack1/ng")); + Assert.assertTrue(returnedNode.getNetworkLocation() + .startsWith("/rack1/ng")); Assert.assertTrue(returnedNode.getParent() != null); // test updating ip address and host name @@ -1923,7 +1924,8 @@ public void testScmRegisterNodeWithUpdatedIpAndHostname() DatanodeDetails returnedUpdatedNode = updatedNodeList.get(0); assertEquals(updatedIpAddress, returnedUpdatedNode.getIpAddress()); assertEquals(updatedHostName, returnedUpdatedNode.getHostName()); - Assert.assertTrue(returnedUpdatedNode.getNetworkLocation().startsWith("/rack1/ng")); + Assert.assertTrue(returnedUpdatedNode.getNetworkLocation() + .startsWith("/rack1/ng")); Assert.assertTrue(returnedUpdatedNode.getParent() != null); } } From e51aa4b7fcc8bd60ab1602307df094aee7703126 Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Tue, 15 Mar 2022 08:45:50 -0700 Subject: [PATCH 03/29] fix findbugs errors --- .../java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java index 770abd32969c..c89460347f7a 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java @@ -465,7 +465,7 @@ private synchronized void addEntryToDnsToUuidMap( } private synchronized void removeEntryFromDnsToUuidMap(String dnsName) { - if (!dnsToUuidMap.contains(dnsName)) { + if (!dnsToUuidMap.containsKey(dnsName)) { return; } Set dnSet = dnsToUuidMap.get(dnsName); From 50ded1c1c0cab54c5eaa101cebf22b4be0ad09c5 Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Wed, 30 Mar 2022 10:54:12 -0700 Subject: [PATCH 04/29] solve a edge case bug --- .../hadoop/hdds/scm/node/NewNodeHandler.java | 1 + .../node/NodeIpOrHostnameUpdateHandler.java | 37 +---- .../hdds/scm/pipeline/PipelineManager.java | 2 + .../scm/pipeline/PipelineManagerImpl.java | 35 +++++ .../scm/pipeline/MockPipelineManager.java | 5 + .../scm/pipeline/TestPipelineManagerImpl.java | 142 ++++++++++++++---- 6 files changed, 154 insertions(+), 68 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NewNodeHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NewNodeHandler.java index 674cf2dfcc7f..612ab048c7c3 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NewNodeHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NewNodeHandler.java @@ -56,6 +56,7 @@ public NewNodeHandler(PipelineManager pipelineManager, public void onMessage(DatanodeDetails datanodeDetails, EventPublisher publisher) { try { + pipelineManager.closeStalePipelines(datanodeDetails); serviceManager.notifyEventTriggered(Event.NEW_NODE_HANDLER_TRIGGERED); if (datanodeDetails.getPersistedOpState() diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java index b559035fc2ed..f9ae0e8962bf 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java @@ -21,17 +21,12 @@ import org.apache.hadoop.hdds.scm.ha.SCMService; import org.apache.hadoop.hdds.scm.ha.SCMServiceManager; import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; -import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; import org.apache.hadoop.hdds.server.events.EventHandler; import org.apache.hadoop.hdds.server.events.EventPublisher; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.List; -import java.util.stream.Collectors; - /** * Handles datanode ip or hostname change event. */ @@ -56,7 +51,7 @@ public NodeIpOrHostnameUpdateHandler(PipelineManager pipelineManager, public void onMessage(DatanodeDetails datanodeDetails, EventPublisher publisher) { try { - closeStalePipelines(datanodeDetails); + pipelineManager.closeStalePipelines(datanodeDetails); serviceManager.notifyEventTriggered(SCMService.Event .NODE_IP_OR_HOSTNAME_UPDATE_HANDLER_TRIGGERED); @@ -73,34 +68,4 @@ public void onMessage(DatanodeDetails datanodeDetails, e); } } - - /** - * close the pipelines whose nodes' IPs are stale. - * - * @param datanodeDetails new datanodeDetails - */ - private void closeStalePipelines(DatanodeDetails datanodeDetails) { - List pipelines = pipelineManager.getPipelines(); - List pipelinesWithStaleIpOrHostname = - pipelines.stream() - .filter(p -> p.getNodes().stream() - .anyMatch(n -> n.getUuid() - .equals(datanodeDetails.getUuid()) - && (!n.getIpAddress() - .equals(datanodeDetails.getIpAddress()) - || !n.getHostName() - .equals(datanodeDetails.getHostName())))) - .collect(Collectors.toList()); - LOG.info("Pipelines with stale IP or Host name: {}", - pipelinesWithStaleIpOrHostname); - pipelinesWithStaleIpOrHostname.forEach(p -> { - try { - LOG.info("Closing pipeline: {}", p.getId()); - pipelineManager.closePipeline(p, false); - LOG.info("Closed pipeline: {}", p.getId()); - } catch (IOException e) { - LOG.error("Close pipeline failed: {}", p, e); - } - }); - } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java index b96d616ca89a..1438b49525e6 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java @@ -90,6 +90,8 @@ NavigableSet getContainersInPipeline(PipelineID pipelineID) void closePipeline(Pipeline pipeline, boolean onTimeout) throws IOException; + void closeStalePipelines(DatanodeDetails datanodeDetails); + void scrubPipeline(ReplicationConfig replicationConfig) throws IOException; diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java index 21681384db4f..b8899fbfbb28 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java @@ -59,6 +59,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.Collectors; /** * SCM Pipeline Manager implementation. @@ -374,6 +375,40 @@ public void closePipeline(Pipeline pipeline, boolean onTimeout) } } + /** close the pipelines whose nodes' IPs are stale + * + * @param datanodeDetails new datanodeDetails + */ + @Override + public void closeStalePipelines(DatanodeDetails datanodeDetails) { + List pipelinesWithStaleIpOrHostname = getStalePipelines(datanodeDetails); + if (pipelinesWithStaleIpOrHostname.isEmpty()) { + LOG.info("No stale pipelines"); + return; + } + LOG.info("Pipelines with stale IP or Host name: {}", pipelinesWithStaleIpOrHostname); + pipelinesWithStaleIpOrHostname.forEach(p -> { + try { + LOG.info("Closing pipeline: {}", p.getId()); + closePipeline(p, false); + LOG.info("Closed pipeline: {}", p.getId()); + } catch (IOException e) { + LOG.error("Close pipeline failed: {}", p, e); + } + }); + } + + @VisibleForTesting + List getStalePipelines(DatanodeDetails datanodeDetails) { + List pipelines = getPipelines(); + return pipelines.stream() + .filter(p -> p.getNodes().stream() + .anyMatch(n -> n.getUuid().equals(datanodeDetails.getUuid()) + && (!n.getIpAddress().equals(datanodeDetails.getIpAddress()) + || !n.getHostName().equals(datanodeDetails.getHostName())))) + .collect(Collectors.toList()); + } + /** * Scrub pipelines. */ diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java index c82006ee73bd..1bc42731c7de 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java @@ -167,6 +167,11 @@ public void closePipeline(final Pipeline pipeline, final boolean onTimeout) throws IOException { } + @Override + public void closeStalePipelines(DatanodeDetails datanodeDetails) { + + } + @Override public void scrubPipeline(ReplicationConfig replicationConfig) throws IOException { diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java index 3a68db18622e..c2f9ad68a965 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java @@ -67,8 +67,13 @@ import static org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState.ALLOCATED; import static org.apache.hadoop.test.MetricsAsserts.getLongCounter; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; /** * Tests for PipelineManagerImpl. @@ -145,12 +150,12 @@ public void testCreatePipeline() throws Exception { Assert.assertTrue(pipelineManager.getPipelines().isEmpty()); Pipeline pipeline1 = pipelineManager.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.THREE)); - Assert.assertEquals(1, pipelineManager.getPipelines().size()); + assertEquals(1, pipelineManager.getPipelines().size()); Assert.assertTrue(pipelineManager.containsPipeline(pipeline1.getId())); Pipeline pipeline2 = pipelineManager.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.ONE)); - Assert.assertEquals(2, pipelineManager.getPipelines().size()); + assertEquals(2, pipelineManager.getPipelines().size()); Assert.assertTrue(pipelineManager.containsPipeline(pipeline2.getId())); buffer1.close(); pipelineManager.close(); @@ -161,11 +166,11 @@ public void testCreatePipeline() throws Exception { createPipelineManager(true, buffer2); // Should be able to load previous pipelines. Assert.assertFalse(pipelineManager2.getPipelines().isEmpty()); - Assert.assertEquals(2, pipelineManager.getPipelines().size()); + assertEquals(2, pipelineManager.getPipelines().size()); Pipeline pipeline3 = pipelineManager2.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.THREE)); buffer2.close(); - Assert.assertEquals(3, pipelineManager2.getPipelines().size()); + assertEquals(3, pipelineManager2.getPipelines().size()); Assert.assertTrue(pipelineManager2.containsPipeline(pipeline3.getId())); pipelineManager2.close(); @@ -196,11 +201,11 @@ public void testUpdatePipelineStates() throws Exception { SCMDBDefinition.PIPELINES.getTable(dbStore); Pipeline pipeline = pipelineManager.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.THREE)); - Assert.assertEquals(1, pipelineManager.getPipelines().size()); + assertEquals(1, pipelineManager.getPipelines().size()); Assert.assertTrue(pipelineManager.containsPipeline(pipeline.getId())); - Assert.assertEquals(ALLOCATED, pipeline.getPipelineState()); + assertEquals(ALLOCATED, pipeline.getPipelineState()); buffer.flush(); - Assert.assertEquals(ALLOCATED, + assertEquals(ALLOCATED, pipelineStore.get(pipeline.getId()).getPipelineState()); PipelineID pipelineID = pipeline.getId(); @@ -214,10 +219,10 @@ public void testUpdatePipelineStates() throws Exception { Assert.assertTrue(pipelineStore.get(pipeline.getId()).isOpen()); pipelineManager.deactivatePipeline(pipeline.getId()); - Assert.assertEquals(Pipeline.PipelineState.DORMANT, + assertEquals(Pipeline.PipelineState.DORMANT, pipelineManager.getPipeline(pipelineID).getPipelineState()); buffer.flush(); - Assert.assertEquals(Pipeline.PipelineState.DORMANT, + assertEquals(Pipeline.PipelineState.DORMANT, pipelineStore.get(pipeline.getId()).getPipelineState()); Assert.assertFalse(pipelineManager .getPipelines(RatisReplicationConfig @@ -239,9 +244,9 @@ public void testOpenPipelineShouldFailOnFollower() throws Exception { PipelineManagerImpl pipelineManager = createPipelineManager(true); Pipeline pipeline = pipelineManager.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.THREE)); - Assert.assertEquals(1, pipelineManager.getPipelines().size()); + assertEquals(1, pipelineManager.getPipelines().size()); Assert.assertTrue(pipelineManager.containsPipeline(pipeline.getId())); - Assert.assertEquals(ALLOCATED, pipeline.getPipelineState()); + assertEquals(ALLOCATED, pipeline.getPipelineState()); // Change to follower assert pipelineManager.getScmhaManager() instanceof SCMHAManagerStub; ((SCMHAManagerStub) pipelineManager.getScmhaManager()).setIsLeader(false); @@ -260,9 +265,9 @@ public void testActivatePipelineShouldFailOnFollower() throws Exception { PipelineManagerImpl pipelineManager = createPipelineManager(true); Pipeline pipeline = pipelineManager.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.THREE)); - Assert.assertEquals(1, pipelineManager.getPipelines().size()); + assertEquals(1, pipelineManager.getPipelines().size()); Assert.assertTrue(pipelineManager.containsPipeline(pipeline.getId())); - Assert.assertEquals(ALLOCATED, pipeline.getPipelineState()); + assertEquals(ALLOCATED, pipeline.getPipelineState()); // Change to follower assert pipelineManager.getScmhaManager() instanceof SCMHAManagerStub; ((SCMHAManagerStub) pipelineManager.getScmhaManager()).setIsLeader(false); @@ -281,9 +286,9 @@ public void testDeactivatePipelineShouldFailOnFollower() throws Exception { PipelineManagerImpl pipelineManager = createPipelineManager(true); Pipeline pipeline = pipelineManager.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.THREE)); - Assert.assertEquals(1, pipelineManager.getPipelines().size()); + assertEquals(1, pipelineManager.getPipelines().size()); Assert.assertTrue(pipelineManager.containsPipeline(pipeline.getId())); - Assert.assertEquals(ALLOCATED, pipeline.getPipelineState()); + assertEquals(ALLOCATED, pipeline.getPipelineState()); // Change to follower assert pipelineManager.getScmhaManager() instanceof SCMHAManagerStub; ((SCMHAManagerStub) pipelineManager.getScmhaManager()).setIsLeader(false); @@ -304,9 +309,9 @@ public void testRemovePipeline() throws Exception { // Create a pipeline Pipeline pipeline = pipelineManager.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.THREE)); - Assert.assertEquals(1, pipelineManager.getPipelines().size()); + assertEquals(1, pipelineManager.getPipelines().size()); Assert.assertTrue(pipelineManager.containsPipeline(pipeline.getId())); - Assert.assertEquals(ALLOCATED, pipeline.getPipelineState()); + assertEquals(ALLOCATED, pipeline.getPipelineState()); // Open the pipeline pipelineManager.openPipeline(pipeline.getId()); @@ -329,7 +334,7 @@ public void testRemovePipeline() throws Exception { fail(); } catch (IOException ioe) { // Should not be able to remove the OPEN pipeline. - Assert.assertEquals(1, pipelineManager.getPipelines().size()); + assertEquals(1, pipelineManager.getPipelines().size()); } catch (Exception e) { Assert.fail("Should not reach here."); } @@ -353,9 +358,9 @@ public void testClosePipelineShouldFailOnFollower() throws Exception { pipelineManager.setScmContext(scmContext); Pipeline pipeline = pipelineManager.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.THREE)); - Assert.assertEquals(1, pipelineManager.getPipelines().size()); + assertEquals(1, pipelineManager.getPipelines().size()); Assert.assertTrue(pipelineManager.containsPipeline(pipeline.getId())); - Assert.assertEquals(ALLOCATED, pipeline.getPipelineState()); + assertEquals(ALLOCATED, pipeline.getPipelineState()); // Change to follower assert pipelineManager.getScmhaManager() instanceof SCMHAManagerStub; ((SCMHAManagerStub) pipelineManager.getScmhaManager()).setIsLeader(false); @@ -429,7 +434,7 @@ public void testPipelineCreationFailedMetric() throws Exception { SCMPipelineMetrics.class.getSimpleName()); long numPipelineAllocated = getLongCounter("NumPipelineAllocated", metrics); - Assert.assertEquals(0, numPipelineAllocated); + assertEquals(0, numPipelineAllocated); // 3 DNs are unhealthy. // Create 5 pipelines (Use up 15 Datanodes) @@ -444,11 +449,11 @@ public void testPipelineCreationFailedMetric() throws Exception { metrics = getMetrics( SCMPipelineMetrics.class.getSimpleName()); numPipelineAllocated = getLongCounter("NumPipelineAllocated", metrics); - Assert.assertEquals(maxPipelineCount, numPipelineAllocated); + assertEquals(maxPipelineCount, numPipelineAllocated); long numPipelineCreateFailed = getLongCounter( "NumPipelineCreationFailed", metrics); - Assert.assertEquals(0, numPipelineCreateFailed); + assertEquals(0, numPipelineCreateFailed); //This should fail... try { @@ -458,18 +463,18 @@ public void testPipelineCreationFailedMetric() throws Exception { fail(); } catch (SCMException ioe) { // pipeline creation failed this time. - Assert.assertEquals(SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE, + assertEquals(SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE, ioe.getResult()); } metrics = getMetrics( SCMPipelineMetrics.class.getSimpleName()); numPipelineAllocated = getLongCounter("NumPipelineAllocated", metrics); - Assert.assertEquals(maxPipelineCount, numPipelineAllocated); + assertEquals(maxPipelineCount, numPipelineAllocated); numPipelineCreateFailed = getLongCounter( "NumPipelineCreationFailed", metrics); - Assert.assertEquals(1, numPipelineCreateFailed); + assertEquals(1, numPipelineCreateFailed); // clean up pipelineManager.close(); @@ -490,7 +495,7 @@ public void testPipelineOpenOnlyWhenLeaderReported() throws Exception { pipelineManager.close(); // new pipeline manager loads the pipelines from the db in ALLOCATED state pipelineManager = createPipelineManager(true); - Assert.assertEquals(Pipeline.PipelineState.ALLOCATED, + assertEquals(Pipeline.PipelineState.ALLOCATED, pipelineManager.getPipeline(pipeline.getId()).getPipelineState()); SCMSafeModeManager scmSafeModeManager = @@ -503,12 +508,12 @@ null, pipelineManager, new EventQueue(), // Report pipelines with leaders List nodes = pipeline.getNodes(); - Assert.assertEquals(3, nodes.size()); + assertEquals(3, nodes.size()); // Send report for all but no leader nodes.forEach(dn -> sendPipelineReport(dn, pipeline, pipelineReportHandler, false)); - Assert.assertEquals(Pipeline.PipelineState.ALLOCATED, + assertEquals(Pipeline.PipelineState.ALLOCATED, pipelineManager.getPipeline(pipeline.getId()).getPipelineState()); nodes.subList(0, 2).forEach(dn -> sendPipelineReport(dn, pipeline, @@ -516,7 +521,7 @@ null, pipelineManager, new EventQueue(), sendPipelineReport(nodes.get(nodes.size() - 1), pipeline, pipelineReportHandler, true); - Assert.assertEquals(Pipeline.PipelineState.OPEN, + assertEquals(Pipeline.PipelineState.OPEN, pipelineManager.getPipeline(pipeline.getId()).getPipelineState()); pipelineManager.close(); @@ -535,7 +540,7 @@ public void testScrubPipeline() throws Exception { .createPipeline(RatisReplicationConfig .getInstance(ReplicationFactor.THREE)); // At this point, pipeline is not at OPEN stage. - Assert.assertEquals(Pipeline.PipelineState.ALLOCATED, + assertEquals(Pipeline.PipelineState.ALLOCATED, pipeline.getPipelineState()); // pipeline should be seen in pipelineManager as ALLOCATED. @@ -569,7 +574,7 @@ public void testScrubPipelineShouldFailOnFollower() throws Exception { .createPipeline(RatisReplicationConfig .getInstance(ReplicationFactor.THREE)); // At this point, pipeline is not at OPEN stage. - Assert.assertEquals(Pipeline.PipelineState.ALLOCATED, + assertEquals(Pipeline.PipelineState.ALLOCATED, pipeline.getPipelineState()); // pipeline should be seen in pipelineManager as ALLOCATED. @@ -724,6 +729,79 @@ public void testPipelineCloseFlow() throws IOException { assertTrue(containerLogIdx < pipelineLogIdx); } + @Test + public void testGetStalePipelines() throws IOException { + SCMHADBTransactionBuffer buffer = new SCMHADBTransactionBufferStub(dbStore); + PipelineManagerImpl pipelineManager = + spy(createPipelineManager(true, buffer)); + + // For existing pipelines + List pipelines = new ArrayList<>(); + UUID[] uuids = new UUID[3]; + String[] ipAddresses = new String[3]; + String[] hostNames = new String[3]; + for (int i = 0; i < 3; i++) { + uuids[i] = UUID.randomUUID(); + ipAddresses[i] = "1.2.3." + (i + 1); + hostNames[i] = "host" + i; + + Pipeline pipeline = mock(Pipeline.class); + DatanodeDetails datanodeDetails = mock(DatanodeDetails.class); + when(datanodeDetails.getUuid()).thenReturn(uuids[i]); + when(datanodeDetails.getIpAddress()).thenReturn(ipAddresses[i]); + when(datanodeDetails.getHostName()).thenReturn(hostNames[i]); + List nodes = new ArrayList<>(); + nodes.add(datanodeDetails); + when(pipeline.getNodes()).thenReturn(nodes); + pipelines.add(pipeline); + } + + List nodes = new ArrayList<>(); + nodes.add(pipelines.get(0).getNodes().get(0)); + nodes.add(pipelines.get(1).getNodes().get(0)); + nodes.add(pipelines.get(2).getNodes().get(0)); + Pipeline pipeline = mock(Pipeline.class); + when(pipeline.getNodes()).thenReturn(nodes); + pipelines.add(pipeline); + + doReturn(pipelines).when(pipelineManager).getPipelines(); + + // node with changed uuid + DatanodeDetails node0 = mock(DatanodeDetails.class); + UUID changedUUID = UUID.randomUUID(); + when(node0.getUuid()).thenReturn(changedUUID); + when(node0.getIpAddress()).thenReturn(ipAddresses[0]); + when(node0.getHostName()).thenReturn(hostNames[0]); + + // test uuid change + assertTrue(pipelineManager.getStalePipelines(node0).isEmpty()); + + // node with changed IP + DatanodeDetails node1 = mock(DatanodeDetails.class); + when(node1.getUuid()).thenReturn(uuids[0]); + when(node1.getIpAddress()).thenReturn("1.2.3.100"); + when(node1.getHostName()).thenReturn(hostNames[0]); + + // test IP change + List pipelineList1 = pipelineManager.getStalePipelines(node1); + assertEquals(2, pipelineList1.size()); + assertEquals(pipelines.get(0), pipelineList1.get(0)); + assertEquals(pipelines.get(3), pipelineList1.get(1)); + + // node with changed host name + DatanodeDetails node2 = mock(DatanodeDetails.class); + when(node2.getUuid()).thenReturn(uuids[0]); + when(node2.getIpAddress()).thenReturn(ipAddresses[0]); + when(node2.getHostName()).thenReturn("host100"); + + // test IP change + List pipelineList2 = pipelineManager.getStalePipelines(node2); + assertEquals(2, pipelineList1.size()); + assertEquals(pipelines.get(0), pipelineList2.get(0)); + assertEquals(pipelines.get(3), pipelineList2.get(1)); + } + + private void sendPipelineReport( DatanodeDetails dn, Pipeline pipeline, PipelineReportHandler pipelineReportHandler, From 5c9addf99d450dac4866d5567a7b4f20488e8ea6 Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Wed, 30 Mar 2022 21:11:27 -0700 Subject: [PATCH 05/29] fix test --- .../java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java | 1 + 1 file changed, 1 insertion(+) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java index c89460347f7a..03de2298236f 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java @@ -362,6 +362,7 @@ public RegisteredCommand register( InetAddress dnAddress = Server.getRemoteIp(); if (dnAddress != null) { // Mostly called inside an RPC, update ip + datanodeDetails.setHostName(dnAddress.getHostName()); datanodeDetails.setIpAddress(dnAddress.getHostAddress()); } From a7e47b7186f6dc227a59c94815aa4d640ac4d4e7 Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Thu, 31 Mar 2022 09:23:56 -0700 Subject: [PATCH 06/29] fix check style error --- .../hdds/scm/pipeline/PipelineManagerImpl.java | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java index b8899fbfbb28..580572deefd2 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java @@ -375,18 +375,20 @@ public void closePipeline(Pipeline pipeline, boolean onTimeout) } } - /** close the pipelines whose nodes' IPs are stale + /** close the pipelines whose nodes' IPs are stale. * * @param datanodeDetails new datanodeDetails */ @Override public void closeStalePipelines(DatanodeDetails datanodeDetails) { - List pipelinesWithStaleIpOrHostname = getStalePipelines(datanodeDetails); + List pipelinesWithStaleIpOrHostname = + getStalePipelines(datanodeDetails); if (pipelinesWithStaleIpOrHostname.isEmpty()) { LOG.info("No stale pipelines"); return; } - LOG.info("Pipelines with stale IP or Host name: {}", pipelinesWithStaleIpOrHostname); + LOG.info("Pipelines with stale IP or Host name: {}", + pipelinesWithStaleIpOrHostname); pipelinesWithStaleIpOrHostname.forEach(p -> { try { LOG.info("Closing pipeline: {}", p.getId()); @@ -403,9 +405,12 @@ List getStalePipelines(DatanodeDetails datanodeDetails) { List pipelines = getPipelines(); return pipelines.stream() .filter(p -> p.getNodes().stream() - .anyMatch(n -> n.getUuid().equals(datanodeDetails.getUuid()) - && (!n.getIpAddress().equals(datanodeDetails.getIpAddress()) - || !n.getHostName().equals(datanodeDetails.getHostName())))) + .anyMatch(n -> n.getUuid() + .equals(datanodeDetails.getUuid()) + && (!n.getIpAddress() + .equals(datanodeDetails.getIpAddress()) + || !n.getHostName() + .equals(datanodeDetails.getHostName())))) .collect(Collectors.toList()); } From 71f1b28d4454ddd665b1400b3a9908a5d011ed88 Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Fri, 1 Apr 2022 13:59:23 -0700 Subject: [PATCH 07/29] address PR comments --- .../org/apache/hadoop/hdds/ratis/RatisHelper.java | 10 ++++++---- .../scm/node/NodeIpOrHostnameUpdateHandler.java | 13 +++++-------- 2 files changed, 11 insertions(+), 12 deletions(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java index 9a7a8e49c4fc..ddd664a653d7 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java @@ -101,11 +101,13 @@ public static UUID toDatanodeId(RaftProtos.RaftPeerProto peerId) { private static String toRaftPeerAddress(DatanodeDetails id, Port.Name port) { if (datanodeUseHostName()) { - LOG.debug("Datanode is using hostname for raft peer address"); - return id.getHostName() + ":" + id.getPort(port).getValue(); + final String address = id.getHostName() + ":" + id.getPort(port).getValue(); + LOG.debug("Datanode is using hostname for raft peer address: {}", address); + return address; } else { - LOG.debug("Datanode is using IP for raft peer address"); - return id.getIpAddress() + ":" + id.getPort(port).getValue(); + final String address = id.getIpAddress() + ":" + id.getPort(port).getValue(); + LOG.debug("Datanode is using IP for raft peer address: {}", address); + return address; } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java index f9ae0e8962bf..cdef9d8eb203 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java @@ -17,7 +17,6 @@ package org.apache.hadoop.hdds.scm.node; import org.apache.hadoop.hdds.protocol.DatanodeDetails; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.ha.SCMService; import org.apache.hadoop.hdds.scm.ha.SCMServiceManager; import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; @@ -40,8 +39,8 @@ public class NodeIpOrHostnameUpdateHandler private final SCMServiceManager serviceManager; public NodeIpOrHostnameUpdateHandler(PipelineManager pipelineManager, - NodeDecommissionManager decommissionManager, - SCMServiceManager serviceManager) { + NodeDecommissionManager decommissionManager, + SCMServiceManager serviceManager) { this.pipelineManager = pipelineManager; this.decommissionManager = decommissionManager; this.serviceManager = serviceManager; @@ -51,18 +50,16 @@ public NodeIpOrHostnameUpdateHandler(PipelineManager pipelineManager, public void onMessage(DatanodeDetails datanodeDetails, EventPublisher publisher) { try { + LOG.info("Closing stale pipelines for datanode: {}", datanodeDetails); pipelineManager.closeStalePipelines(datanodeDetails); serviceManager.notifyEventTriggered(SCMService.Event .NODE_IP_OR_HOSTNAME_UPDATE_HANDLER_TRIGGERED); - if (datanodeDetails.getPersistedOpState() - != HddsProtos.NodeOperationalState.IN_SERVICE) { - decommissionManager.continueAdminForNode(datanodeDetails); - } + decommissionManager.continueAdminForNode(datanodeDetails); } catch (NodeNotFoundException e) { // Should not happen, as the node has just registered to call this event // handler. - LOG.warn( + LOG.error( "NodeNotFound when updating the node Ip or host name to the " + "decommissionManager", e); From d7dbf3621ef0ae1c92a1788ec58c012ecb01a794 Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Fri, 1 Apr 2022 15:35:10 -0700 Subject: [PATCH 08/29] address PR comments --- .../hadoop/hdds/scm/net/NetworkTopology.java | 8 +++ .../hdds/scm/net/NetworkTopologyImpl.java | 71 +++++++++++++++++-- .../hdds/scm/net/TestNetworkTopologyImpl.java | 40 +++++++++++ .../hadoop/hdds/scm/node/SCMNodeManager.java | 17 +++-- 4 files changed, 123 insertions(+), 13 deletions(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopology.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopology.java index 10184ae00e59..c863dc3da557 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopology.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopology.java @@ -39,6 +39,14 @@ public InvalidTopologyException(String msg) { */ void add(Node node); + /** + * Update a node. This will be called when a datanode needs to be updated. + * If the old datanode does not exist, then just add the new datanode. + * @param oldNode node to be updated; can be null + * @param newNode node to update to; cannot be null + */ + void update(Node oldNode, Node newNode); + /** * Remove a node from the network topology. This will be called when a * existing datanode is removed from the system. diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopologyImpl.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopologyImpl.java index 206a0fd73b23..2900c2b8187d 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopologyImpl.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopologyImpl.java @@ -116,6 +116,59 @@ public void add(Node node) { } } + /** + * Update a leaf node. This will be called when a datanode needs to be updated. + * If the old datanode does not exist, then just add the new datanode. + * @param oldNode node to be updated; can be null + * @param newNode node to update to; cannot be null + */ + @Override + public void update(Node oldNode, Node newNode) { + Preconditions.checkArgument(newNode != null, "newNode cannot be null"); + if (oldNode != null && oldNode instanceof InnerNode) { + throw new IllegalArgumentException( + "Not allowed to update an inner node: " + + oldNode.getNetworkFullPath()); + } + + if (newNode instanceof InnerNode) { + throw new IllegalArgumentException( + "Not allowed to update a leaf node to an inner node: " + + newNode.getNetworkFullPath()); + } + + int newDepth = NetUtils.locationToDepth(newNode.getNetworkLocation()) + 1; + // Check depth + if (maxLevel != newDepth) { + throw new InvalidTopologyException("Failed to update to " + + newNode.getNetworkFullPath() + + ": Its path depth is not " + + maxLevel); + } + + netlock.writeLock().lock(); + boolean add; + try { + boolean exist = false; + if (oldNode != null) { + exist = containsNode(oldNode); + } + if (exist) { + clusterTree.remove(oldNode); + } + + add = clusterTree.add(newNode); + } finally { + netlock.writeLock().unlock(); + } + if (add) { + LOG.info("Updated to the new node: {}", newNode.getNetworkFullPath()); + if (LOG.isDebugEnabled()) { + LOG.debug("NetworkTopology became:\n{}", this); + } + } + } + /** * Remove a node from the network topology. This will be called when a * existing datanode is removed from the system. @@ -150,16 +203,20 @@ public boolean contains(Node node) { Preconditions.checkArgument(node != null, "node cannot be null"); netlock.readLock().lock(); try { - Node parent = node.getParent(); - while (parent != null && parent != clusterTree) { - parent = parent.getParent(); - } - if (parent == clusterTree) { - return true; - } + return containsNode(node); } finally { netlock.readLock().unlock(); } + } + + private boolean containsNode(Node node) { + Node parent = node.getParent(); + while (parent != null && parent != clusterTree) { + parent = parent.getParent(); + } + if (parent == clusterTree) { + return true; + } return false; } diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/net/TestNetworkTopologyImpl.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/net/TestNetworkTopologyImpl.java index e50eca2e6908..d906f90c60fa 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/net/TestNetworkTopologyImpl.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/scm/net/TestNetworkTopologyImpl.java @@ -905,6 +905,46 @@ public void testSingleNodeRackWithAffinityNode() { assertNull(chosenNode); } + @Test + public void testUpdateNode() { + List schemas = new ArrayList<>(); + schemas.add(ROOT_SCHEMA); + schemas.add(DATACENTER_SCHEMA); + schemas.add(RACK_SCHEMA); + schemas.add(LEAF_SCHEMA); + + NodeSchemaManager manager = NodeSchemaManager.getInstance(); + manager.init(schemas.toArray(new NodeSchema[0]), true); + NetworkTopology newCluster = + new NetworkTopologyImpl(manager); + Node node = createDatanode("1.1.1.1", "/d1/r1"); + newCluster.add(node); + assertTrue(newCluster.contains(node)); + + // update + Node newNode = createDatanode("1.1.1.2", "/d1/r1"); + assertFalse(newCluster.contains(newNode)); + newCluster.update(node, newNode); + assertFalse(newCluster.contains(node)); + assertTrue(newCluster.contains(newNode)); + + // update a non-existing node + Node nodeExisting = createDatanode("1.1.1.3", "/d1/r1"); + Node newNode2 = createDatanode("1.1.1.4", "/d1/r1"); + assertFalse(newCluster.contains(nodeExisting)); + assertFalse(newCluster.contains(newNode2)); + + newCluster.update(nodeExisting, newNode2); + assertFalse(newCluster.contains(nodeExisting)); + assertTrue(newCluster.contains(newNode2)); + + // old node is null + Node newNode3 = createDatanode("1.1.1.5", "/d1/r1"); + assertFalse(newCluster.contains(newNode3)); + newCluster.update(null, newNode3); + assertTrue(newCluster.contains(newNode3)); + } + private static Node createDatanode(String name, String path) { return new NodeImpl(name, path, NetConstants.NODE_COST_DEFAULT); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java index 03de2298236f..35889d9beecc 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java @@ -412,10 +412,7 @@ public RegisteredCommand register( datanodeDetails.getUuidString(), datanodeInfo, datanodeDetails); - if (clusterMap.contains(datanodeInfo)) { - clusterMap.remove(datanodeInfo); - } - clusterMap.add(datanodeDetails); + clusterMap.update(datanodeInfo, datanodeDetails); String oldDnsName; if (useHostname) { @@ -423,8 +420,9 @@ public RegisteredCommand register( } else { oldDnsName = datanodeInfo.getIpAddress(); } - removeEntryFromDnsToUuidMap(oldDnsName); - addEntryToDnsToUuidMap(dnsName, datanodeDetails.getUuidString()); + updateEntryFromDnsToUuidMap(oldDnsName, + dnsName, + datanodeDetails.getUuidString()); nodeStateManager.updateNode(datanodeDetails, layoutInfo); DatanodeDetails dn = nodeStateManager.getNode(datanodeDetails); @@ -478,6 +476,13 @@ private synchronized void removeEntryFromDnsToUuidMap(String dnsName) { } } + private synchronized void updateEntryFromDnsToUuidMap(String oldDnsName, + String newDnsName, + String uuid) { + removeEntryFromDnsToUuidMap(oldDnsName); + addEntryToDnsToUuidMap(newDnsName, uuid); + } + /** * Send heartbeat to indicate the datanode is alive and doing well. * From b89eed124b2d2946cfa37d159395cd6944879981 Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Sat, 7 May 2022 14:36:52 -0700 Subject: [PATCH 09/29] fix checkstyle error --- .../java/org/apache/hadoop/hdds/ratis/RatisHelper.java | 9 ++++++--- .../apache/hadoop/hdds/scm/net/NetworkTopologyImpl.java | 2 +- .../hdds/scm/node/NodeIpOrHostnameUpdateHandler.java | 3 ++- 3 files changed, 9 insertions(+), 5 deletions(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java index e4d94dbcc9b5..36cf339e4822 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java @@ -102,11 +102,14 @@ public static UUID toDatanodeId(RaftProtos.RaftPeerProto peerId) { private static String toRaftPeerAddress(DatanodeDetails id, Port.Name port) { if (datanodeUseHostName()) { - final String address = id.getHostName() + ":" + id.getPort(port).getValue(); - LOG.debug("Datanode is using hostname for raft peer address: {}", address); + final String address = + id.getHostName() + ":" + id.getPort(port).getValue(); + LOG.debug("Datanode is using hostname for raft peer address: {}", + address); return address; } else { - final String address = id.getIpAddress() + ":" + id.getPort(port).getValue(); + final String address = + id.getIpAddress() + ":" + id.getPort(port).getValue(); LOG.debug("Datanode is using IP for raft peer address: {}", address); return address; } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopologyImpl.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopologyImpl.java index 9f8058523aeb..6b1beed3dd8a 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopologyImpl.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/net/NetworkTopologyImpl.java @@ -117,7 +117,7 @@ public void add(Node node) { } /** - * Update a leaf node. This will be called when a datanode needs to be updated. + * Update a leaf node. It is called when a datanode needs to be updated. * If the old datanode does not exist, then just add the new datanode. * @param oldNode node to be updated; can be null * @param newNode node to update to; cannot be null diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java index cdef9d8eb203..8e13dc63dd09 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java @@ -39,7 +39,8 @@ public class NodeIpOrHostnameUpdateHandler private final SCMServiceManager serviceManager; public NodeIpOrHostnameUpdateHandler(PipelineManager pipelineManager, - NodeDecommissionManager decommissionManager, + NodeDecommissionManager + decommissionManager, SCMServiceManager serviceManager) { this.pipelineManager = pipelineManager; this.decommissionManager = decommissionManager; From e346e8b1a5b2951a50bc8eaa04c714294aa4343e Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" Date: Mon, 9 May 2022 22:25:51 +0200 Subject: [PATCH 10/29] trigger new CI check From 878f9898b0b37abbedccf03fe7594ce9675c8992 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" Date: Wed, 18 May 2022 08:40:58 +0200 Subject: [PATCH 11/29] Avoid new static import for easier merge from master --- .../scm/pipeline/TestPipelineManagerImpl.java | 75 +++++++++---------- 1 file changed, 37 insertions(+), 38 deletions(-) diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java index 726c3b1243ed..6fc074da1f78 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java @@ -76,7 +76,6 @@ import static org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState.ALLOCATED; import static org.apache.hadoop.test.MetricsAsserts.getLongCounter; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; -import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Mockito.doReturn; @@ -163,12 +162,12 @@ public void testCreatePipeline() throws Exception { Assert.assertTrue(pipelineManager.getPipelines().isEmpty()); Pipeline pipeline1 = pipelineManager.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.THREE)); - assertEquals(1, pipelineManager.getPipelines().size()); + Assert.assertEquals(1, pipelineManager.getPipelines().size()); Assert.assertTrue(pipelineManager.containsPipeline(pipeline1.getId())); Pipeline pipeline2 = pipelineManager.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.ONE)); - assertEquals(2, pipelineManager.getPipelines().size()); + Assert.assertEquals(2, pipelineManager.getPipelines().size()); Assert.assertTrue(pipelineManager.containsPipeline(pipeline2.getId())); buffer1.close(); pipelineManager.close(); @@ -179,11 +178,11 @@ public void testCreatePipeline() throws Exception { createPipelineManager(true, buffer2); // Should be able to load previous pipelines. Assert.assertFalse(pipelineManager2.getPipelines().isEmpty()); - assertEquals(2, pipelineManager.getPipelines().size()); + Assert.assertEquals(2, pipelineManager.getPipelines().size()); Pipeline pipeline3 = pipelineManager2.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.THREE)); buffer2.close(); - assertEquals(3, pipelineManager2.getPipelines().size()); + Assert.assertEquals(3, pipelineManager2.getPipelines().size()); Assert.assertTrue(pipelineManager2.containsPipeline(pipeline3.getId())); pipelineManager2.close(); @@ -214,11 +213,11 @@ public void testUpdatePipelineStates() throws Exception { SCMDBDefinition.PIPELINES.getTable(dbStore); Pipeline pipeline = pipelineManager.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.THREE)); - assertEquals(1, pipelineManager.getPipelines().size()); + Assert.assertEquals(1, pipelineManager.getPipelines().size()); Assert.assertTrue(pipelineManager.containsPipeline(pipeline.getId())); - assertEquals(ALLOCATED, pipeline.getPipelineState()); + Assert.assertEquals(ALLOCATED, pipeline.getPipelineState()); buffer.flush(); - assertEquals(ALLOCATED, + Assert.assertEquals(ALLOCATED, pipelineStore.get(pipeline.getId()).getPipelineState()); PipelineID pipelineID = pipeline.getId(); @@ -232,10 +231,10 @@ public void testUpdatePipelineStates() throws Exception { Assert.assertTrue(pipelineStore.get(pipeline.getId()).isOpen()); pipelineManager.deactivatePipeline(pipeline.getId()); - assertEquals(Pipeline.PipelineState.DORMANT, + Assert.assertEquals(Pipeline.PipelineState.DORMANT, pipelineManager.getPipeline(pipelineID).getPipelineState()); buffer.flush(); - assertEquals(Pipeline.PipelineState.DORMANT, + Assert.assertEquals(Pipeline.PipelineState.DORMANT, pipelineStore.get(pipeline.getId()).getPipelineState()); Assert.assertFalse(pipelineManager .getPipelines(RatisReplicationConfig @@ -263,9 +262,9 @@ public void testOpenPipelineShouldFailOnFollower() throws Exception { PipelineManagerImpl pipelineManager = createPipelineManager(true); Pipeline pipeline = pipelineManager.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.THREE)); - assertEquals(1, pipelineManager.getPipelines().size()); + Assert.assertEquals(1, pipelineManager.getPipelines().size()); Assert.assertTrue(pipelineManager.containsPipeline(pipeline.getId())); - assertEquals(ALLOCATED, pipeline.getPipelineState()); + Assert.assertEquals(ALLOCATED, pipeline.getPipelineState()); // Change to follower assert pipelineManager.getScmhaManager() instanceof SCMHAManagerStub; ((SCMHAManagerStub) pipelineManager.getScmhaManager()).setIsLeader(false); @@ -284,9 +283,9 @@ public void testActivatePipelineShouldFailOnFollower() throws Exception { PipelineManagerImpl pipelineManager = createPipelineManager(true); Pipeline pipeline = pipelineManager.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.THREE)); - assertEquals(1, pipelineManager.getPipelines().size()); + Assert.assertEquals(1, pipelineManager.getPipelines().size()); Assert.assertTrue(pipelineManager.containsPipeline(pipeline.getId())); - assertEquals(ALLOCATED, pipeline.getPipelineState()); + Assert.assertEquals(ALLOCATED, pipeline.getPipelineState()); // Change to follower assert pipelineManager.getScmhaManager() instanceof SCMHAManagerStub; ((SCMHAManagerStub) pipelineManager.getScmhaManager()).setIsLeader(false); @@ -305,9 +304,9 @@ public void testDeactivatePipelineShouldFailOnFollower() throws Exception { PipelineManagerImpl pipelineManager = createPipelineManager(true); Pipeline pipeline = pipelineManager.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.THREE)); - assertEquals(1, pipelineManager.getPipelines().size()); + Assert.assertEquals(1, pipelineManager.getPipelines().size()); Assert.assertTrue(pipelineManager.containsPipeline(pipeline.getId())); - assertEquals(ALLOCATED, pipeline.getPipelineState()); + Assert.assertEquals(ALLOCATED, pipeline.getPipelineState()); // Change to follower assert pipelineManager.getScmhaManager() instanceof SCMHAManagerStub; ((SCMHAManagerStub) pipelineManager.getScmhaManager()).setIsLeader(false); @@ -328,9 +327,9 @@ public void testRemovePipeline() throws Exception { // Create a pipeline Pipeline pipeline = pipelineManager.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.THREE)); - assertEquals(1, pipelineManager.getPipelines().size()); + Assert.assertEquals(1, pipelineManager.getPipelines().size()); Assert.assertTrue(pipelineManager.containsPipeline(pipeline.getId())); - assertEquals(ALLOCATED, pipeline.getPipelineState()); + Assert.assertEquals(ALLOCATED, pipeline.getPipelineState()); // Open the pipeline pipelineManager.openPipeline(pipeline.getId()); @@ -353,7 +352,7 @@ public void testRemovePipeline() throws Exception { fail(); } catch (IOException ioe) { // Should not be able to remove the OPEN pipeline. - assertEquals(1, pipelineManager.getPipelines().size()); + Assert.assertEquals(1, pipelineManager.getPipelines().size()); } catch (Exception e) { Assert.fail("Should not reach here."); } @@ -377,9 +376,9 @@ public void testClosePipelineShouldFailOnFollower() throws Exception { pipelineManager.setScmContext(scmContext); Pipeline pipeline = pipelineManager.createPipeline( RatisReplicationConfig.getInstance(ReplicationFactor.THREE)); - assertEquals(1, pipelineManager.getPipelines().size()); + Assert.assertEquals(1, pipelineManager.getPipelines().size()); Assert.assertTrue(pipelineManager.containsPipeline(pipeline.getId())); - assertEquals(ALLOCATED, pipeline.getPipelineState()); + Assert.assertEquals(ALLOCATED, pipeline.getPipelineState()); // Change to follower assert pipelineManager.getScmhaManager() instanceof SCMHAManagerStub; ((SCMHAManagerStub) pipelineManager.getScmhaManager()).setIsLeader(false); @@ -453,7 +452,7 @@ public void testPipelineCreationFailedMetric() throws Exception { SCMPipelineMetrics.class.getSimpleName()); long numPipelineAllocated = getLongCounter("NumPipelineAllocated", metrics); - assertEquals(0, numPipelineAllocated); + Assert.assertEquals(0, numPipelineAllocated); // 3 DNs are unhealthy. // Create 5 pipelines (Use up 15 Datanodes) @@ -468,11 +467,11 @@ public void testPipelineCreationFailedMetric() throws Exception { metrics = getMetrics( SCMPipelineMetrics.class.getSimpleName()); numPipelineAllocated = getLongCounter("NumPipelineAllocated", metrics); - assertEquals(maxPipelineCount, numPipelineAllocated); + Assert.assertEquals(maxPipelineCount, numPipelineAllocated); long numPipelineCreateFailed = getLongCounter( "NumPipelineCreationFailed", metrics); - assertEquals(0, numPipelineCreateFailed); + Assert.assertEquals(0, numPipelineCreateFailed); //This should fail... try { @@ -482,18 +481,18 @@ public void testPipelineCreationFailedMetric() throws Exception { fail(); } catch (SCMException ioe) { // pipeline creation failed this time. - assertEquals(SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE, + Assert.assertEquals(SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE, ioe.getResult()); } metrics = getMetrics( SCMPipelineMetrics.class.getSimpleName()); numPipelineAllocated = getLongCounter("NumPipelineAllocated", metrics); - assertEquals(maxPipelineCount, numPipelineAllocated); + Assert.assertEquals(maxPipelineCount, numPipelineAllocated); numPipelineCreateFailed = getLongCounter( "NumPipelineCreationFailed", metrics); - assertEquals(1, numPipelineCreateFailed); + Assert.assertEquals(1, numPipelineCreateFailed); // clean up pipelineManager.close(); @@ -514,7 +513,7 @@ public void testPipelineOpenOnlyWhenLeaderReported() throws Exception { pipelineManager.close(); // new pipeline manager loads the pipelines from the db in ALLOCATED state pipelineManager = createPipelineManager(true); - assertEquals(Pipeline.PipelineState.ALLOCATED, + Assert.assertEquals(Pipeline.PipelineState.ALLOCATED, pipelineManager.getPipeline(pipeline.getId()).getPipelineState()); SCMSafeModeManager scmSafeModeManager = @@ -527,12 +526,12 @@ null, pipelineManager, new EventQueue(), // Report pipelines with leaders List nodes = pipeline.getNodes(); - assertEquals(3, nodes.size()); + Assert.assertEquals(3, nodes.size()); // Send report for all but no leader nodes.forEach(dn -> sendPipelineReport(dn, pipeline, pipelineReportHandler, false)); - assertEquals(Pipeline.PipelineState.ALLOCATED, + Assert.assertEquals(Pipeline.PipelineState.ALLOCATED, pipelineManager.getPipeline(pipeline.getId()).getPipelineState()); nodes.subList(0, 2).forEach(dn -> sendPipelineReport(dn, pipeline, @@ -540,7 +539,7 @@ null, pipelineManager, new EventQueue(), sendPipelineReport(nodes.get(nodes.size() - 1), pipeline, pipelineReportHandler, true); - assertEquals(Pipeline.PipelineState.OPEN, + Assert.assertEquals(Pipeline.PipelineState.OPEN, pipelineManager.getPipeline(pipeline.getId()).getPipelineState()); pipelineManager.close(); @@ -621,7 +620,7 @@ public void testScrubPipelinesShouldFailOnFollower() throws Exception { .createPipeline(RatisReplicationConfig .getInstance(ReplicationFactor.THREE)); // At this point, pipeline is not at OPEN stage. - assertEquals(Pipeline.PipelineState.ALLOCATED, + Assert.assertEquals(Pipeline.PipelineState.ALLOCATED, pipeline.getPipelineState()); // pipeline should be seen in pipelineManager as ALLOCATED. @@ -830,9 +829,9 @@ public void testGetStalePipelines() throws IOException { // test IP change List pipelineList1 = pipelineManager.getStalePipelines(node1); - assertEquals(2, pipelineList1.size()); - assertEquals(pipelines.get(0), pipelineList1.get(0)); - assertEquals(pipelines.get(3), pipelineList1.get(1)); + Assert.assertEquals(2, pipelineList1.size()); + Assert.assertEquals(pipelines.get(0), pipelineList1.get(0)); + Assert.assertEquals(pipelines.get(3), pipelineList1.get(1)); // node with changed host name DatanodeDetails node2 = mock(DatanodeDetails.class); @@ -842,9 +841,9 @@ public void testGetStalePipelines() throws IOException { // test IP change List pipelineList2 = pipelineManager.getStalePipelines(node2); - assertEquals(2, pipelineList1.size()); - assertEquals(pipelines.get(0), pipelineList2.get(0)); - assertEquals(pipelines.get(3), pipelineList2.get(1)); + Assert.assertEquals(2, pipelineList1.size()); + Assert.assertEquals(pipelines.get(0), pipelineList2.get(0)); + Assert.assertEquals(pipelines.get(3), pipelineList2.get(1)); } public void testCreatePipelineForRead() throws IOException { From 424c788447ab81e133db8a2bed92729cdbf64192 Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Fri, 20 May 2022 11:54:11 -0700 Subject: [PATCH 12/29] address PR comments --- .../hadoop/hdds/scm/events/SCMEvents.java | 4 +- .../apache/hadoop/hdds/scm/ha/SCMService.java | 2 +- ...ler.java => NodeAddressUpdateHandler.java} | 12 +-- .../hadoop/hdds/scm/node/SCMNodeManager.java | 2 +- .../pipeline/BackgroundPipelineCreator.java | 4 +- .../scm/server/StorageContainerManager.java | 10 +- .../scm/pipeline/TestPipelineManagerImpl.java | 6 +- .../k8s/examples/ozone-dn-restart/Flekszible | 36 +++++++ .../examples/ozone-dn-restart/LICENSE.header | 15 +++ .../ozone-dn-restart/config-configmap.yaml | 38 +++++++ .../csi/csi-node-daemonset.yaml | 97 ++++++++++++++++++ .../csi/csi-ozone-clusterrole.yaml | 98 +++++++++++++++++++ .../csi/csi-ozone-clusterrolebinding.yaml | 28 ++++++ .../csi/csi-ozone-serviceaccount.yaml | 21 ++++ .../csi/csi-provisioner-deployment.yaml | 54 ++++++++++ .../org.apache.hadoop.ozone-csidriver.yaml | 22 +++++ .../csi/ozone-storageclass.yaml | 21 ++++ .../ozone-dn-restart/datanode-service.yaml | 28 ++++++ .../datanode-statefulset.yaml | 72 ++++++++++++++ .../freon/freon-deployment.yaml | 46 +++++++++ .../ozone-dn-restart/kustomization.yaml | 26 +++++ .../examples/ozone-dn-restart/om-service.yaml | 28 ++++++ .../ozone-dn-restart/om-statefulset.yaml | 72 ++++++++++++++ .../ozone-csi-test-webserver-deployment.yaml | 50 ++++++++++ ...-test-webserver-persistentvolumeclaim.yaml | 29 ++++++ .../ozone-csi-test-webserver-service.yaml | 29 ++++++ .../ozone-dn-restart/s3g-service.yaml | 28 ++++++ .../ozone-dn-restart/s3g-statefulset.yaml | 61 ++++++++++++ .../ozone-dn-restart/scm-service.yaml | 28 ++++++ .../ozone-dn-restart/scm-statefulset.yaml | 79 +++++++++++++++ .../k8s/examples/ozone-dn-restart/test.sh | 44 +++++++++ 31 files changed, 1070 insertions(+), 20 deletions(-) rename hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/{NodeIpOrHostnameUpdateHandler.java => NodeAddressUpdateHandler.java} (86%) create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/Flekszible create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/LICENSE.header create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/config-configmap.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-node-daemonset.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-clusterrole.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-clusterrolebinding.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-serviceaccount.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-provisioner-deployment.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/org.apache.hadoop.ozone-csidriver.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/ozone-storageclass.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-service.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-statefulset.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/freon/freon-deployment.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/kustomization.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/om-service.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/om-statefulset.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-deployment.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-persistentvolumeclaim.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-service.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/s3g-service.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/s3g-statefulset.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/scm-service.yaml create mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/scm-statefulset.yaml create mode 100755 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/test.sh diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java index 14fe2e3dd0d8..c51d792d4c62 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java @@ -161,8 +161,8 @@ public final class SCMEvents { * This event will be triggered whenever a datanode is registered with * SCM with a different Ip or host name. */ - public static final TypedEvent NODE_IP_OR_HOSTNAME_UPDATE = - new TypedEvent<>(DatanodeDetails.class, "Node_Ip_Or_Hostname_Update"); + public static final TypedEvent NODE_ADDRESS_UPDATE = + new TypedEvent<>(DatanodeDetails.class, "Node_Address_Update"); /** * This event will be triggered whenever a datanode is moved from healthy to diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMService.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMService.java index 4830e677cc37..9d8a90c9c6ae 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMService.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMService.java @@ -60,7 +60,7 @@ enum ServiceStatus { enum Event { PRE_CHECK_COMPLETED, NEW_NODE_HANDLER_TRIGGERED, - NODE_IP_OR_HOSTNAME_UPDATE_HANDLER_TRIGGERED, + NODE_ADDRESS_UPDATE_HANDLER_TRIGGERED, UNHEALTHY_TO_HEALTHY_NODE_HANDLER_TRIGGERED } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeAddressUpdateHandler.java similarity index 86% rename from hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java rename to hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeAddressUpdateHandler.java index 8e13dc63dd09..7c9c6281765c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeIpOrHostnameUpdateHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeAddressUpdateHandler.java @@ -29,19 +29,19 @@ /** * Handles datanode ip or hostname change event. */ -public class NodeIpOrHostnameUpdateHandler +public class NodeAddressUpdateHandler implements EventHandler { private static final Logger LOG = - LoggerFactory.getLogger(NodeIpOrHostnameUpdateHandler.class); + LoggerFactory.getLogger(NodeAddressUpdateHandler.class); private final PipelineManager pipelineManager; private final NodeDecommissionManager decommissionManager; private final SCMServiceManager serviceManager; - public NodeIpOrHostnameUpdateHandler(PipelineManager pipelineManager, - NodeDecommissionManager + public NodeAddressUpdateHandler(PipelineManager pipelineManager, + NodeDecommissionManager decommissionManager, - SCMServiceManager serviceManager) { + SCMServiceManager serviceManager) { this.pipelineManager = pipelineManager; this.decommissionManager = decommissionManager; this.serviceManager = serviceManager; @@ -54,7 +54,7 @@ public void onMessage(DatanodeDetails datanodeDetails, LOG.info("Closing stale pipelines for datanode: {}", datanodeDetails); pipelineManager.closeStalePipelines(datanodeDetails); serviceManager.notifyEventTriggered(SCMService.Event - .NODE_IP_OR_HOSTNAME_UPDATE_HANDLER_TRIGGERED); + .NODE_ADDRESS_UPDATE_HANDLER_TRIGGERED); decommissionManager.continueAdminForNode(datanodeDetails); } catch (NodeNotFoundException e) { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java index 24cf84809130..637a19eef6a6 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java @@ -432,7 +432,7 @@ public RegisteredCommand register( processNodeReport(datanodeDetails, nodeReport); LOG.info("Updated Datanode to: {}", dn); scmNodeEventPublisher - .fireEvent(SCMEvents.NODE_IP_OR_HOSTNAME_UPDATE, dn); + .fireEvent(SCMEvents.NODE_ADDRESS_UPDATE, dn); } } catch (NodeNotFoundException e) { LOG.error("Cannot find datanode {} from nodeStateManager", diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java index 02f1b11cc346..cd528b0a165c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java @@ -44,7 +44,7 @@ import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.RATIS; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType.STAND_ALONE; -import static org.apache.hadoop.hdds.scm.ha.SCMService.Event.NODE_IP_OR_HOSTNAME_UPDATE_HANDLER_TRIGGERED; +import static org.apache.hadoop.hdds.scm.ha.SCMService.Event.NODE_ADDRESS_UPDATE_HANDLER_TRIGGERED; import static org.apache.hadoop.hdds.scm.ha.SCMService.Event.UNHEALTHY_TO_HEALTHY_NODE_HANDLER_TRIGGERED; import static org.apache.hadoop.hdds.scm.ha.SCMService.Event.NEW_NODE_HANDLER_TRIGGERED; import static org.apache.hadoop.hdds.scm.ha.SCMService.Event.PRE_CHECK_COMPLETED; @@ -269,7 +269,7 @@ public void notifyEventTriggered(Event event) { return; } if (event == NEW_NODE_HANDLER_TRIGGERED - || event == NODE_IP_OR_HOSTNAME_UPDATE_HANDLER_TRIGGERED + || event == NODE_ADDRESS_UPDATE_HANDLER_TRIGGERED || event == UNHEALTHY_TO_HEALTHY_NODE_HANDLER_TRIGGERED || event == PRE_CHECK_COMPLETED) { LOG.info("trigger a one-shot run on {}.", THREAD_NAME); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java index 555a44806e19..d7334f04abc0 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java @@ -55,7 +55,7 @@ import org.apache.hadoop.hdds.scm.ha.SCMHAUtils; import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator; import org.apache.hadoop.hdds.scm.ScmInfo; -import org.apache.hadoop.hdds.scm.node.NodeIpOrHostnameUpdateHandler; +import org.apache.hadoop.hdds.scm.node.NodeAddressUpdateHandler; import org.apache.hadoop.hdds.scm.node.CommandQueueReportHandler; import org.apache.hadoop.hdds.scm.ha.StatefulServiceStateManager; import org.apache.hadoop.hdds.scm.ha.StatefulServiceStateManagerImpl; @@ -419,8 +419,8 @@ private void initializeEventHandlers() { NewNodeHandler newNodeHandler = new NewNodeHandler(pipelineManager, scmDecommissionManager, configuration, serviceManager); - NodeIpOrHostnameUpdateHandler nodeIpOrHostnameUpdateHandler = - new NodeIpOrHostnameUpdateHandler(pipelineManager, + NodeAddressUpdateHandler nodeAddressUpdateHandler = + new NodeAddressUpdateHandler(pipelineManager, scmDecommissionManager, serviceManager); StaleNodeHandler staleNodeHandler = new StaleNodeHandler(scmNodeManager, pipelineManager, configuration); @@ -485,8 +485,8 @@ private void initializeEventHandlers() { eventQueue.addHandler(SCMEvents.CONTAINER_ACTIONS, actionsHandler); eventQueue.addHandler(SCMEvents.CLOSE_CONTAINER, closeContainerHandler); eventQueue.addHandler(SCMEvents.NEW_NODE, newNodeHandler); - eventQueue.addHandler(SCMEvents.NODE_IP_OR_HOSTNAME_UPDATE, - nodeIpOrHostnameUpdateHandler); + eventQueue.addHandler(SCMEvents.NODE_ADDRESS_UPDATE, + nodeAddressUpdateHandler); eventQueue.addHandler(SCMEvents.STALE_NODE, staleNodeHandler); eventQueue.addHandler(SCMEvents.HEALTHY_READONLY_TO_HEALTHY_NODE, readOnlyHealthyToHealthyNodeHandler); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java index 32ba0a90f973..f31a8542d7d7 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java @@ -842,9 +842,9 @@ public void testGetStalePipelines() throws IOException { // test IP change List pipelineList2 = pipelineManager.getStalePipelines(node2); - Assertions.assertEquals(2, pipelineList1.size()); - Assertions.assertEquals(pipelines.get(0), pipelineList2.get(0)); - Assertions.assertEquals(pipelines.get(3), pipelineList2.get(1)); + assertEquals(2, pipelineList2.size()); + assertEquals(pipelines.get(0), pipelineList2.get(0)); + assertEquals(pipelines.get(3), pipelineList2.get(1)); } public void testCreatePipelineForRead() throws IOException { diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/Flekszible b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/Flekszible new file mode 100644 index 000000000000..5562aac43c6a --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/Flekszible @@ -0,0 +1,36 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +source: + - path: ../../definitions +import: + - path: ozone + transformations: + - type: Image + image: "@docker.image@" + - type: ozone/persistence + - type: ozone/csi + - path: ozone/freon + destination: freon + transformations: + - type: Image + image: "@docker.image@" + - path: ozone-csi + destination: csi + - path: test-webserver + destination: pv-test +transformations: + - type: Namespace + - type: kustomize diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/LICENSE.header b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/LICENSE.header new file mode 100644 index 000000000000..635f0d9e60e1 --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/LICENSE.header @@ -0,0 +1,15 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/config-configmap.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/config-configmap.yaml new file mode 100644 index 000000000000..c7ac5344d142 --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/config-configmap.yaml @@ -0,0 +1,38 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: v1 +kind: ConfigMap +metadata: + name: config +data: + OZONE-SITE.XML_hdds.datanode.dir: /data/storage + OZONE-SITE.XML_ozone.scm.datanode.id.dir: /data + OZONE-SITE.XML_ozone.metadata.dirs: /data/metadata + OZONE-SITE.XML_ozone.scm.block.client.address: scm-0.scm + OZONE-SITE.XML_ozone.om.address: om-0.om + OZONE-SITE.XML_ozone.scm.client.address: scm-0.scm + OZONE-SITE.XML_ozone.scm.names: scm-0.scm + OZONE-SITE.XML_hdds.scm.safemode.min.datanode: "3" + OZONE-SITE.XML_ozone.datanode.pipeline.limit: "1" + LOG4J.PROPERTIES_log4j.rootLogger: INFO, stdout + LOG4J.PROPERTIES_log4j.appender.stdout: org.apache.log4j.ConsoleAppender + LOG4J.PROPERTIES_log4j.appender.stdout.layout: org.apache.log4j.PatternLayout + LOG4J.PROPERTIES_log4j.appender.stdout.layout.ConversionPattern: '%d{yyyy-MM-dd + HH:mm:ss} %-5p %c{1}:%L - %m%n' + OZONE-SITE.XML_ozone.csi.s3g.address: http://s3g-0.s3g:9878 + OZONE-SITE.XML_ozone.csi.socket: /var/lib/csi/csi.sock + OZONE-SITE.XML_ozone.csi.owner: hadoop diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-node-daemonset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-node-daemonset.yaml new file mode 100644 index 000000000000..1272053720f3 --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-node-daemonset.yaml @@ -0,0 +1,97 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +kind: DaemonSet +apiVersion: apps/v1 +metadata: + name: csi-node +spec: + selector: + matchLabels: + app: csi-node + template: + metadata: + labels: + app: csi-node + spec: + serviceAccount: csi-ozone + containers: + - name: driver-registrar + image: quay.io/k8scsi/csi-node-driver-registrar:v1.0.2 + args: + - --v=4 + - --csi-address=/var/lib/csi/csi.sock + - --kubelet-registration-path=/var/lib/kubelet/plugins/org.apache.hadoop.ozone/csi.sock + env: + - name: KUBE_NODE_NAME + valueFrom: + fieldRef: + fieldPath: spec.nodeName + volumeMounts: + - name: plugin-dir + mountPath: /var/lib/csi + - name: registration-dir + mountPath: /registration/ + - name: csi-node + image: '@docker.image@' + securityContext: + runAsUser: 0 + privileged: true + capabilities: + add: + - SYS_ADMIN + allowPrivilegeEscalation: true + args: + - ozone + - csi + envFrom: + - configMapRef: + name: config + imagePullPolicy: IfNotPresent + volumeMounts: + - name: plugin-dir + mountPath: /var/lib/csi + - name: pods-mount-dir + mountPath: /var/lib/kubelet/pods + mountPropagation: Bidirectional + - name: fuse-device + mountPath: /dev/fuse + - name: dbus + mountPath: /var/run/dbus + - name: systemd + mountPath: /run/systemd + volumes: + - name: plugin-dir + hostPath: + path: /var/lib/kubelet/plugins/org.apache.hadoop.ozone + type: DirectoryOrCreate + - name: registration-dir + hostPath: + path: /var/lib/kubelet/plugins_registry/ + type: DirectoryOrCreate + - name: pods-mount-dir + hostPath: + path: /var/lib/kubelet/pods + type: Directory + - name: fuse-device + hostPath: + path: /dev/fuse + - name: dbus + hostPath: + path: /var/run/dbus + - name: systemd + hostPath: + path: /run/systemd diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-clusterrole.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-clusterrole.yaml new file mode 100644 index 000000000000..927ba6ff7b7f --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-clusterrole.yaml @@ -0,0 +1,98 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +kind: ClusterRole +apiVersion: rbac.authorization.k8s.io/v1 +metadata: + name: csi-ozone-default +rules: +- apiGroups: + - "" + resources: + - secrets + verbs: + - get + - list +- apiGroups: + - "" + resources: + - events + verbs: + - list + - watch + - create + - update + - patch +- apiGroups: + - "" + resources: + - nodes + verbs: + - get + - list + - update + - watch +- apiGroups: + - "" + resources: + - namespaces + verbs: + - get + - list +- apiGroups: + - storage.k8s.io + resources: + - storageclasses + verbs: + - get + - list + - watch +- apiGroups: + - "" + resources: + - persistentvolumeclaims + verbs: + - get + - list + - watch + - update +- apiGroups: + - "" + resources: + - persistentvolumes + verbs: + - get + - list + - watch + - update + - create +- apiGroups: + - storage.k8s.io + resources: + - volumeattachments + verbs: + - get + - list + - watch + - update +- apiGroups: + - storage.k8s.io + resources: + - csinodes + verbs: + - get + - list + - watch diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-clusterrolebinding.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-clusterrolebinding.yaml new file mode 100644 index 000000000000..948e759fbe35 --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-clusterrolebinding.yaml @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +kind: ClusterRoleBinding +apiVersion: rbac.authorization.k8s.io/v1 +metadata: + name: csi-ozone-default +subjects: +- kind: ServiceAccount + name: csi-ozone + namespace: default +roleRef: + kind: ClusterRole + name: csi-ozone-default + apiGroup: rbac.authorization.k8s.io diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-serviceaccount.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-serviceaccount.yaml new file mode 100644 index 000000000000..628d2a1c5957 --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-serviceaccount.yaml @@ -0,0 +1,21 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: v1 +kind: ServiceAccount +metadata: + namespace: default + name: csi-ozone diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-provisioner-deployment.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-provisioner-deployment.yaml new file mode 100644 index 000000000000..81837122d16c --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-provisioner-deployment.yaml @@ -0,0 +1,54 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +kind: Deployment +apiVersion: apps/v1 +metadata: + name: csi-provisioner +spec: + replicas: 1 + selector: + matchLabels: + app: csi-provisioner + template: + metadata: + labels: + app: csi-provisioner + spec: + serviceAccount: csi-ozone + containers: + - name: csi-provisioner + image: quay.io/k8scsi/csi-provisioner:v1.0.1 + args: + - --csi-address=/var/lib/csi/csi.sock + volumeMounts: + - name: socket-dir + mountPath: /var/lib/csi/ + - name: ozone-csi + image: '@docker.image@' + volumeMounts: + - name: socket-dir + mountPath: /var/lib/csi/ + imagePullPolicy: IfNotPresent + envFrom: + - configMapRef: + name: config + args: + - ozone + - csi + volumes: + - name: socket-dir + emptyDir: {} diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/org.apache.hadoop.ozone-csidriver.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/org.apache.hadoop.ozone-csidriver.yaml new file mode 100644 index 000000000000..e657c50f7583 --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/org.apache.hadoop.ozone-csidriver.yaml @@ -0,0 +1,22 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: storage.k8s.io/v1beta1 +kind: CSIDriver +metadata: + name: org.apache.hadoop.ozone +spec: + attachRequired: false diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/ozone-storageclass.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/ozone-storageclass.yaml new file mode 100644 index 000000000000..c6c1c6c9d1e1 --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/ozone-storageclass.yaml @@ -0,0 +1,21 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +kind: StorageClass +apiVersion: storage.k8s.io/v1 +metadata: + name: ozone +provisioner: org.apache.hadoop.ozone diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-service.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-service.yaml new file mode 100644 index 000000000000..929e7a259505 --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-service.yaml @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: v1 +kind: Service +metadata: + name: datanode +spec: + ports: + - port: 9870 + name: rpc + clusterIP: None + selector: + app: ozone + component: datanode diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-statefulset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-statefulset.yaml new file mode 100644 index 000000000000..d7599c60d53f --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-statefulset.yaml @@ -0,0 +1,72 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: apps/v1 +kind: StatefulSet +metadata: + name: datanode + labels: + app.kubernetes.io/component: ozone +spec: + selector: + matchLabels: + app: ozone + component: datanode + serviceName: datanode + replicas: 3 + template: + metadata: + labels: + app: ozone + component: datanode + annotations: + prometheus.io/scrape: "true" + prometheus.io/port: "9882" + prometheus.io/path: /prom + spec: + affinity: + podAntiAffinity: + requiredDuringSchedulingIgnoredDuringExecution: + - labelSelector: + matchExpressions: + - key: component + operator: In + values: + - datanode + topologyKey: kubernetes.io/hostname + securityContext: + fsGroup: 1000 + containers: + - name: datanode + image: '@docker.image@' + args: + - ozone + - datanode + envFrom: + - configMapRef: + name: config + volumeMounts: + - name: data + mountPath: /data + volumeClaimTemplates: + - metadata: + name: data + spec: + accessModes: + - ReadWriteOnce + resources: + requests: + storage: 20Gi diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/freon/freon-deployment.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/freon/freon-deployment.yaml new file mode 100644 index 000000000000..9c140331e0ca --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/freon/freon-deployment.yaml @@ -0,0 +1,46 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: apps/v1 +kind: Deployment +metadata: + name: freon + labels: + app.kubernetes.io/component: ozone +spec: + replicas: 1 + selector: + matchLabels: + app: ozone + component: freon + template: + metadata: + labels: + app: ozone + component: freon + spec: + containers: + - name: freon + image: '@docker.image@' + args: + - ozone + - freon + - rk + - --factor=THREE + - --replication-type=RATIS + envFrom: + - configMapRef: + name: config diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/kustomization.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/kustomization.yaml new file mode 100644 index 000000000000..6b3d553113e6 --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/kustomization.yaml @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +resources: +- config-configmap.yaml +- datanode-service.yaml +- datanode-statefulset.yaml +- om-service.yaml +- om-statefulset.yaml +- s3g-service.yaml +- s3g-statefulset.yaml +- scm-service.yaml +- scm-statefulset.yaml diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/om-service.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/om-service.yaml new file mode 100644 index 000000000000..617277d9b850 --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/om-service.yaml @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: v1 +kind: Service +metadata: + name: om +spec: + ports: + - port: 9874 + name: ui + clusterIP: None + selector: + app: ozone + component: om diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/om-statefulset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/om-statefulset.yaml new file mode 100644 index 000000000000..ad0b16eacae3 --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/om-statefulset.yaml @@ -0,0 +1,72 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: apps/v1 +kind: StatefulSet +metadata: + name: om + labels: + app.kubernetes.io/component: ozone +spec: + selector: + matchLabels: + app: ozone + component: om + serviceName: om + replicas: 1 + template: + metadata: + labels: + app: ozone + component: om + annotations: + prometheus.io/scrape: "true" + prometheus.io/port: "9874" + prometheus.io/path: /prom + spec: + securityContext: + fsGroup: 1000 + containers: + - name: om + image: '@docker.image@' + args: + - ozone + - om + env: + - name: WAITFOR + value: scm-0.scm:9876 + - name: ENSURE_OM_INITIALIZED + value: /data/metadata/om/current/VERSION + livenessProbe: + tcpSocket: + port: 9862 + initialDelaySeconds: 30 + envFrom: + - configMapRef: + name: config + volumeMounts: + - name: data + mountPath: /data + volumes: [] + volumeClaimTemplates: + - metadata: + name: data + spec: + accessModes: + - ReadWriteOnce + resources: + requests: + storage: 2Gi diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-deployment.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-deployment.yaml new file mode 100644 index 000000000000..04edcec9814d --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-deployment.yaml @@ -0,0 +1,50 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: apps/v1 +kind: Deployment +metadata: + name: ozone-csi-test-webserver + labels: + app: ozone-csi-test-webserver + annotations: {} +spec: + replicas: 1 + selector: + matchLabels: + app: ozone-csi-test-webserver + template: + metadata: + labels: + app: ozone-csi-test-webserver + spec: + containers: + - name: web + image: python:3.7.3-alpine3.8 + args: + - python + - -m + - http.server + - --directory + - /www + volumeMounts: + - mountPath: /www + name: webroot + volumes: + - name: webroot + persistentVolumeClaim: + claimName: ozone-csi-test-webserver + readOnly: false diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-persistentvolumeclaim.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-persistentvolumeclaim.yaml new file mode 100644 index 000000000000..4b1e44b206a8 --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-persistentvolumeclaim.yaml @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: v1 +kind: PersistentVolumeClaim +metadata: + name: ozone-csi-test-webserver + labels: {} + annotations: {} +spec: + accessModes: + - ReadWriteOnce + resources: + requests: + storage: 1Gi + storageClassName: ozone diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-service.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-service.yaml new file mode 100644 index 000000000000..6a53a4397f02 --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-service.yaml @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: v1 +kind: Service +metadata: + name: ozone-csi-test-webserver + labels: {} + annotations: {} +spec: + type: NodePort + ports: + - port: 8000 + name: web + selector: + app: ozone-csi-test-webserver diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/s3g-service.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/s3g-service.yaml new file mode 100644 index 000000000000..dd1ca8347991 --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/s3g-service.yaml @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: v1 +kind: Service +metadata: + name: s3g +spec: + ports: + - port: 9878 + name: rest + clusterIP: None + selector: + app: ozone + component: s3g diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/s3g-statefulset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/s3g-statefulset.yaml new file mode 100644 index 000000000000..6e96fb7dbcf9 --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/s3g-statefulset.yaml @@ -0,0 +1,61 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: apps/v1 +kind: StatefulSet +metadata: + name: s3g + labels: + app.kubernetes.io/component: ozone +spec: + selector: + matchLabels: + app: ozone + component: s3g + serviceName: s3g + replicas: 1 + template: + metadata: + labels: + app: ozone + component: s3g + spec: + containers: + - name: s3g + image: '@docker.image@' + args: + - ozone + - s3g + livenessProbe: + httpGet: + path: / + port: 9878 + initialDelaySeconds: 30 + envFrom: + - configMapRef: + name: config + volumeMounts: + - name: data + mountPath: /data + volumeClaimTemplates: + - metadata: + name: data + spec: + accessModes: + - ReadWriteOnce + resources: + requests: + storage: 2Gi diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/scm-service.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/scm-service.yaml new file mode 100644 index 000000000000..0df15d645315 --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/scm-service.yaml @@ -0,0 +1,28 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: v1 +kind: Service +metadata: + name: scm +spec: + ports: + - port: 9876 + name: ui + clusterIP: None + selector: + app: ozone + component: scm diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/scm-statefulset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/scm-statefulset.yaml new file mode 100644 index 000000000000..d4d651349f71 --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/scm-statefulset.yaml @@ -0,0 +1,79 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: apps/v1 +kind: StatefulSet +metadata: + name: scm + labels: + app.kubernetes.io/component: ozone +spec: + selector: + matchLabels: + app: ozone + component: scm + serviceName: scm + replicas: 1 + template: + metadata: + labels: + app: ozone + component: scm + annotations: + prometheus.io/scrape: "true" + prometheus.io/port: "9876" + prometheus.io/path: /prom + spec: + securityContext: + fsGroup: 1000 + initContainers: + - name: init + image: '@docker.image@' + args: + - ozone + - scm + - --init + envFrom: + - configMapRef: + name: config + volumeMounts: + - name: data + mountPath: /data + containers: + - name: scm + image: '@docker.image@' + args: + - ozone + - scm + livenessProbe: + tcpSocket: + port: 9861 + initialDelaySeconds: 30 + envFrom: + - configMapRef: + name: config + volumeMounts: + - name: data + mountPath: /data + volumeClaimTemplates: + - metadata: + name: data + spec: + accessModes: + - ReadWriteOnce + resources: + requests: + storage: 2Gi diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/test.sh b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/test.sh new file mode 100755 index 000000000000..2d58852e515d --- /dev/null +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/test.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +export K8S_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" + +cd "$K8S_DIR" + +# shellcheck source=/dev/null +source "../testlib.sh" + +rm -rf result + +regenerate_resources + +start_k8s_env + +execute_robot_test scm-0 smoketest/basic/basic.robot + +# restart datanodes +kubectl delete pod datanode-0 datanode-1 datanode-2 + +wait_for_startup + +combine_reports + +get_logs + +stop_k8s_env + +revert_resources From e6156e64f3d25957cf5cd8f46d2a0771db407113 Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Fri, 20 May 2022 12:01:22 -0700 Subject: [PATCH 13/29] fix compilation error --- .../hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java | 6 +++--- .../k8s/examples/ozone-dn-restart/datanode-statefulset.yaml | 3 +++ 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java index f31a8542d7d7..47ce604b4b0e 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java @@ -842,9 +842,9 @@ public void testGetStalePipelines() throws IOException { // test IP change List pipelineList2 = pipelineManager.getStalePipelines(node2); - assertEquals(2, pipelineList2.size()); - assertEquals(pipelines.get(0), pipelineList2.get(0)); - assertEquals(pipelines.get(3), pipelineList2.get(1)); + Assertions.assertEquals(2, pipelineList2.size()); + Assertions.assertEquals(pipelines.get(0), pipelineList2.get(0)); + Assertions.assertEquals(pipelines.get(3), pipelineList2.get(1)); } public void testCreatePipelineForRead() throws IOException { diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-statefulset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-statefulset.yaml index d7599c60d53f..3b5116e9c25f 100644 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-statefulset.yaml +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-statefulset.yaml @@ -55,6 +55,9 @@ spec: args: - ozone - datanode + env: + - name: OZONE-SITE.XML_dfs.datanode.use.datanode.hostname + value: "true" envFrom: - configMapRef: name: config From f94bb3df641b15d2d5923f83f88e2c75e26d581a Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" Date: Mon, 23 May 2022 09:34:49 +0200 Subject: [PATCH 14/29] Test read/write after restart --- .../k8s/examples/ozone/datanode-statefulset.yaml | 3 +++ .../dist/src/main/k8s/examples/ozone/test.sh | 14 +++++++++++++- 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone/datanode-statefulset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone/datanode-statefulset.yaml index d7599c60d53f..3b5116e9c25f 100644 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone/datanode-statefulset.yaml +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone/datanode-statefulset.yaml @@ -55,6 +55,9 @@ spec: args: - ozone - datanode + env: + - name: OZONE-SITE.XML_dfs.datanode.use.datanode.hostname + value: "true" envFrom: - configMapRef: name: config diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone/test.sh b/hadoop-ozone/dist/src/main/k8s/examples/ozone/test.sh index 7d6bdfb981e0..cc7e66e37bc2 100755 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone/test.sh +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone/test.sh @@ -28,7 +28,19 @@ regenerate_resources start_k8s_env -execute_robot_test scm-0 smoketest/basic/basic.robot +export SCM=scm-0 + +execute_robot_test ${SCM} -v PREFIX:pre freon/generate.robot +execute_robot_test ${SCM} -v PREFIX:pre freon/validate.robot + +# restart datanodes +kubectl delete pod datanode-0 datanode-1 datanode-2 + +wait_for_startup + +execute_robot_test ${SCM} -v PREFIX:pre freon/validate.robot +execute_robot_test ${SCM} -v PREFIX:post freon/generate.robot +execute_robot_test ${SCM} -v PREFIX:post freon/validate.robot combine_reports From c7993aea856ca9d13fbc5dd4713b776db48b56e1 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" Date: Mon, 23 May 2022 09:35:10 +0200 Subject: [PATCH 15/29] Remove ozone-dn-restart env --- .../k8s/examples/ozone-dn-restart/Flekszible | 36 ------- .../examples/ozone-dn-restart/LICENSE.header | 15 --- .../ozone-dn-restart/config-configmap.yaml | 38 ------- .../csi/csi-node-daemonset.yaml | 97 ------------------ .../csi/csi-ozone-clusterrole.yaml | 98 ------------------- .../csi/csi-ozone-clusterrolebinding.yaml | 28 ------ .../csi/csi-ozone-serviceaccount.yaml | 21 ---- .../csi/csi-provisioner-deployment.yaml | 54 ---------- .../org.apache.hadoop.ozone-csidriver.yaml | 22 ----- .../csi/ozone-storageclass.yaml | 21 ---- .../ozone-dn-restart/datanode-service.yaml | 28 ------ .../datanode-statefulset.yaml | 75 -------------- .../freon/freon-deployment.yaml | 46 --------- .../ozone-dn-restart/kustomization.yaml | 26 ----- .../examples/ozone-dn-restart/om-service.yaml | 28 ------ .../ozone-dn-restart/om-statefulset.yaml | 72 -------------- .../ozone-csi-test-webserver-deployment.yaml | 50 ---------- ...-test-webserver-persistentvolumeclaim.yaml | 29 ------ .../ozone-csi-test-webserver-service.yaml | 29 ------ .../ozone-dn-restart/s3g-service.yaml | 28 ------ .../ozone-dn-restart/s3g-statefulset.yaml | 61 ------------ .../ozone-dn-restart/scm-service.yaml | 28 ------ .../ozone-dn-restart/scm-statefulset.yaml | 79 --------------- .../k8s/examples/ozone-dn-restart/test.sh | 44 --------- 24 files changed, 1053 deletions(-) delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/Flekszible delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/LICENSE.header delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/config-configmap.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-node-daemonset.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-clusterrole.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-clusterrolebinding.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-serviceaccount.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-provisioner-deployment.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/org.apache.hadoop.ozone-csidriver.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/ozone-storageclass.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-service.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-statefulset.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/freon/freon-deployment.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/kustomization.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/om-service.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/om-statefulset.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-deployment.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-persistentvolumeclaim.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-service.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/s3g-service.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/s3g-statefulset.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/scm-service.yaml delete mode 100644 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/scm-statefulset.yaml delete mode 100755 hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/test.sh diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/Flekszible b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/Flekszible deleted file mode 100644 index 5562aac43c6a..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/Flekszible +++ /dev/null @@ -1,36 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -source: - - path: ../../definitions -import: - - path: ozone - transformations: - - type: Image - image: "@docker.image@" - - type: ozone/persistence - - type: ozone/csi - - path: ozone/freon - destination: freon - transformations: - - type: Image - image: "@docker.image@" - - path: ozone-csi - destination: csi - - path: test-webserver - destination: pv-test -transformations: - - type: Namespace - - type: kustomize diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/LICENSE.header b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/LICENSE.header deleted file mode 100644 index 635f0d9e60e1..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/LICENSE.header +++ /dev/null @@ -1,15 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/config-configmap.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/config-configmap.yaml deleted file mode 100644 index c7ac5344d142..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/config-configmap.yaml +++ /dev/null @@ -1,38 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -apiVersion: v1 -kind: ConfigMap -metadata: - name: config -data: - OZONE-SITE.XML_hdds.datanode.dir: /data/storage - OZONE-SITE.XML_ozone.scm.datanode.id.dir: /data - OZONE-SITE.XML_ozone.metadata.dirs: /data/metadata - OZONE-SITE.XML_ozone.scm.block.client.address: scm-0.scm - OZONE-SITE.XML_ozone.om.address: om-0.om - OZONE-SITE.XML_ozone.scm.client.address: scm-0.scm - OZONE-SITE.XML_ozone.scm.names: scm-0.scm - OZONE-SITE.XML_hdds.scm.safemode.min.datanode: "3" - OZONE-SITE.XML_ozone.datanode.pipeline.limit: "1" - LOG4J.PROPERTIES_log4j.rootLogger: INFO, stdout - LOG4J.PROPERTIES_log4j.appender.stdout: org.apache.log4j.ConsoleAppender - LOG4J.PROPERTIES_log4j.appender.stdout.layout: org.apache.log4j.PatternLayout - LOG4J.PROPERTIES_log4j.appender.stdout.layout.ConversionPattern: '%d{yyyy-MM-dd - HH:mm:ss} %-5p %c{1}:%L - %m%n' - OZONE-SITE.XML_ozone.csi.s3g.address: http://s3g-0.s3g:9878 - OZONE-SITE.XML_ozone.csi.socket: /var/lib/csi/csi.sock - OZONE-SITE.XML_ozone.csi.owner: hadoop diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-node-daemonset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-node-daemonset.yaml deleted file mode 100644 index 1272053720f3..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-node-daemonset.yaml +++ /dev/null @@ -1,97 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -kind: DaemonSet -apiVersion: apps/v1 -metadata: - name: csi-node -spec: - selector: - matchLabels: - app: csi-node - template: - metadata: - labels: - app: csi-node - spec: - serviceAccount: csi-ozone - containers: - - name: driver-registrar - image: quay.io/k8scsi/csi-node-driver-registrar:v1.0.2 - args: - - --v=4 - - --csi-address=/var/lib/csi/csi.sock - - --kubelet-registration-path=/var/lib/kubelet/plugins/org.apache.hadoop.ozone/csi.sock - env: - - name: KUBE_NODE_NAME - valueFrom: - fieldRef: - fieldPath: spec.nodeName - volumeMounts: - - name: plugin-dir - mountPath: /var/lib/csi - - name: registration-dir - mountPath: /registration/ - - name: csi-node - image: '@docker.image@' - securityContext: - runAsUser: 0 - privileged: true - capabilities: - add: - - SYS_ADMIN - allowPrivilegeEscalation: true - args: - - ozone - - csi - envFrom: - - configMapRef: - name: config - imagePullPolicy: IfNotPresent - volumeMounts: - - name: plugin-dir - mountPath: /var/lib/csi - - name: pods-mount-dir - mountPath: /var/lib/kubelet/pods - mountPropagation: Bidirectional - - name: fuse-device - mountPath: /dev/fuse - - name: dbus - mountPath: /var/run/dbus - - name: systemd - mountPath: /run/systemd - volumes: - - name: plugin-dir - hostPath: - path: /var/lib/kubelet/plugins/org.apache.hadoop.ozone - type: DirectoryOrCreate - - name: registration-dir - hostPath: - path: /var/lib/kubelet/plugins_registry/ - type: DirectoryOrCreate - - name: pods-mount-dir - hostPath: - path: /var/lib/kubelet/pods - type: Directory - - name: fuse-device - hostPath: - path: /dev/fuse - - name: dbus - hostPath: - path: /var/run/dbus - - name: systemd - hostPath: - path: /run/systemd diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-clusterrole.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-clusterrole.yaml deleted file mode 100644 index 927ba6ff7b7f..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-clusterrole.yaml +++ /dev/null @@ -1,98 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -kind: ClusterRole -apiVersion: rbac.authorization.k8s.io/v1 -metadata: - name: csi-ozone-default -rules: -- apiGroups: - - "" - resources: - - secrets - verbs: - - get - - list -- apiGroups: - - "" - resources: - - events - verbs: - - list - - watch - - create - - update - - patch -- apiGroups: - - "" - resources: - - nodes - verbs: - - get - - list - - update - - watch -- apiGroups: - - "" - resources: - - namespaces - verbs: - - get - - list -- apiGroups: - - storage.k8s.io - resources: - - storageclasses - verbs: - - get - - list - - watch -- apiGroups: - - "" - resources: - - persistentvolumeclaims - verbs: - - get - - list - - watch - - update -- apiGroups: - - "" - resources: - - persistentvolumes - verbs: - - get - - list - - watch - - update - - create -- apiGroups: - - storage.k8s.io - resources: - - volumeattachments - verbs: - - get - - list - - watch - - update -- apiGroups: - - storage.k8s.io - resources: - - csinodes - verbs: - - get - - list - - watch diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-clusterrolebinding.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-clusterrolebinding.yaml deleted file mode 100644 index 948e759fbe35..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-clusterrolebinding.yaml +++ /dev/null @@ -1,28 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -kind: ClusterRoleBinding -apiVersion: rbac.authorization.k8s.io/v1 -metadata: - name: csi-ozone-default -subjects: -- kind: ServiceAccount - name: csi-ozone - namespace: default -roleRef: - kind: ClusterRole - name: csi-ozone-default - apiGroup: rbac.authorization.k8s.io diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-serviceaccount.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-serviceaccount.yaml deleted file mode 100644 index 628d2a1c5957..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-ozone-serviceaccount.yaml +++ /dev/null @@ -1,21 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -apiVersion: v1 -kind: ServiceAccount -metadata: - namespace: default - name: csi-ozone diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-provisioner-deployment.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-provisioner-deployment.yaml deleted file mode 100644 index 81837122d16c..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/csi-provisioner-deployment.yaml +++ /dev/null @@ -1,54 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -kind: Deployment -apiVersion: apps/v1 -metadata: - name: csi-provisioner -spec: - replicas: 1 - selector: - matchLabels: - app: csi-provisioner - template: - metadata: - labels: - app: csi-provisioner - spec: - serviceAccount: csi-ozone - containers: - - name: csi-provisioner - image: quay.io/k8scsi/csi-provisioner:v1.0.1 - args: - - --csi-address=/var/lib/csi/csi.sock - volumeMounts: - - name: socket-dir - mountPath: /var/lib/csi/ - - name: ozone-csi - image: '@docker.image@' - volumeMounts: - - name: socket-dir - mountPath: /var/lib/csi/ - imagePullPolicy: IfNotPresent - envFrom: - - configMapRef: - name: config - args: - - ozone - - csi - volumes: - - name: socket-dir - emptyDir: {} diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/org.apache.hadoop.ozone-csidriver.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/org.apache.hadoop.ozone-csidriver.yaml deleted file mode 100644 index e657c50f7583..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/org.apache.hadoop.ozone-csidriver.yaml +++ /dev/null @@ -1,22 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -apiVersion: storage.k8s.io/v1beta1 -kind: CSIDriver -metadata: - name: org.apache.hadoop.ozone -spec: - attachRequired: false diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/ozone-storageclass.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/ozone-storageclass.yaml deleted file mode 100644 index c6c1c6c9d1e1..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/csi/ozone-storageclass.yaml +++ /dev/null @@ -1,21 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -kind: StorageClass -apiVersion: storage.k8s.io/v1 -metadata: - name: ozone -provisioner: org.apache.hadoop.ozone diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-service.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-service.yaml deleted file mode 100644 index 929e7a259505..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-service.yaml +++ /dev/null @@ -1,28 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -apiVersion: v1 -kind: Service -metadata: - name: datanode -spec: - ports: - - port: 9870 - name: rpc - clusterIP: None - selector: - app: ozone - component: datanode diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-statefulset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-statefulset.yaml deleted file mode 100644 index 3b5116e9c25f..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/datanode-statefulset.yaml +++ /dev/null @@ -1,75 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -apiVersion: apps/v1 -kind: StatefulSet -metadata: - name: datanode - labels: - app.kubernetes.io/component: ozone -spec: - selector: - matchLabels: - app: ozone - component: datanode - serviceName: datanode - replicas: 3 - template: - metadata: - labels: - app: ozone - component: datanode - annotations: - prometheus.io/scrape: "true" - prometheus.io/port: "9882" - prometheus.io/path: /prom - spec: - affinity: - podAntiAffinity: - requiredDuringSchedulingIgnoredDuringExecution: - - labelSelector: - matchExpressions: - - key: component - operator: In - values: - - datanode - topologyKey: kubernetes.io/hostname - securityContext: - fsGroup: 1000 - containers: - - name: datanode - image: '@docker.image@' - args: - - ozone - - datanode - env: - - name: OZONE-SITE.XML_dfs.datanode.use.datanode.hostname - value: "true" - envFrom: - - configMapRef: - name: config - volumeMounts: - - name: data - mountPath: /data - volumeClaimTemplates: - - metadata: - name: data - spec: - accessModes: - - ReadWriteOnce - resources: - requests: - storage: 20Gi diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/freon/freon-deployment.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/freon/freon-deployment.yaml deleted file mode 100644 index 9c140331e0ca..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/freon/freon-deployment.yaml +++ /dev/null @@ -1,46 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -apiVersion: apps/v1 -kind: Deployment -metadata: - name: freon - labels: - app.kubernetes.io/component: ozone -spec: - replicas: 1 - selector: - matchLabels: - app: ozone - component: freon - template: - metadata: - labels: - app: ozone - component: freon - spec: - containers: - - name: freon - image: '@docker.image@' - args: - - ozone - - freon - - rk - - --factor=THREE - - --replication-type=RATIS - envFrom: - - configMapRef: - name: config diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/kustomization.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/kustomization.yaml deleted file mode 100644 index 6b3d553113e6..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/kustomization.yaml +++ /dev/null @@ -1,26 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -resources: -- config-configmap.yaml -- datanode-service.yaml -- datanode-statefulset.yaml -- om-service.yaml -- om-statefulset.yaml -- s3g-service.yaml -- s3g-statefulset.yaml -- scm-service.yaml -- scm-statefulset.yaml diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/om-service.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/om-service.yaml deleted file mode 100644 index 617277d9b850..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/om-service.yaml +++ /dev/null @@ -1,28 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -apiVersion: v1 -kind: Service -metadata: - name: om -spec: - ports: - - port: 9874 - name: ui - clusterIP: None - selector: - app: ozone - component: om diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/om-statefulset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/om-statefulset.yaml deleted file mode 100644 index ad0b16eacae3..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/om-statefulset.yaml +++ /dev/null @@ -1,72 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -apiVersion: apps/v1 -kind: StatefulSet -metadata: - name: om - labels: - app.kubernetes.io/component: ozone -spec: - selector: - matchLabels: - app: ozone - component: om - serviceName: om - replicas: 1 - template: - metadata: - labels: - app: ozone - component: om - annotations: - prometheus.io/scrape: "true" - prometheus.io/port: "9874" - prometheus.io/path: /prom - spec: - securityContext: - fsGroup: 1000 - containers: - - name: om - image: '@docker.image@' - args: - - ozone - - om - env: - - name: WAITFOR - value: scm-0.scm:9876 - - name: ENSURE_OM_INITIALIZED - value: /data/metadata/om/current/VERSION - livenessProbe: - tcpSocket: - port: 9862 - initialDelaySeconds: 30 - envFrom: - - configMapRef: - name: config - volumeMounts: - - name: data - mountPath: /data - volumes: [] - volumeClaimTemplates: - - metadata: - name: data - spec: - accessModes: - - ReadWriteOnce - resources: - requests: - storage: 2Gi diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-deployment.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-deployment.yaml deleted file mode 100644 index 04edcec9814d..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-deployment.yaml +++ /dev/null @@ -1,50 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -apiVersion: apps/v1 -kind: Deployment -metadata: - name: ozone-csi-test-webserver - labels: - app: ozone-csi-test-webserver - annotations: {} -spec: - replicas: 1 - selector: - matchLabels: - app: ozone-csi-test-webserver - template: - metadata: - labels: - app: ozone-csi-test-webserver - spec: - containers: - - name: web - image: python:3.7.3-alpine3.8 - args: - - python - - -m - - http.server - - --directory - - /www - volumeMounts: - - mountPath: /www - name: webroot - volumes: - - name: webroot - persistentVolumeClaim: - claimName: ozone-csi-test-webserver - readOnly: false diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-persistentvolumeclaim.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-persistentvolumeclaim.yaml deleted file mode 100644 index 4b1e44b206a8..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-persistentvolumeclaim.yaml +++ /dev/null @@ -1,29 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -apiVersion: v1 -kind: PersistentVolumeClaim -metadata: - name: ozone-csi-test-webserver - labels: {} - annotations: {} -spec: - accessModes: - - ReadWriteOnce - resources: - requests: - storage: 1Gi - storageClassName: ozone diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-service.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-service.yaml deleted file mode 100644 index 6a53a4397f02..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/pv-test/ozone-csi-test-webserver-service.yaml +++ /dev/null @@ -1,29 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -apiVersion: v1 -kind: Service -metadata: - name: ozone-csi-test-webserver - labels: {} - annotations: {} -spec: - type: NodePort - ports: - - port: 8000 - name: web - selector: - app: ozone-csi-test-webserver diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/s3g-service.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/s3g-service.yaml deleted file mode 100644 index dd1ca8347991..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/s3g-service.yaml +++ /dev/null @@ -1,28 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -apiVersion: v1 -kind: Service -metadata: - name: s3g -spec: - ports: - - port: 9878 - name: rest - clusterIP: None - selector: - app: ozone - component: s3g diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/s3g-statefulset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/s3g-statefulset.yaml deleted file mode 100644 index 6e96fb7dbcf9..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/s3g-statefulset.yaml +++ /dev/null @@ -1,61 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -apiVersion: apps/v1 -kind: StatefulSet -metadata: - name: s3g - labels: - app.kubernetes.io/component: ozone -spec: - selector: - matchLabels: - app: ozone - component: s3g - serviceName: s3g - replicas: 1 - template: - metadata: - labels: - app: ozone - component: s3g - spec: - containers: - - name: s3g - image: '@docker.image@' - args: - - ozone - - s3g - livenessProbe: - httpGet: - path: / - port: 9878 - initialDelaySeconds: 30 - envFrom: - - configMapRef: - name: config - volumeMounts: - - name: data - mountPath: /data - volumeClaimTemplates: - - metadata: - name: data - spec: - accessModes: - - ReadWriteOnce - resources: - requests: - storage: 2Gi diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/scm-service.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/scm-service.yaml deleted file mode 100644 index 0df15d645315..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/scm-service.yaml +++ /dev/null @@ -1,28 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -apiVersion: v1 -kind: Service -metadata: - name: scm -spec: - ports: - - port: 9876 - name: ui - clusterIP: None - selector: - app: ozone - component: scm diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/scm-statefulset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/scm-statefulset.yaml deleted file mode 100644 index d4d651349f71..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/scm-statefulset.yaml +++ /dev/null @@ -1,79 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -apiVersion: apps/v1 -kind: StatefulSet -metadata: - name: scm - labels: - app.kubernetes.io/component: ozone -spec: - selector: - matchLabels: - app: ozone - component: scm - serviceName: scm - replicas: 1 - template: - metadata: - labels: - app: ozone - component: scm - annotations: - prometheus.io/scrape: "true" - prometheus.io/port: "9876" - prometheus.io/path: /prom - spec: - securityContext: - fsGroup: 1000 - initContainers: - - name: init - image: '@docker.image@' - args: - - ozone - - scm - - --init - envFrom: - - configMapRef: - name: config - volumeMounts: - - name: data - mountPath: /data - containers: - - name: scm - image: '@docker.image@' - args: - - ozone - - scm - livenessProbe: - tcpSocket: - port: 9861 - initialDelaySeconds: 30 - envFrom: - - configMapRef: - name: config - volumeMounts: - - name: data - mountPath: /data - volumeClaimTemplates: - - metadata: - name: data - spec: - accessModes: - - ReadWriteOnce - resources: - requests: - storage: 2Gi diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/test.sh b/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/test.sh deleted file mode 100755 index 2d58852e515d..000000000000 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone-dn-restart/test.sh +++ /dev/null @@ -1,44 +0,0 @@ -#!/usr/bin/env bash -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -export K8S_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" - -cd "$K8S_DIR" - -# shellcheck source=/dev/null -source "../testlib.sh" - -rm -rf result - -regenerate_resources - -start_k8s_env - -execute_robot_test scm-0 smoketest/basic/basic.robot - -# restart datanodes -kubectl delete pod datanode-0 datanode-1 datanode-2 - -wait_for_startup - -combine_reports - -get_logs - -stop_k8s_env - -revert_resources From 52d20d206a0d9c71d22c4445adb8bd5b448dbf54 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" Date: Mon, 23 May 2022 10:12:39 +0200 Subject: [PATCH 16/29] Fix smoketest path --- hadoop-ozone/dist/src/main/k8s/examples/ozone/test.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone/test.sh b/hadoop-ozone/dist/src/main/k8s/examples/ozone/test.sh index cc7e66e37bc2..fd5b0331d0b4 100755 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone/test.sh +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone/test.sh @@ -30,17 +30,17 @@ start_k8s_env export SCM=scm-0 -execute_robot_test ${SCM} -v PREFIX:pre freon/generate.robot -execute_robot_test ${SCM} -v PREFIX:pre freon/validate.robot +execute_robot_test ${SCM} -v PREFIX:pre smoketest/freon/generate.robot +execute_robot_test ${SCM} -v PREFIX:pre smoketest/freon/validate.robot # restart datanodes kubectl delete pod datanode-0 datanode-1 datanode-2 wait_for_startup -execute_robot_test ${SCM} -v PREFIX:pre freon/validate.robot -execute_robot_test ${SCM} -v PREFIX:post freon/generate.robot -execute_robot_test ${SCM} -v PREFIX:post freon/validate.robot +execute_robot_test ${SCM} -v PREFIX:pre smoketest/freon/validate.robot +execute_robot_test ${SCM} -v PREFIX:post smoketest/freon/generate.robot +execute_robot_test ${SCM} -v PREFIX:post smoketest/freon/validate.robot combine_reports From aa2fb97056dcc91706f6c5cf42e82418768a0739 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" Date: Mon, 23 May 2022 10:46:58 +0200 Subject: [PATCH 17/29] Default values for variables --- hadoop-ozone/dist/src/main/smoketest/ozone-lib/freon.robot | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/hadoop-ozone/dist/src/main/smoketest/ozone-lib/freon.robot b/hadoop-ozone/dist/src/main/smoketest/ozone-lib/freon.robot index 563af1ed3cd7..8d10cc81e900 100644 --- a/hadoop-ozone/dist/src/main/smoketest/ozone-lib/freon.robot +++ b/hadoop-ozone/dist/src/main/smoketest/ozone-lib/freon.robot @@ -16,6 +16,10 @@ *** Settings *** Resource ../lib/os.robot +*** Variables *** +${OM_HA_PARAM} ${EMPTY} +${SECURITY_ENABLED} false + *** Keywords *** Freon DCG [arguments] ${prefix}=dcg ${n}=1 ${threads}=1 ${args}=${EMPTY} From 627a568e559ceee7dd3a27d1f0823dc38368f6b9 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" Date: Mon, 23 May 2022 11:42:09 +0200 Subject: [PATCH 18/29] Skip chunk generator/validator in kubernetes --- .../dist/src/main/compose/restart/test.sh | 5 ++++ .../main/smoketest/freon/generate-chunk.robot | 26 +++++++++++++++++++ .../src/main/smoketest/freon/generate.robot | 3 --- .../main/smoketest/freon/validate-chunk.robot | 26 +++++++++++++++++++ .../src/main/smoketest/freon/validate.robot | 3 --- 5 files changed, 57 insertions(+), 6 deletions(-) create mode 100644 hadoop-ozone/dist/src/main/smoketest/freon/generate-chunk.robot create mode 100644 hadoop-ozone/dist/src/main/smoketest/freon/validate-chunk.robot diff --git a/hadoop-ozone/dist/src/main/compose/restart/test.sh b/hadoop-ozone/dist/src/main/compose/restart/test.sh index cf0f53242da5..f73c263b5c9d 100644 --- a/hadoop-ozone/dist/src/main/compose/restart/test.sh +++ b/hadoop-ozone/dist/src/main/compose/restart/test.sh @@ -33,15 +33,20 @@ fix_data_dir_permissions start_docker_env execute_robot_test scm -v PREFIX:pre freon/generate.robot execute_robot_test scm -v PREFIX:pre freon/validate.robot +execute_robot_test scm -v PREFIX:pre freon/generate-chunk.robot +execute_robot_test scm -v PREFIX:pre freon/validate-chunk.robot KEEP_RUNNING=false stop_docker_env # re-start cluster with new version and check after upgrade export OZONE_KEEP_RESULTS=true start_docker_env execute_robot_test scm -v PREFIX:pre freon/validate.robot +execute_robot_test scm -v PREFIX:pre freon/validate-chunk.robot # test write key to old bucket after upgrade execute_robot_test scm -v PREFIX:post freon/generate.robot execute_robot_test scm -v PREFIX:post freon/validate.robot +execute_robot_test scm -v PREFIX:post freon/generate-chunk.robot +execute_robot_test scm -v PREFIX:post freon/validate-chunk.robot stop_docker_env generate_report diff --git a/hadoop-ozone/dist/src/main/smoketest/freon/generate-chunk.robot b/hadoop-ozone/dist/src/main/smoketest/freon/generate-chunk.robot new file mode 100644 index 000000000000..5742338c399a --- /dev/null +++ b/hadoop-ozone/dist/src/main/smoketest/freon/generate-chunk.robot @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +*** Settings *** +Documentation Test freon chunk generation commands +Resource ../ozone-lib/freon.robot +Test Timeout 5 minutes + +*** Variables *** +${PREFIX} ${EMPTY} + +*** Test Cases *** +DN Chunk Generator + Freon DCG prefix=dcg${PREFIX} n=100 diff --git a/hadoop-ozone/dist/src/main/smoketest/freon/generate.robot b/hadoop-ozone/dist/src/main/smoketest/freon/generate.robot index 4611a3c33ea2..7af2003c51d7 100644 --- a/hadoop-ozone/dist/src/main/smoketest/freon/generate.robot +++ b/hadoop-ozone/dist/src/main/smoketest/freon/generate.robot @@ -30,6 +30,3 @@ OM Key Generator OM Bucket Generator Freon OMBG prefix=ombg${PREFIX} - -DN Chunk Generator - Freon DCG prefix=dcg${PREFIX} n=100 diff --git a/hadoop-ozone/dist/src/main/smoketest/freon/validate-chunk.robot b/hadoop-ozone/dist/src/main/smoketest/freon/validate-chunk.robot new file mode 100644 index 000000000000..26d3c2aac06a --- /dev/null +++ b/hadoop-ozone/dist/src/main/smoketest/freon/validate-chunk.robot @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +*** Settings *** +Documentation Test freon chunk validation commands +Resource ../ozone-lib/freon.robot +Test Timeout 5 minutes + +*** Variables *** +${PREFIX} ${EMPTY} + +*** Test Cases *** +DN Chunk Validator + Freon DCV prefix=dcg${PREFIX} n=100 diff --git a/hadoop-ozone/dist/src/main/smoketest/freon/validate.robot b/hadoop-ozone/dist/src/main/smoketest/freon/validate.robot index 4f782a5bfec4..243da4ba1e39 100644 --- a/hadoop-ozone/dist/src/main/smoketest/freon/validate.robot +++ b/hadoop-ozone/dist/src/main/smoketest/freon/validate.robot @@ -24,6 +24,3 @@ ${PREFIX} ${EMPTY} *** Test Cases *** Ozone Client Key Validator Freon OCKV prefix=ockg${PREFIX} - -DN Chunk Validator - Freon DCV prefix=dcg${PREFIX} n=100 From 4404106407a9d519bedfd30b0a12d477a0b96f05 Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Mon, 30 May 2022 15:59:11 -0700 Subject: [PATCH 19/29] merge ozone and ozne-dn-restart environment --- .../dist/src/main/k8s/examples/ozone/config-configmap.yaml | 1 + .../dist/src/main/k8s/examples/ozone/datanode-statefulset.yaml | 2 ++ .../dist/src/main/k8s/examples/ozone/om-statefulset.yaml | 1 + .../dist/src/main/k8s/examples/ozone/s3g-statefulset.yaml | 1 + .../dist/src/main/k8s/examples/ozone/scm-statefulset.yaml | 2 ++ 5 files changed, 7 insertions(+) diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone/config-configmap.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone/config-configmap.yaml index c7ac5344d142..92fe9166d031 100644 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone/config-configmap.yaml +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone/config-configmap.yaml @@ -28,6 +28,7 @@ data: OZONE-SITE.XML_ozone.scm.names: scm-0.scm OZONE-SITE.XML_hdds.scm.safemode.min.datanode: "3" OZONE-SITE.XML_ozone.datanode.pipeline.limit: "1" + OZONE-SITE.XML_dfs.datanode.use.datanode.hostname: "true" LOG4J.PROPERTIES_log4j.rootLogger: INFO, stdout LOG4J.PROPERTIES_log4j.appender.stdout: org.apache.log4j.ConsoleAppender LOG4J.PROPERTIES_log4j.appender.stdout.layout: org.apache.log4j.PatternLayout diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone/datanode-statefulset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone/datanode-statefulset.yaml index 3b5116e9c25f..276b1ce8873f 100644 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone/datanode-statefulset.yaml +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone/datanode-statefulset.yaml @@ -27,6 +27,7 @@ spec: component: datanode serviceName: datanode replicas: 3 + podManagementPolicy: Parallel template: metadata: labels: @@ -52,6 +53,7 @@ spec: containers: - name: datanode image: '@docker.image@' + imagePullPolicy: Always args: - ozone - datanode diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone/om-statefulset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone/om-statefulset.yaml index ad0b16eacae3..84b81dd1bf49 100644 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone/om-statefulset.yaml +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone/om-statefulset.yaml @@ -42,6 +42,7 @@ spec: containers: - name: om image: '@docker.image@' + imagePullPolicy: Always args: - ozone - om diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone/s3g-statefulset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone/s3g-statefulset.yaml index 6e96fb7dbcf9..8a17c72c3857 100644 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone/s3g-statefulset.yaml +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone/s3g-statefulset.yaml @@ -36,6 +36,7 @@ spec: containers: - name: s3g image: '@docker.image@' + imagePullPolicy: Always args: - ozone - s3g diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone/scm-statefulset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone/scm-statefulset.yaml index d4d651349f71..6efc374d00fe 100644 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone/scm-statefulset.yaml +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone/scm-statefulset.yaml @@ -42,6 +42,7 @@ spec: initContainers: - name: init image: '@docker.image@' + imagePullPolicy: Always args: - ozone - scm @@ -55,6 +56,7 @@ spec: containers: - name: scm image: '@docker.image@' + imagePullPolicy: Always args: - ozone - scm From 8bbca39a9758e884361070b38aa0c603fc62eb6a Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Mon, 30 May 2022 16:01:46 -0700 Subject: [PATCH 20/29] delete unecessary dn env setting --- .../dist/src/main/k8s/examples/ozone/datanode-statefulset.yaml | 3 --- 1 file changed, 3 deletions(-) diff --git a/hadoop-ozone/dist/src/main/k8s/examples/ozone/datanode-statefulset.yaml b/hadoop-ozone/dist/src/main/k8s/examples/ozone/datanode-statefulset.yaml index 276b1ce8873f..4d510ba26b84 100644 --- a/hadoop-ozone/dist/src/main/k8s/examples/ozone/datanode-statefulset.yaml +++ b/hadoop-ozone/dist/src/main/k8s/examples/ozone/datanode-statefulset.yaml @@ -57,9 +57,6 @@ spec: args: - ozone - datanode - env: - - name: OZONE-SITE.XML_dfs.datanode.use.datanode.hostname - value: "true" envFrom: - configMapRef: name: config From aeeee2720a0fff617d8f097855fc50df21648f7b Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Mon, 30 May 2022 22:55:30 -0700 Subject: [PATCH 21/29] trun two logs from debug to info --- .../main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java index 36cf339e4822..811bd7fb11f1 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java @@ -104,13 +104,13 @@ private static String toRaftPeerAddress(DatanodeDetails id, Port.Name port) { if (datanodeUseHostName()) { final String address = id.getHostName() + ":" + id.getPort(port).getValue(); - LOG.debug("Datanode is using hostname for raft peer address: {}", + LOG.info("Datanode is using hostname for raft peer address: {}", address); return address; } else { final String address = id.getIpAddress() + ":" + id.getPort(port).getValue(); - LOG.debug("Datanode is using IP for raft peer address: {}", address); + LOG.info("Datanode is using IP for raft peer address: {}", address); return address; } } From db20d845ecbefcc8afcd34f3eae2af33624e6242 Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Wed, 1 Jun 2022 02:16:34 -0700 Subject: [PATCH 22/29] make block allocation more reliable when there are pipelines are in allocated state --- .../WritableRatisContainerProvider.java | 63 +++++++++++++------ .../main/k8s/definitions/ozone/config.yaml | 1 + 2 files changed, 45 insertions(+), 19 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableRatisContainerProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableRatisContainerProvider.java index e5605ccdf56a..8af66ad1762c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableRatisContainerProvider.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableRatisContainerProvider.java @@ -33,6 +33,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.Comparator; import java.util.List; /** @@ -92,15 +93,10 @@ public ContainerInfo getContainer(final long size, // mentioned in HDDS-5655. pipelineManager.acquireReadLock(); try { - availablePipelines = pipelineManager.getPipelines(repConfig, - Pipeline.PipelineState.OPEN, excludeList.getDatanodes(), - excludeList.getPipelineIds()); - if (availablePipelines.size() == 0 && !excludeList.isEmpty()) { - // if no pipelines can be found, try finding pipeline without - // exclusion - availablePipelines = pipelineManager - .getPipelines(repConfig, Pipeline.PipelineState.OPEN); - } + availablePipelines = + findPipelinesByState(repConfig, + excludeList, + Pipeline.PipelineState.OPEN); if (availablePipelines.size() != 0) { containerInfo = selectContainer(availablePipelines, size, owner, excludeList); @@ -124,7 +120,27 @@ public ContainerInfo getContainer(final long size, } catch (SCMException se) { LOG.warn("Pipeline creation failed for repConfig {} " + "Datanodes may be used up.", repConfig, se); - break; + LOG.info("Try to see if any pipeline is in ALLOCATED state, " + + "and then will wait for it to be OPEN"); + List allocatedPipelines = findPipelinesByState(repConfig, + excludeList, + Pipeline.PipelineState.ALLOCATED); + if (!allocatedPipelines.isEmpty()) { + // wait until the oldest allocated pipeline is ready + Pipeline oldestAllocatedPipeline = + allocatedPipelines + .stream() + .min(Comparator + .comparing(p -> p.getCreationTimestamp())) + .get(); + try { + pipelineManager + .waitPipelineReady(oldestAllocatedPipeline.getId(), 0); + } catch (IOException e) { + LOG.warn("Waiting for pipeline {} to be OPEN failed. ", + oldestAllocatedPipeline, e); + } + } } catch (IOException e) { LOG.warn("Pipeline creation failed for repConfig: {}. " + "Retrying get pipelines call once.", repConfig, e); @@ -134,15 +150,9 @@ public ContainerInfo getContainer(final long size, try { // If Exception occurred or successful creation of pipeline do one // final try to fetch pipelines. - availablePipelines = pipelineManager - .getPipelines(repConfig, Pipeline.PipelineState.OPEN, - excludeList.getDatanodes(), excludeList.getPipelineIds()); - if (availablePipelines.size() == 0 && !excludeList.isEmpty()) { - // if no pipelines can be found, try finding pipeline without - // exclusion - availablePipelines = pipelineManager - .getPipelines(repConfig, Pipeline.PipelineState.OPEN); - } + availablePipelines = findPipelinesByState(repConfig, + excludeList, + Pipeline.PipelineState.OPEN); if (availablePipelines.size() == 0) { LOG.info("Could not find available pipeline of repConfig: {} " + "even after retrying", repConfig); @@ -167,6 +177,21 @@ public ContainerInfo getContainer(final long size, return null; } + private List findPipelinesByState( + final ReplicationConfig repConfig, + final ExcludeList excludeList, + final Pipeline.PipelineState pipelineState) { + List pipelines = pipelineManager.getPipelines(repConfig, + pipelineState, excludeList.getDatanodes(), + excludeList.getPipelineIds()); + if (pipelines.size() == 0 && !excludeList.isEmpty()) { + // if no pipelines can be found, try finding pipeline without + // exclusion + pipelines = pipelineManager.getPipelines(repConfig, pipelineState); + } + return pipelines; + } + private ContainerInfo selectContainer(List availablePipelines, long size, String owner, ExcludeList excludeList) { Pipeline pipeline; diff --git a/hadoop-ozone/dist/src/main/k8s/definitions/ozone/config.yaml b/hadoop-ozone/dist/src/main/k8s/definitions/ozone/config.yaml index 7b65a3ecc40c..88a36835c290 100644 --- a/hadoop-ozone/dist/src/main/k8s/definitions/ozone/config.yaml +++ b/hadoop-ozone/dist/src/main/k8s/definitions/ozone/config.yaml @@ -28,6 +28,7 @@ data: OZONE-SITE.XML_ozone.scm.names: "scm-0.scm" OZONE-SITE.XML_hdds.scm.safemode.min.datanode: "3" OZONE-SITE.XML_ozone.datanode.pipeline.limit: "1" + OZONE-SITE.XML_dfs.datanode.use.datanode.hostname: "true" LOG4J.PROPERTIES_log4j.rootLogger: "INFO, stdout" LOG4J.PROPERTIES_log4j.appender.stdout: "org.apache.log4j.ConsoleAppender" LOG4J.PROPERTIES_log4j.appender.stdout.layout: "org.apache.log4j.PatternLayout" From 62c46eb5bc388cf54a5c97e8510401a196cbc550 Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Wed, 1 Jun 2022 15:06:49 -0700 Subject: [PATCH 23/29] address PR comments --- .../apache/hadoop/hdds/ratis/RatisHelper.java | 4 +- .../hdds/scm/pipeline/PipelineManager.java | 13 ++++ .../scm/pipeline/PipelineManagerImpl.java | 63 +++++++++++++------ .../WritableRatisContainerProvider.java | 25 ++++---- 4 files changed, 70 insertions(+), 35 deletions(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java index 811bd7fb11f1..36cf339e4822 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java @@ -104,13 +104,13 @@ private static String toRaftPeerAddress(DatanodeDetails id, Port.Name port) { if (datanodeUseHostName()) { final String address = id.getHostName() + ":" + id.getPort(port).getValue(); - LOG.info("Datanode is using hostname for raft peer address: {}", + LOG.debug("Datanode is using hostname for raft peer address: {}", address); return address; } else { final String address = id.getIpAddress() + ":" + id.getPort(port).getValue(); - LOG.info("Datanode is using IP for raft peer address: {}", address); + LOG.debug("Datanode is using IP for raft peer address: {}", address); return address; } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java index 5b42c6cd85e6..d3c4391bd795 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java @@ -152,6 +152,19 @@ default void waitPipelineReady(PipelineID pipelineID, long timeout) throws IOException { } + /** + * Wait one pipeline to be OPEN among a collection pipelines + * @param pipelineIDs ID collection of the pipelines to wait for + * @param timeout wait timeout(millisecond), if 0, use default timeout + * @return Pipeline the pipeline which is OPEN + * @throws IOException in case of any Exception, such as timeout + */ + default Pipeline waitOnePipelineReady(Collection pipelineIDs, + long timeout) + throws IOException { + return null; + } + /** * Get SafeMode status. * @return boolean diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java index 06c95f37f3e8..a587fb96243e 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java @@ -42,6 +42,7 @@ import org.apache.hadoop.metrics2.util.MBeans; import org.apache.hadoop.ozone.ClientVersion; import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException; +import org.apache.hadoop.thirdparty.com.google.common.collect.Lists; import org.apache.ratis.protocol.exceptions.NotLeaderException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -429,18 +430,19 @@ public void closeStalePipelines(DatanodeDetails datanodeDetails) { List pipelinesWithStaleIpOrHostname = getStalePipelines(datanodeDetails); if (pipelinesWithStaleIpOrHostname.isEmpty()) { - LOG.info("No stale pipelines"); + LOG.debug("No stale pipelines for datanode {}", + datanodeDetails.getUuidString()); return; } - LOG.info("Pipelines with stale IP or Host name: {}", - pipelinesWithStaleIpOrHostname); + LOG.info("Found {} stale pipelines", + pipelinesWithStaleIpOrHostname.size()); pipelinesWithStaleIpOrHostname.forEach(p -> { try { - LOG.info("Closing pipeline: {}", p.getId()); + LOG.info("Closing the stale pipeline: {}", p.getId()); closePipeline(p, false); - LOG.info("Closed pipeline: {}", p.getId()); + LOG.info("Closed the stale pipeline: {}", p.getId()); } catch (IOException e) { - LOG.error("Close pipeline failed: {}", p, e); + LOG.error("Closing the stale pipeline failed: {}", p, e); } }); } @@ -570,34 +572,57 @@ public void deactivatePipeline(PipelineID pipelineID) @Override public void waitPipelineReady(PipelineID pipelineID, long timeout) throws IOException { + waitOnePipelineReady(Lists.newArrayList(pipelineID), timeout); + } + + @Override + public Pipeline waitOnePipelineReady(Collection pipelineIDs, + long timeout) + throws IOException { long st = clock.millis(); if (timeout == 0) { timeout = pipelineWaitDefaultTimeout; } - - boolean ready; - Pipeline pipeline; + List pipelineIDStrs = + pipelineIDs.stream() + .map(id -> id.getId().toString()) + .collect(Collectors.toList()); + String piplineIdsStr = String.join(",", pipelineIDStrs); + Pipeline pipeline = null; do { - try { - pipeline = stateManager.getPipeline(pipelineID); - } catch (PipelineNotFoundException e) { - throw new PipelineNotFoundException(String.format( - "Pipeline %s cannot be found", pipelineID)); + boolean found = false; + for (PipelineID pipelineID : pipelineIDs) { + try { + Pipeline tempPipeline = stateManager.getPipeline(pipelineID); + found = true; + if (tempPipeline.isOpen()) { + pipeline = tempPipeline; + break; + } + } catch (PipelineNotFoundException e) { + LOG.warn("Pipeline {} cannot be found", pipelineID); + } } - ready = pipeline.isOpen(); - if (!ready) { + + if (!found) { + throw new PipelineNotFoundException("The input pipeline IDs " + + piplineIdsStr + " cannot be found"); + } + + if (pipeline == null) { try { Thread.sleep((long)100); } catch (InterruptedException e) { Thread.currentThread().interrupt(); } } - } while (!ready && clock.millis() - st < timeout); + } while (pipeline == null && clock.millis() - st < timeout); - if (!ready) { + if (pipeline == null) { throw new IOException(String.format("Pipeline %s is not ready in %d ms", - pipelineID, timeout)); + piplineIdsStr, timeout)); } + return pipeline; } @Override diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableRatisContainerProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableRatisContainerProvider.java index 8af66ad1762c..a113a0cd140f 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableRatisContainerProvider.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/WritableRatisContainerProvider.java @@ -33,8 +33,8 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.Comparator; import java.util.List; +import java.util.stream.Collectors; /** * Class to obtain a writable container for Ratis and Standalone pipelines. @@ -119,26 +119,23 @@ public ContainerInfo getContainer(final long size, } catch (SCMException se) { LOG.warn("Pipeline creation failed for repConfig {} " + - "Datanodes may be used up.", repConfig, se); - LOG.info("Try to see if any pipeline is in ALLOCATED state, " + - "and then will wait for it to be OPEN"); + "Datanodes may be used up. Try to see if any pipeline is in " + + "ALLOCATED state, and then will wait for it to be OPEN", + repConfig, se); List allocatedPipelines = findPipelinesByState(repConfig, excludeList, Pipeline.PipelineState.ALLOCATED); if (!allocatedPipelines.isEmpty()) { - // wait until the oldest allocated pipeline is ready - Pipeline oldestAllocatedPipeline = - allocatedPipelines - .stream() - .min(Comparator - .comparing(p -> p.getCreationTimestamp())) - .get(); + List allocatedPipelineIDs = + allocatedPipelines.stream() + .map(p -> p.getId()) + .collect(Collectors.toList()); try { pipelineManager - .waitPipelineReady(oldestAllocatedPipeline.getId(), 0); + .waitOnePipelineReady(allocatedPipelineIDs, 0); } catch (IOException e) { - LOG.warn("Waiting for pipeline {} to be OPEN failed. ", - oldestAllocatedPipeline, e); + LOG.warn("Waiting for one of pipelines {} to be OPEN failed. ", + allocatedPipelineIDs, e); } } } catch (IOException e) { From 1a81703624ca370e420f000b1bbc794d0b5d2ea0 Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Wed, 1 Jun 2022 17:36:28 -0700 Subject: [PATCH 24/29] fix checkstyle error --- .../org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java index d3c4391bd795..afc663a893a2 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java @@ -153,7 +153,7 @@ default void waitPipelineReady(PipelineID pipelineID, long timeout) } /** - * Wait one pipeline to be OPEN among a collection pipelines + * Wait one pipeline to be OPEN among a collection pipelines. * @param pipelineIDs ID collection of the pipelines to wait for * @param timeout wait timeout(millisecond), if 0, use default timeout * @return Pipeline the pipeline which is OPEN From 2c5f81263ea4cfad4087ce9accf366a59d2bccaa Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Fri, 10 Jun 2022 18:02:38 -0700 Subject: [PATCH 25/29] fix a comment --- .../hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java index cd528b0a165c..cf3ab0baf5ad 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java @@ -65,7 +65,7 @@ public class BackgroundPipelineCreator implements SCMService { * SCMService related variables. * 1) after leaving safe mode, BackgroundPipelineCreator needs to * wait for a while before really take effect. - * 2) NewNodeHandler, NodeIpOrHostnameUpdateHandler, + * 2) NewNodeHandler, NodeAddressUpdateHandler, * NonHealthyToHealthyNodeHandler, PreCheckComplete * will trigger a one-shot run of BackgroundPipelineCreator, * no matter in safe mode or not. From 27f2d76dfb932fe735c941bd930ca3e051522183 Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Wed, 15 Jun 2022 22:30:49 -0700 Subject: [PATCH 26/29] add more unit test --- .../hadoop/hdds/scm/node/SCMNodeManager.java | 1 - .../scm/pipeline/TestPipelineManagerImpl.java | 29 ++++++++++++++++++- 2 files changed, 28 insertions(+), 2 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java index 637a19eef6a6..53cd39d9a519 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java @@ -364,7 +364,6 @@ public RegisteredCommand register( InetAddress dnAddress = Server.getRemoteIp(); if (dnAddress != null) { // Mostly called inside an RPC, update ip - datanodeDetails.setHostName(dnAddress.getHostName()); datanodeDetails.setIpAddress(dnAddress.getHostAddress()); } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java index 47ce604b4b0e..33fb149f39f5 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java @@ -52,6 +52,7 @@ import org.apache.ozone.test.GenericTestUtils; import org.apache.ozone.test.TestClock; import org.apache.ratis.protocol.exceptions.NotLeaderException; +import org.assertj.core.util.Lists; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -81,6 +82,8 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; /** @@ -777,7 +780,8 @@ public void testPipelineCloseFlow() throws IOException { @Test public void testGetStalePipelines() throws IOException { - SCMHADBTransactionBuffer buffer = new SCMHADBTransactionBufferStub(dbStore); + SCMHADBTransactionBuffer buffer = + new SCMHADBTransactionBufferStub(dbStore); PipelineManagerImpl pipelineManager = spy(createPipelineManager(true, buffer)); @@ -847,6 +851,29 @@ public void testGetStalePipelines() throws IOException { Assertions.assertEquals(pipelines.get(3), pipelineList2.get(1)); } + @Test + public void testCloseStalePipelines() throws IOException { + SCMHADBTransactionBuffer buffer = + new SCMHADBTransactionBufferStub(dbStore); + PipelineManagerImpl pipelineManager = + spy(createPipelineManager(true, buffer)); + + Pipeline pipeline0 = mock(Pipeline.class); + Pipeline pipeline1 = mock(Pipeline.class); + when(pipeline0.getId()).thenReturn(mock(PipelineID.class)); + when(pipeline1.getId()).thenReturn(mock(PipelineID.class)); + DatanodeDetails datanodeDetails = mock(DatanodeDetails.class); + List stalePipelines = Lists.newArrayList(pipeline0, pipeline1); + doReturn(stalePipelines).when(pipelineManager) + .getStalePipelines(datanodeDetails); + + pipelineManager.closeStalePipelines(datanodeDetails); + verify(pipelineManager, times(1)) + .closePipeline(stalePipelines.get(0), false); + verify(pipelineManager, times(1)) + .closePipeline(stalePipelines.get(1), false); + } + public void testCreatePipelineForRead() throws IOException { PipelineManager pipelineManager = createPipelineManager(true); List dns = nodeManager From 7ce007710d18e3d71b1b82e7148ad1793a434d7b Mon Sep 17 00:00:00 2001 From: zhiheng xie Date: Thu, 16 Jun 2022 23:21:58 -0700 Subject: [PATCH 27/29] fix test failure --- .../java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java index d6f91648b8e8..78a1b632f273 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java @@ -365,6 +365,9 @@ public RegisteredCommand register( InetAddress dnAddress = Server.getRemoteIp(); if (dnAddress != null) { // Mostly called inside an RPC, update ip + if (!useHostname) { + datanodeDetails.setHostName(dnAddress.getHostName()); + } datanodeDetails.setIpAddress(dnAddress.getHostAddress()); } From e742158578a3e25557e6ac7315542763fc389344 Mon Sep 17 00:00:00 2001 From: George Jahad Date: Fri, 17 Jun 2022 10:36:05 -0700 Subject: [PATCH 28/29] working --- .../scm/pipeline/TestPipelineManagerImpl.java | 82 +++++++++++++++++++ 1 file changed, 82 insertions(+) diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java index 33fb149f39f5..df3a1fcb1e31 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java @@ -21,17 +21,20 @@ import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hdds.HddsConfigKeys; 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.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; import org.apache.hadoop.hdds.scm.HddsTestUtils; +import org.apache.hadoop.hdds.scm.PipelineChoosePolicy; import org.apache.hadoop.hdds.scm.container.ContainerID; import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.ContainerManager; import org.apache.hadoop.hdds.scm.container.ContainerReplica; import org.apache.hadoop.hdds.scm.container.MockNodeManager; +import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; import org.apache.hadoop.hdds.scm.exceptions.SCMException; import org.apache.hadoop.hdds.scm.ha.SCMHADBTransactionBuffer; import org.apache.hadoop.hdds.scm.ha.SCMHADBTransactionBufferStub; @@ -40,6 +43,7 @@ import org.apache.hadoop.hdds.scm.ha.SCMServiceManager; import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition; import org.apache.hadoop.hdds.scm.node.NodeStatus; +import org.apache.hadoop.hdds.scm.pipeline.choose.algorithms.HealthyPipelineChoosePolicy; import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager; import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; @@ -57,6 +61,7 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; import org.slf4j.LoggerFactory; import java.io.File; @@ -64,6 +69,7 @@ import java.time.Instant; import java.time.ZoneOffset; import java.util.ArrayList; +import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -75,11 +81,19 @@ import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_DATANODE_PIPELINE_LIMIT_DEFAULT; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_PIPELINE_ALLOCATED_TIMEOUT; import static org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState.ALLOCATED; +import static org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState.OPEN; import static org.apache.hadoop.test.MetricsAsserts.getLongCounter; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.anyList; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -874,6 +888,74 @@ public void testCloseStalePipelines() throws IOException { .closePipeline(stalePipelines.get(1), false); } + @Test + public void testWaitForAllocatedPipeline() throws IOException { + SCMHADBTransactionBuffer buffer = + new SCMHADBTransactionBufferStub(dbStore); + PipelineManagerImpl pipelineManager = + createPipelineManager(true, buffer); + + PipelineManagerImpl pipelineManagerSpy = spy(pipelineManager); + ReplicationConfig repConfig = RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE); + PipelineChoosePolicy pipelineChoosingPolicy + = new HealthyPipelineChoosePolicy(); + ContainerManager containerManager + = mock(ContainerManager.class); + + WritableContainerProvider provider; + String OWNER = "TEST"; + Pipeline allocatedPipeline; + + // Throw on pipeline creates, so no new pipelines can be created + doThrow(SCMException.class).when(pipelineManagerSpy).createPipeline(any(), any(), anyList()); + provider = new WritableRatisContainerProvider( + conf, pipelineManagerSpy, containerManager, pipelineChoosingPolicy); + + // Add a single pipeline to manager, (in the allocated state) + allocatedPipeline = pipelineManager.createPipeline(repConfig); + pipelineManager.getStateManager().updatePipelineState(allocatedPipeline.getId() + .getProtobuf(), HddsProtos.PipelineState.PIPELINE_ALLOCATED); + + // Assign a container to that pipeline + ContainerInfo container = HddsTestUtils. + getContainer(HddsProtos.LifeCycleState.OPEN, allocatedPipeline.getId()); + + pipelineManager.addContainerToPipeline( + allocatedPipeline.getId(), container.containerID()); + doReturn(container).when(containerManager).getMatchingContainer(anyLong(), + anyString(), eq(allocatedPipeline), any()); + + + Assertions.assertTrue(pipelineManager.getPipelines(repConfig, OPEN).isEmpty(), "No open pipelines exist"); + Assertions.assertTrue(pipelineManager.getPipelines(repConfig, ALLOCATED).contains(allocatedPipeline), "An allocated pipeline exists"); + + // Instrument waitOnePipelineReady to open pipeline a bit after it is called + Runnable r = () -> { + try { + Thread.sleep(100); + pipelineManager.openPipeline(allocatedPipeline.getId()); + } catch (Exception e) { + fail("exception on opening pipeline", e); + } + }; + doAnswer(call -> { + new Thread(r).start(); + return call.callRealMethod(); + }).when(pipelineManagerSpy).waitOnePipelineReady(any(), anyLong()); + + + ContainerInfo c = provider.getContainer(1, repConfig, OWNER, new ExcludeList()); + Assertions.assertTrue(c.equals(container), "Expected container was returned"); + + // Confirm that waitOnePipelineReady was called on allocated pipelines + ArgumentCaptor> captor = ArgumentCaptor.forClass(Collection.class); + verify(pipelineManagerSpy, times(1)).waitOnePipelineReady(captor.capture(), anyLong()); + Collection coll = captor.getValue(); + Assertions.assertTrue(coll.contains(allocatedPipeline.getId()), + "waitOnePipelineReady() was called on allocated pipeline"); + pipelineManager.close(); + } + public void testCreatePipelineForRead() throws IOException { PipelineManager pipelineManager = createPipelineManager(true); List dns = nodeManager From d8f482251e0663103c510ba7eba1b6397330a01a Mon Sep 17 00:00:00 2001 From: George Jahad Date: Fri, 17 Jun 2022 10:49:08 -0700 Subject: [PATCH 29/29] cleanup --- .../scm/pipeline/TestPipelineManagerImpl.java | 38 ++++++++++++------- 1 file changed, 24 insertions(+), 14 deletions(-) diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java index df3a1fcb1e31..0e60921da56e 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java @@ -893,32 +893,36 @@ public void testWaitForAllocatedPipeline() throws IOException { SCMHADBTransactionBuffer buffer = new SCMHADBTransactionBufferStub(dbStore); PipelineManagerImpl pipelineManager = - createPipelineManager(true, buffer); + createPipelineManager(true, buffer); PipelineManagerImpl pipelineManagerSpy = spy(pipelineManager); - ReplicationConfig repConfig = RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE); + ReplicationConfig repConfig = + RatisReplicationConfig.getInstance(HddsProtos.ReplicationFactor.THREE); PipelineChoosePolicy pipelineChoosingPolicy - = new HealthyPipelineChoosePolicy(); + = new HealthyPipelineChoosePolicy(); ContainerManager containerManager - = mock(ContainerManager.class); + = mock(ContainerManager.class); WritableContainerProvider provider; - String OWNER = "TEST"; + String owner = "TEST"; Pipeline allocatedPipeline; // Throw on pipeline creates, so no new pipelines can be created - doThrow(SCMException.class).when(pipelineManagerSpy).createPipeline(any(), any(), anyList()); + doThrow(SCMException.class).when(pipelineManagerSpy) + .createPipeline(any(), any(), anyList()); provider = new WritableRatisContainerProvider( conf, pipelineManagerSpy, containerManager, pipelineChoosingPolicy); // Add a single pipeline to manager, (in the allocated state) allocatedPipeline = pipelineManager.createPipeline(repConfig); - pipelineManager.getStateManager().updatePipelineState(allocatedPipeline.getId() + pipelineManager.getStateManager() + .updatePipelineState(allocatedPipeline.getId() .getProtobuf(), HddsProtos.PipelineState.PIPELINE_ALLOCATED); // Assign a container to that pipeline ContainerInfo container = HddsTestUtils. - getContainer(HddsProtos.LifeCycleState.OPEN, allocatedPipeline.getId()); + getContainer(HddsProtos.LifeCycleState.OPEN, + allocatedPipeline.getId()); pipelineManager.addContainerToPipeline( allocatedPipeline.getId(), container.containerID()); @@ -926,8 +930,10 @@ public void testWaitForAllocatedPipeline() throws IOException { anyString(), eq(allocatedPipeline), any()); - Assertions.assertTrue(pipelineManager.getPipelines(repConfig, OPEN).isEmpty(), "No open pipelines exist"); - Assertions.assertTrue(pipelineManager.getPipelines(repConfig, ALLOCATED).contains(allocatedPipeline), "An allocated pipeline exists"); + Assertions.assertTrue(pipelineManager.getPipelines(repConfig, OPEN) + .isEmpty(), "No open pipelines exist"); + Assertions.assertTrue(pipelineManager.getPipelines(repConfig, ALLOCATED) + .contains(allocatedPipeline), "An allocated pipeline exists"); // Instrument waitOnePipelineReady to open pipeline a bit after it is called Runnable r = () -> { @@ -944,12 +950,16 @@ public void testWaitForAllocatedPipeline() throws IOException { }).when(pipelineManagerSpy).waitOnePipelineReady(any(), anyLong()); - ContainerInfo c = provider.getContainer(1, repConfig, OWNER, new ExcludeList()); - Assertions.assertTrue(c.equals(container), "Expected container was returned"); + ContainerInfo c = provider.getContainer(1, repConfig, + owner, new ExcludeList()); + Assertions.assertTrue(c.equals(container), + "Expected container was returned"); // Confirm that waitOnePipelineReady was called on allocated pipelines - ArgumentCaptor> captor = ArgumentCaptor.forClass(Collection.class); - verify(pipelineManagerSpy, times(1)).waitOnePipelineReady(captor.capture(), anyLong()); + ArgumentCaptor> captor = + ArgumentCaptor.forClass(Collection.class); + verify(pipelineManagerSpy, times(1)) + .waitOnePipelineReady(captor.capture(), anyLong()); Collection coll = captor.getValue(); Assertions.assertTrue(coll.contains(allocatedPipeline.getId()), "waitOnePipelineReady() was called on allocated pipeline");