HDDS-242. Introduce NEW_NODE, STALE_NODE and DEAD_NODE event

and corresponding event handlers in SCM.
Contributed by Nanda Kumar.

Recommitting after making sure that patch is clean.
This commit is contained in:
Anu Engineer 2018-07-11 12:08:50 -07:00
parent b56785873a
commit 632aca5793
23 changed files with 415 additions and 57 deletions

View File

@ -25,9 +25,12 @@
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.hadoop.hdds.scm.events.SCMEvents.DATANODE_COMMAND;
/**
* In case of a node failure, volume failure, volume out of spapce, node
* out of space etc, CLOSE_CONTAINER will be triggered.
@ -73,9 +76,11 @@ public void onMessage(ContainerID containerID, EventPublisher publisher) {
if (info.getState() == HddsProtos.LifeCycleState.OPEN) {
for (DatanodeDetails datanode :
containerWithPipeline.getPipeline().getMachines()) {
containerManager.getNodeManager().addDatanodeCommand(datanode.getUuid(),
CommandForDatanode closeContainerCommand = new CommandForDatanode<>(
datanode.getUuid(),
new CloseContainerCommand(containerID.getId(),
info.getReplicationType()));
publisher.fireEvent(DATANODE_COMMAND, closeContainerCommand);
}
try {
// Finalize event will make sure the state of the container transitions

View File

@ -709,11 +709,6 @@ public void flushContainerInfo() throws IOException {
}
}
@Override
public NodeManager getNodeManager() {
return nodeManager;
}
@VisibleForTesting
public MetadataStore getContainerStore() {
return containerStore;

View File

@ -0,0 +1,47 @@
/**
* 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.container;
import org.apache.hadoop.hdds.scm.node.states.Node2ContainerMap;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher
.ContainerReportFromDatanode;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
/**
* Handles container reports from datanode.
*/
public class ContainerReportHandler implements
EventHandler<ContainerReportFromDatanode> {
private final Mapping containerMapping;
private final Node2ContainerMap node2ContainerMap;
public ContainerReportHandler(Mapping containerMapping,
Node2ContainerMap node2ContainerMap) {
this.containerMapping = containerMapping;
this.node2ContainerMap = node2ContainerMap;
}
@Override
public void onMessage(ContainerReportFromDatanode containerReportFromDatanode,
EventPublisher publisher) {
// TODO: process container report.
}
}

View File

@ -25,7 +25,6 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import java.io.Closeable;
import java.io.IOException;
@ -129,17 +128,11 @@ void processContainerReports(DatanodeDetails datanodeDetails,
void updateDeleteTransactionId(Map<Long, Long> deleteTransactionMap)
throws IOException;
/**
* Returns the nodeManager.
* @return NodeManager
*/
NodeManager getNodeManager();
/**
* Returns the ContainerWithPipeline.
* @return NodeManager
*/
public ContainerWithPipeline getMatchingContainerWithPipeline(final long size,
ContainerWithPipeline getMatchingContainerWithPipeline(long size,
String owner, ReplicationType type, ReplicationFactor factor,
LifeCycleState state) throws IOException;
}

View File

@ -26,7 +26,6 @@
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.hdds.scm.events;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.ContainerReportFromDatanode;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.NodeReportFromDatanode;
@ -71,6 +72,27 @@ public final class SCMEvents {
public static final TypedEvent<ContainerID> CLOSE_CONTAINER =
new TypedEvent<>(ContainerID.class, "Close_Container");
/**
* This event will be triggered whenever a new datanode is
* registered with SCM.
*/
public static final TypedEvent<DatanodeDetails> NEW_NODE =
new TypedEvent<>(DatanodeDetails.class, "New_Node");
/**
* This event will be triggered whenever a datanode is moved from healthy to
* stale state.
*/
public static final TypedEvent<DatanodeDetails> STALE_NODE =
new TypedEvent<>(DatanodeDetails.class, "Stale_Node");
/**
* This event will be triggered whenever a datanode is moved from stale to
* dead state.
*/
public static final TypedEvent<DatanodeDetails> DEAD_NODE =
new TypedEvent<>(DatanodeDetails.class, "Dead_Node");
/**
* Private Ctor. Never Constructed.
*/

View File

@ -106,4 +106,15 @@ public List<StorageReportProto> getStorageReports() {
lock.readLock().unlock();
}
}
@Override
public int hashCode() {
return super.hashCode();
}
@Override
public boolean equals(Object obj) {
return super.equals(obj);
}
}

View File

@ -0,0 +1,42 @@
/**
* 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;
import org.apache.hadoop.hdds.scm.node.states.Node2ContainerMap;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
/**
* Handles Dead Node event.
*/
public class DeadNodeHandler implements EventHandler<DatanodeDetails> {
private final Node2ContainerMap node2ContainerMap;
public DeadNodeHandler(Node2ContainerMap node2ContainerMap) {
this.node2ContainerMap = node2ContainerMap;
}
@Override
public void onMessage(DatanodeDetails datanodeDetails,
EventPublisher publisher) {
//TODO: add logic to handle dead node.
}
}

View File

@ -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.
*/
package org.apache.hadoop.hdds.scm.node;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.node.states.Node2ContainerMap;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import java.util.Collections;
/**
* Handles New Node event.
*/
public class NewNodeHandler implements EventHandler<DatanodeDetails> {
private final Node2ContainerMap node2ContainerMap;
public NewNodeHandler(Node2ContainerMap node2ContainerMap) {
this.node2ContainerMap = node2ContainerMap;
}
@Override
public void onMessage(DatanodeDetails datanodeDetails,
EventPublisher publisher) {
try {
node2ContainerMap.insertNewDatanode(datanodeDetails.getUuid(),
Collections.emptySet());
} catch (SCMException e) {
// TODO: log exception message.
}
}
}

View File

@ -22,7 +22,9 @@
import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.ozone.protocol.StorageContainerNodeProtocol;
import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import java.io.Closeable;
@ -53,7 +55,7 @@
* list, by calling removeNode. We will throw away this nodes info soon.
*/
public interface NodeManager extends StorageContainerNodeProtocol,
NodeManagerMXBean, Closeable {
EventHandler<CommandForDatanode>, NodeManagerMXBean, Closeable {
/**
* Removes a data node from the management of this Node Manager.
*

View File

@ -0,0 +1,42 @@
/**
* 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.scm.server.SCMDatanodeHeartbeatDispatcher
.NodeReportFromDatanode;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
/**
* Handles Node Reports from datanode.
*/
public class NodeReportHandler implements EventHandler<NodeReportFromDatanode> {
private final NodeManager nodeManager;
public NodeReportHandler(NodeManager nodeManager) {
this.nodeManager = nodeManager;
}
@Override
public void onMessage(NodeReportFromDatanode nodeReportFromDatanode,
EventPublisher publisher) {
//TODO: process node report.
}
}

View File

@ -24,9 +24,12 @@
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
import org.apache.hadoop.hdds.scm.HddsServerUtil;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.node.states.NodeAlreadyExistsException;
import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
import org.apache.hadoop.hdds.scm.node.states.NodeStateMap;
import org.apache.hadoop.hdds.server.events.Event;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.apache.hadoop.ozone.common.statemachine
.InvalidStateTransitionException;
import org.apache.hadoop.ozone.common.statemachine.StateMachine;
@ -36,9 +39,11 @@
import org.slf4j.LoggerFactory;
import java.io.Closeable;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ScheduledExecutorService;
@ -86,6 +91,14 @@ private enum NodeLifeCycleEvent {
* This is the map which maintains the current state of all datanodes.
*/
private final NodeStateMap nodeStateMap;
/**
* Used for publishing node state change events.
*/
private final EventPublisher eventPublisher;
/**
* Maps the event to be triggered when a node state us updated.
*/
private final Map<NodeState, Event<DatanodeDetails>> state2EventMap;
/**
* ExecutorService used for scheduling heartbeat processing thread.
*/
@ -108,8 +121,11 @@ private enum NodeLifeCycleEvent {
*
* @param conf Configuration
*/
public NodeStateManager(Configuration conf) {
nodeStateMap = new NodeStateMap();
public NodeStateManager(Configuration conf, EventPublisher eventPublisher) {
this.nodeStateMap = new NodeStateMap();
this.eventPublisher = eventPublisher;
this.state2EventMap = new HashMap<>();
initialiseState2EventMap();
Set<NodeState> finalStates = new HashSet<>();
finalStates.add(NodeState.DECOMMISSIONED);
this.stateMachine = new StateMachine<>(NodeState.HEALTHY, finalStates);
@ -130,6 +146,14 @@ public NodeStateManager(Configuration conf) {
TimeUnit.MILLISECONDS);
}
/**
* Populates state2event map.
*/
private void initialiseState2EventMap() {
state2EventMap.put(NodeState.STALE, SCMEvents.STALE_NODE);
state2EventMap.put(NodeState.DEAD, SCMEvents.DEAD_NODE);
}
/*
*
* Node and State Transition Mapping:
@ -220,6 +244,7 @@ private void initializeStateMachine() {
public void addNode(DatanodeDetails datanodeDetails)
throws NodeAlreadyExistsException {
nodeStateMap.addNode(datanodeDetails, stateMachine.getInitialState());
eventPublisher.fireEvent(SCMEvents.NEW_NODE, datanodeDetails);
}
/**
@ -548,6 +573,9 @@ private void updateNodeState(DatanodeInfo node, Predicate<Long> condition,
if (condition.test(node.getLastHeartbeatTime())) {
NodeState newState = stateMachine.getNextState(state, lifeCycleEvent);
nodeStateMap.updateNodeState(node.getUuid(), state, newState);
if (state2EventMap.containsKey(newState)) {
eventPublisher.fireEvent(state2EventMap.get(newState), node);
}
}
} catch (InvalidStateTransitionException e) {
LOG.warn("Invalid state transition of node {}." +

View File

@ -25,7 +25,6 @@
import org.apache.hadoop.hdds.scm.VersionInfo;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@ -78,8 +77,7 @@
* as soon as you read it.
*/
public class SCMNodeManager
implements NodeManager, StorageContainerNodeProtocol,
EventHandler<CommandForDatanode> {
implements NodeManager, StorageContainerNodeProtocol {
@VisibleForTesting
static final Logger LOG =
@ -117,14 +115,13 @@ public class SCMNodeManager
// Node pool manager.
private final StorageContainerManager scmManager;
/**
* Constructs SCM machine Manager.
*/
public SCMNodeManager(OzoneConfiguration conf, String clusterID,
StorageContainerManager scmManager) throws IOException {
this.nodeStateManager = new NodeStateManager(conf);
StorageContainerManager scmManager, EventPublisher eventPublisher)
throws IOException {
this.nodeStateManager = new NodeStateManager(conf, eventPublisher);
this.nodeStats = new ConcurrentHashMap<>();
this.scmStat = new SCMNodeStat();
this.clusterID = clusterID;
@ -462,14 +459,25 @@ public Map<String, Integer> getNodeCount() {
return nodeCountMap;
}
// TODO:
// Since datanode commands are added through event queue, onMessage method
// should take care of adding commands to command queue.
// Refactor and remove all the usage of this method and delete this method.
@Override
public void addDatanodeCommand(UUID dnId, SCMCommand command) {
this.commandQueue.addCommand(dnId, command);
}
/**
* This method is called by EventQueue whenever someone adds a new
* DATANODE_COMMAND to the Queue.
*
* @param commandForDatanode DatanodeCommand
* @param ignored publisher
*/
@Override
public void onMessage(CommandForDatanode commandForDatanode,
EventPublisher publisher) {
EventPublisher ignored) {
addDatanodeCommand(commandForDatanode.getDatanodeId(),
commandForDatanode.getCommand());
}

View File

@ -0,0 +1,42 @@
/**
* 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;
import org.apache.hadoop.hdds.scm.node.states.Node2ContainerMap;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
/**
* Handles Stale node event.
*/
public class StaleNodeHandler implements EventHandler<DatanodeDetails> {
private final Node2ContainerMap node2ContainerMap;
public StaleNodeHandler(Node2ContainerMap node2ContainerMap) {
this.node2ContainerMap = node2ContainerMap;
}
@Override
public void onMessage(DatanodeDetails datanodeDetails,
EventPublisher publisher) {
//TODO: logic to handle stale node.
}
}

View File

@ -17,6 +17,7 @@
package org.apache.hadoop.hdds.scm.server;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
@ -24,12 +25,16 @@
.StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import com.google.protobuf.GeneratedMessage;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.List;
import static org.apache.hadoop.hdds.scm.events.SCMEvents.CONTAINER_REPORT;
import static org.apache.hadoop.hdds.scm.events.SCMEvents.NODE_REPORT;
@ -42,10 +47,15 @@ public final class SCMDatanodeHeartbeatDispatcher {
private static final Logger LOG =
LoggerFactory.getLogger(SCMDatanodeHeartbeatDispatcher.class);
private EventPublisher eventPublisher;
private final NodeManager nodeManager;
private final EventPublisher eventPublisher;
public SCMDatanodeHeartbeatDispatcher(EventPublisher eventPublisher) {
public SCMDatanodeHeartbeatDispatcher(NodeManager nodeManager,
EventPublisher eventPublisher) {
Preconditions.checkNotNull(nodeManager);
Preconditions.checkNotNull(eventPublisher);
this.nodeManager = nodeManager;
this.eventPublisher = eventPublisher;
}
@ -54,11 +64,14 @@ public SCMDatanodeHeartbeatDispatcher(EventPublisher eventPublisher) {
* Dispatches heartbeat to registered event handlers.
*
* @param heartbeat heartbeat to be dispatched.
*
* @return list of SCMCommand
*/
public void dispatch(SCMHeartbeatRequestProto heartbeat) {
public List<SCMCommand> dispatch(SCMHeartbeatRequestProto heartbeat) {
DatanodeDetails datanodeDetails =
DatanodeDetails.getFromProtoBuf(heartbeat.getDatanodeDetails());
// should we dispatch heartbeat through eventPublisher?
List<SCMCommand> commands = nodeManager.processHeartbeat(datanodeDetails);
if (heartbeat.hasNodeReport()) {
LOG.debug("Dispatching Node Report.");
eventPublisher.fireEvent(NODE_REPORT,
@ -73,6 +86,7 @@ public void dispatch(SCMHeartbeatRequestProto heartbeat) {
heartbeat.getContainerReport()));
}
return commands;
}
/**

View File

@ -133,7 +133,8 @@ public SCMDatanodeProtocolServer(final OzoneConfiguration conf,
conf.getInt(OZONE_SCM_HANDLER_COUNT_KEY,
OZONE_SCM_HANDLER_COUNT_DEFAULT);
heartbeatDispatcher = new SCMDatanodeHeartbeatDispatcher(eventPublisher);
heartbeatDispatcher = new SCMDatanodeHeartbeatDispatcher(
scm.getScmNodeManager(), eventPublisher);
RPC.setProtocolEngine(conf, StorageContainerDatanodeProtocolPB.class,
ProtobufRpcEngine.class);
@ -214,22 +215,13 @@ public static SCMRegisteredResponseProto getRegisteredResponse(
@Override
public SCMHeartbeatResponseProto sendHeartbeat(
SCMHeartbeatRequestProto heartbeat)
throws IOException {
heartbeatDispatcher.dispatch(heartbeat);
// TODO: Remove the below code after SCM refactoring.
DatanodeDetails datanodeDetails = DatanodeDetails
.getFromProtoBuf(heartbeat.getDatanodeDetails());
NodeReportProto nodeReport = heartbeat.getNodeReport();
List<SCMCommand> commands =
scm.getScmNodeManager().processHeartbeat(datanodeDetails);
SCMHeartbeatRequestProto heartbeat) throws IOException {
List<SCMCommandProto> cmdResponses = new LinkedList<>();
for (SCMCommand cmd : commands) {
for (SCMCommand cmd : heartbeatDispatcher.dispatch(heartbeat)) {
cmdResponses.add(getCommandResponse(cmd));
}
return SCMHeartbeatResponseProto.newBuilder()
.setDatanodeUUID(datanodeDetails.getUuidString())
.setDatanodeUUID(heartbeat.getDatanodeDetails().getUuid())
.addAllCommands(cmdResponses).build();
}

View File

@ -33,15 +33,23 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
import org.apache.hadoop.hdds.scm.block.BlockManager;
import org.apache.hadoop.hdds.scm.block.BlockManagerImpl;
import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler;
import org.apache.hadoop.hdds.scm.container.ContainerMapping;
import org.apache.hadoop.hdds.scm.container.ContainerReportHandler;
import org.apache.hadoop.hdds.scm.container.Mapping;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.placement.metrics.ContainerStat;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMMetrics;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes;
import org.apache.hadoop.hdds.scm.node.DeadNodeHandler;
import org.apache.hadoop.hdds.scm.node.NewNodeHandler;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.node.NodeReportHandler;
import org.apache.hadoop.hdds.scm.node.SCMNodeManager;
import org.apache.hadoop.hdds.scm.node.StaleNodeHandler;
import org.apache.hadoop.hdds.scm.node.states.Node2ContainerMap;
import org.apache.hadoop.hdds.server.ServiceRuntimeInfoImpl;
import org.apache.hadoop.hdds.server.events.EventQueue;
import org.apache.hadoop.hdfs.DFSUtil;
@ -71,7 +79,6 @@
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DB_CACHE_SIZE_DEFAULT;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DB_CACHE_SIZE_MB;
import static org.apache.hadoop.hdds.scm.events.SCMEvents.DATANODE_COMMAND;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED;
import static org.apache.hadoop.util.ExitUtil.terminate;
@ -126,6 +133,8 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
private final Mapping scmContainerManager;
private final BlockManager scmBlockManager;
private final SCMStorage scmStorage;
private final EventQueue eventQueue;
/*
* HTTP endpoint for JMX access.
*/
@ -164,18 +173,35 @@ private StorageContainerManager(OzoneConfiguration conf) throws IOException {
throw new SCMException("SCM not initialized.", ResultCodes
.SCM_NOT_INITIALIZED);
}
EventQueue eventQueue = new EventQueue();
SCMNodeManager nm =
new SCMNodeManager(conf, scmStorage.getClusterID(), this);
scmNodeManager = nm;
eventQueue.addHandler(DATANODE_COMMAND, nm);
eventQueue = new EventQueue();
scmContainerManager = new ContainerMapping(conf, getScmNodeManager(),
cacheSize);
scmNodeManager = new SCMNodeManager(
conf, scmStorage.getClusterID(), this, eventQueue);
scmContainerManager = new ContainerMapping(
conf, getScmNodeManager(), cacheSize);
scmBlockManager = new BlockManagerImpl(
conf, getScmNodeManager(), scmContainerManager);
scmBlockManager =
new BlockManagerImpl(conf, getScmNodeManager(), scmContainerManager);
Node2ContainerMap node2ContainerMap = new Node2ContainerMap();
CloseContainerEventHandler closeContainerHandler =
new CloseContainerEventHandler(scmContainerManager);
NodeReportHandler nodeReportHandler =
new NodeReportHandler(scmNodeManager);
ContainerReportHandler containerReportHandler =
new ContainerReportHandler(scmContainerManager, node2ContainerMap);
NewNodeHandler newNodeHandler = new NewNodeHandler(node2ContainerMap);
StaleNodeHandler staleNodeHandler = new StaleNodeHandler(node2ContainerMap);
DeadNodeHandler deadNodeHandler = new DeadNodeHandler(node2ContainerMap);
eventQueue.addHandler(SCMEvents.DATANODE_COMMAND, scmNodeManager);
eventQueue.addHandler(SCMEvents.NODE_REPORT, nodeReportHandler);
eventQueue.addHandler(SCMEvents.CONTAINER_REPORT, containerReportHandler);
eventQueue.addHandler(SCMEvents.CLOSE_CONTAINER, closeContainerHandler);
eventQueue.addHandler(SCMEvents.NEW_NODE, newNodeHandler);
eventQueue.addHandler(SCMEvents.STALE_NODE, staleNodeHandler);
eventQueue.addHandler(SCMEvents.DEAD_NODE, deadNodeHandler);
scmAdminUsernames = conf.getTrimmedStringCollection(OzoneConfigKeys
.OZONE_ADMINISTRATORS);
@ -189,7 +215,6 @@ private StorageContainerManager(OzoneConfiguration conf) throws IOException {
blockProtocolServer = new SCMBlockProtocolServer(conf, this);
clientProtocolServer = new SCMClientProtocolServer(conf, this);
httpServer = new StorageContainerManagerHttpServer(conf);
registerMXBean();
}

View File

@ -26,8 +26,10 @@
.StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.protocol.VersionResponse;
import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode;
import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.assertj.core.util.Preconditions;
@ -399,6 +401,13 @@ public void delContainer(DatanodeDetails datanodeDetails, long size) {
}
}
@Override
public void onMessage(CommandForDatanode commandForDatanode,
EventPublisher publisher) {
addDatanodeCommand(commandForDatanode.getDatanodeId(),
commandForDatanode.getCommand());
}
/**
* A class to declare some values for the nodes so that our tests
* won't fail.

View File

@ -41,6 +41,7 @@
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB;
import static org.apache.hadoop.hdds.scm.events.SCMEvents.CLOSE_CONTAINER;
import static org.apache.hadoop.hdds.scm.events.SCMEvents.DATANODE_COMMAND;
/**
* Tests the closeContainerEventHandler class.
@ -69,6 +70,7 @@ public static void setUp() throws Exception {
eventQueue = new EventQueue();
eventQueue.addHandler(CLOSE_CONTAINER,
new CloseContainerEventHandler(mapping));
eventQueue.addHandler(DATANODE_COMMAND, nodeManager);
}
@AfterClass

View File

@ -34,6 +34,8 @@
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.server.events.EventQueue;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.test.PathUtils;
@ -41,6 +43,7 @@
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.mockito.Mockito;
import java.io.File;
import java.io.IOException;
@ -86,8 +89,15 @@ OzoneConfiguration getConf() {
SCMNodeManager createNodeManager(OzoneConfiguration config)
throws IOException {
EventQueue eventQueue = new EventQueue();
eventQueue.addHandler(SCMEvents.NEW_NODE,
Mockito.mock(NewNodeHandler.class));
eventQueue.addHandler(SCMEvents.STALE_NODE,
Mockito.mock(StaleNodeHandler.class));
eventQueue.addHandler(SCMEvents.DEAD_NODE,
Mockito.mock(DeadNodeHandler.class));
SCMNodeManager nodeManager = new SCMNodeManager(config,
UUID.randomUUID().toString(), null);
UUID.randomUUID().toString(), null, eventQueue);
assertFalse("Node manager should be in chill mode",
nodeManager.isOutOfChillMode());
return nodeManager;

View File

@ -30,6 +30,7 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.server.events.EventQueue;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
@ -45,6 +46,7 @@
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.mockito.Mockito;
import java.io.File;
import java.io.IOException;
@ -124,8 +126,15 @@ OzoneConfiguration getConf() {
SCMNodeManager createNodeManager(OzoneConfiguration config)
throws IOException {
EventQueue eventQueue = new EventQueue();
eventQueue.addHandler(SCMEvents.NEW_NODE,
Mockito.mock(NewNodeHandler.class));
eventQueue.addHandler(SCMEvents.STALE_NODE,
Mockito.mock(StaleNodeHandler.class));
eventQueue.addHandler(SCMEvents.DEAD_NODE,
Mockito.mock(DeadNodeHandler.class));
SCMNodeManager nodeManager = new SCMNodeManager(config,
UUID.randomUUID().toString(), null);
UUID.randomUUID().toString(), null, eventQueue);
assertFalse("Node manager should be in chill mode",
nodeManager.isOutOfChillMode());
return nodeManager;

View File

@ -28,6 +28,7 @@
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher
.ContainerReportFromDatanode;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher
@ -37,6 +38,7 @@
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
import static org.apache.hadoop.hdds.scm.events.SCMEvents.CONTAINER_REPORT;
import static org.apache.hadoop.hdds.scm.events.SCMEvents.NODE_REPORT;
@ -55,7 +57,8 @@ public void testNodeReportDispatcher() throws IOException {
NodeReportProto nodeReport = NodeReportProto.getDefaultInstance();
SCMDatanodeHeartbeatDispatcher dispatcher =
new SCMDatanodeHeartbeatDispatcher(new EventPublisher() {
new SCMDatanodeHeartbeatDispatcher(Mockito.mock(NodeManager.class),
new EventPublisher() {
@Override
public <PAYLOAD, EVENT_TYPE extends Event<PAYLOAD>> void fireEvent(
EVENT_TYPE event, PAYLOAD payload) {
@ -90,7 +93,8 @@ public void testContainerReportDispatcher() throws IOException {
ContainerReportsProto.getDefaultInstance();
SCMDatanodeHeartbeatDispatcher dispatcher =
new SCMDatanodeHeartbeatDispatcher(new EventPublisher() {
new SCMDatanodeHeartbeatDispatcher(Mockito.mock(NodeManager.class),
new EventPublisher() {
@Override
public <PAYLOAD, EVENT_TYPE extends Event<PAYLOAD>> void fireEvent(
EVENT_TYPE event, PAYLOAD payload) {

View File

@ -28,7 +28,9 @@
.StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.apache.hadoop.ozone.protocol.VersionResponse;
import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode;
import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
@ -287,4 +289,9 @@ public void addDatanodeCommand(UUID dnId, SCMCommand command) {
this.commandQueue.addCommand(dnId, command);
}
@Override
public void onMessage(CommandForDatanode commandForDatanode,
EventPublisher publisher) {
// do nothing.
}
}