HDDS-81. Moving ContainerReport inside Datanode heartbeat.

Contributed by Nanda Kumar.
This commit is contained in:
Anu Engineer 2018-05-29 12:40:27 -07:00
parent 4827e9a908
commit 201440b987
44 changed files with 608 additions and 1217 deletions

View File

@ -35,11 +35,11 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos; .StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto; .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport; .StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMStorageReport; .StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.OzoneConsts;
@ -854,11 +854,11 @@ public class ContainerManagerImpl implements ContainerManager {
* @return node report. * @return node report.
*/ */
@Override @Override
public SCMNodeReport getNodeReport() throws IOException { public NodeReportProto getNodeReport() throws IOException {
StorageLocationReport[] reports = locationManager.getLocationReport(); StorageLocationReport[] reports = locationManager.getLocationReport();
SCMNodeReport.Builder nrb = SCMNodeReport.newBuilder(); NodeReportProto.Builder nrb = NodeReportProto.newBuilder();
for (int i = 0; i < reports.length; i++) { for (int i = 0; i < reports.length; i++) {
SCMStorageReport.Builder srb = SCMStorageReport.newBuilder(); StorageReportProto.Builder srb = StorageReportProto.newBuilder();
nrb.addStorageReport(reports[i].getProtoBufMessage()); nrb.addStorageReport(reports[i].getProtoBufMessage());
} }
return nrb.build(); return nrb.build();
@ -891,7 +891,7 @@ public class ContainerManagerImpl implements ContainerManager {
* @throws IOException * @throws IOException
*/ */
@Override @Override
public ContainerReportsRequestProto getContainerReport() throws IOException { public ContainerReportsProto getContainerReport() throws IOException {
LOG.debug("Starting container report iteration."); LOG.debug("Starting container report iteration.");
// No need for locking since containerMap is a ConcurrentSkipListMap // No need for locking since containerMap is a ConcurrentSkipListMap
// And we can never get the exact state since close might happen // And we can never get the exact state since close might happen
@ -899,12 +899,8 @@ public class ContainerManagerImpl implements ContainerManager {
List<ContainerData> containers = containerMap.values().stream() List<ContainerData> containers = containerMap.values().stream()
.collect(Collectors.toList()); .collect(Collectors.toList());
ContainerReportsRequestProto.Builder crBuilder = ContainerReportsProto.Builder crBuilder =
ContainerReportsRequestProto.newBuilder(); ContainerReportsProto.newBuilder();
// TODO: support delta based container report
crBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage())
.setType(ContainerReportsRequestProto.reportType.fullReport);
for (ContainerData container: containers) { for (ContainerData container: containers) {
long containerId = container.getContainerID(); long containerId = container.getContainerID();

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.ozone.container.common.impl;
import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdds.protocol.proto. import org.apache.hadoop.hdds.protocol.proto.
StorageContainerDatanodeProtocolProtos.SCMStorageReport; StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.hdds.protocol.proto. import org.apache.hadoop.hdds.protocol.proto.
StorageContainerDatanodeProtocolProtos.StorageTypeProto; StorageContainerDatanodeProtocolProtos.StorageTypeProto;
@ -137,8 +137,8 @@ public class StorageLocationReport {
* @return SCMStorageReport * @return SCMStorageReport
* @throws IOException In case, the storage type specified is invalid. * @throws IOException In case, the storage type specified is invalid.
*/ */
public SCMStorageReport getProtoBufMessage() throws IOException{ public StorageReportProto getProtoBufMessage() throws IOException{
SCMStorageReport.Builder srb = SCMStorageReport.newBuilder(); StorageReportProto.Builder srb = StorageReportProto.newBuilder();
return srb.setStorageUuid(getId()) return srb.setStorageUuid(getId())
.setCapacity(getCapacity()) .setCapacity(getCapacity())
.setScmUsed(getScmUsed()) .setScmUsed(getScmUsed())
@ -156,7 +156,7 @@ public class StorageLocationReport {
* @throws IOException in case of invalid storage type * @throws IOException in case of invalid storage type
*/ */
public static StorageLocationReport getFromProtobuf(SCMStorageReport report) public static StorageLocationReport getFromProtobuf(StorageReportProto report)
throws IOException { throws IOException {
StorageLocationReport.Builder builder = StorageLocationReport.newBuilder(); StorageLocationReport.Builder builder = StorageLocationReport.newBuilder();
builder.setId(report.getStorageUuid()) builder.setId(report.getStorageUuid())

View File

@ -27,9 +27,9 @@ import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.hdfs.util.RwLock; import org.apache.hadoop.hdfs.util.RwLock;
import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto; .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport; .StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData; import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import java.io.IOException; import java.io.IOException;
@ -171,14 +171,14 @@ public interface ContainerManager extends RwLock {
* Get the Node Report of container storage usage. * Get the Node Report of container storage usage.
* @return node report. * @return node report.
*/ */
SCMNodeReport getNodeReport() throws IOException; NodeReportProto getNodeReport() throws IOException;
/** /**
* Gets container report. * Gets container report.
* @return container report. * @return container report.
* @throws IOException * @throws IOException
*/ */
ContainerReportsRequestProto getContainerReport() throws IOException; ContainerReportsProto getContainerReport() throws IOException;
/** /**
* Gets container reports. * Gets container reports.

View File

@ -21,8 +21,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.CloseContainerCommandHandler;
.CloseContainerHandler;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
.CommandDispatcher; .CommandDispatcher;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
@ -86,7 +85,7 @@ public class DatanodeStateMachine implements Closeable {
// When we add new handlers just adding a new handler here should do the // When we add new handlers just adding a new handler here should do the
// trick. // trick.
commandDispatcher = CommandDispatcher.newBuilder() commandDispatcher = CommandDispatcher.newBuilder()
.addHandler(new CloseContainerHandler()) .addHandler(new CloseContainerCommandHandler())
.addHandler(new DeleteBlocksCommandHandler( .addHandler(new DeleteBlocksCommandHandler(
container.getContainerManager(), conf)) container.getContainerManager(), conf))
.setConnectionManager(connectionManager) .setConnectionManager(connectionManager)
@ -131,7 +130,7 @@ public class DatanodeStateMachine implements Closeable {
try { try {
LOG.debug("Executing cycle Number : {}", context.getExecutionCount()); LOG.debug("Executing cycle Number : {}", context.getExecutionCount());
nextHB.set(Time.monotonicNow() + heartbeatFrequency); nextHB.set(Time.monotonicNow() + heartbeatFrequency);
context.setReportState(container.getNodeReport()); context.setNodeReport(container.getNodeReport());
context.execute(executorService, heartbeatFrequency, context.execute(executorService, heartbeatFrequency,
TimeUnit.MILLISECONDS); TimeUnit.MILLISECONDS);
now = Time.monotonicNow(); now = Time.monotonicNow();

View File

@ -18,7 +18,7 @@ package org.apache.hadoop.ozone.container.common.statemachine;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport; .StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.ozone.container.common.states.DatanodeState; import org.apache.hadoop.ozone.container.common.states.DatanodeState;
import org.apache.hadoop.ozone.container.common.states.datanode import org.apache.hadoop.ozone.container.common.states.datanode
.InitDatanodeState; .InitDatanodeState;
@ -52,7 +52,7 @@ public class StateContext {
private final AtomicLong stateExecutionCount; private final AtomicLong stateExecutionCount;
private final Configuration conf; private final Configuration conf;
private DatanodeStateMachine.DatanodeStates state; private DatanodeStateMachine.DatanodeStates state;
private SCMNodeReport nrState; private NodeReportProto dnReport;
/** /**
* Constructs a StateContext. * Constructs a StateContext.
@ -69,7 +69,7 @@ public class StateContext {
commandQueue = new LinkedList<>(); commandQueue = new LinkedList<>();
lock = new ReentrantLock(); lock = new ReentrantLock();
stateExecutionCount = new AtomicLong(0); stateExecutionCount = new AtomicLong(0);
nrState = SCMNodeReport.getDefaultInstance(); dnReport = NodeReportProto.getDefaultInstance();
} }
/** /**
@ -144,16 +144,16 @@ public class StateContext {
* Returns the node report of the datanode state context. * Returns the node report of the datanode state context.
* @return the node report. * @return the node report.
*/ */
public SCMNodeReport getNodeReport() { public NodeReportProto getNodeReport() {
return nrState; return dnReport;
} }
/** /**
* Sets the storage location report of the datanode state context. * Sets the storage location report of the datanode state context.
* @param nrReport - node report * @param nodeReport node report
*/ */
public void setReportState(SCMNodeReport nrReport) { public void setNodeReport(NodeReportProto nodeReport) {
this.nrState = nrReport; this.dnReport = nodeReport;
} }
/** /**

View File

@ -17,9 +17,9 @@
package org.apache.hadoop.ozone.container.common.statemachine.commandhandler; package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCloseContainerCmdResponseProto; .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCmdType; .StorageContainerDatanodeProtocolProtos.CloseContainerCommandProto;
import org.apache.hadoop.ozone.container.common.statemachine import org.apache.hadoop.ozone.container.common.statemachine
.SCMConnectionManager; .SCMConnectionManager;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext; import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
@ -30,18 +30,18 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
/** /**
* Container Report handler. * Handler for close container command received from SCM.
*/ */
public class CloseContainerHandler implements CommandHandler { public class CloseContainerCommandHandler implements CommandHandler {
static final Logger LOG = static final Logger LOG =
LoggerFactory.getLogger(CloseContainerHandler.class); LoggerFactory.getLogger(CloseContainerCommandHandler.class);
private int invocationCount; private int invocationCount;
private long totalTime; private long totalTime;
/** /**
* Constructs a ContainerReport handler. * Constructs a ContainerReport handler.
*/ */
public CloseContainerHandler() { public CloseContainerCommandHandler() {
} }
/** /**
@ -62,9 +62,9 @@ public class CloseContainerHandler implements CommandHandler {
long containerID = -1; long containerID = -1;
try { try {
SCMCloseContainerCmdResponseProto CloseContainerCommandProto
closeContainerProto = closeContainerProto =
SCMCloseContainerCmdResponseProto CloseContainerCommandProto
.parseFrom(command.getProtoBufMessage()); .parseFrom(command.getProtoBufMessage());
containerID = closeContainerProto.getContainerID(); containerID = closeContainerProto.getContainerID();
@ -84,8 +84,8 @@ public class CloseContainerHandler implements CommandHandler {
* @return Type * @return Type
*/ */
@Override @Override
public SCMCmdType getCommandType() { public SCMCommandProto.Type getCommandType() {
return SCMCmdType.closeContainerCommand; return SCMCommandProto.Type.closeContainerCommand;
} }
/** /**

View File

@ -18,7 +18,8 @@
package org.apache.hadoop.ozone.container.common.statemachine.commandhandler; package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType; import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type;
import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager; import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext; import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
@ -38,7 +39,7 @@ public final class CommandDispatcher {
static final Logger LOG = static final Logger LOG =
LoggerFactory.getLogger(CommandDispatcher.class); LoggerFactory.getLogger(CommandDispatcher.class);
private final StateContext context; private final StateContext context;
private final Map<SCMCmdType, CommandHandler> handlerMap; private final Map<Type, CommandHandler> handlerMap;
private final OzoneContainer container; private final OzoneContainer container;
private final SCMConnectionManager connectionManager; private final SCMConnectionManager connectionManager;

View File

@ -17,8 +17,10 @@
package org.apache.hadoop.ozone.container.common.statemachine.commandhandler; package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType; import org.apache.hadoop.hdds.protocol.proto
import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager; .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.ozone.container.common.statemachine
.SCMConnectionManager;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext; import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand; import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
@ -42,7 +44,7 @@ public interface CommandHandler {
* Returns the command type that this command handler handles. * Returns the command type that this command handler handles.
* @return Type * @return Type
*/ */
SCMCmdType getCommandType(); SCMCommandProto.Type getCommandType();
/** /**
* Returns number of times this handler has been invoked. * Returns number of times this handler has been invoked.

View File

@ -18,6 +18,8 @@ package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
import com.google.common.primitives.Longs; import com.google.common.primitives.Longs;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto; .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
@ -26,8 +28,6 @@ import org.apache.hadoop.hdds.protocol.proto
.DeleteBlockTransactionResult; .DeleteBlockTransactionResult;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction; .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCmdType;
import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData; import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.helpers import org.apache.hadoop.ozone.container.common.helpers
@ -73,10 +73,10 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
@Override @Override
public void handle(SCMCommand command, OzoneContainer container, public void handle(SCMCommand command, OzoneContainer container,
StateContext context, SCMConnectionManager connectionManager) { StateContext context, SCMConnectionManager connectionManager) {
if (command.getType() != SCMCmdType.deleteBlocksCommand) { if (command.getType() != SCMCommandProto.Type.deleteBlocksCommand) {
LOG.warn("Skipping handling command, expected command " LOG.warn("Skipping handling command, expected command "
+ "type {} but found {}", + "type {} but found {}",
SCMCmdType.deleteBlocksCommand, command.getType()); SCMCommandProto.Type.deleteBlocksCommand, command.getType());
return; return;
} }
LOG.debug("Processing block deletion command."); LOG.debug("Processing block deletion command.");
@ -193,8 +193,8 @@ public class DeleteBlocksCommandHandler implements CommandHandler {
} }
@Override @Override
public SCMCmdType getCommandType() { public SCMCommandProto.Type getCommandType() {
return SCMCmdType.deleteBlocksCommand; return SCMCommandProto.Type.deleteBlocksCommand;
} }
@Override @Override

View File

@ -23,7 +23,9 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCommandResponseProto; .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto; .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
import org.apache.hadoop.ozone.container.common.helpers import org.apache.hadoop.ozone.container.common.helpers
@ -97,8 +99,13 @@ public class HeartbeatEndpointTask
try { try {
Preconditions.checkState(this.datanodeDetailsProto != null); Preconditions.checkState(this.datanodeDetailsProto != null);
SCMHeartbeatRequestProto request = SCMHeartbeatRequestProto.newBuilder()
.setDatanodeDetails(datanodeDetailsProto)
.setNodeReport(context.getNodeReport())
.build();
SCMHeartbeatResponseProto reponse = rpcEndpoint.getEndPoint() SCMHeartbeatResponseProto reponse = rpcEndpoint.getEndPoint()
.sendHeartbeat(datanodeDetailsProto, this.context.getNodeReport()); .sendHeartbeat(request);
processResponse(reponse, datanodeDetailsProto); processResponse(reponse, datanodeDetailsProto);
rpcEndpoint.setLastSuccessfulHeartbeat(ZonedDateTime.now()); rpcEndpoint.setLastSuccessfulHeartbeat(ZonedDateTime.now());
rpcEndpoint.zeroMissedCount(); rpcEndpoint.zeroMissedCount();
@ -125,13 +132,13 @@ public class HeartbeatEndpointTask
*/ */
private void processResponse(SCMHeartbeatResponseProto response, private void processResponse(SCMHeartbeatResponseProto response,
final DatanodeDetailsProto datanodeDetails) { final DatanodeDetailsProto datanodeDetails) {
for (SCMCommandResponseProto commandResponseProto : response Preconditions.checkState(response.getDatanodeUUID()
.equalsIgnoreCase(datanodeDetails.getUuid()),
"Unexpected datanode ID in the response.");
// Verify the response is indeed for this datanode.
for (SCMCommandProto commandResponseProto : response
.getCommandsList()) { .getCommandsList()) {
// Verify the response is indeed for this datanode. switch (commandResponseProto.getCommandType()) {
Preconditions.checkState(commandResponseProto.getDatanodeUUID()
.equalsIgnoreCase(datanodeDetails.getUuid()),
"Unexpected datanode ID in the response.");
switch (commandResponseProto.getCmdType()) {
case reregisterCommand: case reregisterCommand:
if (rpcEndpoint.getState() == EndPointStates.HEARTBEAT) { if (rpcEndpoint.getState() == EndPointStates.HEARTBEAT) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
@ -148,7 +155,8 @@ public class HeartbeatEndpointTask
break; break;
case deleteBlocksCommand: case deleteBlocksCommand:
DeleteBlocksCommand db = DeleteBlocksCommand DeleteBlocksCommand db = DeleteBlocksCommand
.getFromProtobuf(commandResponseProto.getDeleteBlocksProto()); .getFromProtobuf(
commandResponseProto.getDeleteBlocksCommandProto());
if (!db.blocksTobeDeleted().isEmpty()) { if (!db.blocksTobeDeleted().isEmpty()) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug(DeletedContainerBlocksSummary LOG.debug(DeletedContainerBlocksSummary
@ -161,7 +169,7 @@ public class HeartbeatEndpointTask
case closeContainerCommand: case closeContainerCommand:
CloseContainerCommand closeContainer = CloseContainerCommand closeContainer =
CloseContainerCommand.getFromProtobuf( CloseContainerCommand.getFromProtobuf(
commandResponseProto.getCloseContainerProto()); commandResponseProto.getCloseContainerCommandProto());
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Received SCM container close request for container {}", LOG.debug("Received SCM container close request for container {}",
closeContainer.getContainerID()); closeContainer.getContainerID());
@ -170,7 +178,7 @@ public class HeartbeatEndpointTask
break; break;
default: default:
throw new IllegalArgumentException("Unknown response : " throw new IllegalArgumentException("Unknown response : "
+ commandResponseProto.getCmdType().name()); + commandResponseProto.getCommandType().name());
} }
} }
} }

View File

@ -24,11 +24,11 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.container.common.statemachine import org.apache.hadoop.ozone.container.common.statemachine
.EndpointStateMachine; .EndpointStateMachine;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport; .StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto; .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto; .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -104,11 +104,11 @@ public final class RegisterEndpointTask implements
rpcEndPoint.lock(); rpcEndPoint.lock();
try { try {
ContainerReportsRequestProto contianerReport = datanodeContainerManager ContainerReportsProto contianerReport = datanodeContainerManager
.getContainerReport(); .getContainerReport();
SCMNodeReport nodeReport = datanodeContainerManager.getNodeReport(); NodeReportProto nodeReport = datanodeContainerManager.getNodeReport();
// TODO : Add responses to the command Queue. // TODO : Add responses to the command Queue.
SCMRegisteredCmdResponseProto response = rpcEndPoint.getEndPoint() SCMRegisteredResponseProto response = rpcEndPoint.getEndPoint()
.register(datanodeDetails.getProtoBufMessage(), nodeReport, .register(datanodeDetails.getProtoBufMessage(), nodeReport,
contianerReport); contianerReport);
Preconditions.checkState(UUID.fromString(response.getDatanodeUUID()) Preconditions.checkState(UUID.fromString(response.getDatanodeUUID())

View File

@ -19,14 +19,14 @@ package org.apache.hadoop.ozone.container.ozoneimpl;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation; import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto; .StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport;
import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData; import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl; import org.apache.hadoop.ozone.container.common.impl.ChunkManagerImpl;
@ -219,7 +219,7 @@ public class OzoneContainer {
/** /**
* Returns node report of container storage usage. * Returns node report of container storage usage.
*/ */
public SCMNodeReport getNodeReport() throws IOException { public NodeReportProto getNodeReport() throws IOException {
return this.manager.getNodeReport(); return this.manager.getNodeReport();
} }
@ -255,7 +255,7 @@ public class OzoneContainer {
* @return - container report. * @return - container report.
* @throws IOException * @throws IOException
*/ */
public ContainerReportsRequestProto getContainerReport() throws IOException { public ContainerReportsProto getContainerReport() throws IOException {
return this.manager.getContainerReport(); return this.manager.getContainerReport();
} }

View File

@ -18,21 +18,21 @@ package org.apache.hadoop.ozone.protocol;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto; .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos .StorageContainerDatanodeProtocolProtos
.ContainerBlocksDeletionACKResponseProto; .ContainerBlocksDeletionACKResponseProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto; .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsResponseProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto; .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport; .StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto; .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto; .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
@ -55,13 +55,12 @@ public interface StorageContainerDatanodeProtocol {
/** /**
* Used by data node to send a Heartbeat. * Used by data node to send a Heartbeat.
* @param datanodeDetails - Datanode Details. * @param heartbeat Heartbeat
* @param nodeReport - node report state
* @return - SCMHeartbeatResponseProto * @return - SCMHeartbeatResponseProto
* @throws IOException * @throws IOException
*/ */
SCMHeartbeatResponseProto sendHeartbeat(DatanodeDetailsProto datanodeDetails, SCMHeartbeatResponseProto sendHeartbeat(SCMHeartbeatRequestProto heartbeat)
SCMNodeReport nodeReport) throws IOException; throws IOException;
/** /**
* Register Datanode. * Register Datanode.
@ -70,19 +69,10 @@ public interface StorageContainerDatanodeProtocol {
* @param containerReportsRequestProto - Container Reports. * @param containerReportsRequestProto - Container Reports.
* @return SCM Command. * @return SCM Command.
*/ */
SCMRegisteredCmdResponseProto register(DatanodeDetailsProto datanodeDetails, SCMRegisteredResponseProto register(DatanodeDetailsProto datanodeDetails,
SCMNodeReport nodeReport, ContainerReportsRequestProto NodeReportProto nodeReport, ContainerReportsProto
containerReportsRequestProto) throws IOException; containerReportsRequestProto) throws IOException;
/**
* Send a container report.
* @param reports -- Container report.
* @return container reports response.
* @throws IOException
*/
ContainerReportsResponseProto sendContainerReport(
ContainerReportsRequestProto reports) throws IOException;
/** /**
* Used by datanode to send block deletion ACK to SCM. * Used by datanode to send block deletion ACK to SCM.
* @param request block deletion transactions. * @param request block deletion transactions.

View File

@ -18,11 +18,12 @@
package org.apache.hadoop.ozone.protocol; package org.apache.hadoop.ozone.protocol;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto; import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport; .StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto; .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand; import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import java.util.List; import java.util.List;
@ -49,11 +50,11 @@ public interface StorageContainerNodeProtocol {
/** /**
* Register the node if the node finds that it is not registered with any SCM. * Register the node if the node finds that it is not registered with any SCM.
* @param datanodeDetails DatanodeDetails * @param datanodeDetails DatanodeDetails
* @param nodeReport SCMNodeReport * @param nodeReport NodeReportProto
* @return SCMHeartbeatResponseProto * @return SCMHeartbeatResponseProto
*/ */
SCMCommand register(DatanodeDetailsProto datanodeDetails, SCMNodeReport RegisteredCommand register(DatanodeDetails datanodeDetails,
nodeReport); NodeReportProto nodeReport);
/** /**
* Send heartbeat to indicate the datanode is alive and doing well. * Send heartbeat to indicate the datanode is alive and doing well.
@ -61,7 +62,7 @@ public interface StorageContainerNodeProtocol {
* @param nodeReport - node report. * @param nodeReport - node report.
* @return SCMheartbeat response list * @return SCMheartbeat response list
*/ */
List<SCMCommand> sendHeartbeat(DatanodeDetailsProto datanodeDetails, List<SCMCommand> sendHeartbeat(DatanodeDetails datanodeDetails,
SCMNodeReport nodeReport); NodeReportProto nodeReport);
} }

View File

@ -19,18 +19,16 @@ package org.apache.hadoop.ozone.protocol.commands;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCloseContainerCmdResponseProto; .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCmdType; .StorageContainerDatanodeProtocolProtos.CloseContainerCommandProto;
import static org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCmdType.closeContainerCommand;
/** /**
* Asks datanode to close a container. * Asks datanode to close a container.
*/ */
public class CloseContainerCommand public class CloseContainerCommand
extends SCMCommand<SCMCloseContainerCmdResponseProto> { extends SCMCommand<CloseContainerCommandProto> {
private long containerID; private long containerID;
@ -44,8 +42,8 @@ public class CloseContainerCommand
* @return Type * @return Type
*/ */
@Override @Override
public SCMCmdType getType() { public SCMCommandProto.Type getType() {
return closeContainerCommand; return SCMCommandProto.Type.closeContainerCommand;
} }
/** /**
@ -58,13 +56,13 @@ public class CloseContainerCommand
return getProto().toByteArray(); return getProto().toByteArray();
} }
public SCMCloseContainerCmdResponseProto getProto() { public CloseContainerCommandProto getProto() {
return SCMCloseContainerCmdResponseProto.newBuilder() return CloseContainerCommandProto.newBuilder()
.setContainerID(containerID).build(); .setContainerID(containerID).build();
} }
public static CloseContainerCommand getFromProtobuf( public static CloseContainerCommand getFromProtobuf(
SCMCloseContainerCmdResponseProto closeContainerProto) { CloseContainerCommandProto closeContainerProto) {
Preconditions.checkNotNull(closeContainerProto); Preconditions.checkNotNull(closeContainerProto);
return new CloseContainerCommand(closeContainerProto.getContainerID()); return new CloseContainerCommand(closeContainerProto.getContainerID());

View File

@ -17,12 +17,12 @@
*/ */
package org.apache.hadoop.ozone.protocol.commands; package org.apache.hadoop.ozone.protocol.commands;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction; .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCmdType; .StorageContainerDatanodeProtocolProtos.DeleteBlocksCommandProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMDeleteBlocksCmdResponseProto;
import java.util.List; import java.util.List;
@ -30,7 +30,7 @@ import java.util.List;
* A SCM command asks a datanode to delete a number of blocks. * A SCM command asks a datanode to delete a number of blocks.
*/ */
public class DeleteBlocksCommand extends public class DeleteBlocksCommand extends
SCMCommand<SCMDeleteBlocksCmdResponseProto> { SCMCommand<DeleteBlocksCommandProto> {
private List<DeletedBlocksTransaction> blocksTobeDeleted; private List<DeletedBlocksTransaction> blocksTobeDeleted;
@ -44,8 +44,8 @@ public class DeleteBlocksCommand extends
} }
@Override @Override
public SCMCmdType getType() { public SCMCommandProto.Type getType() {
return SCMCmdType.deleteBlocksCommand; return SCMCommandProto.Type.deleteBlocksCommand;
} }
@Override @Override
@ -54,13 +54,13 @@ public class DeleteBlocksCommand extends
} }
public static DeleteBlocksCommand getFromProtobuf( public static DeleteBlocksCommand getFromProtobuf(
SCMDeleteBlocksCmdResponseProto deleteBlocksProto) { DeleteBlocksCommandProto deleteBlocksProto) {
return new DeleteBlocksCommand(deleteBlocksProto return new DeleteBlocksCommand(deleteBlocksProto
.getDeletedBlocksTransactionsList()); .getDeletedBlocksTransactionsList());
} }
public SCMDeleteBlocksCmdResponseProto getProto() { public DeleteBlocksCommandProto getProto() {
return SCMDeleteBlocksCmdResponseProto.newBuilder() return DeleteBlocksCommandProto.newBuilder()
.addAllDeletedBlocksTransactions(blocksTobeDeleted).build(); .addAllDeletedBlocksTransactions(blocksTobeDeleted).build();
} }
} }

View File

@ -19,18 +19,15 @@ package org.apache.hadoop.ozone.protocol.commands;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCmdType; .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto; .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto
.ErrorCode; .ErrorCode;
/** /**
* Response to Datanode Register call. * Response to Datanode Register call.
*/ */
public class RegisteredCommand extends public class RegisteredCommand {
SCMCommand<SCMRegisteredCmdResponseProto> {
private String datanodeUUID; private String datanodeUUID;
private String clusterID; private String clusterID;
private ErrorCode error; private ErrorCode error;
@ -59,16 +56,6 @@ public class RegisteredCommand extends
return new Builder(); return new Builder();
} }
/**
* Returns the type of this command.
*
* @return Type
*/
@Override
public SCMCmdType getType() {
return SCMCmdType.registeredCommand;
}
/** /**
* Returns datanode UUID. * Returns datanode UUID.
* *
@ -117,10 +104,9 @@ public class RegisteredCommand extends
* *
* @return A protobuf message. * @return A protobuf message.
*/ */
@Override
public byte[] getProtoBufMessage() { public byte[] getProtoBufMessage() {
SCMRegisteredCmdResponseProto.Builder builder = SCMRegisteredResponseProto.Builder builder =
SCMRegisteredCmdResponseProto.newBuilder() SCMRegisteredResponseProto.newBuilder()
.setClusterID(this.clusterID) .setClusterID(this.clusterID)
.setDatanodeUUID(this.datanodeUUID) .setDatanodeUUID(this.datanodeUUID)
.setErrorCode(this.error); .setErrorCode(this.error);
@ -157,7 +143,7 @@ public class RegisteredCommand extends
* @param response - RegisteredCmdResponseProto * @param response - RegisteredCmdResponseProto
* @return RegisteredCommand * @return RegisteredCommand
*/ */
public RegisteredCommand getFromProtobuf(SCMRegisteredCmdResponseProto public RegisteredCommand getFromProtobuf(SCMRegisteredResponseProto
response) { response) {
Preconditions.checkNotNull(response); Preconditions.checkNotNull(response);
if (response.hasHostname() && response.hasIpAddress()) { if (response.hasHostname() && response.hasIpAddress()) {

View File

@ -18,18 +18,16 @@
package org.apache.hadoop.ozone.protocol.commands; package org.apache.hadoop.ozone.protocol.commands;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCmdType; .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import static org.apache.hadoop.hdds.protocol.proto import static org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCmdType.reregisterCommand; .StorageContainerDatanodeProtocolProtos.ReregisterCommandProto;
import static org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMReregisterCmdResponseProto;
/** /**
* Informs a datanode to register itself with SCM again. * Informs a datanode to register itself with SCM again.
*/ */
public class ReregisterCommand extends public class ReregisterCommand extends
SCMCommand<SCMReregisterCmdResponseProto>{ SCMCommand<ReregisterCommandProto>{
/** /**
* Returns the type of this command. * Returns the type of this command.
@ -37,8 +35,8 @@ public class ReregisterCommand extends
* @return Type * @return Type
*/ */
@Override @Override
public SCMCmdType getType() { public SCMCommandProto.Type getType() {
return reregisterCommand; return SCMCommandProto.Type.reregisterCommand;
} }
/** /**
@ -51,8 +49,8 @@ public class ReregisterCommand extends
return getProto().toByteArray(); return getProto().toByteArray();
} }
public SCMReregisterCmdResponseProto getProto() { public ReregisterCommandProto getProto() {
return SCMReregisterCmdResponseProto return ReregisterCommandProto
.newBuilder() .newBuilder()
.build(); .build();
} }

View File

@ -19,7 +19,7 @@ package org.apache.hadoop.ozone.protocol.commands;
import com.google.protobuf.GeneratedMessage; import com.google.protobuf.GeneratedMessage;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCmdType; .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
/** /**
* A class that acts as the base class to convert between Java and SCM * A class that acts as the base class to convert between Java and SCM
@ -31,7 +31,7 @@ public abstract class SCMCommand<T extends GeneratedMessage> {
* Returns the type of this command. * Returns the type of this command.
* @return Type * @return Type
*/ */
public abstract SCMCmdType getType(); public abstract SCMCommandProto.Type getType();
/** /**
* Gets the protobuf message of this object. * Gets the protobuf message of this object.

View File

@ -19,25 +19,24 @@ package org.apache.hadoop.ozone.protocolPB;
import com.google.protobuf.RpcController; import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException; import com.google.protobuf.ServiceException;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto; .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos .StorageContainerDatanodeProtocolProtos
.ContainerBlocksDeletionACKResponseProto; .ContainerBlocksDeletionACKResponseProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsResponseProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto; .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto; .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport; .StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMRegisterRequestProto; .StorageContainerDatanodeProtocolProtos.SCMRegisterRequestProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto; .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto; .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
@ -123,22 +122,16 @@ public class StorageContainerDatanodeProtocolClientSideTranslatorPB
/** /**
* Send by datanode to SCM. * Send by datanode to SCM.
* *
* @param datanodeDetailsProto - Datanode Details * @param heartbeat node heartbeat
* @param nodeReport - node report
* @throws IOException * @throws IOException
*/ */
@Override @Override
public SCMHeartbeatResponseProto sendHeartbeat( public SCMHeartbeatResponseProto sendHeartbeat(
DatanodeDetailsProto datanodeDetailsProto, SCMHeartbeatRequestProto heartbeat) throws IOException {
SCMNodeReport nodeReport) throws IOException {
SCMHeartbeatRequestProto.Builder req = SCMHeartbeatRequestProto
.newBuilder();
req.setDatanodeDetails(datanodeDetailsProto);
req.setNodeReport(nodeReport);
final SCMHeartbeatResponseProto resp; final SCMHeartbeatResponseProto resp;
try { try {
resp = rpcProxy.sendHeartbeat(NULL_RPC_CONTROLLER, req.build()); resp = rpcProxy.sendHeartbeat(NULL_RPC_CONTROLLER, heartbeat);
} catch (ServiceException e) { } catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e); throw ProtobufHelper.getRemoteException(e);
} }
@ -154,16 +147,16 @@ public class StorageContainerDatanodeProtocolClientSideTranslatorPB
* @return SCM Command. * @return SCM Command.
*/ */
@Override @Override
public SCMRegisteredCmdResponseProto register( public SCMRegisteredResponseProto register(
DatanodeDetailsProto datanodeDetailsProto, SCMNodeReport nodeReport, DatanodeDetailsProto datanodeDetailsProto, NodeReportProto nodeReport,
ContainerReportsRequestProto containerReportsRequestProto) ContainerReportsProto containerReportsRequestProto)
throws IOException { throws IOException {
SCMRegisterRequestProto.Builder req = SCMRegisterRequestProto.Builder req =
SCMRegisterRequestProto.newBuilder(); SCMRegisterRequestProto.newBuilder();
req.setDatanodeDetails(datanodeDetailsProto); req.setDatanodeDetails(datanodeDetailsProto);
req.setContainerReport(containerReportsRequestProto); req.setContainerReport(containerReportsRequestProto);
req.setNodeReport(nodeReport); req.setNodeReport(nodeReport);
final SCMRegisteredCmdResponseProto response; final SCMRegisteredResponseProto response;
try { try {
response = rpcProxy.register(NULL_RPC_CONTROLLER, req.build()); response = rpcProxy.register(NULL_RPC_CONTROLLER, req.build());
} catch (ServiceException e) { } catch (ServiceException e) {
@ -172,25 +165,6 @@ public class StorageContainerDatanodeProtocolClientSideTranslatorPB
return response; return response;
} }
/**
* Send a container report.
*
* @param reports -- Container report
* @return HeartbeatRespose.nullcommand.
* @throws IOException
*/
@Override
public ContainerReportsResponseProto sendContainerReport(
ContainerReportsRequestProto reports) throws IOException {
final ContainerReportsResponseProto resp;
try {
resp = rpcProxy.sendContainerReport(NULL_RPC_CONTROLLER, reports);
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
return resp;
}
@Override @Override
public ContainerBlocksDeletionACKResponseProto sendContainerBlocksDeletionACK( public ContainerBlocksDeletionACKResponseProto sendContainerBlocksDeletionACK(
ContainerBlocksDeletionACKProto deletedBlocks) throws IOException { ContainerBlocksDeletionACKProto deletedBlocks) throws IOException {

View File

@ -19,18 +19,22 @@ package org.apache.hadoop.ozone.protocolPB;
import com.google.protobuf.RpcController; import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException; import com.google.protobuf.ServiceException;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos; .StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMRegisterRequestProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto; .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos .StorageContainerDatanodeProtocolProtos
.ContainerBlocksDeletionACKResponseProto; .ContainerBlocksDeletionACKResponseProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsResponseProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto; .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
@ -55,9 +59,8 @@ public class StorageContainerDatanodeProtocolServerSideTranslatorPB
} }
@Override @Override
public StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto public SCMVersionResponseProto getVersion(RpcController controller,
getVersion(RpcController controller, SCMVersionRequestProto request)
StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto request)
throws ServiceException { throws ServiceException {
try { try {
return impl.getVersion(request); return impl.getVersion(request);
@ -67,15 +70,13 @@ public class StorageContainerDatanodeProtocolServerSideTranslatorPB
} }
@Override @Override
public StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto public SCMRegisteredResponseProto register(RpcController controller,
register(RpcController controller, StorageContainerDatanodeProtocolProtos SCMRegisterRequestProto request) throws ServiceException {
.SCMRegisterRequestProto request) throws ServiceException {
try { try {
ContainerReportsRequestProto containerRequestProto = null; ContainerReportsProto containerRequestProto = request
SCMNodeReport scmNodeReport = null; .getContainerReport();
containerRequestProto = request.getContainerReport(); NodeReportProto dnNodeReport = request.getNodeReport();
scmNodeReport = request.getNodeReport(); return impl.register(request.getDatanodeDetails(), dnNodeReport,
return impl.register(request.getDatanodeDetails(), scmNodeReport,
containerRequestProto); containerRequestProto);
} catch (IOException e) { } catch (IOException e) {
throw new ServiceException(e); throw new ServiceException(e);
@ -83,27 +84,15 @@ public class StorageContainerDatanodeProtocolServerSideTranslatorPB
} }
@Override @Override
public SCMHeartbeatResponseProto public SCMHeartbeatResponseProto sendHeartbeat(RpcController controller,
sendHeartbeat(RpcController controller,
SCMHeartbeatRequestProto request) throws ServiceException { SCMHeartbeatRequestProto request) throws ServiceException {
try { try {
return impl.sendHeartbeat(request.getDatanodeDetails(), return impl.sendHeartbeat(request);
request.getNodeReport());
} catch (IOException e) { } catch (IOException e) {
throw new ServiceException(e); throw new ServiceException(e);
} }
} }
@Override
public ContainerReportsResponseProto sendContainerReport(
RpcController controller, ContainerReportsRequestProto request)
throws ServiceException {
try {
return impl.sendContainerReport(request);
} catch (IOException e) {
throw new ServiceException(e);
}
}
@Override @Override
public ContainerBlocksDeletionACKResponseProto sendContainerBlocksDeletionACK( public ContainerBlocksDeletionACKResponseProto sendContainerBlocksDeletionACK(

View File

@ -34,6 +34,42 @@ package hadoop.hdds;
import "hdds.proto"; import "hdds.proto";
/**
* Request for version info of the software stack on the server.
*/
message SCMVersionRequestProto {}
/**
* Generic response that is send to a version request. This allows keys to be
* added on the fly and protocol to remain stable.
*/
message SCMVersionResponseProto {
required uint32 softwareVersion = 1;
repeated hadoop.hdds.KeyValue keys = 2;
}
message SCMRegisterRequestProto {
required DatanodeDetailsProto datanodeDetails = 1;
required NodeReportProto nodeReport = 2;
required ContainerReportsProto containerReport = 3;
}
/**
* Datanode ID returned by the SCM. This is similar to name node
* registeration of a datanode.
*/
message SCMRegisteredResponseProto {
enum ErrorCode {
success = 1;
errorNodeNotPermitted = 2;
}
required ErrorCode errorCode = 1;
required string datanodeUUID = 2;
required string clusterID = 3;
optional SCMNodeAddressList addressList = 4;
optional string hostname = 5;
optional string ipAddress = 6;
}
/** /**
* This message is send by data node to indicate that it is alive or it is * This message is send by data node to indicate that it is alive or it is
@ -41,60 +77,38 @@ import "hdds.proto";
*/ */
message SCMHeartbeatRequestProto { message SCMHeartbeatRequestProto {
required DatanodeDetailsProto datanodeDetails = 1; required DatanodeDetailsProto datanodeDetails = 1;
optional SCMNodeReport nodeReport = 2; optional NodeReportProto nodeReport = 2;
optional ContainerReportsProto containerReport = 3;
} }
/** /*
A container report contains the following information. * A group of commands for the datanode to execute
*/ */
message ContainerInfo { message SCMHeartbeatResponseProto {
optional string finalhash = 2; required string datanodeUUID = 1;
optional int64 size = 3; repeated SCMCommandProto commands = 2;
optional int64 used = 4;
optional int64 keyCount = 5;
// TODO: move the io count to separate message
optional int64 readCount = 6;
optional int64 writeCount = 7;
optional int64 readBytes = 8;
optional int64 writeBytes = 9;
required int64 containerID = 10;
optional hadoop.hdds.LifeCycleState state = 11;
} }
// The deleted blocks which are stored in deletedBlock.db of scm. message SCMNodeAddressList {
// We don't use BlockID because this only contians multiple localIDs repeated string addressList = 1;
// of the same containerID.
message DeletedBlocksTransaction {
required int64 txID = 1;
required int64 containerID = 2;
repeated int64 localID = 3;
// the retry time of sending deleting command to datanode.
required int32 count = 4;
}
/**
A set of container reports, max count is generally set to
8192 since that keeps the size of the reports under 1 MB.
*/
message ContainerReportsRequestProto {
enum reportType {
fullReport = 0;
deltaReport = 1;
}
required DatanodeDetailsProto datanodeDetails = 1;
repeated ContainerInfo reports = 2;
required reportType type = 3;
}
message ContainerReportsResponseProto {
} }
/** /**
* This message is send along with the heart beat to report datanode * This message is send along with the heart beat to report datanode
* storage utilization by SCM. * storage utilization to SCM.
*/ */
message SCMNodeReport { message NodeReportProto {
repeated SCMStorageReport storageReport = 1; repeated StorageReportProto storageReport = 1;
}
message StorageReportProto {
required string storageUuid = 1;
required string storageLocation = 2;
optional uint64 capacity = 3 [default = 0];
optional uint64 scmUsed = 4 [default = 0];
optional uint64 remaining = 5 [default = 0];
optional StorageTypeProto storageType = 6 [default = DISK];
optional bool failed = 7 [default = false];
} }
/** /**
@ -108,117 +122,71 @@ enum StorageTypeProto {
PROVIDED = 5; PROVIDED = 5;
} }
message SCMStorageReport {
required string storageUuid = 1;
required string storageLocation = 2;
optional uint64 capacity = 3 [default = 0];
optional uint64 scmUsed = 4 [default = 0];
optional uint64 remaining = 5 [default = 0];
optional StorageTypeProto storageType = 6 [default = DISK];
optional bool failed = 7 [default = false];
}
message SCMRegisterRequestProto {
required DatanodeDetailsProto datanodeDetails = 1;
required SCMNodeReport nodeReport = 2;
required ContainerReportsRequestProto containerReport = 3;
}
/** /**
* Request for version info of the software stack on the server. A set of container reports, max count is generally set to
*/ 8192 since that keeps the size of the reports under 1 MB.
message SCMVersionRequestProto {
}
/**
* Generic response that is send to a version request. This allows keys to be
* added on the fly and protocol to remain stable.
*/ */
message SCMVersionResponseProto { message ContainerReportsProto {
required uint32 softwareVersion = 1; repeated ContainerInfo reports = 2;
repeated hadoop.hdds.KeyValue keys = 2;
} }
message SCMNodeAddressList {
repeated string addressList = 1;
}
/** /**
* Datanode ID returned by the SCM. This is similar to name node A container report contains the following information.
* registeration of a datanode. */
message ContainerInfo {
optional string finalhash = 1;
optional int64 size = 2;
optional int64 used = 3;
optional int64 keyCount = 4;
// TODO: move the io count to separate message
optional int64 readCount = 5;
optional int64 writeCount = 6;
optional int64 readBytes = 7;
optional int64 writeBytes = 8;
required int64 containerID = 9;
optional hadoop.hdds.LifeCycleState state = 10;
}
/*
* These are commands returned by SCM for to the datanode to execute.
*/ */
message SCMRegisteredCmdResponseProto { message SCMCommandProto {
enum ErrorCode { enum Type {
success = 1; reregisterCommand = 1;
errorNodeNotPermitted = 2; deleteBlocksCommand = 2;
closeContainerCommand = 3;
deleteContainerCommand = 4;
} }
required ErrorCode errorCode = 2; // TODO: once we start using protoc 3.x, refactor this message using "oneof"
required string datanodeUUID = 3; required Type commandType = 1;
required string clusterID = 4; optional ReregisterCommandProto reregisterCommandProto = 2;
optional SCMNodeAddressList addressList = 5; optional DeleteBlocksCommandProto deleteBlocksCommandProto = 3;
optional string hostname = 6; optional CloseContainerCommandProto closeContainerCommandProto = 4;
optional string ipAddress = 7; optional DeleteContainerCommandProto deleteContainerCommandProto = 5;
} }
/** /**
* SCM informs a datanode to register itself again. * SCM informs a datanode to register itself again.
* With recieving this command, datanode will transit to REGISTER state. * With recieving this command, datanode will transit to REGISTER state.
*/ */
message SCMReregisterCmdResponseProto {} message ReregisterCommandProto {}
/**
This command tells the data node to send in the container report when possible
*/
message SendContainerReportProto {
}
/**
This command asks the datanode to close a specific container.
*/
message SCMCloseContainerCmdResponseProto {
required int64 containerID = 1;
}
/**
Type of commands supported by SCM to datanode protocol.
*/
enum SCMCmdType {
versionCommand = 2;
registeredCommand = 3;
reregisterCommand = 4;
deleteBlocksCommand = 5;
closeContainerCommand = 6;
}
/*
* These are commands returned by SCM for to the datanode to execute.
*/
message SCMCommandResponseProto {
required SCMCmdType cmdType = 2; // Type of the command
optional SCMRegisteredCmdResponseProto registeredProto = 3;
optional SCMVersionResponseProto versionProto = 4;
optional SCMReregisterCmdResponseProto reregisterProto = 5;
optional SCMDeleteBlocksCmdResponseProto deleteBlocksProto = 6;
required string datanodeUUID = 7;
optional SCMCloseContainerCmdResponseProto closeContainerProto = 8;
}
/*
* A group of commands for the datanode to execute
*/
message SCMHeartbeatResponseProto {
repeated SCMCommandResponseProto commands = 1;
}
// HB response from SCM, contains a list of block deletion transactions. // HB response from SCM, contains a list of block deletion transactions.
message SCMDeleteBlocksCmdResponseProto { message DeleteBlocksCommandProto {
repeated DeletedBlocksTransaction deletedBlocksTransactions = 1; repeated DeletedBlocksTransaction deletedBlocksTransactions = 1;
} }
// SendACK response returned by datanode to SCM, currently empty. // The deleted blocks which are stored in deletedBlock.db of scm.
message ContainerBlocksDeletionACKResponseProto { // We don't use BlockID because this only contians multiple localIDs
// of the same containerID.
message DeletedBlocksTransaction {
required int64 txID = 1;
required int64 containerID = 2;
repeated int64 localID = 3;
// the retry time of sending deleting command to datanode.
required int32 count = 4;
} }
// ACK message datanode sent to SCM, contains the result of // ACK message datanode sent to SCM, contains the result of
@ -231,6 +199,24 @@ message ContainerBlocksDeletionACKProto {
repeated DeleteBlockTransactionResult results = 1; repeated DeleteBlockTransactionResult results = 1;
} }
// SendACK response returned by datanode to SCM, currently empty.
message ContainerBlocksDeletionACKResponseProto {
}
/**
This command asks the datanode to close a specific container.
*/
message CloseContainerCommandProto {
required int64 containerID = 1;
}
/**
This command asks the datanode to close a specific container.
*/
message DeleteContainerCommandProto {
required int64 containerID = 1;
}
/** /**
* Protocol used from a datanode to StorageContainerManager. * Protocol used from a datanode to StorageContainerManager.
* *
@ -305,7 +291,7 @@ service StorageContainerDatanodeProtocolService {
/** /**
* Registers a data node with SCM. * Registers a data node with SCM.
*/ */
rpc register (SCMRegisterRequestProto) returns (SCMRegisteredCmdResponseProto); rpc register (SCMRegisterRequestProto) returns (SCMRegisteredResponseProto);
/** /**
* Send heartbeat from datanode to SCM. HB's under SCM looks more * Send heartbeat from datanode to SCM. HB's under SCM looks more
@ -314,12 +300,6 @@ service StorageContainerDatanodeProtocolService {
*/ */
rpc sendHeartbeat (SCMHeartbeatRequestProto) returns (SCMHeartbeatResponseProto); rpc sendHeartbeat (SCMHeartbeatRequestProto) returns (SCMHeartbeatResponseProto);
/**
send container reports sends the container report to SCM. This will
return a null command as response.
*/
rpc sendContainerReport(ContainerReportsRequestProto) returns (ContainerReportsResponseProto);
/** /**
* Sends the block deletion ACK to SCM. * Sends the block deletion ACK to SCM.
*/ */

View File

@ -16,12 +16,12 @@
*/ */
package org.apache.hadoop.ozone.container.common; package org.apache.hadoop.ozone.container.common;
import com.google.common.base.Preconditions; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.scm.VersionInfo; import org.apache.hadoop.hdds.scm.VersionInfo;
import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos; .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto; .StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
@ -30,13 +30,13 @@ import org.apache.hadoop.hdds.protocol.proto
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerInfo; .StorageContainerDatanodeProtocolProtos.ContainerInfo;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCommandResponseProto; .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto; .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport; .StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMStorageReport; .StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.ozone.protocol.StorageContainerDatanodeProtocol; import org.apache.hadoop.ozone.protocol.StorageContainerDatanodeProtocol;
import org.apache.hadoop.ozone.protocol.VersionResponse; import org.apache.hadoop.ozone.protocol.VersionResponse;
@ -56,7 +56,7 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
// Map of datanode to containers // Map of datanode to containers
private Map<DatanodeDetails, Map<String, ContainerInfo>> nodeContainers = private Map<DatanodeDetails, Map<String, ContainerInfo>> nodeContainers =
new HashMap(); new HashMap();
private Map<DatanodeDetails, SCMNodeReport> nodeReports = new HashMap<>(); private Map<DatanodeDetails, NodeReportProto> nodeReports = new HashMap<>();
/** /**
* Returns the number of heartbeats made to this class. * Returns the number of heartbeats made to this class.
* *
@ -166,20 +166,17 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
/** /**
* Used by data node to send a Heartbeat. * Used by data node to send a Heartbeat.
* *
* @param datanodeDetailsProto - DatanodeDetailsProto. * @param heartbeat - node heartbeat.
* @param nodeReport - node report.
* @return - SCMHeartbeatResponseProto * @return - SCMHeartbeatResponseProto
* @throws IOException * @throws IOException
*/ */
@Override @Override
public StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto public StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto
sendHeartbeat(DatanodeDetailsProto datanodeDetailsProto, sendHeartbeat(SCMHeartbeatRequestProto heartbeat) throws IOException {
SCMNodeReport nodeReport)
throws IOException {
rpcCount.incrementAndGet(); rpcCount.incrementAndGet();
heartbeatCount.incrementAndGet(); heartbeatCount.incrementAndGet();
sleepIfNeeded(); sleepIfNeeded();
List<SCMCommandResponseProto> List<SCMCommandProto>
cmdResponses = new LinkedList<>(); cmdResponses = new LinkedList<>();
return SCMHeartbeatResponseProto.newBuilder().addAllCommands(cmdResponses) return SCMHeartbeatResponseProto.newBuilder().addAllCommands(cmdResponses)
.build(); .build();
@ -193,21 +190,19 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
*/ */
@Override @Override
public StorageContainerDatanodeProtocolProtos public StorageContainerDatanodeProtocolProtos
.SCMRegisteredCmdResponseProto register( .SCMRegisteredResponseProto register(
DatanodeDetailsProto datanodeDetailsProto, SCMNodeReport nodeReport, DatanodeDetailsProto datanodeDetailsProto, NodeReportProto nodeReport,
StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto StorageContainerDatanodeProtocolProtos.ContainerReportsProto
containerReportsRequestProto) containerReportsRequestProto)
throws IOException { throws IOException {
rpcCount.incrementAndGet(); rpcCount.incrementAndGet();
sendContainerReport(containerReportsRequestProto);
updateNodeReport(datanodeDetailsProto, nodeReport); updateNodeReport(datanodeDetailsProto, nodeReport);
sleepIfNeeded(); sleepIfNeeded();
return StorageContainerDatanodeProtocolProtos return StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto
.SCMRegisteredCmdResponseProto
.newBuilder().setClusterID(UUID.randomUUID().toString()) .newBuilder().setClusterID(UUID.randomUUID().toString())
.setDatanodeUUID(datanodeDetailsProto.getUuid()).setErrorCode( .setDatanodeUUID(datanodeDetailsProto.getUuid()).setErrorCode(
StorageContainerDatanodeProtocolProtos StorageContainerDatanodeProtocolProtos
.SCMRegisteredCmdResponseProto.ErrorCode.success).build(); .SCMRegisteredResponseProto.ErrorCode.success).build();
} }
/** /**
@ -216,19 +211,19 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
* @param nodeReport * @param nodeReport
*/ */
public void updateNodeReport(DatanodeDetailsProto datanodeDetailsProto, public void updateNodeReport(DatanodeDetailsProto datanodeDetailsProto,
SCMNodeReport nodeReport) { NodeReportProto nodeReport) {
DatanodeDetails datanode = DatanodeDetails.getFromProtoBuf( DatanodeDetails datanode = DatanodeDetails.getFromProtoBuf(
datanodeDetailsProto); datanodeDetailsProto);
SCMNodeReport.Builder datanodeReport = SCMNodeReport.newBuilder(); NodeReportProto.Builder nodeReportProto = NodeReportProto.newBuilder();
List<SCMStorageReport> storageReports = List<StorageReportProto> storageReports =
nodeReport.getStorageReportList(); nodeReport.getStorageReportList();
for(SCMStorageReport report : storageReports) { for(StorageReportProto report : storageReports) {
datanodeReport.addStorageReport(report); nodeReportProto.addStorageReport(report);
} }
nodeReports.put(datanode, datanodeReport.build()); nodeReports.put(datanode, nodeReportProto.build());
} }
@ -254,39 +249,6 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
return 0; return 0;
} }
/**
* Send a container report.
*
* @param reports -- Container report
* @return HeartbeatResponse.nullcommand.
* @throws IOException
*/
@Override
public StorageContainerDatanodeProtocolProtos.ContainerReportsResponseProto
sendContainerReport(StorageContainerDatanodeProtocolProtos
.ContainerReportsRequestProto reports) throws IOException {
Preconditions.checkNotNull(reports);
containerReportsCount.incrementAndGet();
DatanodeDetails datanode = DatanodeDetails.getFromProtoBuf(
reports.getDatanodeDetails());
if (reports.getReportsCount() > 0) {
Map containers = nodeContainers.get(datanode);
if (containers == null) {
containers = new LinkedHashMap();
nodeContainers.put(datanode, containers);
}
for (StorageContainerDatanodeProtocolProtos.ContainerInfo report:
reports.getReportsList()) {
containers.put(report.getContainerID(), report);
}
}
return StorageContainerDatanodeProtocolProtos
.ContainerReportsResponseProto.newBuilder().build();
}
@Override @Override
public ContainerBlocksDeletionACKResponseProto sendContainerBlocksDeletionACK( public ContainerBlocksDeletionACKResponseProto sendContainerBlocksDeletionACK(
ContainerBlocksDeletionACKProto request) throws IOException { ContainerBlocksDeletionACKProto request) throws IOException {

View File

@ -20,6 +20,7 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.primitives.Longs; import com.google.common.primitives.Longs;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.container.closer.ContainerCloser; import org.apache.hadoop.hdds.scm.container.closer.ContainerCloser;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
@ -33,7 +34,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos; .StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto; .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.lease.Lease; import org.apache.hadoop.ozone.lease.Lease;
import org.apache.hadoop.ozone.lease.LeaseException; import org.apache.hadoop.ozone.lease.LeaseException;
@ -368,11 +369,12 @@ public class ContainerMapping implements Mapping {
* @param reports Container report * @param reports Container report
*/ */
@Override @Override
public void processContainerReports(ContainerReportsRequestProto reports) public void processContainerReports(DatanodeDetails datanodeDetails,
ContainerReportsProto reports)
throws IOException { throws IOException {
List<StorageContainerDatanodeProtocolProtos.ContainerInfo> List<StorageContainerDatanodeProtocolProtos.ContainerInfo>
containerInfos = reports.getReportsList(); containerInfos = reports.getReportsList();
containerSupervisor.handleContainerReport(reports); containerSupervisor.handleContainerReport(datanodeDetails, reports);
for (StorageContainerDatanodeProtocolProtos.ContainerInfo datanodeState : for (StorageContainerDatanodeProtocolProtos.ContainerInfo datanodeState :
containerInfos) { containerInfos) {
byte[] dbKey = Longs.toByteArray(datanodeState.getContainerID()); byte[] dbKey = Longs.toByteArray(datanodeState.getContainerID());
@ -402,7 +404,7 @@ public class ContainerMapping implements Mapping {
// Container not found in our container db. // Container not found in our container db.
LOG.error("Error while processing container report from datanode :" + LOG.error("Error while processing container report from datanode :" +
" {}, for container: {}, reason: container doesn't exist in" + " {}, for container: {}, reason: container doesn't exist in" +
"container database.", reports.getDatanodeDetails(), "container database.", datanodeDetails,
datanodeState.getContainerID()); datanodeState.getContainerID());
} }
} finally { } finally {

View File

@ -16,10 +16,11 @@
*/ */
package org.apache.hadoop.hdds.scm.container; package org.apache.hadoop.hdds.scm.container;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto; .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
@ -98,7 +99,8 @@ public interface Mapping extends Closeable {
* *
* @param reports Container report * @param reports Container report
*/ */
void processContainerReports(ContainerReportsRequestProto reports) void processContainerReports(DatanodeDetails datanodeDetails,
ContainerReportsProto reports)
throws IOException; throws IOException;
} }

View File

@ -25,7 +25,7 @@ import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.node.NodePoolManager; import org.apache.hadoop.hdds.scm.node.NodePoolManager;
import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto; .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.concurrent.HadoopExecutors; import org.apache.hadoop.util.concurrent.HadoopExecutors;
import org.apache.hadoop.util.concurrent.HadoopThreadPoolExecutor; import org.apache.hadoop.util.concurrent.HadoopThreadPoolExecutor;
@ -295,24 +295,21 @@ public class ContainerSupervisor implements Closeable {
* @param containerReport -- Container report for a specific container from * @param containerReport -- Container report for a specific container from
* a datanode. * a datanode.
*/ */
public void handleContainerReport( public void handleContainerReport(DatanodeDetails datanodeDetails,
ContainerReportsRequestProto containerReport) { ContainerReportsProto containerReport) {
DatanodeDetails datanodeDetails = DatanodeDetails.getFromProtoBuf(
containerReport.getDatanodeDetails());
inProgressPoolListLock.readLock().lock(); inProgressPoolListLock.readLock().lock();
try { try {
String poolName = poolManager.getNodePool(datanodeDetails); String poolName = poolManager.getNodePool(datanodeDetails);
for (InProgressPool ppool : inProgressPoolList) { for (InProgressPool ppool : inProgressPoolList) {
if (ppool.getPoolName().equalsIgnoreCase(poolName)) { if (ppool.getPoolName().equalsIgnoreCase(poolName)) {
ppool.handleContainerReport(containerReport); ppool.handleContainerReport(datanodeDetails, containerReport);
return; return;
} }
} }
// TODO: Decide if we can do anything else with this report. // TODO: Decide if we can do anything else with this report.
LOG.debug("Discarding the container report for pool {}. " + LOG.debug("Discarding the container report for pool {}. " +
"That pool is not currently in the pool reconciliation process." + "That pool is not currently in the pool reconciliation process." +
" Container Name: {}", poolName, " Container Name: {}", poolName, datanodeDetails);
containerReport.getDatanodeDetails());
} catch (SCMException e) { } catch (SCMException e) {
LOG.warn("Skipping processing container report from datanode {}, " LOG.warn("Skipping processing container report from datanode {}, "
+ "cause: failed to get the corresponding node pool", + "cause: failed to get the corresponding node pool",

View File

@ -24,7 +24,7 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerInfo; .StorageContainerDatanodeProtocolProtos.ContainerInfo;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto; .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Time;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -178,21 +178,20 @@ public final class InProgressPool {
* *
* @param containerReport - ContainerReport * @param containerReport - ContainerReport
*/ */
public void handleContainerReport( public void handleContainerReport(DatanodeDetails datanodeDetails,
ContainerReportsRequestProto containerReport) { ContainerReportsProto containerReport) {
if (status == ProgressStatus.InProgress) { if (status == ProgressStatus.InProgress) {
executorService.submit(processContainerReport(containerReport)); executorService.submit(processContainerReport(datanodeDetails,
containerReport));
} else { } else {
LOG.debug("Cannot handle container report when the pool is in {} status.", LOG.debug("Cannot handle container report when the pool is in {} status.",
status); status);
} }
} }
private Runnable processContainerReport( private Runnable processContainerReport(DatanodeDetails datanodeDetails,
ContainerReportsRequestProto reports) { ContainerReportsProto reports) {
return () -> { return () -> {
DatanodeDetails datanodeDetails =
DatanodeDetails.getFromProtoBuf(reports.getDatanodeDetails());
if (processedNodeSet.computeIfAbsent(datanodeDetails.getUuid(), if (processedNodeSet.computeIfAbsent(datanodeDetails.getUuid(),
(k) -> true)) { (k) -> true)) {
nodeProcessed.incrementAndGet(); nodeProcessed.incrementAndGet();

View File

@ -21,7 +21,7 @@ package org.apache.hadoop.hdds.scm.node;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport; .StorageContainerDatanodeProtocolProtos.NodeReportProto;
import static org.apache.hadoop.util.Time.monotonicNow; import static org.apache.hadoop.util.Time.monotonicNow;
@ -31,7 +31,7 @@ import static org.apache.hadoop.util.Time.monotonicNow;
public class HeartbeatQueueItem { public class HeartbeatQueueItem {
private DatanodeDetails datanodeDetails; private DatanodeDetails datanodeDetails;
private long recvTimestamp; private long recvTimestamp;
private SCMNodeReport nodeReport; private NodeReportProto nodeReport;
/** /**
* *
@ -40,7 +40,7 @@ public class HeartbeatQueueItem {
* @param nodeReport - node report associated with the heartbeat if any. * @param nodeReport - node report associated with the heartbeat if any.
*/ */
HeartbeatQueueItem(DatanodeDetails datanodeDetails, long recvTimestamp, HeartbeatQueueItem(DatanodeDetails datanodeDetails, long recvTimestamp,
SCMNodeReport nodeReport) { NodeReportProto nodeReport) {
this.datanodeDetails = datanodeDetails; this.datanodeDetails = datanodeDetails;
this.recvTimestamp = recvTimestamp; this.recvTimestamp = recvTimestamp;
this.nodeReport = nodeReport; this.nodeReport = nodeReport;
@ -56,7 +56,7 @@ public class HeartbeatQueueItem {
/** /**
* @return node report. * @return node report.
*/ */
public SCMNodeReport getNodeReport() { public NodeReportProto getNodeReport() {
return nodeReport; return nodeReport;
} }
@ -72,7 +72,7 @@ public class HeartbeatQueueItem {
*/ */
public static class Builder { public static class Builder {
private DatanodeDetails datanodeDetails; private DatanodeDetails datanodeDetails;
private SCMNodeReport nodeReport; private NodeReportProto nodeReport;
private long recvTimestamp = monotonicNow(); private long recvTimestamp = monotonicNow();
public Builder setDatanodeDetails(DatanodeDetails dnDetails) { public Builder setDatanodeDetails(DatanodeDetails dnDetails) {
@ -80,8 +80,8 @@ public class HeartbeatQueueItem {
return this; return this;
} }
public Builder setNodeReport(SCMNodeReport scmNodeReport) { public Builder setNodeReport(NodeReportProto report) {
this.nodeReport = scmNodeReport; this.nodeReport = report;
return this; return this;
} }

View File

@ -28,15 +28,14 @@ import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException; import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DatanodeDetailsProto;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport; .StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto
.ErrorCode; .ErrorCode;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMStorageReport; .StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto; .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
import org.apache.hadoop.ipc.Server; import org.apache.hadoop.ipc.Server;
@ -592,7 +591,7 @@ public class SCMNodeManager
DatanodeDetails datanodeDetails = hbItem.getDatanodeDetails(); DatanodeDetails datanodeDetails = hbItem.getDatanodeDetails();
UUID datanodeUuid = datanodeDetails.getUuid(); UUID datanodeUuid = datanodeDetails.getUuid();
SCMNodeReport nodeReport = hbItem.getNodeReport(); NodeReportProto nodeReport = hbItem.getNodeReport();
long recvTimestamp = hbItem.getRecvTimestamp(); long recvTimestamp = hbItem.getRecvTimestamp();
long processTimestamp = Time.monotonicNow(); long processTimestamp = Time.monotonicNow();
if (LOG.isTraceEnabled()) { if (LOG.isTraceEnabled()) {
@ -637,7 +636,7 @@ public class SCMNodeManager
new ReregisterCommand()); new ReregisterCommand());
} }
private void updateNodeStat(UUID dnId, SCMNodeReport nodeReport) { private void updateNodeStat(UUID dnId, NodeReportProto nodeReport) {
SCMNodeStat stat = nodeStats.get(dnId); SCMNodeStat stat = nodeStats.get(dnId);
if (stat == null) { if (stat == null) {
LOG.debug("SCM updateNodeStat based on heartbeat from previous" + LOG.debug("SCM updateNodeStat based on heartbeat from previous" +
@ -649,8 +648,9 @@ public class SCMNodeManager
long totalCapacity = 0; long totalCapacity = 0;
long totalRemaining = 0; long totalRemaining = 0;
long totalScmUsed = 0; long totalScmUsed = 0;
List<SCMStorageReport> storageReports = nodeReport.getStorageReportList(); List<StorageReportProto> storageReports = nodeReport
for (SCMStorageReport report : storageReports) { .getStorageReportList();
for (StorageReportProto report : storageReports) {
totalCapacity += report.getCapacity(); totalCapacity += report.getCapacity();
totalRemaining += report.getRemaining(); totalRemaining += report.getRemaining();
totalScmUsed+= report.getScmUsed(); totalScmUsed+= report.getScmUsed();
@ -710,7 +710,7 @@ public class SCMNodeManager
* Register the node if the node finds that it is not registered with any * Register the node if the node finds that it is not registered with any
* SCM. * SCM.
* *
* @param datanodeDetailsProto - Send datanodeDetails with Node info. * @param datanodeDetails - Send datanodeDetails with Node info.
* This function generates and assigns new datanode ID * This function generates and assigns new datanode ID
* for the datanode. This allows SCM to be run independent * for the datanode. This allows SCM to be run independent
* of Namenode if required. * of Namenode if required.
@ -719,13 +719,11 @@ public class SCMNodeManager
* @return SCMHeartbeatResponseProto * @return SCMHeartbeatResponseProto
*/ */
@Override @Override
public SCMCommand register(DatanodeDetailsProto datanodeDetailsProto, public RegisteredCommand register(
SCMNodeReport nodeReport) { DatanodeDetails datanodeDetails, NodeReportProto nodeReport) {
String hostname = null; String hostname = null;
String ip = null; String ip = null;
DatanodeDetails datanodeDetails = DatanodeDetails.getFromProtoBuf(
datanodeDetailsProto);
InetAddress dnAddress = Server.getRemoteIp(); InetAddress dnAddress = Server.getRemoteIp();
if (dnAddress != null) { if (dnAddress != null) {
// Mostly called inside an RPC, update ip and peer hostname // Mostly called inside an RPC, update ip and peer hostname
@ -734,7 +732,7 @@ public class SCMNodeManager
datanodeDetails.setHostName(hostname); datanodeDetails.setHostName(hostname);
datanodeDetails.setIpAddress(ip); datanodeDetails.setIpAddress(ip);
} }
SCMCommand responseCommand = verifyDatanodeUUID(datanodeDetails); RegisteredCommand responseCommand = verifyDatanodeUUID(datanodeDetails);
if (responseCommand != null) { if (responseCommand != null) {
return responseCommand; return responseCommand;
} }
@ -785,7 +783,8 @@ public class SCMNodeManager
* @param datanodeDetails - Datanode Details. * @param datanodeDetails - Datanode Details.
* @return SCMCommand * @return SCMCommand
*/ */
private SCMCommand verifyDatanodeUUID(DatanodeDetails datanodeDetails) { private RegisteredCommand verifyDatanodeUUID(
DatanodeDetails datanodeDetails) {
if (datanodeDetails.getUuid() != null && if (datanodeDetails.getUuid() != null &&
nodes.containsKey(datanodeDetails.getUuid())) { nodes.containsKey(datanodeDetails.getUuid())) {
LOG.trace("Datanode is already registered. Datanode: {}", LOG.trace("Datanode is already registered. Datanode: {}",
@ -802,34 +801,23 @@ public class SCMNodeManager
/** /**
* Send heartbeat to indicate the datanode is alive and doing well. * Send heartbeat to indicate the datanode is alive and doing well.
* *
* @param datanodeDetailsProto - DatanodeDetailsProto. * @param datanodeDetails - DatanodeDetailsProto.
* @param nodeReport - node report. * @param nodeReport - node report.
* @return SCMheartbeat response. * @return SCMheartbeat response.
* @throws IOException * @throws IOException
*/ */
@Override @Override
public List<SCMCommand> sendHeartbeat( public List<SCMCommand> sendHeartbeat(
DatanodeDetailsProto datanodeDetailsProto, SCMNodeReport nodeReport) { DatanodeDetails datanodeDetails, NodeReportProto nodeReport) {
Preconditions.checkNotNull(datanodeDetailsProto, "Heartbeat is missing " + Preconditions.checkNotNull(datanodeDetails, "Heartbeat is missing " +
"DatanodeDetails."); "DatanodeDetails.");
DatanodeDetails datanodeDetails = DatanodeDetails heartbeatQueue.add(
.getFromProtoBuf(datanodeDetailsProto); new HeartbeatQueueItem.Builder()
// Checking for NULL to make sure that we don't get .setDatanodeDetails(datanodeDetails)
// an exception from ConcurrentList. .setNodeReport(nodeReport)
// This could be a problem in tests, if this function is invoked via .build());
// protobuf, transport layer will guarantee that this is not null. return commandQueue.getCommand(datanodeDetails.getUuid());
if (datanodeDetails != null) {
heartbeatQueue.add(
new HeartbeatQueueItem.Builder()
.setDatanodeDetails(datanodeDetails)
.setNodeReport(nodeReport)
.build());
return commandQueue.getCommand(datanodeDetails.getUuid());
} else {
LOG.error("Datanode ID in heartbeat is null");
}
return null;
} }
/** /**

View File

@ -23,7 +23,7 @@ import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto. import org.apache.hadoop.hdds.protocol.proto.
StorageContainerDatanodeProtocolProtos.SCMStorageReport; StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.hdds.scm.exceptions.SCMException; import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.metrics2.util.MBeans; import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConfigKeys;
@ -33,7 +33,11 @@ import org.slf4j.LoggerFactory;
import javax.management.ObjectName; import javax.management.ObjectName;
import java.io.IOException; import java.io.IOException;
import java.util.*; import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors; import java.util.stream.Collectors;
@ -159,7 +163,7 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
} }
public StorageReportResult processNodeReport(UUID datanodeID, public StorageReportResult processNodeReport(UUID datanodeID,
StorageContainerDatanodeProtocolProtos.SCMNodeReport nodeReport) StorageContainerDatanodeProtocolProtos.NodeReportProto nodeReport)
throws IOException { throws IOException {
Preconditions.checkNotNull(datanodeID); Preconditions.checkNotNull(datanodeID);
Preconditions.checkNotNull(nodeReport); Preconditions.checkNotNull(nodeReport);
@ -170,9 +174,9 @@ public class SCMNodeStorageStatMap implements SCMNodeStorageStatMXBean {
Set<StorageLocationReport> storagReportSet = new HashSet<>(); Set<StorageLocationReport> storagReportSet = new HashSet<>();
Set<StorageLocationReport> fullVolumeSet = new HashSet<>(); Set<StorageLocationReport> fullVolumeSet = new HashSet<>();
Set<StorageLocationReport> failedVolumeSet = new HashSet<>(); Set<StorageLocationReport> failedVolumeSet = new HashSet<>();
List<SCMStorageReport> List<StorageReportProto>
storageReports = nodeReport.getStorageReportList(); storageReports = nodeReport.getStorageReportList();
for (SCMStorageReport report : storageReports) { for (StorageReportProto report : storageReports) {
StorageLocationReport storageReport = StorageLocationReport storageReport =
StorageLocationReport.getFromProtobuf(report); StorageLocationReport.getFromProtobuf(report);
storagReportSet.add(storageReport); storagReportSet.add(storageReport);

View File

@ -25,29 +25,47 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.protobuf.BlockingService; import com.google.protobuf.BlockingService;
import org.apache.hadoop.hdds.conf.OzoneConfiguration; 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;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMReregisterCmdResponseProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandResponseProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport; .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto; import org.apache.hadoop.hdds.protocol.proto
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto; .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsResponseProto; import org.apache.hadoop.hdds.protocol.proto
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKResponseProto; .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto.DeleteBlockTransactionResult; import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ReregisterCommandProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerBlocksDeletionACKProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos
.ContainerBlocksDeletionACKResponseProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos
.ContainerBlocksDeletionACKProto.DeleteBlockTransactionResult;
import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType.versionCommand; import static org.apache.hadoop.hdds.protocol.proto
import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType.registeredCommand; .StorageContainerDatanodeProtocolProtos.SCMCommandProto
import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType.reregisterCommand; .Type.closeContainerCommand;
import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType.deleteBlocksCommand; import static org.apache.hadoop.hdds.protocol.proto
import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType.closeContainerCommand; .StorageContainerDatanodeProtocolProtos.SCMCommandProto
.Type.deleteBlocksCommand;
import static org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCommandProto
.Type.reregisterCommand;
import org.apache.hadoop.hdds.scm.HddsServerUtil; import org.apache.hadoop.hdds.scm.HddsServerUtil;
@ -150,96 +168,81 @@ public class SCMDatanodeProtocolServer implements
@Override @Override
public SCMHeartbeatResponseProto sendHeartbeat( public SCMHeartbeatResponseProto sendHeartbeat(
HddsProtos.DatanodeDetailsProto datanodeDetails, SCMHeartbeatRequestProto heartbeat)
StorageContainerDatanodeProtocolProtos.SCMNodeReport nodeReport)
throws IOException { throws IOException {
// TODO: Add a heartbeat dispatcher.
DatanodeDetails datanodeDetails = DatanodeDetails
.getFromProtoBuf(heartbeat.getDatanodeDetails());
NodeReportProto nodeReport = heartbeat.getNodeReport();
List<SCMCommand> commands = List<SCMCommand> commands =
scm.getScmNodeManager().sendHeartbeat(datanodeDetails, nodeReport); scm.getScmNodeManager().sendHeartbeat(datanodeDetails, nodeReport);
List<SCMCommandResponseProto> cmdResponses = new LinkedList<>(); List<SCMCommandProto> cmdResponses = new LinkedList<>();
for (SCMCommand cmd : commands) { for (SCMCommand cmd : commands) {
cmdResponses.add(getCommandResponse(cmd, datanodeDetails.getUuid())); cmdResponses.add(getCommandResponse(cmd));
} }
return SCMHeartbeatResponseProto.newBuilder() return SCMHeartbeatResponseProto.newBuilder()
.setDatanodeUUID(datanodeDetails.getUuidString())
.addAllCommands(cmdResponses).build(); .addAllCommands(cmdResponses).build();
} }
@Override @Override
public SCMRegisteredCmdResponseProto register( public SCMRegisteredResponseProto register(
HddsProtos.DatanodeDetailsProto datanodeDetails, SCMNodeReport nodeReport, HddsProtos.DatanodeDetailsProto datanodeDetailsProto,
ContainerReportsRequestProto containerReportsRequestProto) NodeReportProto nodeReport,
ContainerReportsProto containerReportsProto)
throws IOException { throws IOException {
DatanodeDetails datanodeDetails = DatanodeDetails
.getFromProtoBuf(datanodeDetailsProto);
// TODO : Return the list of Nodes that forms the SCM HA. // TODO : Return the list of Nodes that forms the SCM HA.
RegisteredCommand registeredCommand = (RegisteredCommand) scm RegisteredCommand registeredCommand = scm.getScmNodeManager()
.getScmNodeManager().register(datanodeDetails, nodeReport); .register(datanodeDetails, nodeReport);
SCMCmdType type = registeredCommand.getType(); if (registeredCommand.getError()
if (type == SCMCmdType.registeredCommand && registeredCommand.getError() == SCMRegisteredResponseProto.ErrorCode.success) {
== SCMRegisteredCmdResponseProto.ErrorCode.success) { scm.getScmContainerManager().processContainerReports(datanodeDetails,
scm.getScmContainerManager().processContainerReports( containerReportsProto);
containerReportsRequestProto);
} }
return getRegisteredResponse(registeredCommand); return getRegisteredResponse(registeredCommand);
} }
@VisibleForTesting @VisibleForTesting
public static SCMRegisteredCmdResponseProto getRegisteredResponse( public static SCMRegisteredResponseProto getRegisteredResponse(
SCMCommand cmd) { RegisteredCommand cmd) {
Preconditions.checkState(cmd.getClass() == RegisteredCommand.class); return SCMRegisteredResponseProto.newBuilder()
RegisteredCommand rCmd = (RegisteredCommand) cmd;
SCMCmdType type = cmd.getType();
if (type != SCMCmdType.registeredCommand) {
throw new IllegalArgumentException(
"Registered command is not well " + "formed. Internal Error.");
}
return SCMRegisteredCmdResponseProto.newBuilder()
// TODO : Fix this later when we have multiple SCM support. // TODO : Fix this later when we have multiple SCM support.
// .setAddressList(addressList) // .setAddressList(addressList)
.setErrorCode(rCmd.getError()) .setErrorCode(cmd.getError())
.setClusterID(rCmd.getClusterID()) .setClusterID(cmd.getClusterID())
.setDatanodeUUID(rCmd.getDatanodeUUID()) .setDatanodeUUID(cmd.getDatanodeUUID())
.build(); .build();
} }
@Override public void processContainerReports(DatanodeDetails datanodeDetails,
public ContainerReportsResponseProto sendContainerReport( ContainerReportsProto reports)
ContainerReportsRequestProto reports)
throws IOException { throws IOException {
updateContainerReportMetrics(reports); updateContainerReportMetrics(datanodeDetails, reports);
// should we process container reports async? // should we process container reports async?
scm.getScmContainerManager().processContainerReports(reports); scm.getScmContainerManager()
return ContainerReportsResponseProto.newBuilder().build(); .processContainerReports(datanodeDetails, reports);
} }
private void updateContainerReportMetrics( private void updateContainerReportMetrics(DatanodeDetails datanodeDetails,
ContainerReportsRequestProto reports) { ContainerReportsProto reports) {
ContainerStat newStat = null; ContainerStat newStat = new ContainerStat();
// TODO: We should update the logic once incremental container report for (StorageContainerDatanodeProtocolProtos.ContainerInfo info : reports
// type is supported. .getReportsList()) {
if (reports newStat.add(new ContainerStat(info.getSize(), info.getUsed(),
.getType() == StorageContainerDatanodeProtocolProtos info.getKeyCount(), info.getReadBytes(), info.getWriteBytes(),
.ContainerReportsRequestProto.reportType.fullReport) { info.getReadCount(), info.getWriteCount()));
newStat = new ContainerStat();
for (StorageContainerDatanodeProtocolProtos.ContainerInfo info : reports
.getReportsList()) {
newStat.add(new ContainerStat(info.getSize(), info.getUsed(),
info.getKeyCount(), info.getReadBytes(), info.getWriteBytes(),
info.getReadCount(), info.getWriteCount()));
}
// update container metrics
StorageContainerManager.getMetrics().setLastContainerStat(newStat);
} }
// update container metrics
StorageContainerManager.getMetrics().setLastContainerStat(newStat);
// Update container stat entry, this will trigger a removal operation if it // Update container stat entry, this will trigger a removal operation if it
// exists in cache. // exists in cache.
synchronized (scm.getContainerReportCache()) { String datanodeUuid = datanodeDetails.getUuidString();
String datanodeUuid = reports.getDatanodeDetails().getUuid(); scm.getContainerReportCache().put(datanodeUuid, newStat);
if (datanodeUuid != null && newStat != null) { // update global view container metrics
scm.getContainerReportCache().put(datanodeUuid, newStat); StorageContainerManager.getMetrics().incrContainerStat(newStat);
// update global view container metrics
StorageContainerManager.getMetrics().incrContainerStat(newStat);
}
}
} }
@ -298,28 +301,15 @@ public class SCMDatanodeProtocolServer implements
* @throws IOException * @throws IOException
*/ */
@VisibleForTesting @VisibleForTesting
public StorageContainerDatanodeProtocolProtos.SCMCommandResponseProto public SCMCommandProto getCommandResponse(SCMCommand cmd)
getCommandResponse( throws IOException {
SCMCommand cmd, final String datanodeID) throws IOException { SCMCommandProto.Builder builder =
SCMCmdType type = cmd.getType(); SCMCommandProto.newBuilder();
SCMCommandResponseProto.Builder builder = switch (cmd.getType()) {
SCMCommandResponseProto.newBuilder().setDatanodeUUID(datanodeID);
switch (type) {
case registeredCommand:
return builder
.setCmdType(registeredCommand)
.setRegisteredProto(SCMRegisteredCmdResponseProto
.getDefaultInstance())
.build();
case versionCommand:
return builder
.setCmdType(versionCommand)
.setVersionProto(SCMVersionResponseProto.getDefaultInstance())
.build();
case reregisterCommand: case reregisterCommand:
return builder return builder
.setCmdType(reregisterCommand) .setCommandType(reregisterCommand)
.setReregisterProto(SCMReregisterCmdResponseProto .setReregisterCommandProto(ReregisterCommandProto
.getDefaultInstance()) .getDefaultInstance())
.build(); .build();
case deleteBlocksCommand: case deleteBlocksCommand:
@ -335,13 +325,14 @@ public class SCMDatanodeProtocolServer implements
.collect(Collectors.toList()); .collect(Collectors.toList());
scm.getScmBlockManager().getDeletedBlockLog().incrementCount(txs); scm.getScmBlockManager().getDeletedBlockLog().incrementCount(txs);
return builder return builder
.setCmdType(deleteBlocksCommand) .setCommandType(deleteBlocksCommand)
.setDeleteBlocksProto(((DeleteBlocksCommand) cmd).getProto()) .setDeleteBlocksCommandProto(((DeleteBlocksCommand) cmd).getProto())
.build(); .build();
case closeContainerCommand: case closeContainerCommand:
return builder return builder
.setCmdType(closeContainerCommand) .setCommandType(closeContainerCommand)
.setCloseContainerProto(((CloseContainerCommand) cmd).getProto()) .setCloseContainerCommandProto(
((CloseContainerCommand) cmd).getProto())
.build(); .build();
default: default:
throw new IllegalArgumentException("Not implemented"); throw new IllegalArgumentException("Not implemented");

View File

@ -18,9 +18,9 @@ package org.apache.hadoop.hdds.scm;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol import org.apache.hadoop.hdds.protocol
.proto.StorageContainerDatanodeProtocolProtos.SCMNodeReport; .proto.StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMStorageReport; .StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.StorageTypeProto; .StorageContainerDatanodeProtocolProtos.StorageTypeProto;
import org.apache.hadoop.hdds.scm.node.SCMNodeManager; import org.apache.hadoop.hdds.scm.node.SCMNodeManager;
@ -53,16 +53,17 @@ public final class TestUtils {
public static DatanodeDetails getDatanodeDetails(SCMNodeManager nodeManager, public static DatanodeDetails getDatanodeDetails(SCMNodeManager nodeManager,
String uuid) { String uuid) {
DatanodeDetails datanodeDetails = getDatanodeDetails(uuid); DatanodeDetails datanodeDetails = getDatanodeDetails(uuid);
nodeManager.register(datanodeDetails.getProtoBufMessage(), null); nodeManager.register(datanodeDetails, null);
return datanodeDetails; return datanodeDetails;
} }
/** /**
* Create Node Report object. * Create Node Report object.
* @return SCMNodeReport * @return NodeReportProto
*/ */
public static SCMNodeReport createNodeReport(List<SCMStorageReport> reports) { public static NodeReportProto createNodeReport(
SCMNodeReport.Builder nodeReport = SCMNodeReport.newBuilder(); List<StorageReportProto> reports) {
NodeReportProto.Builder nodeReport = NodeReportProto.newBuilder();
nodeReport.addAllStorageReport(reports); nodeReport.addAllStorageReport(reports);
return nodeReport.build(); return nodeReport.build();
} }
@ -71,14 +72,14 @@ public final class TestUtils {
* Create SCM Storage Report object. * Create SCM Storage Report object.
* @return list of SCMStorageReport * @return list of SCMStorageReport
*/ */
public static List<SCMStorageReport> createStorageReport(long capacity, public static List<StorageReportProto> createStorageReport(long capacity,
long used, long remaining, String path, StorageTypeProto type, String id, long used, long remaining, String path, StorageTypeProto type, String id,
int count) { int count) {
List<SCMStorageReport> reportList = new ArrayList<>(); List<StorageReportProto> reportList = new ArrayList<>();
for (int i = 0; i < count; i++) { for (int i = 0; i < count; i++) {
Preconditions.checkNotNull(path); Preconditions.checkNotNull(path);
Preconditions.checkNotNull(id); Preconditions.checkNotNull(id);
SCMStorageReport.Builder srb = SCMStorageReport.newBuilder(); StorageReportProto.Builder srb = StorageReportProto.newBuilder();
srb.setStorageUuid(id).setStorageLocation(path).setCapacity(capacity) srb.setStorageUuid(id).setStorageLocation(path).setCapacity(capacity)
.setScmUsed(used).setRemaining(remaining); .setScmUsed(used).setRemaining(remaining);
StorageTypeProto storageTypeProto = StorageTypeProto storageTypeProto =

View File

@ -24,13 +24,14 @@ import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport; .StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMStorageReport; .StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto; .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.protocol.VersionResponse; import org.apache.hadoop.ozone.protocol.VersionResponse;
import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand; import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.assertj.core.util.Preconditions; import org.assertj.core.util.Preconditions;
import org.mockito.Mockito; import org.mockito.Mockito;
@ -370,13 +371,13 @@ public class MockNodeManager implements NodeManager {
* Register the node if the node finds that it is not registered with any * Register the node if the node finds that it is not registered with any
* SCM. * SCM.
* *
* @param datanodeDetails DatanodeDetailsProto * @param datanodeDetails DatanodeDetails
* @param nodeReport SCMNodeReport * @param nodeReport NodeReportProto
* @return SCMHeartbeatResponseProto * @return SCMHeartbeatResponseProto
*/ */
@Override @Override
public SCMCommand register(HddsProtos.DatanodeDetailsProto datanodeDetails, public RegisteredCommand register(DatanodeDetails datanodeDetails,
SCMNodeReport nodeReport) { NodeReportProto nodeReport) {
return null; return null;
} }
@ -388,9 +389,8 @@ public class MockNodeManager implements NodeManager {
* @return SCMheartbeat response list * @return SCMheartbeat response list
*/ */
@Override @Override
public List<SCMCommand> sendHeartbeat( public List<SCMCommand> sendHeartbeat(DatanodeDetails datanodeDetails,
HddsProtos.DatanodeDetailsProto datanodeDetails, NodeReportProto nodeReport) {
SCMNodeReport nodeReport) {
if ((datanodeDetails != null) && (nodeReport != null) && (nodeReport if ((datanodeDetails != null) && (nodeReport != null) && (nodeReport
.getStorageReportCount() > 0)) { .getStorageReportCount() > 0)) {
SCMNodeStat stat = this.nodeMetricMap.get(datanodeDetails.getUuid()); SCMNodeStat stat = this.nodeMetricMap.get(datanodeDetails.getUuid());
@ -398,8 +398,9 @@ public class MockNodeManager implements NodeManager {
long totalCapacity = 0L; long totalCapacity = 0L;
long totalRemaining = 0L; long totalRemaining = 0L;
long totalScmUsed = 0L; long totalScmUsed = 0L;
List<SCMStorageReport> storageReports = nodeReport.getStorageReportList(); List<StorageReportProto> storageReports = nodeReport
for (SCMStorageReport report : storageReports) { .getStorageReportList();
for (StorageReportProto report : storageReports) {
totalCapacity += report.getCapacity(); totalCapacity += report.getCapacity();
totalRemaining += report.getRemaining(); totalRemaining += report.getRemaining();
totalScmUsed += report.getScmUsed(); totalScmUsed += report.getScmUsed();
@ -407,8 +408,7 @@ public class MockNodeManager implements NodeManager {
aggregateStat.subtract(stat); aggregateStat.subtract(stat);
stat.set(totalCapacity, totalScmUsed, totalRemaining); stat.set(totalCapacity, totalScmUsed, totalRemaining);
aggregateStat.add(stat); aggregateStat.add(stat);
nodeMetricMap.put(DatanodeDetails nodeMetricMap.put(datanodeDetails.getUuid(), stat);
.getFromProtoBuf(datanodeDetails).getUuid(), stat);
} }
return null; return null;

View File

@ -28,7 +28,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos; .StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto; .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.SCMTestUtils; import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
@ -191,8 +191,6 @@ public class TestContainerMapping {
public void testFullContainerReport() throws IOException { public void testFullContainerReport() throws IOException {
ContainerInfo info = createContainer(); ContainerInfo info = createContainer();
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(); DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
ContainerReportsRequestProto.reportType reportType =
ContainerReportsRequestProto.reportType.fullReport;
List<StorageContainerDatanodeProtocolProtos.ContainerInfo> reports = List<StorageContainerDatanodeProtocolProtos.ContainerInfo> reports =
new ArrayList<>(); new ArrayList<>();
StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder = StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder =
@ -209,12 +207,11 @@ public class TestContainerMapping {
reports.add(ciBuilder.build()); reports.add(ciBuilder.build());
ContainerReportsRequestProto.Builder crBuilder = ContainerReportsProto.Builder crBuilder = ContainerReportsProto
ContainerReportsRequestProto.newBuilder(); .newBuilder();
crBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage()) crBuilder.addAllReports(reports);
.setType(reportType).addAllReports(reports);
mapping.processContainerReports(crBuilder.build()); mapping.processContainerReports(datanodeDetails, crBuilder.build());
ContainerInfo updatedContainer = ContainerInfo updatedContainer =
mapping.getContainer(info.getContainerID()); mapping.getContainer(info.getContainerID());
@ -227,8 +224,6 @@ public class TestContainerMapping {
public void testContainerCloseWithContainerReport() throws IOException { public void testContainerCloseWithContainerReport() throws IOException {
ContainerInfo info = createContainer(); ContainerInfo info = createContainer();
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(); DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
ContainerReportsRequestProto.reportType reportType =
ContainerReportsRequestProto.reportType.fullReport;
List<StorageContainerDatanodeProtocolProtos.ContainerInfo> reports = List<StorageContainerDatanodeProtocolProtos.ContainerInfo> reports =
new ArrayList<>(); new ArrayList<>();
@ -246,12 +241,11 @@ public class TestContainerMapping {
reports.add(ciBuilder.build()); reports.add(ciBuilder.build());
ContainerReportsRequestProto.Builder crBuilder = ContainerReportsProto.Builder crBuilder =
ContainerReportsRequestProto.newBuilder(); ContainerReportsProto.newBuilder();
crBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage()) crBuilder.addAllReports(reports);
.setType(reportType).addAllReports(reports);
mapping.processContainerReports(crBuilder.build()); mapping.processContainerReports(datanodeDetails, crBuilder.build());
ContainerInfo updatedContainer = ContainerInfo updatedContainer =
mapping.getContainer(info.getContainerID()); mapping.getContainer(info.getContainerID());

View File

@ -30,7 +30,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos; .StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto; .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.SCMTestUtils; import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
@ -199,9 +199,8 @@ public class TestContainerCloser {
private void sendContainerReport(ContainerInfo info, long used) throws private void sendContainerReport(ContainerInfo info, long used) throws
IOException { IOException {
ContainerReportsRequestProto.Builder ContainerReportsProto.Builder
reports = ContainerReportsRequestProto.newBuilder(); reports = ContainerReportsProto.newBuilder();
reports.setType(ContainerReportsRequestProto.reportType.fullReport);
StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder = StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder =
StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder(); StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder();
@ -214,9 +213,8 @@ public class TestContainerCloser {
.setWriteCount(100000000L) .setWriteCount(100000000L)
.setReadBytes(2000000000L) .setReadBytes(2000000000L)
.setWriteBytes(2000000000L); .setWriteBytes(2000000000L);
reports.setDatanodeDetails(
TestUtils.getDatanodeDetails().getProtoBufMessage());
reports.addReports(ciBuilder); reports.addReports(ciBuilder);
mapping.processContainerReports(reports.build()); mapping.processContainerReports(TestUtils.getDatanodeDetails(),
reports.build());
} }
} }

View File

@ -33,7 +33,7 @@ import org.apache.hadoop.hdds.scm.container.placement.algorithms
import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMStorageReport; .StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
@ -133,9 +133,9 @@ public class TestContainerPlacement {
for (DatanodeDetails datanodeDetails : datanodes) { for (DatanodeDetails datanodeDetails : datanodes) {
String id = UUID.randomUUID().toString(); String id = UUID.randomUUID().toString();
String path = testDir.getAbsolutePath() + "/" + id; String path = testDir.getAbsolutePath() + "/" + id;
List<SCMStorageReport> reports = TestUtils List<StorageReportProto> reports = TestUtils
.createStorageReport(capacity, used, remaining, path, null, id, 1); .createStorageReport(capacity, used, remaining, path, null, id, 1);
nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(), nodeManager.sendHeartbeat(datanodeDetails,
TestUtils.createNodeReport(reports)); TestUtils.createNodeReport(reports));
} }

View File

@ -19,6 +19,8 @@ package org.apache.hadoop.hdds.scm.node;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.TestUtils; import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat; import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
@ -26,7 +28,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMStorageReport; .StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand; import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
@ -63,8 +65,6 @@ import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DEAD;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState
.HEALTHY; .HEALTHY;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE;
import static org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCmdType;
import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.core.StringStartsWith.startsWith; import static org.hamcrest.core.StringStartsWith.startsWith;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
@ -144,7 +144,7 @@ public class TestNodeManager {
for (int x = 0; x < nodeManager.getMinimumChillModeNodes(); x++) { for (int x = 0; x < nodeManager.getMinimumChillModeNodes(); x++) {
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails( DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(
nodeManager); nodeManager);
nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(), nodeManager.sendHeartbeat(datanodeDetails,
null); null);
} }
@ -191,8 +191,8 @@ public class TestNodeManager {
// Need 100 nodes to come out of chill mode, only one node is sending HB. // Need 100 nodes to come out of chill mode, only one node is sending HB.
nodeManager.setMinimumChillModeNodes(100); nodeManager.setMinimumChillModeNodes(100);
nodeManager.sendHeartbeat(TestUtils.getDatanodeDetails(nodeManager) nodeManager.sendHeartbeat(TestUtils.getDatanodeDetails(nodeManager),
.getProtoBufMessage(), null); null);
GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(), GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(),
100, 4 * 1000); 100, 4 * 1000);
assertFalse("Not enough heartbeat, Node manager should have" + assertFalse("Not enough heartbeat, Node manager should have" +
@ -219,7 +219,7 @@ public class TestNodeManager {
// Send 10 heartbeat from same node, and assert we never leave chill mode. // Send 10 heartbeat from same node, and assert we never leave chill mode.
for (int x = 0; x < 10; x++) { for (int x = 0; x < 10; x++) {
nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(), nodeManager.sendHeartbeat(datanodeDetails,
null); null);
} }
@ -250,7 +250,7 @@ public class TestNodeManager {
nodeManager.close(); nodeManager.close();
// These should never be processed. // These should never be processed.
nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(), nodeManager.sendHeartbeat(datanodeDetails,
null); null);
// Let us just wait for 2 seconds to prove that HBs are not processed. // Let us just wait for 2 seconds to prove that HBs are not processed.
@ -274,13 +274,13 @@ public class TestNodeManager {
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(); DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
String dnId = datanodeDetails.getUuidString(); String dnId = datanodeDetails.getUuidString();
String storagePath = testDir.getAbsolutePath() + "/" + dnId; String storagePath = testDir.getAbsolutePath() + "/" + dnId;
List<SCMStorageReport> reports = List<StorageReportProto> reports =
TestUtils.createStorageReport(100, 10, 90, storagePath, null, dnId, 1); TestUtils.createStorageReport(100, 10, 90, storagePath, null, dnId, 1);
try (SCMNodeManager nodemanager = createNodeManager(conf)) { try (SCMNodeManager nodemanager = createNodeManager(conf)) {
nodemanager.register(datanodeDetails.getProtoBufMessage(), nodemanager.register(datanodeDetails,
TestUtils.createNodeReport(reports)); TestUtils.createNodeReport(reports));
List<SCMCommand> command = nodemanager.sendHeartbeat( List<SCMCommand> command = nodemanager.sendHeartbeat(
datanodeDetails.getProtoBufMessage(), null); datanodeDetails, null);
Assert.assertTrue(nodemanager.getAllNodes().contains(datanodeDetails)); Assert.assertTrue(nodemanager.getAllNodes().contains(datanodeDetails));
Assert.assertTrue("On regular HB calls, SCM responses a " Assert.assertTrue("On regular HB calls, SCM responses a "
+ "datanode with an empty command list", command.isEmpty()); + "datanode with an empty command list", command.isEmpty());
@ -298,10 +298,10 @@ public class TestNodeManager {
GenericTestUtils.waitFor(new Supplier<Boolean>() { GenericTestUtils.waitFor(new Supplier<Boolean>() {
@Override public Boolean get() { @Override public Boolean get() {
List<SCMCommand> command = List<SCMCommand> command =
nodemanager.sendHeartbeat(datanodeDetails.getProtoBufMessage(), nodemanager.sendHeartbeat(datanodeDetails,
null); null);
return command.size() == 1 && command.get(0).getType() return command.size() == 1 && command.get(0).getType()
.equals(SCMCmdType.reregisterCommand); .equals(SCMCommandProto.Type.reregisterCommand);
} }
}, 100, 3 * 1000); }, 100, 3 * 1000);
} catch (TimeoutException e) { } catch (TimeoutException e) {
@ -330,7 +330,7 @@ public class TestNodeManager {
for (int x = 0; x < count; x++) { for (int x = 0; x < count; x++) {
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails( DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(
nodeManager); nodeManager);
nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(), nodeManager.sendHeartbeat(datanodeDetails,
null); null);
} }
GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(), GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(),
@ -422,19 +422,19 @@ public class TestNodeManager {
DatanodeDetails staleNode = TestUtils.getDatanodeDetails(nodeManager); DatanodeDetails staleNode = TestUtils.getDatanodeDetails(nodeManager);
// Heartbeat once // Heartbeat once
nodeManager.sendHeartbeat(staleNode.getProtoBufMessage(), nodeManager.sendHeartbeat(staleNode,
null); null);
// Heartbeat all other nodes. // Heartbeat all other nodes.
for (DatanodeDetails dn : nodeList) { for (DatanodeDetails dn : nodeList) {
nodeManager.sendHeartbeat(dn.getProtoBufMessage(), null); nodeManager.sendHeartbeat(dn, null);
} }
// Wait for 2 seconds .. and heartbeat good nodes again. // Wait for 2 seconds .. and heartbeat good nodes again.
Thread.sleep(2 * 1000); Thread.sleep(2 * 1000);
for (DatanodeDetails dn : nodeList) { for (DatanodeDetails dn : nodeList) {
nodeManager.sendHeartbeat(dn.getProtoBufMessage(), null); nodeManager.sendHeartbeat(dn, null);
} }
// Wait for 2 seconds, wait a total of 4 seconds to make sure that the // Wait for 2 seconds, wait a total of 4 seconds to make sure that the
@ -451,7 +451,7 @@ public class TestNodeManager {
// heartbeat good nodes again. // heartbeat good nodes again.
for (DatanodeDetails dn : nodeList) { for (DatanodeDetails dn : nodeList) {
nodeManager.sendHeartbeat(dn.getProtoBufMessage(), null); nodeManager.sendHeartbeat(dn, null);
} }
// 6 seconds is the dead window for this test , so we wait a total of // 6 seconds is the dead window for this test , so we wait a total of
@ -565,11 +565,11 @@ public class TestNodeManager {
DatanodeDetails deadNode = DatanodeDetails deadNode =
TestUtils.getDatanodeDetails(nodeManager); TestUtils.getDatanodeDetails(nodeManager);
nodeManager.sendHeartbeat( nodeManager.sendHeartbeat(
healthyNode.getProtoBufMessage(), null); healthyNode, null);
nodeManager.sendHeartbeat( nodeManager.sendHeartbeat(
staleNode.getProtoBufMessage(), null); staleNode, null);
nodeManager.sendHeartbeat( nodeManager.sendHeartbeat(
deadNode.getProtoBufMessage(), null); deadNode, null);
// Sleep so that heartbeat processing thread gets to run. // Sleep so that heartbeat processing thread gets to run.
Thread.sleep(500); Thread.sleep(500);
@ -596,15 +596,15 @@ public class TestNodeManager {
*/ */
nodeManager.sendHeartbeat( nodeManager.sendHeartbeat(
healthyNode.getProtoBufMessage(), null); healthyNode, null);
nodeManager.sendHeartbeat( nodeManager.sendHeartbeat(
staleNode.getProtoBufMessage(), null); staleNode, null);
nodeManager.sendHeartbeat( nodeManager.sendHeartbeat(
deadNode.getProtoBufMessage(), null); deadNode, null);
Thread.sleep(1500); Thread.sleep(1500);
nodeManager.sendHeartbeat( nodeManager.sendHeartbeat(
healthyNode.getProtoBufMessage(), null); healthyNode, null);
Thread.sleep(2 * 1000); Thread.sleep(2 * 1000);
assertEquals(1, nodeManager.getNodeCount(HEALTHY)); assertEquals(1, nodeManager.getNodeCount(HEALTHY));
@ -625,12 +625,12 @@ public class TestNodeManager {
*/ */
nodeManager.sendHeartbeat( nodeManager.sendHeartbeat(
healthyNode.getProtoBufMessage(), null); healthyNode, null);
nodeManager.sendHeartbeat( nodeManager.sendHeartbeat(
staleNode.getProtoBufMessage(), null); staleNode, null);
Thread.sleep(1500); Thread.sleep(1500);
nodeManager.sendHeartbeat( nodeManager.sendHeartbeat(
healthyNode.getProtoBufMessage(), null); healthyNode, null);
Thread.sleep(2 * 1000); Thread.sleep(2 * 1000);
// 3.5 seconds have elapsed for stale node, so it moves into Stale. // 3.5 seconds have elapsed for stale node, so it moves into Stale.
@ -664,11 +664,11 @@ public class TestNodeManager {
* back all the nodes in healthy state. * back all the nodes in healthy state.
*/ */
nodeManager.sendHeartbeat( nodeManager.sendHeartbeat(
healthyNode.getProtoBufMessage(), null); healthyNode, null);
nodeManager.sendHeartbeat( nodeManager.sendHeartbeat(
staleNode.getProtoBufMessage(), null); staleNode, null);
nodeManager.sendHeartbeat( nodeManager.sendHeartbeat(
deadNode.getProtoBufMessage(), null); deadNode, null);
Thread.sleep(500); Thread.sleep(500);
//Assert all nodes are healthy. //Assert all nodes are healthy.
assertEquals(3, nodeManager.getAllNodes().size()); assertEquals(3, nodeManager.getAllNodes().size());
@ -689,7 +689,7 @@ public class TestNodeManager {
int sleepDuration) throws InterruptedException { int sleepDuration) throws InterruptedException {
while (!Thread.currentThread().isInterrupted()) { while (!Thread.currentThread().isInterrupted()) {
for (DatanodeDetails dn : list) { for (DatanodeDetails dn : list) {
manager.sendHeartbeat(dn.getProtoBufMessage(), null); manager.sendHeartbeat(dn, null);
} }
Thread.sleep(sleepDuration); Thread.sleep(sleepDuration);
} }
@ -775,7 +775,7 @@ public class TestNodeManager {
// No Thread just one time HBs the node manager, so that these will be // No Thread just one time HBs the node manager, so that these will be
// marked as dead nodes eventually. // marked as dead nodes eventually.
for (DatanodeDetails dn : deadNodeList) { for (DatanodeDetails dn : deadNodeList) {
nodeManager.sendHeartbeat(dn.getProtoBufMessage(), null); nodeManager.sendHeartbeat(dn, null);
} }
@ -940,7 +940,7 @@ public class TestNodeManager {
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails( DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails(
nodeManager); nodeManager);
nodeManager.sendHeartbeat( nodeManager.sendHeartbeat(
datanodeDetails.getProtoBufMessage(), null); datanodeDetails, null);
String status = nodeManager.getChillModeStatus(); String status = nodeManager.getChillModeStatus();
Assert.assertThat(status, containsString("Still in chill " + Assert.assertThat(status, containsString("Still in chill " +
"mode, waiting on nodes to report in.")); "mode, waiting on nodes to report in."));
@ -967,8 +967,7 @@ public class TestNodeManager {
// Assert that node manager force enter cannot be overridden by nodes HBs. // Assert that node manager force enter cannot be overridden by nodes HBs.
for (int x = 0; x < 20; x++) { for (int x = 0; x < 20; x++) {
DatanodeDetails datanode = TestUtils.getDatanodeDetails(nodeManager); DatanodeDetails datanode = TestUtils.getDatanodeDetails(nodeManager);
nodeManager.sendHeartbeat(datanode.getProtoBufMessage(), nodeManager.sendHeartbeat(datanode, null);
null);
} }
Thread.sleep(500); Thread.sleep(500);
@ -1009,10 +1008,10 @@ public class TestNodeManager {
String dnId = datanodeDetails.getUuidString(); String dnId = datanodeDetails.getUuidString();
long free = capacity - used; long free = capacity - used;
String storagePath = testDir.getAbsolutePath() + "/" + dnId; String storagePath = testDir.getAbsolutePath() + "/" + dnId;
List<SCMStorageReport> reports = TestUtils List<StorageReportProto> reports = TestUtils
.createStorageReport(capacity, used, free, storagePath, .createStorageReport(capacity, used, free, storagePath,
null, dnId, 1); null, dnId, 1);
nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(), nodeManager.sendHeartbeat(datanodeDetails,
TestUtils.createNodeReport(reports)); TestUtils.createNodeReport(reports));
} }
GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(), GenericTestUtils.waitFor(() -> nodeManager.waitForHeartbeatProcessed(),
@ -1058,11 +1057,11 @@ public class TestNodeManager {
long scmUsed = x * usedPerHeartbeat; long scmUsed = x * usedPerHeartbeat;
long remaining = capacity - scmUsed; long remaining = capacity - scmUsed;
String storagePath = testDir.getAbsolutePath() + "/" + dnId; String storagePath = testDir.getAbsolutePath() + "/" + dnId;
List<SCMStorageReport> reports = TestUtils List<StorageReportProto> reports = TestUtils
.createStorageReport(capacity, scmUsed, remaining, storagePath, .createStorageReport(capacity, scmUsed, remaining, storagePath,
null, dnId, 1); null, dnId, 1);
nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(), nodeManager.sendHeartbeat(datanodeDetails,
TestUtils.createNodeReport(reports)); TestUtils.createNodeReport(reports));
Thread.sleep(100); Thread.sleep(100);
} }
@ -1140,10 +1139,10 @@ public class TestNodeManager {
// Send a new report to bring the dead node back to healthy // Send a new report to bring the dead node back to healthy
String storagePath = testDir.getAbsolutePath() + "/" + dnId; String storagePath = testDir.getAbsolutePath() + "/" + dnId;
List<SCMStorageReport> reports = TestUtils List<StorageReportProto> reports = TestUtils
.createStorageReport(capacity, expectedScmUsed, expectedRemaining, .createStorageReport(capacity, expectedScmUsed, expectedRemaining,
storagePath, null, dnId, 1); storagePath, null, dnId, 1);
nodeManager.sendHeartbeat(datanodeDetails.getProtoBufMessage(), nodeManager.sendHeartbeat(datanodeDetails,
TestUtils.createNodeReport(reports)); TestUtils.createNodeReport(reports));
// Wait up to 5 seconds so that the dead node becomes healthy // Wait up to 5 seconds so that the dead node becomes healthy

View File

@ -21,9 +21,9 @@ import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto. import org.apache.hadoop.hdds.protocol.proto.
StorageContainerDatanodeProtocolProtos.SCMNodeReport; StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto. import org.apache.hadoop.hdds.protocol.proto.
StorageContainerDatanodeProtocolProtos.SCMStorageReport; StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.hdds.scm.TestUtils; import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.exceptions.SCMException; import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.OzoneConsts;
@ -134,7 +134,7 @@ public class TestSCMNodeStorageStatMap {
@Test @Test
public void testProcessNodeReportCheckOneNode() throws IOException { public void testProcessNodeReportCheckOneNode() throws IOException {
UUID key = getFirstKey(); UUID key = getFirstKey();
List<SCMStorageReport> reportList = new ArrayList<>(); List<StorageReportProto> reportList = new ArrayList<>();
Set<StorageLocationReport> reportSet = testData.get(key); Set<StorageLocationReport> reportSet = testData.get(key);
SCMNodeStorageStatMap map = new SCMNodeStorageStatMap(conf); SCMNodeStorageStatMap map = new SCMNodeStorageStatMap(conf);
map.insertNewDatanode(key, reportSet); map.insertNewDatanode(key, reportSet);
@ -146,16 +146,16 @@ public class TestSCMNodeStorageStatMap {
long reportCapacity = report.getCapacity(); long reportCapacity = report.getCapacity();
long reportScmUsed = report.getScmUsed(); long reportScmUsed = report.getScmUsed();
long reportRemaining = report.getRemaining(); long reportRemaining = report.getRemaining();
List<SCMStorageReport> reports = TestUtils List<StorageReportProto> reports = TestUtils
.createStorageReport(reportCapacity, reportScmUsed, reportRemaining, .createStorageReport(reportCapacity, reportScmUsed, reportRemaining,
path, null, storageId, 1); path, null, storageId, 1);
StorageReportResult result = StorageReportResult result =
map.processNodeReport(key, TestUtils.createNodeReport(reports)); map.processNodeReport(key, TestUtils.createNodeReport(reports));
Assert.assertEquals(result.getStatus(), Assert.assertEquals(result.getStatus(),
SCMNodeStorageStatMap.ReportStatus.ALL_IS_WELL); SCMNodeStorageStatMap.ReportStatus.ALL_IS_WELL);
StorageContainerDatanodeProtocolProtos.SCMNodeReport.Builder nrb = StorageContainerDatanodeProtocolProtos.NodeReportProto.Builder nrb =
SCMNodeReport.newBuilder(); NodeReportProto.newBuilder();
SCMStorageReport srb = reportSet.iterator().next().getProtoBufMessage(); StorageReportProto srb = reportSet.iterator().next().getProtoBufMessage();
reportList.add(srb); reportList.add(srb);
result = map.processNodeReport(key, TestUtils.createNodeReport(reportList)); result = map.processNodeReport(key, TestUtils.createNodeReport(reportList));
Assert.assertEquals(result.getStatus(), Assert.assertEquals(result.getStatus(),
@ -168,7 +168,7 @@ public class TestSCMNodeStorageStatMap {
Assert.assertEquals(result.getStatus(), Assert.assertEquals(result.getStatus(),
SCMNodeStorageStatMap.ReportStatus.STORAGE_OUT_OF_SPACE); SCMNodeStorageStatMap.ReportStatus.STORAGE_OUT_OF_SPACE);
// Mark a disk failed // Mark a disk failed
SCMStorageReport srb2 = SCMStorageReport.newBuilder() StorageReportProto srb2 = StorageReportProto.newBuilder()
.setStorageUuid(UUID.randomUUID().toString()) .setStorageUuid(UUID.randomUUID().toString())
.setStorageLocation(srb.getStorageLocation()).setScmUsed(reportCapacity) .setStorageLocation(srb.getStorageLocation()).setScmUsed(reportCapacity)
.setCapacity(reportCapacity).setRemaining(0).setFailed(true).build(); .setCapacity(reportCapacity).setRemaining(0).setFailed(true).build();

View File

@ -20,22 +20,21 @@ import org.apache.commons.codec.digest.DigestUtils;
import org.apache.commons.lang3.RandomUtils; import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.scm.TestUtils; import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.VersionInfo; import org.apache.hadoop.hdds.scm.VersionInfo;
import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos; .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto; .StorageContainerDatanodeProtocolProtos.SCMHeartbeatRequestProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsResponseProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto; .StorageContainerDatanodeProtocolProtos.SCMHeartbeatResponseProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMRegisteredCmdResponseProto; .StorageContainerDatanodeProtocolProtos.SCMRegisteredResponseProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMStorageReport; .StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto; .StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RPC;
@ -200,7 +199,7 @@ public class TestEndPoint {
DatanodeDetails nodeToRegister = getDatanodeDetails(); DatanodeDetails nodeToRegister = getDatanodeDetails();
try (EndpointStateMachine rpcEndPoint = createEndpoint( try (EndpointStateMachine rpcEndPoint = createEndpoint(
SCMTestUtils.getConf(), serverAddress, 1000)) { SCMTestUtils.getConf(), serverAddress, 1000)) {
SCMRegisteredCmdResponseProto responseProto = rpcEndPoint.getEndPoint() SCMRegisteredResponseProto responseProto = rpcEndPoint.getEndPoint()
.register(nodeToRegister.getProtoBufMessage(), TestUtils .register(nodeToRegister.getProtoBufMessage(), TestUtils
.createNodeReport( .createNodeReport(
getStorageReports(nodeToRegister.getUuidString())), getStorageReports(nodeToRegister.getUuidString())),
@ -215,7 +214,7 @@ public class TestEndPoint {
} }
} }
private List<SCMStorageReport> getStorageReports(String id) { private List<StorageReportProto> getStorageReports(String id) {
String storagePath = testDir.getAbsolutePath() + "/" + id; String storagePath = testDir.getAbsolutePath() + "/" + id;
return TestUtils.createStorageReport(100, 10, 90, storagePath, null, id, 1); return TestUtils.createStorageReport(100, 10, 90, storagePath, null, id, 1);
} }
@ -293,9 +292,14 @@ public class TestEndPoint {
createEndpoint(SCMTestUtils.getConf(), createEndpoint(SCMTestUtils.getConf(),
serverAddress, 1000)) { serverAddress, 1000)) {
String storageId = UUID.randomUUID().toString(); String storageId = UUID.randomUUID().toString();
SCMHeartbeatRequestProto request = SCMHeartbeatRequestProto.newBuilder()
.setDatanodeDetails(dataNode.getProtoBufMessage())
.setNodeReport(TestUtils.createNodeReport(
getStorageReports(storageId)))
.build();
SCMHeartbeatResponseProto responseProto = rpcEndPoint.getEndPoint() SCMHeartbeatResponseProto responseProto = rpcEndPoint.getEndPoint()
.sendHeartbeat(dataNode.getProtoBufMessage(), .sendHeartbeat(request);
TestUtils.createNodeReport(getStorageReports(storageId)));
Assert.assertNotNull(responseProto); Assert.assertNotNull(responseProto);
Assert.assertEquals(0, responseProto.getCommandsCount()); Assert.assertEquals(0, responseProto.getCommandsCount());
} }
@ -361,86 +365,11 @@ public class TestEndPoint {
lessThanOrEqualTo(rpcTimeout + tolerance)); lessThanOrEqualTo(rpcTimeout + tolerance));
} }
/** private ContainerReportsProto createContainerReport(
* Returns a new container report.
* @return
*/
ContainerReport getRandomContainerReport() {
return new ContainerReport(RandomUtils.nextLong(),
DigestUtils.sha256Hex("Random"));
}
/**
* Creates dummy container reports.
* @param count - The number of closed containers to create.
* @return ContainerReportsProto
*/
StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto
createDummyContainerReports(int count) {
StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto.Builder
reportsBuilder = StorageContainerDatanodeProtocolProtos
.ContainerReportsRequestProto.newBuilder();
for (int x = 0; x < count; x++) {
reportsBuilder.addReports(getRandomContainerReport()
.getProtoBufMessage());
}
reportsBuilder.setDatanodeDetails(getDatanodeDetails()
.getProtoBufMessage());
reportsBuilder.setType(StorageContainerDatanodeProtocolProtos
.ContainerReportsRequestProto.reportType.fullReport);
return reportsBuilder.build();
}
/**
* Tests that rpcEndpoint sendContainerReport works as expected.
* @throws Exception
*/
@Test
public void testContainerReportSend() throws Exception {
final int count = 1000;
scmServerImpl.reset();
try (EndpointStateMachine rpcEndPoint =
createEndpoint(SCMTestUtils.getConf(),
serverAddress, 1000)) {
ContainerReportsResponseProto responseProto = rpcEndPoint
.getEndPoint().sendContainerReport(createDummyContainerReports(
count));
Assert.assertNotNull(responseProto);
}
Assert.assertEquals(1, scmServerImpl.getContainerReportsCount());
Assert.assertEquals(count, scmServerImpl.getContainerCount());
}
/**
* Tests that rpcEndpoint sendContainerReport works as expected.
* @throws Exception
*/
@Test
public void testContainerReport() throws Exception {
final int count = 1000;
scmServerImpl.reset();
try (EndpointStateMachine rpcEndPoint =
createEndpoint(SCMTestUtils.getConf(),
serverAddress, 1000)) {
ContainerReportsResponseProto responseProto = rpcEndPoint
.getEndPoint().sendContainerReport(createContainerReport(count,
null));
Assert.assertNotNull(responseProto);
}
Assert.assertEquals(1, scmServerImpl.getContainerReportsCount());
Assert.assertEquals(count, scmServerImpl.getContainerCount());
final long expectedKeyCount = count * 1000;
Assert.assertEquals(expectedKeyCount, scmServerImpl.getKeyCount());
final long expectedBytesUsed = count * OzoneConsts.GB * 2;
Assert.assertEquals(expectedBytesUsed, scmServerImpl.getBytesUsed());
}
private ContainerReportsRequestProto createContainerReport(
int count, DatanodeDetails datanodeDetails) { int count, DatanodeDetails datanodeDetails) {
StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto.Builder StorageContainerDatanodeProtocolProtos.ContainerReportsProto.Builder
reportsBuilder = StorageContainerDatanodeProtocolProtos reportsBuilder = StorageContainerDatanodeProtocolProtos
.ContainerReportsRequestProto.newBuilder(); .ContainerReportsProto.newBuilder();
for (int x = 0; x < count; x++) { for (int x = 0; x < count; x++) {
long containerID = RandomUtils.nextLong(); long containerID = RandomUtils.nextLong();
ContainerReport report = new ContainerReport(containerID, ContainerReport report = new ContainerReport(containerID,
@ -455,14 +384,6 @@ public class TestEndPoint {
reportsBuilder.addReports(report.getProtoBufMessage()); reportsBuilder.addReports(report.getProtoBufMessage());
} }
if(datanodeDetails == null) {
reportsBuilder.setDatanodeDetails(getDatanodeDetails()
.getProtoBufMessage());
} else {
reportsBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage());
}
reportsBuilder.setType(StorageContainerDatanodeProtocolProtos
.ContainerReportsRequestProto.reportType.fullReport);
return reportsBuilder.build(); return reportsBuilder.build();
} }
} }

View File

@ -1,275 +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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.ozone.container.replication;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.container.replication.ContainerSupervisor;
import org.apache.hadoop.hdds.scm.container.replication.InProgressPool;
import org.apache.hadoop.hdds.scm.node.CommandQueue;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.node.NodePoolManager;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.ozone.container.testutils
.ReplicationDatanodeStateManager;
import org.apache.hadoop.ozone.container.testutils.ReplicationNodeManagerMock;
import org.apache.hadoop.ozone.container.testutils
.ReplicationNodePoolManagerMock;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.event.Level;
import java.io.IOException;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
.OZONE_SCM_CONTAINER_REPORTS_WAIT_TIMEOUT;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
.OZONE_SCM_CONTAINER_REPORT_PROCESSING_INTERVAL;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState
.HEALTHY;
import static org.apache.ratis.shaded.com.google.common.util.concurrent
.Uninterruptibles.sleepUninterruptibly;
/**
* Tests for the container manager.
*/
public class TestContainerSupervisor {
final static String POOL_NAME_TEMPLATE = "Pool%d";
static final int MAX_DATANODES = 72;
static final int POOL_SIZE = 24;
static final int POOL_COUNT = 3;
private LogCapturer logCapturer = LogCapturer.captureLogs(
LogFactory.getLog(ContainerSupervisor.class));
private List<DatanodeDetails> datanodes = new LinkedList<>();
private NodeManager nodeManager;
private NodePoolManager poolManager;
private CommandQueue commandQueue;
private ContainerSupervisor containerSupervisor;
private ReplicationDatanodeStateManager datanodeStateManager;
@After
public void tearDown() throws Exception {
logCapturer.stopCapturing();
GenericTestUtils.setLogLevel(ContainerSupervisor.LOG, Level.INFO);
}
@Before
public void setUp() throws Exception {
GenericTestUtils.setLogLevel(ContainerSupervisor.LOG, Level.DEBUG);
Map<DatanodeDetails, NodeState> nodeStateMap = new HashMap<>();
// We are setting up 3 pools with 24 nodes each in this cluster.
// First we create 72 Datanodes.
for (int x = 0; x < MAX_DATANODES; x++) {
DatanodeDetails datanode = TestUtils.getDatanodeDetails();
datanodes.add(datanode);
nodeStateMap.put(datanode, HEALTHY);
}
commandQueue = new CommandQueue();
// All nodes in this cluster are healthy for time being.
nodeManager = new ReplicationNodeManagerMock(nodeStateMap, commandQueue);
poolManager = new ReplicationNodePoolManagerMock();
Assert.assertEquals("Max datanodes should be equal to POOL_SIZE * " +
"POOL_COUNT", POOL_COUNT * POOL_SIZE, MAX_DATANODES);
// Start from 1 instead of zero so we can multiply and get the node index.
for (int y = 1; y <= POOL_COUNT; y++) {
String poolName = String.format(POOL_NAME_TEMPLATE, y);
for (int z = 0; z < POOL_SIZE; z++) {
DatanodeDetails id = datanodes.get(y * z);
poolManager.addNode(poolName, id);
}
}
OzoneConfiguration config = SCMTestUtils.getOzoneConf();
config.setTimeDuration(OZONE_SCM_CONTAINER_REPORTS_WAIT_TIMEOUT, 2,
TimeUnit.SECONDS);
config.setTimeDuration(OZONE_SCM_CONTAINER_REPORT_PROCESSING_INTERVAL, 1,
TimeUnit.SECONDS);
containerSupervisor = new ContainerSupervisor(config,
nodeManager, poolManager);
datanodeStateManager = new ReplicationDatanodeStateManager(nodeManager,
poolManager);
// Sleep for one second to make sure all threads get time to run.
sleepUninterruptibly(1, TimeUnit.SECONDS);
}
@Test
/**
* Asserts that at least one pool is picked up for processing.
*/
public void testAssertPoolsAreProcessed() {
// This asserts that replication manager has started processing at least
// one pool.
Assert.assertTrue(containerSupervisor.getInProgressPoolCount() > 0);
// Since all datanodes are flagged as healthy in this test, for each
// datanode we must have queued a command.
Assert.assertEquals("Commands are in queue :",
POOL_SIZE * containerSupervisor.getInProgressPoolCount(),
commandQueue.getCommandsInQueue());
}
@Test
/**
* This test sends container reports for 2 containers to a pool in progress.
* Asserts that we are able to find a container with single replica and do
* not find container with 3 replicas.
*/
public void testDetectSingleContainerReplica() throws TimeoutException,
InterruptedException {
long singleNodeContainerID = 9001;
long threeNodeContainerID = 9003;
InProgressPool ppool = containerSupervisor.getInProcessPoolList().get(0);
// Only single datanode reporting that "SingleNodeContainer" exists.
List<ContainerReportsRequestProto> clist =
datanodeStateManager.getContainerReport(singleNodeContainerID,
ppool.getPool().getPoolName(), 1);
ppool.handleContainerReport(clist.get(0));
// Three nodes are going to report that ThreeNodeContainer exists.
clist = datanodeStateManager.getContainerReport(threeNodeContainerID,
ppool.getPool().getPoolName(), 3);
for (ContainerReportsRequestProto reportsProto : clist) {
ppool.handleContainerReport(reportsProto);
}
GenericTestUtils.waitFor(() -> ppool.getContainerProcessedCount() == 4,
200, 1000);
ppool.setDoneProcessing();
List<Map.Entry<Long, Integer>> containers = ppool.filterContainer(p -> p
.getValue() == 1);
Assert.assertEquals(singleNodeContainerID,
containers.get(0).getKey().longValue());
int count = containers.get(0).getValue();
Assert.assertEquals(1L, count);
}
@Test
/**
* We create three containers, Normal,OveReplicated and WayOverReplicated
* containers. This test asserts that we are able to find the
* over replicated containers.
*/
public void testDetectOverReplica() throws TimeoutException,
InterruptedException {
long normalContainerID = 9000;
long overReplicatedContainerID = 9001;
long wayOverReplicatedContainerID = 9002;
InProgressPool ppool = containerSupervisor.getInProcessPoolList().get(0);
List<ContainerReportsRequestProto> clist =
datanodeStateManager.getContainerReport(normalContainerID,
ppool.getPool().getPoolName(), 3);
ppool.handleContainerReport(clist.get(0));
clist = datanodeStateManager.getContainerReport(overReplicatedContainerID,
ppool.getPool().getPoolName(), 4);
for (ContainerReportsRequestProto reportsProto : clist) {
ppool.handleContainerReport(reportsProto);
}
clist = datanodeStateManager.getContainerReport(
wayOverReplicatedContainerID, ppool.getPool().getPoolName(), 7);
for (ContainerReportsRequestProto reportsProto : clist) {
ppool.handleContainerReport(reportsProto);
}
// We ignore container reports from the same datanodes.
// it is possible that these each of these containers get placed
// on same datanodes, so allowing for 4 duplicates in the set of 14.
GenericTestUtils.waitFor(() -> ppool.getContainerProcessedCount() > 10,
200, 1000);
ppool.setDoneProcessing();
List<Map.Entry<Long, Integer>> containers = ppool.filterContainer(p -> p
.getValue() > 3);
Assert.assertEquals(2, containers.size());
}
@Test
/**
* This test verifies that all pools are picked up for replica processing.
*
*/
public void testAllPoolsAreProcessed() throws TimeoutException,
InterruptedException {
// Verify that we saw all three pools being picked up for processing.
GenericTestUtils.waitFor(() -> containerSupervisor.getPoolProcessCount()
>= 3, 200, 15 * 1000);
Assert.assertTrue(logCapturer.getOutput().contains("Pool1") &&
logCapturer.getOutput().contains("Pool2") &&
logCapturer.getOutput().contains("Pool3"));
}
@Test
/**
* Adds a new pool and tests that we are able to pick up that new pool for
* processing as well as handle container reports for datanodes in that pool.
* @throws TimeoutException
* @throws InterruptedException
*/
public void testAddingNewPoolWorks()
throws TimeoutException, InterruptedException, IOException {
LogCapturer inProgressLog = LogCapturer.captureLogs(
LogFactory.getLog(InProgressPool.class));
GenericTestUtils.setLogLevel(InProgressPool.LOG, Level.DEBUG);
try {
DatanodeDetails id = TestUtils.getDatanodeDetails();
((ReplicationNodeManagerMock) (nodeManager)).addNode(id, HEALTHY);
poolManager.addNode("PoolNew", id);
GenericTestUtils.waitFor(() ->
logCapturer.getOutput().contains("PoolNew"),
200, 15 * 1000);
long newContainerID = 7001;
// Assert that we are able to send a container report to this new
// pool and datanode.
List<ContainerReportsRequestProto> clist =
datanodeStateManager.getContainerReport(newContainerID,
"PoolNew", 1);
containerSupervisor.handleContainerReport(clist.get(0));
GenericTestUtils.waitFor(() ->
inProgressLog.getOutput()
.contains(Long.toString(newContainerID)) && inProgressLog
.getOutput().contains(id.getUuidString()),
200, 10 * 1000);
} finally {
inProgressLog.stopCapturing();
}
}
}

View File

@ -1,101 +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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.ozone.container.testutils;
import com.google.common.primitives.Longs;
import org.apache.commons.codec.digest.DigestUtils;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.node.NodePoolManager;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerInfo;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto;
import java.util.LinkedList;
import java.util.List;
import java.util.Random;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState
.HEALTHY;
/**
* This class manages the state of datanode
* in conjunction with the node pool and node managers.
*/
public class ReplicationDatanodeStateManager {
private final NodeManager nodeManager;
private final NodePoolManager poolManager;
private final Random r;
/**
* The datanode state Manager.
*
* @param nodeManager
* @param poolManager
*/
public ReplicationDatanodeStateManager(NodeManager nodeManager,
NodePoolManager poolManager) {
this.nodeManager = nodeManager;
this.poolManager = poolManager;
r = new Random();
}
/**
* Get Container Report as if it is from a datanode in the cluster.
* @param containerID - Container ID.
* @param poolName - Pool Name.
* @param dataNodeCount - Datanode Count.
* @return List of Container Reports.
*/
public List<ContainerReportsRequestProto> getContainerReport(
long containerID, String poolName, int dataNodeCount) {
List<ContainerReportsRequestProto> containerList = new LinkedList<>();
List<DatanodeDetails> nodesInPool = poolManager.getNodes(poolName);
if (nodesInPool == null) {
return containerList;
}
if (nodesInPool.size() < dataNodeCount) {
throw new IllegalStateException("Not enough datanodes to create " +
"required container reports");
}
while (containerList.size() < dataNodeCount && nodesInPool.size() > 0) {
DatanodeDetails id = nodesInPool.get(r.nextInt(nodesInPool.size()));
nodesInPool.remove(id);
containerID++;
// We return container reports only for nodes that are healthy.
if (nodeManager.getNodeState(id) == HEALTHY) {
ContainerInfo info = ContainerInfo.newBuilder()
.setContainerID(containerID)
.setFinalhash(DigestUtils.sha256Hex(
Longs.toByteArray(containerID)))
.setContainerID(containerID)
.build();
ContainerReportsRequestProto containerReport =
ContainerReportsRequestProto.newBuilder().addReports(info)
.setDatanodeDetails(id.getProtoBufMessage())
.setType(ContainerReportsRequestProto.reportType.fullReport)
.build();
containerList.add(containerReport);
}
}
return containerList;
}
}

View File

@ -24,13 +24,13 @@ import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.node.NodePoolManager; import org.apache.hadoop.hdds.scm.node.NodePoolManager;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException; import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdds.protocol.DatanodeDetails; 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.HddsProtos.NodeState;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport; .StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto; .StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto;
import org.apache.hadoop.ozone.protocol.VersionResponse; import org.apache.hadoop.ozone.protocol.VersionResponse;
import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand; import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.mockito.Mockito; import org.mockito.Mockito;
@ -277,12 +277,12 @@ public class ReplicationNodeManagerMock implements NodeManager {
* Register the node if the node finds that it is not registered with any SCM. * Register the node if the node finds that it is not registered with any SCM.
* *
* @param dd DatanodeDetailsProto * @param dd DatanodeDetailsProto
* @param nodeReport SCMNodeReport * @param nodeReport NodeReportProto
* @return SCMHeartbeatResponseProto * @return SCMHeartbeatResponseProto
*/ */
@Override @Override
public SCMCommand register(HddsProtos.DatanodeDetailsProto dd, public RegisteredCommand register(DatanodeDetails dd,
SCMNodeReport nodeReport) { NodeReportProto nodeReport) {
return null; return null;
} }
@ -294,8 +294,8 @@ public class ReplicationNodeManagerMock implements NodeManager {
* @return SCMheartbeat response list * @return SCMheartbeat response list
*/ */
@Override @Override
public List<SCMCommand> sendHeartbeat(HddsProtos.DatanodeDetailsProto dd, public List<SCMCommand> sendHeartbeat(DatanodeDetails dd,
SCMNodeReport nodeReport) { NodeReportProto nodeReport) {
return null; return null;
} }

View File

@ -32,8 +32,10 @@ import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction; import org.apache.hadoop.hdds.protocol.proto
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCmdType; .StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager.StartupOption; import org.apache.hadoop.hdds.scm.server.StorageContainerManager.StartupOption;
import org.apache.hadoop.hdds.scm.block.DeletedBlockLog; import org.apache.hadoop.hdds.scm.block.DeletedBlockLog;
@ -302,12 +304,11 @@ public class TestStorageContainerManager {
NodeManager nodeManager = cluster.getStorageContainerManager() NodeManager nodeManager = cluster.getStorageContainerManager()
.getScmNodeManager(); .getScmNodeManager();
List<SCMCommand> commands = nodeManager.sendHeartbeat( List<SCMCommand> commands = nodeManager.sendHeartbeat(
nodeManager.getNodes(NodeState.HEALTHY).get(0).getProtoBufMessage(), nodeManager.getNodes(NodeState.HEALTHY).get(0), null);
null);
if (commands != null) { if (commands != null) {
for (SCMCommand cmd : commands) { for (SCMCommand cmd : commands) {
if (cmd.getType() == SCMCmdType.deleteBlocksCommand) { if (cmd.getType() == SCMCommandProto.Type.deleteBlocksCommand) {
List<DeletedBlocksTransaction> deletedTXs = List<DeletedBlocksTransaction> deletedTXs =
((DeleteBlocksCommand) cmd).blocksTobeDeleted(); ((DeleteBlocksCommand) cmd).blocksTobeDeleted();
return deletedTXs != null && deletedTXs.size() == limitSize; return deletedTXs != null && deletedTXs.size() == limitSize;

View File

@ -32,8 +32,10 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.helpers.ContainerReport; import org.apache.hadoop.ozone.container.common.helpers.ContainerReport;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos; import org.apache.hadoop.hdds.protocol.proto
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto; .StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.scm.container.placement.metrics.ContainerStat; 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.container.placement.metrics.SCMMetrics;
import org.apache.hadoop.hdds.scm.node.SCMNodeManager; import org.apache.hadoop.hdds.scm.node.SCMNodeManager;
@ -75,11 +77,11 @@ public class TestSCMMetrics {
ContainerStat stat = new ContainerStat(size, used, keyCount, readBytes, ContainerStat stat = new ContainerStat(size, used, keyCount, readBytes,
writeBytes, readCount, writeCount); writeBytes, readCount, writeCount);
StorageContainerManager scmManager = cluster.getStorageContainerManager(); StorageContainerManager scmManager = cluster.getStorageContainerManager();
DatanodeDetails fstDatanodeDetails = TestUtils.getDatanodeDetails();
ContainerReportsRequestProto request = createContainerReport(numReport, ContainerReportsProto request = createContainerReport(numReport, stat);
stat, null); String fstDatanodeUuid = fstDatanodeDetails.getUuidString();
String fstDatanodeUuid = request.getDatanodeDetails().getUuid(); scmManager.getDatanodeProtocolServer().processContainerReports(
scmManager.getDatanodeProtocolServer().sendContainerReport(request); fstDatanodeDetails, request);
// verify container stat metrics // verify container stat metrics
MetricsRecordBuilder scmMetrics = getMetrics(SCMMetrics.SOURCE_NAME); MetricsRecordBuilder scmMetrics = getMetrics(SCMMetrics.SOURCE_NAME);
@ -100,9 +102,11 @@ public class TestSCMMetrics {
getLongGauge("LastContainerReportWriteCount", scmMetrics)); getLongGauge("LastContainerReportWriteCount", scmMetrics));
// add one new report // add one new report
request = createContainerReport(1, stat, null); DatanodeDetails sndDatanodeDetails = TestUtils.getDatanodeDetails();
String sndDatanodeUuid = request.getDatanodeDetails().getUuid(); request = createContainerReport(1, stat);
scmManager.getDatanodeProtocolServer().sendContainerReport(request); String sndDatanodeUuid = sndDatanodeDetails.getUuidString();
scmManager.getDatanodeProtocolServer().processContainerReports(
sndDatanodeDetails, request);
scmMetrics = getMetrics(SCMMetrics.SOURCE_NAME); scmMetrics = getMetrics(SCMMetrics.SOURCE_NAME);
assertEquals(size * (numReport + 1), assertEquals(size * (numReport + 1),
@ -124,12 +128,12 @@ public class TestSCMMetrics {
// Re-send reports but with different value for validating // Re-send reports but with different value for validating
// the aggregation. // the aggregation.
stat = new ContainerStat(100, 50, 3, 50, 60, 5, 6); stat = new ContainerStat(100, 50, 3, 50, 60, 5, 6);
scmManager.getDatanodeProtocolServer().sendContainerReport( scmManager.getDatanodeProtocolServer().processContainerReports(
createContainerReport(1, stat, fstDatanodeUuid)); fstDatanodeDetails, createContainerReport(1, stat));
stat = new ContainerStat(1, 1, 1, 1, 1, 1, 1); stat = new ContainerStat(1, 1, 1, 1, 1, 1, 1);
scmManager.getDatanodeProtocolServer().sendContainerReport( scmManager.getDatanodeProtocolServer().processContainerReports(
createContainerReport(1, stat, sndDatanodeUuid)); sndDatanodeDetails, createContainerReport(1, stat));
// the global container metrics value should be updated // the global container metrics value should be updated
scmMetrics = getMetrics(SCMMetrics.SOURCE_NAME); scmMetrics = getMetrics(SCMMetrics.SOURCE_NAME);
@ -170,11 +174,11 @@ public class TestSCMMetrics {
writeBytes, readCount, writeCount); writeBytes, readCount, writeCount);
StorageContainerManager scmManager = cluster.getStorageContainerManager(); StorageContainerManager scmManager = cluster.getStorageContainerManager();
String datanodeUuid = cluster.getHddsDatanodes().get(0) DatanodeDetails datanodeDetails = cluster.getHddsDatanodes().get(0)
.getDatanodeDetails().getUuidString(); .getDatanodeDetails();
ContainerReportsRequestProto request = createContainerReport(numReport, ContainerReportsProto request = createContainerReport(numReport, stat);
stat, datanodeUuid); scmManager.getDatanodeProtocolServer().processContainerReports(
scmManager.getDatanodeProtocolServer().sendContainerReport(request); datanodeDetails, request);
MetricsRecordBuilder scmMetrics = getMetrics(SCMMetrics.SOURCE_NAME); MetricsRecordBuilder scmMetrics = getMetrics(SCMMetrics.SOURCE_NAME);
assertEquals(size * numReport, assertEquals(size * numReport,
@ -216,11 +220,11 @@ public class TestSCMMetrics {
} }
} }
private ContainerReportsRequestProto createContainerReport(int numReport, private ContainerReportsProto createContainerReport(int numReport,
ContainerStat stat, String datanodeUuid) { ContainerStat stat) {
StorageContainerDatanodeProtocolProtos.ContainerReportsRequestProto.Builder StorageContainerDatanodeProtocolProtos.ContainerReportsProto.Builder
reportsBuilder = StorageContainerDatanodeProtocolProtos reportsBuilder = StorageContainerDatanodeProtocolProtos
.ContainerReportsRequestProto.newBuilder(); .ContainerReportsProto.newBuilder();
for (int i = 0; i < numReport; i++) { for (int i = 0; i < numReport; i++) {
ContainerReport report = new ContainerReport( ContainerReport report = new ContainerReport(
@ -234,24 +238,6 @@ public class TestSCMMetrics {
report.setWriteBytes(stat.getWriteBytes().get()); report.setWriteBytes(stat.getWriteBytes().get());
reportsBuilder.addReports(report.getProtoBufMessage()); reportsBuilder.addReports(report.getProtoBufMessage());
} }
DatanodeDetails datanodeDetails;
if (datanodeUuid == null) {
datanodeDetails = TestUtils.getDatanodeDetails();
} else {
datanodeDetails = DatanodeDetails.newBuilder()
.setUuid(datanodeUuid)
.setIpAddress("127.0.0.1")
.setHostName("localhost")
.setContainerPort(0)
.setRatisPort(0)
.setOzoneRestPort(0)
.build();
}
reportsBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage());
reportsBuilder.setType(StorageContainerDatanodeProtocolProtos
.ContainerReportsRequestProto.reportType.fullReport);
return reportsBuilder.build(); return reportsBuilder.build();
} }
} }