diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java index 5e161b33ce13..99972ae90038 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java @@ -81,12 +81,7 @@ public final class HddsConfigKeys { public static final String HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK = "hdds.scm.safemode.pipeline-availability.check"; public static final boolean - HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK_DEFAULT = true; - - public static final String HDDS_SCM_SAFEMODE_PIPELINE_CREATION = - "hdds.scm.safemode.pipeline.creation"; - public static final boolean - HDDS_SCM_SAFEMODE_PIPELINE_CREATION_DEFAULT = true; + HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK_DEFAULT = false; // % of containers which should have at least one reported replica // before SCM comes out of safe mode. @@ -94,16 +89,13 @@ public final class HddsConfigKeys { "hdds.scm.safemode.threshold.pct"; public static final double HDDS_SCM_SAFEMODE_THRESHOLD_PCT_DEFAULT = 0.99; + // percentage of healthy pipelines, where all 3 datanodes are reported in the // pipeline. public static final String HDDS_SCM_SAFEMODE_HEALTHY_PIPELINE_THRESHOLD_PCT = "hdds.scm.safemode.healthy.pipelie.pct"; public static final double HDDS_SCM_SAFEMODE_HEALTHY_PIPELINE_THRESHOLD_PCT_DEFAULT = 0.10; - // number of healthy RATIS pipeline(ONE or THREE factor) - public static final String HDDS_SCM_SAFEMODE_MIN_PIPELINE = - "hdds.scm.safemode.min.pipeline"; - public static final int HDDS_SCM_SAFEMODE_MIN_PIPELINE_DEFAULT = 1; public static final String HDDS_SCM_SAFEMODE_ONE_NODE_REPORTED_PIPELINE_PCT = "hdds.scm.safemode.atleast.one.node.reported.pipeline.pct"; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java index 522a7477c8dd..15a7bea84dea 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java @@ -155,11 +155,6 @@ public boolean isOpen() { return state == PipelineState.OPEN; } - public boolean isAllocationTimeout() { - //TODO: define a system property to control the timeout value - return false; - } - public void setNodesInOrder(List nodes) { nodesInOrder.set(nodes); } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/Scheduler.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/Scheduler.java index f5e55c10e1bd..9edc10448101 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/Scheduler.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/Scheduler.java @@ -23,7 +23,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; /** @@ -73,9 +72,9 @@ public void schedule(CheckedRunnable runnable, long delay, }, delay, timeUnit); } - public ScheduledFuture scheduleWithFixedDelay(Runnable runnable, - long initialDelay, long fixedDelay, TimeUnit timeUnit) { - return scheduler + public void scheduleWithFixedDelay(Runnable runnable, long initialDelay, + long fixedDelay, TimeUnit timeUnit) { + scheduler .scheduleWithFixedDelay(runnable, initialDelay, fixedDelay, timeUnit); } diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml index c20a6a4d7550..2f9ce31424b4 100644 --- a/hadoop-hdds/common/src/main/resources/ozone-default.xml +++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml @@ -318,6 +318,15 @@ datanode periodically send container report to SCM. Unit could be defined with postfix (ns,ms,s,m,h,d) + + hdds.command.status.report.interval + 60000ms + OZONE, CONTAINER, MANAGEMENT + Time interval of the datanode to send status of command + execution. Each datanode periodically the execution status of commands + received from SCM to SCM. Unit could be defined with postfix + (ns,ms,s,m,h,d) + hdds.pipeline.report.interval 60000ms @@ -1291,7 +1300,7 @@ hdds.scm.safemode.pipeline-availability.check - true + false HDDS,SCM,OPERATION Boolean value to enable pipeline availability check during SCM safe mode. @@ -1376,25 +1385,6 @@ - - hdds.scm.safemode.pipeline.creation - true - HDDS,SCM,OPERATION - - Boolean value to enable background pipeline creation in SCM safe mode. - - - - - hdds.scm.safemode.min.pipeline - 1 - HDDS,SCM,OPERATION - - Minimum RATIS pipeline number to exit SCM safe mode. Considered only when - "hdds.scm.safemode.pipeline.creation" is True. - - - hdds.lock.max.concurrency 100 diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java index 0fbb2270a55c..2763278b0d2b 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java @@ -38,12 +38,8 @@ import org.apache.hadoop.ozone.container.common.report.ReportManager; import org.apache.hadoop.ozone.container.common.statemachine.commandhandler .CloseContainerCommandHandler; -import org.apache.hadoop.ozone.container.common.statemachine.commandhandler - .ClosePipelineCommandHandler; import org.apache.hadoop.ozone.container.common.statemachine.commandhandler .CommandDispatcher; -import org.apache.hadoop.ozone.container.common.statemachine.commandhandler - .CreatePipelineCommandHandler; import org.apache.hadoop.ozone.container.common.statemachine.commandhandler .DeleteBlocksCommandHandler; import org.apache.hadoop.ozone.container.common.statemachine.commandhandler @@ -135,8 +131,6 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails, conf)) .addHandler(new ReplicateContainerCommandHandler(conf, supervisor)) .addHandler(new DeleteContainerCommandHandler()) - .addHandler(new ClosePipelineCommandHandler()) - .addHandler(new CreatePipelineCommandHandler()) .setConnectionManager(connectionManager) .setContainer(container) .setContext(context) diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java index 3c4e24a427b9..ef06c1482017 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java @@ -39,7 +39,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.concurrent.atomic.AtomicLong; /** * Handler for close container command received from SCM. @@ -49,7 +48,7 @@ public class CloseContainerCommandHandler implements CommandHandler { private static final Logger LOG = LoggerFactory.getLogger(CloseContainerCommandHandler.class); - private AtomicLong invocationCount = new AtomicLong(0); + private int invocationCount; private long totalTime; /** @@ -70,7 +69,7 @@ public CloseContainerCommandHandler() { public void handle(SCMCommand command, OzoneContainer ozoneContainer, StateContext context, SCMConnectionManager connectionManager) { LOG.debug("Processing Close Container command."); - invocationCount.incrementAndGet(); + invocationCount++; final long startTime = Time.monotonicNow(); final DatanodeDetails datanodeDetails = context.getParent() .getDatanodeDetails(); @@ -163,7 +162,7 @@ public SCMCommandProto.Type getCommandType() { */ @Override public int getInvocationCount() { - return (int)invocationCount.get(); + return invocationCount; } /** @@ -173,8 +172,8 @@ public int getInvocationCount() { */ @Override public long getAverageRunTime() { - if (invocationCount.get() > 0) { - return totalTime / invocationCount.get(); + if (invocationCount > 0) { + return totalTime / invocationCount; } return 0; } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ClosePipelineCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ClosePipelineCommandHandler.java deleted file mode 100644 index b1c609064b74..000000000000 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ClosePipelineCommandHandler.java +++ /dev/null @@ -1,120 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - *

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

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package org.apache.hadoop.ozone.container.common.statemachine.commandhandler; - -import org.apache.hadoop.hdds.protocol.DatanodeDetails; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.hdds.protocol.proto. - StorageContainerDatanodeProtocolProtos.ClosePipelineCommandProto; -import org.apache.hadoop.hdds.protocol.proto. - 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.transport.server - .XceiverServerSpi; -import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; -import org.apache.hadoop.ozone.protocol.commands.ClosePipelineCommand; -import org.apache.hadoop.ozone.protocol.commands.SCMCommand; -import org.apache.hadoop.util.Time; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.concurrent.atomic.AtomicLong; - -/** - * Handler for close pipeline command received from SCM. - */ -public class ClosePipelineCommandHandler implements CommandHandler { - - private static final Logger LOG = - LoggerFactory.getLogger(ClosePipelineCommandHandler.class); - - private AtomicLong invocationCount = new AtomicLong(0); - private long totalTime; - - /** - * Constructs a closePipelineCommand handler. - */ - public ClosePipelineCommandHandler() { - } - - /** - * Handles a given SCM command. - * - * @param command - SCM Command - * @param ozoneContainer - Ozone Container. - * @param context - Current Context. - * @param connectionManager - The SCMs that we are talking to. - */ - @Override - public void handle(SCMCommand command, OzoneContainer ozoneContainer, - StateContext context, SCMConnectionManager connectionManager) { - invocationCount.incrementAndGet(); - final long startTime = Time.monotonicNow(); - final DatanodeDetails dn = context.getParent().getDatanodeDetails(); - final ClosePipelineCommandProto closeCommand = - ((ClosePipelineCommand)command).getProto(); - final HddsProtos.PipelineID pipelineID = closeCommand.getPipelineID(); - - try { - XceiverServerSpi server = ozoneContainer.getWriteChannel(); - server.removeGroup(pipelineID); - LOG.info("Close Pipeline #{} command on datanode #{}.", pipelineID, - dn.getUuidString()); - } catch (IOException e) { - LOG.error("Can't close pipeline #{}", pipelineID, e); - } finally { - long endTime = Time.monotonicNow(); - totalTime += endTime - startTime; - } - } - - /** - * Returns the command type that this command handler handles. - * - * @return Type - */ - @Override - public SCMCommandProto.Type getCommandType() { - return SCMCommandProto.Type.closePipelineCommand; - } - - /** - * Returns number of times this handler has been invoked. - * - * @return int - */ - @Override - public int getInvocationCount() { - return (int)invocationCount.get(); - } - - /** - * Returns the average time this function takes to run. - * - * @return long - */ - @Override - public long getAverageRunTime() { - if (invocationCount.get() > 0) { - return totalTime / invocationCount.get(); - } - return 0; - } -} diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandHandler.java index dca02f6e6228..1ea0ea845150 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CommandHandler.java @@ -68,7 +68,7 @@ void handle(SCMCommand command, OzoneContainer container, default void updateCommandStatus(StateContext context, SCMCommand command, Consumer cmdStatusUpdater, Logger log) { if (!context.updateCommandStatus(command.getId(), cmdStatusUpdater)) { - log.warn("{} with Id:{} not found.", command.getType(), + log.debug("{} with Id:{} not found.", command.getType(), command.getId()); } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CreatePipelineCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CreatePipelineCommandHandler.java deleted file mode 100644 index 3a60d7eb3084..000000000000 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CreatePipelineCommandHandler.java +++ /dev/null @@ -1,135 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - *

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

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package org.apache.hadoop.ozone.container.common.statemachine.commandhandler; - -import org.apache.hadoop.hdds.protocol.DatanodeDetails; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.hdds.protocol.proto. - StorageContainerDatanodeProtocolProtos.CreatePipelineCommandProto; -import org.apache.hadoop.hdds.protocol.proto. - 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.transport.server - .XceiverServerSpi; -import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; -import org.apache.hadoop.ozone.protocol.commands.CreatePipelineCommand; -import org.apache.hadoop.ozone.protocol.commands.SCMCommand; -import org.apache.hadoop.util.Time; -import org.apache.ratis.protocol.NotLeaderException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.Collection; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; - -/** - * Handler for create pipeline command received from SCM. - */ -public class CreatePipelineCommandHandler implements CommandHandler { - - private static final Logger LOG = - LoggerFactory.getLogger(CreatePipelineCommandHandler.class); - - private AtomicLong invocationCount = new AtomicLong(0); - private long totalTime; - - /** - * Constructs a createPipelineCommand handler. - */ - public CreatePipelineCommandHandler() { - } - - /** - * Handles a given SCM command. - * - * @param command - SCM Command - * @param ozoneContainer - Ozone Container. - * @param context - Current Context. - * @param connectionManager - The SCMs that we are talking to. - */ - @Override - public void handle(SCMCommand command, OzoneContainer ozoneContainer, - StateContext context, SCMConnectionManager connectionManager) { - invocationCount.incrementAndGet(); - final long startTime = Time.monotonicNow(); - final DatanodeDetails dn = context.getParent() - .getDatanodeDetails(); - final CreatePipelineCommandProto createCommand = - ((CreatePipelineCommand)command).getProto(); - final HddsProtos.PipelineID pipelineID = createCommand.getPipelineID(); - Collection peers = - createCommand.getDatanodeList().stream() - .map(DatanodeDetails::getFromProtoBuf) - .collect(Collectors.toList()); - - try { - XceiverServerSpi server = ozoneContainer.getWriteChannel(); - server.addGroup(pipelineID, peers); - LOG.info("Create Pipeline {} {} #{} command succeed on datanode {}.", - createCommand.getType(), createCommand.getFactor(), pipelineID, - dn.getUuidString()); - // Trigger heartbeat report - context.addReport(context.getParent().getContainer().getPipelineReport()); - context.getParent().triggerHeartbeat(); - } catch (NotLeaderException e) { - LOG.debug("Follower cannot create pipeline #{}.", pipelineID); - } catch (IOException e) { - LOG.error("Can't create pipeline {} {} #{}", createCommand.getType(), - createCommand.getFactor(), pipelineID, e); - } finally { - long endTime = Time.monotonicNow(); - totalTime += endTime - startTime; - } - } - - /** - * Returns the command type that this command handler handles. - * - * @return Type - */ - @Override - public SCMCommandProto.Type getCommandType() { - return SCMCommandProto.Type.createPipelineCommand; - } - - /** - * Returns number of times this handler has been invoked. - * - * @return int - */ - @Override - public int getInvocationCount() { - return (int)invocationCount.get(); - } - - /** - * Returns the average time this function takes to run. - * - * @return long - */ - @Override - public long getAverageRunTime() { - if (invocationCount.get() > 0) { - return totalTime / invocationCount.get(); - } - return 0; - } -} diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java index a55d0d647d54..c50f4573d07f 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java @@ -46,8 +46,6 @@ .EndpointStateMachine.EndPointStates; import org.apache.hadoop.ozone.container.common.statemachine.StateContext; import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand; -import org.apache.hadoop.ozone.protocol.commands.ClosePipelineCommand; -import org.apache.hadoop.ozone.protocol.commands.CreatePipelineCommand; import org.apache.hadoop.ozone.protocol.commands.DeleteBlocksCommand; import org.apache.hadoop.ozone.protocol.commands.DeleteContainerCommand; import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand; @@ -311,26 +309,6 @@ private void processResponse(SCMHeartbeatResponseProto response, } this.context.addCommand(deleteContainerCommand); break; - case createPipelineCommand: - CreatePipelineCommand createPipelineCommand = - CreatePipelineCommand.getFromProtobuf( - commandResponseProto.getCreatePipelineCommandProto()); - if (LOG.isDebugEnabled()) { - LOG.debug("Received SCM create pipeline request {}", - createPipelineCommand.getPipelineID()); - } - this.context.addCommand(createPipelineCommand); - break; - case closePipelineCommand: - ClosePipelineCommand closePipelineCommand = - ClosePipelineCommand.getFromProtobuf( - commandResponseProto.getClosePipelineCommandProto()); - if (LOG.isDebugEnabled()) { - LOG.debug("Received SCM close pipeline request {}", - closePipelineCommand.getPipelineID()); - } - this.context.addCommand(closePipelineCommand); - break; default: throw new IllegalArgumentException("Unknown response : " + commandResponseProto.getCommandType().name()); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java index 01f463c5cddd..4e0d34384ce2 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java @@ -18,7 +18,6 @@ package org.apache.hadoop.ozone.container.common.transport.server; -import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos .ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -26,7 +25,6 @@ .StorageContainerDatanodeProtocolProtos.PipelineReport; import java.io.IOException; -import java.util.Collection; import java.util.List; /** A server endpoint that acts as the communication layer for Ozone @@ -62,22 +60,6 @@ void submitRequest(ContainerCommandRequestProto request, */ boolean isExist(HddsProtos.PipelineID pipelineId); - - /** - * Join a new pipeline. - */ - default void addGroup(HddsProtos.PipelineID pipelineId, - Collection peers) throws IOException { - } - - - /** - * Exit a pipeline. - */ - default void removeGroup(HddsProtos.PipelineID pipelineId) - throws IOException { - } - /** * Get pipeline report for the XceiverServer instance. * @return list of report for each pipeline. diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java index a76944b9f070..1146394fee91 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java @@ -67,7 +67,6 @@ import java.io.File; import java.io.IOException; -import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -621,41 +620,6 @@ public List getPipelineIds() { return pipelineIDs; } - @Override - public void addGroup(HddsProtos.PipelineID pipelineId, - Collection peers) throws IOException { - final PipelineID pipelineID = PipelineID.getFromProtobuf(pipelineId); - final RaftGroupId groupId = RaftGroupId.valueOf(pipelineID.getId()); - final RaftGroup group = RatisHelper.newRaftGroup(groupId, peers); - GroupManagementRequest request = GroupManagementRequest.newAdd( - clientId, server.getId(), nextCallId(), group); - - RaftClientReply reply; - try { - reply = server.groupManagement(request); - } catch (Exception e) { - throw new IOException(e.getMessage(), e); - } - processReply(reply); - } - - @Override - public void removeGroup(HddsProtos.PipelineID pipelineId) - throws IOException { - GroupManagementRequest request = GroupManagementRequest.newRemove( - clientId, server.getId(), nextCallId(), - RaftGroupId.valueOf(PipelineID.getFromProtobuf(pipelineId).getId()), - true); - - RaftClientReply reply; - try { - reply = server.groupManagement(request); - } catch (Exception e) { - throw new IOException(e.getMessage(), e); - } - processReply(reply); - } - void handleNodeSlowness(RaftGroupId groupId, RoleInfoProto roleInfoProto) { handlePipelineFailure(groupId, roleInfoProto); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ClosePipelineCommand.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ClosePipelineCommand.java deleted file mode 100644 index 1f75bc375ff1..000000000000 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/ClosePipelineCommand.java +++ /dev/null @@ -1,73 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - *

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

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package org.apache.hadoop.ozone.protocol.commands; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.ClosePipelineCommandProto; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMCommandProto; -import org.apache.hadoop.hdds.scm.pipeline.PipelineID; - -/** - * Asks datanode to close a pipeline. - */ -public class ClosePipelineCommand - extends SCMCommand { - - private final PipelineID pipelineID; - - public ClosePipelineCommand(final PipelineID pipelineID) { - super(); - this.pipelineID = pipelineID; - } - - public ClosePipelineCommand(long cmdId, final PipelineID pipelineID) { - super(cmdId); - this.pipelineID = pipelineID; - } - - /** - * Returns the type of this command. - * - * @return Type - */ - @Override - public SCMCommandProto.Type getType() { - return SCMCommandProto.Type.closePipelineCommand; - } - - @Override - public ClosePipelineCommandProto getProto() { - ClosePipelineCommandProto.Builder builder = - ClosePipelineCommandProto.newBuilder(); - builder.setCmdId(getId()); - builder.setPipelineID(pipelineID.getProtobuf()); - return builder.build(); - } - - public static ClosePipelineCommand getFromProtobuf( - ClosePipelineCommandProto createPipelineProto) { - Preconditions.checkNotNull(createPipelineProto); - return new ClosePipelineCommand(createPipelineProto.getCmdId(), - PipelineID.getFromProtobuf(createPipelineProto.getPipelineID())); - } - - public PipelineID getPipelineID() { - return pipelineID; - } -} diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CreatePipelineCommand.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CreatePipelineCommand.java deleted file mode 100644 index 9e22cbcce330..000000000000 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CreatePipelineCommand.java +++ /dev/null @@ -1,100 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - *

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

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package org.apache.hadoop.ozone.protocol.commands; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.hdds.protocol.DatanodeDetails; -import org.apache.hadoop.hdds.protocol.proto. - StorageContainerDatanodeProtocolProtos.CreatePipelineCommandProto; -import org.apache.hadoop.hdds.protocol.proto. - StorageContainerDatanodeProtocolProtos.SCMCommandProto; -import org.apache.hadoop.hdds.scm.pipeline.PipelineID; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; - -import java.util.List; -import java.util.stream.Collectors; - -/** - * Asks datanode to create a pipeline. - */ -public class CreatePipelineCommand - extends SCMCommand { - - private final PipelineID pipelineID; - private final ReplicationFactor factor; - private final ReplicationType type; - private final List nodelist; - - public CreatePipelineCommand(final PipelineID pipelineID, - final ReplicationType type, final ReplicationFactor factor, - final List datanodeList) { - super(); - this.pipelineID = pipelineID; - this.factor = factor; - this.type = type; - this.nodelist = datanodeList; - } - - public CreatePipelineCommand(long cmdId, final PipelineID pipelineID, - final ReplicationType type, final ReplicationFactor factor, - final List datanodeList) { - super(cmdId); - this.pipelineID = pipelineID; - this.factor = factor; - this.type = type; - this.nodelist = datanodeList; - } - - /** - * Returns the type of this command. - * - * @return Type - */ - @Override - public SCMCommandProto.Type getType() { - return SCMCommandProto.Type.createPipelineCommand; - } - - @Override - public CreatePipelineCommandProto getProto() { - return CreatePipelineCommandProto.newBuilder() - .setCmdId(getId()) - .setPipelineID(pipelineID.getProtobuf()) - .setFactor(factor) - .setType(type) - .addAllDatanode(nodelist.stream() - .map(DatanodeDetails::getProtoBufMessage) - .collect(Collectors.toList())) - .build(); - } - - public static CreatePipelineCommand getFromProtobuf( - CreatePipelineCommandProto createPipelineProto) { - Preconditions.checkNotNull(createPipelineProto); - return new CreatePipelineCommand(createPipelineProto.getCmdId(), - PipelineID.getFromProtobuf(createPipelineProto.getPipelineID()), - createPipelineProto.getType(), createPipelineProto.getFactor(), - createPipelineProto.getDatanodeList().stream() - .map(DatanodeDetails::getFromProtoBuf) - .collect(Collectors.toList())); - } - - public PipelineID getPipelineID() { - return pipelineID; - } -} diff --git a/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto b/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto index 8b272c82cc22..45a1db681542 100644 --- a/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto +++ b/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto @@ -283,8 +283,6 @@ message SCMCommandProto { closeContainerCommand = 3; deleteContainerCommand = 4; replicateContainerCommand = 5; - createPipelineCommand = 6; - closePipelineCommand = 7; } // TODO: once we start using protoc 3.x, refactor this message using "oneof" required Type commandType = 1; @@ -293,8 +291,6 @@ message SCMCommandProto { optional CloseContainerCommandProto closeContainerCommandProto = 4; optional DeleteContainerCommandProto deleteContainerCommandProto = 5; optional ReplicateContainerCommandProto replicateContainerCommandProto = 6; - optional CreatePipelineCommandProto createPipelineCommandProto = 7; - optional ClosePipelineCommandProto closePipelineCommandProto = 8; } /** @@ -363,25 +359,6 @@ message ReplicateContainerCommandProto { required int64 cmdId = 3; } -/** -This command asks the datanode to create a pipeline. -*/ -message CreatePipelineCommandProto { - required PipelineID pipelineID = 1; - required ReplicationType type = 2; - required ReplicationFactor factor = 3; - repeated DatanodeDetailsProto datanode = 4; - required int64 cmdId = 5; -} - -/** -This command asks the datanode to close a pipeline. -*/ -message ClosePipelineCommandProto { - required PipelineID pipelineID = 1; - required int64 cmdId = 2; -} - /** * Protocol used from a datanode to StorageContainerManager. * diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java index b7a7525cbaca..845bdf154b9d 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java @@ -24,7 +24,6 @@ import java.util.Objects; import java.util.concurrent.TimeUnit; import javax.management.ObjectName; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.StorageUnit; import org.apache.hadoop.hdds.client.BlockID; @@ -197,8 +196,6 @@ public AllocatedBlock allocateBlock(final long size, ReplicationType type, // TODO: #CLUTIL Remove creation logic when all replication types and // factors are handled by pipeline creator pipeline = pipelineManager.createPipeline(type, factor); - // wait until pipeline is ready - pipelineManager.waitPipelineReady(pipeline.getId(), 0); } catch (IOException e) { LOG.warn("Pipeline creation failed for type:{} factor:{}. Retrying " + "get pipelines call once.", type, factor, e); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java index cefc185c58ca..7dde8d75f94d 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java @@ -253,7 +253,6 @@ ContainerInfo allocateContainer(final PipelineManager pipelineManager, // TODO: #CLUTIL remove creation logic when all replication types and // factors are handled by pipeline creator job. pipeline = pipelineManager.createPipeline(type, replicationFactor); - pipelineManager.waitPipelineReady(pipeline.getId(), 0); } catch (IOException e) { final List pipelines = pipelineManager .getPipelines(type, replicationFactor, Pipeline.PipelineState.OPEN); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java index 6de05fd38019..43d396e0cb12 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java @@ -21,7 +21,6 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.scm.block.PendingDeleteStatusList; -import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager.SafeModeStatus; import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler; import org.apache.hadoop.hdds.scm.container.ContainerID; @@ -98,14 +97,15 @@ public final class SCMEvents { new TypedEvent<>(PipelineReportFromDatanode.class, "Pipeline_Report"); /** - * Open pipeline event sent by PipelineReportHandler. This event is + * PipelineReport processed by pipeline report handler. This event is * received by HealthyPipelineSafeModeRule. */ - public static final TypedEvent - OPEN_PIPELINE = new TypedEvent<>(Pipeline.class, "Open_Pipeline"); + public static final TypedEvent + PROCESSED_PIPELINE_REPORT = new TypedEvent<>( + PipelineReportFromDatanode.class, "Processed_Pipeline_Report"); /** - * PipelineActions are sent by Datanode to close a pipeline. It's received by + * PipelineActions are sent by Datanode. This event is received by * SCMDatanodeHeartbeatDispatcher and PIPELINE_ACTIONS event is generated. */ public static final TypedEvent @@ -113,7 +113,7 @@ public final class SCMEvents { "Pipeline_Actions"); /** - * A Command status report will be sent by datanodes. This report is received + * A Command status report will be sent by datanodes. This repoort is received * by SCMDatanodeHeartbeatDispatcher and CommandReport event is generated. */ public static final TypedEvent diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java index 4065c2f3e7bb..687356648c3c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/BackgroundPipelineCreator.java @@ -26,7 +26,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -42,7 +41,6 @@ class BackgroundPipelineCreator { private final AtomicBoolean isPipelineCreatorRunning; private final PipelineManager pipelineManager; private final Configuration conf; - private ScheduledFuture periodicTask; BackgroundPipelineCreator(PipelineManager pipelineManager, Scheduler scheduler, Configuration conf) { @@ -59,16 +57,13 @@ private boolean shouldSchedulePipelineCreator() { /** * Schedules a fixed interval job to create pipelines. */ - synchronized void startFixedIntervalPipelineCreator() { - if (periodicTask != null) { - return; - } + void startFixedIntervalPipelineCreator() { long intervalInMillis = conf .getTimeDuration(ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_INTERVAL, ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS); // TODO: #CLUTIL We can start the job asap - periodicTask = scheduler.scheduleWithFixedDelay(() -> { + scheduler.scheduleWithFixedDelay(() -> { if (!shouldSchedulePipelineCreator()) { return; } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineFactory.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineFactory.java index 86ad5eed9745..77e037a07117 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineFactory.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineFactory.java @@ -24,8 +24,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; import org.apache.hadoop.hdds.scm.node.NodeManager; -import org.apache.hadoop.hdds.server.events.EventPublisher; - +import org.apache.ratis.grpc.GrpcTlsConfig; import java.io.IOException; import java.util.HashMap; @@ -40,13 +39,12 @@ public final class PipelineFactory { private Map providers; PipelineFactory(NodeManager nodeManager, PipelineStateManager stateManager, - Configuration conf, EventPublisher eventPublisher) { + Configuration conf, GrpcTlsConfig tlsConfig) { providers = new HashMap<>(); providers.put(ReplicationType.STAND_ALONE, new SimplePipelineProvider(nodeManager)); providers.put(ReplicationType.RATIS, - new RatisPipelineProvider(nodeManager, stateManager, conf, - eventPublisher)); + new RatisPipelineProvider(nodeManager, stateManager, conf, tlsConfig)); } @VisibleForTesting @@ -65,11 +63,6 @@ public Pipeline create(ReplicationType type, ReplicationFactor factor, return providers.get(type).create(factor, nodes); } - public void close(ReplicationType type, Pipeline pipeline) - throws IOException { - providers.get(type).close(pipeline); - } - public void shutdown() { providers.values().forEach(provider -> provider.shutdown()); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java index 779008f9a460..9ba5f3189f76 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManager.java @@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; import org.apache.hadoop.hdds.scm.container.ContainerID; +import org.apache.ratis.grpc.GrpcTlsConfig; import java.io.Closeable; import java.io.IOException; @@ -49,9 +50,6 @@ Pipeline createPipeline(ReplicationType type, ReplicationFactor factor, List getPipelines(ReplicationType type, ReplicationFactor factor); - List getPipelines(ReplicationType type, - Pipeline.PipelineState state); - List getPipelines(ReplicationType type, ReplicationFactor factor, Pipeline.PipelineState state); @@ -97,14 +95,5 @@ void finalizeAndDestroyPipeline(Pipeline pipeline, boolean onTimeout) */ void deactivatePipeline(PipelineID pipelineID) throws IOException; - /** - * Wait a pipeline to be OPEN. - * - * @param pipelineID ID of the pipeline to wait for. - * @param timeout wait timeout(millisecond), if 0, use default timeout - * @throws IOException in case of any Exception, such as timeout - */ - default void waitPipelineReady(PipelineID pipelineID, long timeout) - throws IOException { - } + GrpcTlsConfig getGrpcTlsConfig(); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java index c00ff7881b8e..a0ce21626723 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java @@ -33,7 +33,5 @@ public interface PipelineProvider { Pipeline create(ReplicationFactor factor, List nodes); - void close(Pipeline pipeline) throws IOException; - void shutdown(); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java index a7e2bf12197b..b8cb7b4246c7 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java @@ -19,23 +19,18 @@ package org.apache.hadoop.hdds.scm.pipeline; import java.io.IOException; +import java.util.Objects; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.protocol.DatanodeDetails; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.PipelineReport; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.PipelineReportsProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReport; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto; import org.apache.hadoop.hdds.scm.events.SCMEvents; import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager; -import org.apache.hadoop.hdds.scm.server - .SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode; +import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode; import org.apache.hadoop.hdds.server.events.EventHandler; import org.apache.hadoop.hdds.server.events.EventPublisher; -import org.apache.hadoop.ozone.protocol.commands.ClosePipelineCommand; -import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,8 +50,10 @@ public class PipelineReportHandler implements private final boolean pipelineAvailabilityCheck; public PipelineReportHandler(SCMSafeModeManager scmSafeModeManager, - PipelineManager pipelineManager, Configuration conf) { + PipelineManager pipelineManager, + Configuration conf) { Preconditions.checkNotNull(pipelineManager); + Objects.requireNonNull(scmSafeModeManager); this.scmSafeModeManager = scmSafeModeManager; this.pipelineManager = pipelineManager; this.conf = conf; @@ -79,45 +76,48 @@ public void onMessage(PipelineReportFromDatanode pipelineReportFromDatanode, } for (PipelineReport report : pipelineReport.getPipelineReportList()) { try { - processPipelineReport(report, dn, publisher); + processPipelineReport(report, dn); } catch (IOException e) { LOGGER.error("Could not process pipeline report={} from dn={} {}", report, dn, e); } } + if (pipelineAvailabilityCheck && scmSafeModeManager.getInSafeMode()) { + publisher.fireEvent(SCMEvents.PROCESSED_PIPELINE_REPORT, + pipelineReportFromDatanode); + } } - private void processPipelineReport(PipelineReport report, DatanodeDetails dn, - EventPublisher publisher) throws IOException { + private void processPipelineReport(PipelineReport report, DatanodeDetails dn) + throws IOException { PipelineID pipelineID = PipelineID.getFromProtobuf(report.getPipelineID()); Pipeline pipeline; try { pipeline = pipelineManager.getPipeline(pipelineID); } catch (PipelineNotFoundException e) { - final ClosePipelineCommand closeCommand = - new ClosePipelineCommand(pipelineID); - final CommandForDatanode datanodeCommand = - new CommandForDatanode<>(dn.getUuid(), closeCommand); - publisher.fireEvent(SCMEvents.DATANODE_COMMAND, datanodeCommand); + RatisPipelineUtils.destroyPipeline(dn, pipelineID, conf, + pipelineManager.getGrpcTlsConfig()); return; } pipeline.reportDatanode(dn); - // ONE replica pipeline doesn't have leader flag - if (report.getIsLeader() || - pipeline.getFactor() == HddsProtos.ReplicationFactor.ONE) { + if (report.getIsLeader()) { pipeline.setLeaderId(dn.getUuid()); } + if ((pipeline.getPipelineState() == Pipeline.PipelineState.ALLOCATED) + && pipeline.isHealthy()) { + pipelineManager.openPipeline(pipelineID); + } if (pipeline.getPipelineState() == Pipeline.PipelineState.ALLOCATED) { - LOGGER.info("Pipeline {} {} reported by {}", pipeline.getFactor(), - pipeline.getId(), dn); - if (pipeline.isHealthy()) { + + + if (report.getIsLeader()) { + // Pipeline reported as the leader + pipeline.setLeaderId(dn.getUuid()); pipelineManager.openPipeline(pipelineID); - if (pipelineAvailabilityCheck && scmSafeModeManager.getInSafeMode()) { - publisher.fireEvent(SCMEvents.OPEN_PIPELINE, pipeline); - } } } + pipeline.reportDatanode(dn); } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManager.java index 180d0bfa388c..2410b544581c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineStateManager.java @@ -129,9 +129,9 @@ Pipeline openPipeline(PipelineID pipelineId) throws IOException { throw new IOException("Closed pipeline can not be opened"); } if (pipeline.getPipelineState() == PipelineState.ALLOCATED) { + pipeline = pipelineStateMap.updatePipelineState( + pipelineId, PipelineState.OPEN); LOG.info("Pipeline {} moved to OPEN state", pipeline.toString()); - pipeline = pipelineStateMap - .updatePipelineState(pipelineId, PipelineState.OPEN); } return pipeline; } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java index a2ee50a35ff0..94443dd7a193 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java @@ -24,28 +24,37 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState; import org.apache.hadoop.hdds.scm.ScmConfigKeys; -import org.apache.hadoop.hdds.scm.container.placement.algorithms - .ContainerPlacementPolicy; -import org.apache.hadoop.hdds.scm.container.placement.algorithms - .SCMContainerPlacementRandom; -import org.apache.hadoop.hdds.scm.events.SCMEvents; +import org.apache.hadoop.hdds.scm.client.HddsClientUtils; +import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicy; +import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRandom; import org.apache.hadoop.hdds.scm.node.NodeManager; import org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState; -import org.apache.hadoop.hdds.server.events.EventPublisher; -import org.apache.hadoop.ozone.protocol.commands.ClosePipelineCommand; -import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode; -import org.apache.hadoop.ozone.protocol.commands.CreatePipelineCommand; +import org.apache.hadoop.io.MultipleIOException; +import org.apache.hadoop.hdds.ratis.RatisHelper; +import org.apache.ratis.client.RaftClient; +import org.apache.ratis.grpc.GrpcTlsConfig; +import org.apache.ratis.protocol.RaftClientReply; +import org.apache.ratis.protocol.RaftGroup; +import org.apache.ratis.protocol.RaftPeer; +import org.apache.ratis.retry.RetryPolicy; +import org.apache.ratis.rpc.SupportedRpcType; +import org.apache.ratis.util.TimeDuration; +import org.apache.ratis.util.function.CheckedBiConsumer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ForkJoinPool; import java.util.concurrent.ForkJoinWorkerThread; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -60,7 +69,6 @@ public class RatisPipelineProvider implements PipelineProvider { private final NodeManager nodeManager; private final PipelineStateManager stateManager; private final Configuration conf; - private final EventPublisher eventPublisher; // Set parallelism at 3, as now in Ratis we create 1 and 3 node pipelines. private final int parallelismForPool = 3; @@ -75,14 +83,15 @@ public class RatisPipelineProvider implements PipelineProvider { private final ForkJoinPool forkJoinPool = new ForkJoinPool( parallelismForPool, factory, null, false); + private final GrpcTlsConfig tlsConfig; RatisPipelineProvider(NodeManager nodeManager, PipelineStateManager stateManager, Configuration conf, - EventPublisher eventPublisher) { + GrpcTlsConfig tlsConfig) { this.nodeManager = nodeManager; this.stateManager = stateManager; this.conf = conf; - this.eventPublisher = eventPublisher; + this.tlsConfig = tlsConfig; } @@ -144,27 +153,8 @@ public Pipeline create(ReplicationFactor factor) throws IOException { throw new InsufficientDatanodesException(e); } - Pipeline pipeline = Pipeline.newBuilder() - .setId(PipelineID.randomId()) - .setState(PipelineState.ALLOCATED) - .setType(ReplicationType.RATIS) - .setFactor(factor) - .setNodes(dns) - .build(); - - // Send command to datanodes to create pipeline - final CreatePipelineCommand createCommand = - new CreatePipelineCommand(pipeline.getId(), pipeline.getType(), - factor, dns); - - dns.stream().forEach(node -> { - final CommandForDatanode datanodeCommand = - new CommandForDatanode<>(node.getUuid(), createCommand); - LOG.info("Send pipeline:{} create command to datanode {}", - pipeline.getId(), datanodeCommand.getDatanodeId()); - eventPublisher.fireEvent(SCMEvents.DATANODE_COMMAND, datanodeCommand); - }); - + Pipeline pipeline = create(factor, dns); + initializePipeline(pipeline); return pipeline; } @@ -191,22 +181,69 @@ public void shutdown() { } } - /** - * Removes pipeline from SCM. Sends command to destroy pipeline on all - * the datanodes. - * - * @param pipeline - Pipeline to be destroyed - * @throws IOException - */ - public void close(Pipeline pipeline) { - final ClosePipelineCommand closeCommand = - new ClosePipelineCommand(pipeline.getId()); - pipeline.getNodes().stream().forEach(node -> { - final CommandForDatanode datanodeCommand = - new CommandForDatanode<>(node.getUuid(), closeCommand); - LOG.info("Send pipeline:{} close command to datanode {}", - pipeline.getId(), datanodeCommand.getDatanodeId()); - eventPublisher.fireEvent(SCMEvents.DATANODE_COMMAND, datanodeCommand); - }); + protected void initializePipeline(Pipeline pipeline) throws IOException { + final RaftGroup group = RatisHelper.newRaftGroup(pipeline); + if (LOG.isDebugEnabled()) { + LOG.debug("creating pipeline:{} with {}", pipeline.getId(), group); + } + callRatisRpc(pipeline.getNodes(), + (raftClient, peer) -> { + RaftClientReply reply = raftClient.groupAdd(group, peer.getId()); + if (reply == null || !reply.isSuccess()) { + String msg = "Pipeline initialization failed for pipeline:" + + pipeline.getId() + " node:" + peer.getId(); + LOG.error(msg); + throw new IOException(msg); + } + }); + } + + private void callRatisRpc(List datanodes, + CheckedBiConsumer< RaftClient, RaftPeer, IOException> rpc) + throws IOException { + if (datanodes.isEmpty()) { + return; + } + + final String rpcType = conf + .get(ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY, + ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT); + final RetryPolicy retryPolicy = RatisHelper.createRetryPolicy(conf); + final List< IOException > exceptions = + Collections.synchronizedList(new ArrayList<>()); + final int maxOutstandingRequests = + HddsClientUtils.getMaxOutstandingRequests(conf); + final TimeDuration requestTimeout = + RatisHelper.getClientRequestTimeout(conf); + try { + forkJoinPool.submit(() -> { + datanodes.parallelStream().forEach(d -> { + final RaftPeer p = RatisHelper.toRaftPeer(d); + try (RaftClient client = RatisHelper + .newRaftClient(SupportedRpcType.valueOfIgnoreCase(rpcType), p, + retryPolicy, maxOutstandingRequests, tlsConfig, + requestTimeout)) { + rpc.accept(client, p); + } catch (IOException ioe) { + String errMsg = + "Failed invoke Ratis rpc " + rpc + " for " + d.getUuid(); + LOG.error(errMsg, ioe); + exceptions.add(new IOException(errMsg, ioe)); + } + }); + }).get(); + } catch (ExecutionException | RejectedExecutionException ex) { + LOG.error(ex.getClass().getName() + " exception occurred during " + + "createPipeline", ex); + throw new IOException(ex.getClass().getName() + " exception occurred " + + "during createPipeline", ex); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + throw new IOException("Interrupt exception occurred during " + + "createPipeline", ex); + } + if (!exceptions.isEmpty()) { + throw MultipleIOException.createIOException(exceptions); + } } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineUtils.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineUtils.java new file mode 100644 index 000000000000..497e717b1f70 --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineUtils.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdds.scm.pipeline; + +import java.io.IOException; + +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.client.HddsClientUtils; +import org.apache.hadoop.hdds.ratis.RatisHelper; +import org.apache.ratis.client.RaftClient; +import org.apache.ratis.grpc.GrpcTlsConfig; +import org.apache.ratis.protocol.RaftGroup; +import org.apache.ratis.protocol.RaftGroupId; +import org.apache.ratis.protocol.RaftPeer; +import org.apache.ratis.retry.RetryPolicy; +import org.apache.ratis.rpc.SupportedRpcType; +import org.apache.ratis.util.TimeDuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Utility class for Ratis pipelines. Contains methods to create and destroy + * ratis pipelines. + */ +public final class RatisPipelineUtils { + + private static final Logger LOG = + LoggerFactory.getLogger(RatisPipelineUtils.class); + + private RatisPipelineUtils() { + } + /** + * Removes pipeline from SCM. Sends ratis command to destroy pipeline on all + * the datanodes. + * + * @param pipeline - Pipeline to be destroyed + * @param ozoneConf - Ozone configuration + * @param grpcTlsConfig + * @throws IOException + */ + public static void destroyPipeline(Pipeline pipeline, Configuration ozoneConf, + GrpcTlsConfig grpcTlsConfig) { + final RaftGroup group = RatisHelper.newRaftGroup(pipeline); + if (LOG.isDebugEnabled()) { + LOG.debug("destroying pipeline:{} with {}", pipeline.getId(), group); + } + for (DatanodeDetails dn : pipeline.getNodes()) { + try { + destroyPipeline(dn, pipeline.getId(), ozoneConf, grpcTlsConfig); + } catch (IOException e) { + LOG.warn("Pipeline destroy failed for pipeline={} dn={}", + pipeline.getId(), dn); + } + } + } + + /** + * Sends ratis command to destroy pipeline on the given datanode. + * + * @param dn - Datanode on which pipeline needs to be destroyed + * @param pipelineID - ID of pipeline to be destroyed + * @param ozoneConf - Ozone configuration + * @param grpcTlsConfig - grpc tls configuration + * @throws IOException + */ + static void destroyPipeline(DatanodeDetails dn, PipelineID pipelineID, + Configuration ozoneConf, GrpcTlsConfig grpcTlsConfig) throws IOException { + final String rpcType = ozoneConf + .get(ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY, + ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT); + final RetryPolicy retryPolicy = RatisHelper.createRetryPolicy(ozoneConf); + final RaftPeer p = RatisHelper.toRaftPeer(dn); + final int maxOutstandingRequests = + HddsClientUtils.getMaxOutstandingRequests(ozoneConf); + final TimeDuration requestTimeout = + RatisHelper.getClientRequestTimeout(ozoneConf); + try(RaftClient client = RatisHelper + .newRaftClient(SupportedRpcType.valueOfIgnoreCase(rpcType), p, + retryPolicy, maxOutstandingRequests, grpcTlsConfig, + requestTimeout)) { + client.groupRemove(RaftGroupId.valueOf(pipelineID.getId()), + true, p.getId()); + } + } +} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java index 00a442983cef..0964f6d4db29 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java @@ -21,7 +21,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; @@ -38,7 +37,7 @@ import org.apache.hadoop.hdds.utils.MetadataStore; import org.apache.hadoop.hdds.utils.MetadataStoreBuilder; import org.apache.hadoop.hdds.utils.Scheduler; -import org.apache.hadoop.util.Time; +import org.apache.ratis.grpc.GrpcTlsConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -82,18 +81,18 @@ public class SCMPipelineManager implements PipelineManager { private final NodeManager nodeManager; private final SCMPipelineMetrics metrics; private final Configuration conf; - private long pipelineWaitDefaultTimeout; // Pipeline Manager MXBean private ObjectName pmInfoBean; + private GrpcTlsConfig grpcTlsConfig; public SCMPipelineManager(Configuration conf, NodeManager nodeManager, - EventPublisher eventPublisher) + EventPublisher eventPublisher, GrpcTlsConfig grpcTlsConfig) throws IOException { this.lock = new ReentrantReadWriteLock(); this.conf = conf; this.stateManager = new PipelineStateManager(conf); this.pipelineFactory = new PipelineFactory(nodeManager, stateManager, - conf, eventPublisher); + conf, grpcTlsConfig); // TODO: See if thread priority needs to be set for these threads scheduler = new Scheduler("RatisPipelineUtilsThread", false, 1); this.backgroundPipelineCreator = @@ -114,11 +113,8 @@ public SCMPipelineManager(Configuration conf, NodeManager nodeManager, this.metrics = SCMPipelineMetrics.create(); this.pmInfoBean = MBeans.register("SCMPipelineManager", "SCMPipelineManagerInfo", this); - this.pipelineWaitDefaultTimeout = conf.getTimeDuration( - HddsConfigKeys.HDDS_PIPELINE_REPORT_INTERVAL, - HddsConfigKeys.HDDS_PIPELINE_REPORT_INTERVAL_DEFAULT, - TimeUnit.MILLISECONDS); initializePipelineState(); + this.grpcTlsConfig = grpcTlsConfig; } public PipelineStateManager getStateManager() { @@ -152,8 +148,8 @@ private void initializePipelineState() throws IOException { } @Override - public synchronized Pipeline createPipeline(ReplicationType type, - ReplicationFactor factor) throws IOException { + public synchronized Pipeline createPipeline( + ReplicationType type, ReplicationFactor factor) throws IOException { lock.writeLock().lock(); try { Pipeline pipeline = pipelineFactory.create(type, factor); @@ -161,11 +157,8 @@ public synchronized Pipeline createPipeline(ReplicationType type, pipeline.getProtobufMessage().toByteArray()); stateManager.addPipeline(pipeline); nodeManager.addPipeline(pipeline); - metrics.incNumPipelineAllocated(); - if (pipeline.isOpen()) { - metrics.incNumPipelineCreated(); - metrics.createPerPipelineMetrics(pipeline); - } + metrics.incNumPipelineCreated(); + metrics.createPerPipelineMetrics(pipeline); return pipeline; } catch (InsufficientDatanodesException idEx) { throw idEx; @@ -232,16 +225,6 @@ public List getPipelines(ReplicationType type, } } - public List getPipelines(ReplicationType type, - Pipeline.PipelineState state) { - lock.readLock().lock(); - try { - return stateManager.getPipelines(type, state); - } finally { - lock.readLock().unlock(); - } - } - @Override public List getPipelines(ReplicationType type, ReplicationFactor factor, Pipeline.PipelineState state) { @@ -310,7 +293,6 @@ public void openPipeline(PipelineID pipelineId) throws IOException { lock.writeLock().lock(); try { Pipeline pipeline = stateManager.openPipeline(pipelineId); - metrics.incNumPipelineCreated(); metrics.createPerPipelineMetrics(pipeline); } finally { lock.writeLock().unlock(); @@ -397,45 +379,6 @@ public void deactivatePipeline(PipelineID pipelineID) stateManager.deactivatePipeline(pipelineID); } - /** - * Wait a pipeline to be OPEN. - * - * @param pipelineID ID of the pipeline to wait for. - * @param timeout wait timeout, millisecond, 0 to use default value - * @throws IOException in case of any Exception, such as timeout - */ - @Override - public void waitPipelineReady(PipelineID pipelineID, long timeout) - throws IOException { - long st = Time.monotonicNow(); - if (timeout == 0) { - timeout = pipelineWaitDefaultTimeout; - } - - boolean ready; - Pipeline pipeline; - do { - try { - pipeline = stateManager.getPipeline(pipelineID); - } catch (PipelineNotFoundException e) { - throw new PipelineNotFoundException(String.format( - "Pipeline %s cannot be found", pipelineID)); - } - ready = pipeline.isOpen(); - if (!ready) { - try { - Thread.sleep((long)100); - } catch (InterruptedException e) { - } - } - } while (!ready && Time.monotonicNow() - st < timeout); - - if (!ready) { - throw new IOException(String.format("Pipeline %s is not ready in %d ms", - pipelineID, timeout)); - } - } - /** * Moves the pipeline to CLOSED state and sends close container command for * all the containers in the pipeline. @@ -465,7 +408,7 @@ private void finalizePipeline(PipelineID pipelineId) throws IOException { * @throws IOException */ private void destroyPipeline(Pipeline pipeline) throws IOException { - pipelineFactory.close(pipeline.getType(), pipeline); + RatisPipelineUtils.destroyPipeline(pipeline, conf, grpcTlsConfig); // remove the pipeline from the pipeline manager removePipeline(pipeline.getId()); triggerPipelineCreation(); @@ -497,6 +440,11 @@ public void incNumBlocksAllocatedMetric(PipelineID id) { metrics.incNumBlocksAllocated(id); } + @Override + public GrpcTlsConfig getGrpcTlsConfig() { + return grpcTlsConfig; + } + @Override public void close() throws IOException { if (scheduler != null) { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineMetrics.java index 40a6f290b54e..b6a14450f22d 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineMetrics.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineMetrics.java @@ -47,7 +47,6 @@ public final class SCMPipelineMetrics implements MetricsSource { private MetricsRegistry registry; - private @Metric MutableCounterLong numPipelineAllocated; private @Metric MutableCounterLong numPipelineCreated; private @Metric MutableCounterLong numPipelineCreationFailed; private @Metric MutableCounterLong numPipelineDestroyed; @@ -85,7 +84,6 @@ public void unRegister() { @SuppressWarnings("SuspiciousMethodCalls") public void getMetrics(MetricsCollector collector, boolean all) { MetricsRecordBuilder recordBuilder = collector.addRecord(SOURCE_NAME); - numPipelineAllocated.snapshot(recordBuilder, true); numPipelineCreated.snapshot(recordBuilder, true); numPipelineCreationFailed.snapshot(recordBuilder, true); numPipelineDestroyed.snapshot(recordBuilder, true); @@ -119,14 +117,6 @@ void incNumBlocksAllocated(PipelineID pipelineID) { MutableCounterLong::incr); } - /** - * Increments number of pipeline allocation count, including succeeded - * and failed. - */ - void incNumPipelineAllocated() { - numPipelineAllocated.incr(); - } - /** * Increments number of successful pipeline creation count. */ diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java index 00cb7ae164b3..ab98dfa3ed7b 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java @@ -73,11 +73,6 @@ public Pipeline create(ReplicationFactor factor, .build(); } - @Override - public void close(Pipeline pipeline) throws IOException { - - } - @Override public void shutdown() { // Do nothing. diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/HealthyPipelineSafeModeRule.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/HealthyPipelineSafeModeRule.java index 9b19acf8f78b..2f9a66f27b52 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/HealthyPipelineSafeModeRule.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/HealthyPipelineSafeModeRule.java @@ -17,19 +17,27 @@ */ package org.apache.hadoop.hdds.scm.safemode; +import java.util.HashMap; +import java.util.Map; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReport; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto; import org.apache.hadoop.hdds.scm.events.SCMEvents; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; -import com.google.common.base.Preconditions; +import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException; +import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode; import org.apache.hadoop.hdds.server.events.EventQueue; import org.apache.hadoop.hdds.server.events.TypedEvent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; /** * Class defining Safe mode exit criteria for Pipelines. @@ -39,55 +47,43 @@ * through in a cluster. */ public class HealthyPipelineSafeModeRule - extends SafeModeExitRule{ + extends SafeModeExitRule{ public static final Logger LOG = LoggerFactory.getLogger(HealthyPipelineSafeModeRule.class); + private final PipelineManager pipelineManager; private int healthyPipelineThresholdCount; private int currentHealthyPipelineCount = 0; + private final Map processedPipelines = new HashMap<>(); private final double healthyPipelinesPercent; HealthyPipelineSafeModeRule(String ruleName, EventQueue eventQueue, PipelineManager pipelineManager, SCMSafeModeManager manager, Configuration configuration) { super(manager, ruleName, eventQueue); + this.pipelineManager = pipelineManager; healthyPipelinesPercent = configuration.getDouble(HddsConfigKeys. HDDS_SCM_SAFEMODE_HEALTHY_PIPELINE_THRESHOLD_PCT, HddsConfigKeys. HDDS_SCM_SAFEMODE_HEALTHY_PIPELINE_THRESHOLD_PCT_DEFAULT); - int minHealthyPipelines = 0; - - boolean createPipelineInSafemode = configuration.getBoolean( - HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, - HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION_DEFAULT); - - if (createPipelineInSafemode) { - minHealthyPipelines = - configuration.getInt(HddsConfigKeys.HDDS_SCM_SAFEMODE_MIN_PIPELINE, - HddsConfigKeys.HDDS_SCM_SAFEMODE_MIN_PIPELINE_DEFAULT); - } - Preconditions.checkArgument( (healthyPipelinesPercent >= 0.0 && healthyPipelinesPercent <= 1.0), HddsConfigKeys. HDDS_SCM_SAFEMODE_HEALTHY_PIPELINE_THRESHOLD_PCT + " value should be >= 0.0 and <= 1.0"); - // We want to wait for RATIS THREE factor write pipelines - int pipelineCount = pipelineManager.getPipelines( - HddsProtos.ReplicationType.RATIS, HddsProtos.ReplicationFactor.THREE, - Pipeline.PipelineState.OPEN).size() + + // As we want to wait for 3 node pipelines + int pipelineCount = pipelineManager.getPipelines(HddsProtos.ReplicationType.RATIS, - HddsProtos.ReplicationFactor.THREE, - Pipeline.PipelineState.ALLOCATED).size(); + HddsProtos.ReplicationFactor.THREE).size(); // This value will be zero when pipeline count is 0. // On a fresh installed cluster, there will be zero pipelines in the SCM // pipeline DB. - healthyPipelineThresholdCount = Math.max(minHealthyPipelines, - (int) Math.ceil(healthyPipelinesPercent * pipelineCount)); + healthyPipelineThresholdCount = + (int) Math.ceil(healthyPipelinesPercent * pipelineCount); LOG.info(" Total pipeline count is {}, healthy pipeline " + "threshold count is {}", pipelineCount, healthyPipelineThresholdCount); @@ -103,8 +99,8 @@ public void setHealthyPipelineThresholdCount(int actualPipelineCount) { } @Override - protected TypedEvent getEventType() { - return SCMEvents.OPEN_PIPELINE; + protected TypedEvent getEventType() { + return SCMEvents.PROCESSED_PIPELINE_REPORT; } @Override @@ -116,18 +112,38 @@ protected boolean validate() { } @Override - protected void process(Pipeline pipeline) { + protected void process(PipelineReportFromDatanode + pipelineReportFromDatanode) { // When SCM is in safe mode for long time, already registered - // datanode can send pipeline report again, or SCMPipelineManager will - // create new pipelines. - Preconditions.checkNotNull(pipeline); - if (pipeline.getType() == HddsProtos.ReplicationType.RATIS && - pipeline.getFactor() == HddsProtos.ReplicationFactor.THREE) { - getSafeModeMetrics().incCurrentHealthyPipelinesCount(); - currentHealthyPipelineCount++; + // datanode can send pipeline report again, then pipeline handler fires + // processed report event, we should not consider this pipeline report + // from datanode again during threshold calculation. + Preconditions.checkNotNull(pipelineReportFromDatanode); + + PipelineReportsProto pipelineReport = + pipelineReportFromDatanode.getReport(); + + for (PipelineReport report : pipelineReport.getPipelineReportList()) { + PipelineID pipelineID = PipelineID.getFromProtobuf( + report.getPipelineID()); + Pipeline pipeline; + try { + pipeline = pipelineManager.getPipeline(pipelineID); + } catch (PipelineNotFoundException e) { + continue; + } + + if (!processedPipelines.containsKey(pipelineID)) { + if (pipeline.getFactor() == HddsProtos.ReplicationFactor.THREE && + report.getIsLeader()) { + // If the pipeline gets reported with a leader we mark it as healthy + currentHealthyPipelineCount++; + getSafeModeMetrics().incCurrentHealthyPipelinesCount(); + processedPipelines.put(pipelineID, Boolean.TRUE); + } + } } - if (scmInSafeMode()) { SCMSafeModeManager.getLogger().info( "SCM in safe mode. Healthy pipelines reported count is {}, " + @@ -138,6 +154,7 @@ protected void process(Pipeline pipeline) { @Override protected void cleanup() { + processedPipelines.clear(); } @VisibleForTesting diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/OneReplicaPipelineSafeModeRule.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/OneReplicaPipelineSafeModeRule.java index 0783d0270a9e..841d8ff6654c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/OneReplicaPipelineSafeModeRule.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/OneReplicaPipelineSafeModeRule.java @@ -22,10 +22,17 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.protocol.proto + .StorageContainerDatanodeProtocolProtos.PipelineReport; +import org.apache.hadoop.hdds.protocol.proto + .StorageContainerDatanodeProtocolProtos.PipelineReportsProto; import org.apache.hadoop.hdds.scm.events.SCMEvents; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; +import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException; +import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher. + PipelineReportFromDatanode; import org.apache.hadoop.hdds.server.events.EventQueue; import org.apache.hadoop.hdds.server.events.TypedEvent; import org.slf4j.Logger; @@ -33,7 +40,6 @@ import java.util.HashSet; import java.util.Set; -import java.util.stream.Collectors; /** * This rule covers whether we have at least one datanode is reported for each @@ -41,14 +47,14 @@ * replica available for read when we exit safe mode. */ public class OneReplicaPipelineSafeModeRule extends - SafeModeExitRule { + SafeModeExitRule { private static final Logger LOG = LoggerFactory.getLogger(OneReplicaPipelineSafeModeRule.class); private int thresholdCount; private Set reportedPipelineIDSet = new HashSet<>(); - private Set oldPipelineIDSet; + private final PipelineManager pipelineManager; private int currentReportedPipelineCount = 0; @@ -56,6 +62,7 @@ public OneReplicaPipelineSafeModeRule(String ruleName, EventQueue eventQueue, PipelineManager pipelineManager, SCMSafeModeManager safeModeManager, Configuration configuration) { super(safeModeManager, ruleName, eventQueue); + this.pipelineManager = pipelineManager; double percent = configuration.getDouble( @@ -68,25 +75,24 @@ public OneReplicaPipelineSafeModeRule(String ruleName, EventQueue eventQueue, HDDS_SCM_SAFEMODE_ONE_NODE_REPORTED_PIPELINE_PCT + " value should be >= 0.0 and <= 1.0"); - oldPipelineIDSet = pipelineManager.getPipelines( - HddsProtos.ReplicationType.RATIS, - HddsProtos.ReplicationFactor.THREE) - .stream().map(p -> p.getId()).collect(Collectors.toSet()); - int totalPipelineCount = oldPipelineIDSet.size(); + int totalPipelineCount = + pipelineManager.getPipelines(HddsProtos.ReplicationType.RATIS, + HddsProtos.ReplicationFactor.THREE).size(); thresholdCount = (int) Math.ceil(percent * totalPipelineCount); - LOG.info("Total pipeline count is {}, pipeline's with at least one " + + LOG.info(" Total pipeline count is {}, pipeline's with atleast one " + "datanode reported threshold count is {}", totalPipelineCount, thresholdCount); getSafeModeMetrics().setNumPipelinesWithAtleastOneReplicaReportedThreshold( thresholdCount); + } @Override - protected TypedEvent getEventType() { - return SCMEvents.OPEN_PIPELINE; + protected TypedEvent getEventType() { + return SCMEvents.PROCESSED_PIPELINE_REPORT; } @Override @@ -98,26 +104,40 @@ protected boolean validate() { } @Override - protected void process(Pipeline pipeline) { - Preconditions.checkNotNull(pipeline); - if (pipeline.getType() == HddsProtos.ReplicationType.RATIS && - pipeline.getFactor() == HddsProtos.ReplicationFactor.THREE && - !reportedPipelineIDSet.contains(pipeline.getId())) { - if (oldPipelineIDSet.contains(pipeline.getId())) { + protected void process(PipelineReportFromDatanode + pipelineReportFromDatanode) { + Pipeline pipeline; + Preconditions.checkNotNull(pipelineReportFromDatanode); + PipelineReportsProto pipelineReport = + pipelineReportFromDatanode.getReport(); + + for (PipelineReport report : pipelineReport.getPipelineReportList()) { + PipelineID pipelineID = PipelineID + .getFromProtobuf(report.getPipelineID()); + try { + pipeline = pipelineManager.getPipeline(pipelineID); + } catch (PipelineNotFoundException e) { + continue; + } + + if (pipeline.getFactor() == HddsProtos.ReplicationFactor.THREE && + !reportedPipelineIDSet.contains(pipelineID)) { + reportedPipelineIDSet.add(pipelineID); getSafeModeMetrics() .incCurrentHealthyPipelinesWithAtleastOneReplicaReportedCount(); - currentReportedPipelineCount++; - reportedPipelineIDSet.add(pipeline.getId()); } } + currentReportedPipelineCount = reportedPipelineIDSet.size(); + if (scmInSafeMode()) { SCMSafeModeManager.getLogger().info( - "SCM in safe mode. Pipelines with at least one datanode reported " + - "count is {}, required at least one datanode reported per " + + "SCM in safe mode. Pipelines with atleast one datanode reported " + + "count is {}, required atleast one datanode reported per " + "pipeline count is {}", currentReportedPipelineCount, thresholdCount); } + } @Override @@ -134,4 +154,5 @@ public int getThresholdCount() { public int getCurrentReportedPipelineCount() { return currentReportedPipelineCount; } -} \ No newline at end of file + +} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SCMSafeModeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SCMSafeModeManager.java index 1e83bc4eb14b..a22d1623fdcd 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SCMSafeModeManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SCMSafeModeManager.java @@ -59,17 +59,17 @@ * number of datanode registered is met or not. * * 3. HealthyPipelineSafeModeRule: - * Once the PipelineReportHandler processes the + * Once the pipelineReportHandler processes the * {@link SCMEvents#PIPELINE_REPORT}, it fires - * {@link SCMEvents#OPEN_PIPELINE}. This rule handles this + * {@link SCMEvents#PROCESSED_PIPELINE_REPORT}. This rule handles this * event. This rule processes this report, and check if pipeline is healthy * and increments current healthy pipeline count. Then validate it cutoff * threshold for healthy pipeline is met or not. * * 4. OneReplicaPipelineSafeModeRule: - * Once the PipelineReportHandler processes the + * Once the pipelineReportHandler processes the * {@link SCMEvents#PIPELINE_REPORT}, it fires - * {@link SCMEvents#OPEN_PIPELINE}. This rule handles this + * {@link SCMEvents#PROCESSED_PIPELINE_REPORT}. This rule handles this * event. This rule processes this report, and add the reported pipeline to * reported pipeline set. Then validate it cutoff threshold for one replica * per pipeline is met or not. @@ -135,13 +135,6 @@ public SCMSafeModeManager(Configuration conf, oneReplicaPipelineSafeModeRule); } emitSafeModeStatus(); - boolean createPipelineInSafemode = conf.getBoolean( - HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, - HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION_DEFAULT); - - if (createPipelineInSafemode) { - pipelineManager.startPipelineCreator(); - } } else { this.safeModeMetrics = null; exitSafeMode(eventQueue); @@ -173,7 +166,6 @@ public synchronized void validateSafeModeExitRules(String ruleName, if (exitRules.get(ruleName) != null) { validatedRules.add(ruleName); - LOG.info("{} rule is successfully validated", ruleName); } else { // This should never happen LOG.error("No Such Exit rule {}", ruleName); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeHandler.java index 2fbe89361412..44d1c941774b 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeHandler.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeHandler.java @@ -128,8 +128,7 @@ private void cleanupPipelines() { List pipelineList = scmPipelineManager.getPipelines(); pipelineList.forEach((pipeline) -> { try { - if (pipeline.getPipelineState() == Pipeline.PipelineState.ALLOCATED && - pipeline.isAllocationTimeout()) { + if (!pipeline.isHealthy()) { scmPipelineManager.finalizeAndDestroyPipeline(pipeline, false); } } catch (IOException ex) { @@ -142,4 +141,6 @@ private void cleanupPipelines() { public boolean getSafeModeStatus() { return isInSafeMode.get(); } + + } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeHeartbeatDispatcher.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeHeartbeatDispatcher.java index 3dbb4cb78f0f..9f6077b4f706 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeHeartbeatDispatcher.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeHeartbeatDispatcher.java @@ -164,7 +164,6 @@ public List dispatch(SCMHeartbeatRequestProto heartbeat) { } if (heartbeat.getCommandStatusReportsCount() != 0) { - LOG.debug("Dispatching Command Status Report."); for (CommandStatusReportsProto commandStatusReport : heartbeat .getCommandStatusReportsList()) { eventPublisher.fireEvent( diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java index 901bc2ca9100..530c0a6d2383 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java @@ -61,8 +61,6 @@ import org.apache.hadoop.ozone.audit.SCMAction; import org.apache.hadoop.ozone.protocol.StorageContainerDatanodeProtocol; import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand; -import org.apache.hadoop.ozone.protocol.commands.ClosePipelineCommand; -import org.apache.hadoop.ozone.protocol.commands.CreatePipelineCommand; import org.apache.hadoop.ozone.protocol.commands.DeleteBlocksCommand; import org.apache.hadoop.ozone.protocol.commands.DeleteContainerCommand; import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand; @@ -81,12 +79,6 @@ import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type.deleteContainerCommand; import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type.replicateContainerCommand; import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type.reregisterCommand; -import static org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type - .createPipelineCommand; -import static org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type - .closePipelineCommand; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_DEFAULT; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_KEY; @@ -337,18 +329,6 @@ public SCMCommandProto getCommandResponse(SCMCommand cmd) .setReplicateContainerCommandProto( ((ReplicateContainerCommand)cmd).getProto()) .build(); - case createPipelineCommand: - return builder - .setCommandType(createPipelineCommand) - .setCreatePipelineCommandProto( - ((CreatePipelineCommand)cmd).getProto()) - .build(); - case closePipelineCommand: - return builder - .setCommandType(closePipelineCommand) - .setClosePipelineCommandProto( - ((ClosePipelineCommand)cmd).getProto()) - .build(); default: throw new IllegalArgumentException("Scm command " + cmd.getType().toString() + " is not implemented"); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java index 16ea0942d719..21127f4daec0 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java @@ -400,7 +400,8 @@ private void initializeSystemManagers(OzoneConfiguration conf, pipelineManager = configurator.getPipelineManager(); } else { pipelineManager = - new SCMPipelineManager(conf, scmNodeManager, eventQueue); + new SCMPipelineManager(conf, scmNodeManager, eventQueue, + grpcTlsConfig); } if (configurator.getContainerManager() != null) { diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java index 89c4eb3d64f1..a475f9b5e535 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java @@ -33,9 +33,7 @@ .StorageContainerDatanodeProtocolProtos.PipelineReportsProto; import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.ContainerReplica; -import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.PipelineID; -import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; import org.apache.hadoop.hdds.scm.server.SCMConfigurator; import org.apache.hadoop.hdds.scm.server .SCMDatanodeHeartbeatDispatcher.PipelineActionsFromDatanode; @@ -362,17 +360,6 @@ public static PipelineReportFromDatanode getPipelineReportFromDatanode( return new PipelineReportFromDatanode(dn, reportBuilder.build()); } - public static void openAllRatisPipelines(PipelineManager pipelineManager) - throws IOException { - // Pipeline is created by background thread - List pipelines = - pipelineManager.getPipelines(HddsProtos.ReplicationType.RATIS); - // Trigger the processed pipeline report event - for (Pipeline pipeline : pipelines) { - pipelineManager.openPipeline(pipeline.getId()); - } - } - public static PipelineActionsFromDatanode getPipelineActionFromDatanode( DatanodeDetails dn, PipelineID... pipelineIDs) { PipelineActionsProto.Builder actionsProtoBuilder = diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java index aa190f4d0909..a012d64677bd 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java @@ -24,14 +24,11 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.hdds.protocol.proto - .StorageContainerDatanodeProtocolProtos.SCMCommandProto; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.TestUtils; import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager.SafeModeStatus; @@ -48,13 +45,9 @@ import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager; import org.apache.hadoop.hdds.scm.server.SCMConfigurator; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; -import org.apache.hadoop.hdds.server.events.EventHandler; -import org.apache.hadoop.hdds.server.events.EventPublisher; import org.apache.hadoop.hdds.server.events.EventQueue; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.container.common.SCMTestUtils; -import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode; -import org.apache.hadoop.ozone.protocol.commands.CreatePipelineCommand; import org.apache.hadoop.test.GenericTestUtils; import org.junit.After; import org.junit.Assert; @@ -101,18 +94,14 @@ public void setUp() throws Exception { conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, folder.newFolder().toString()); - conf.setBoolean(HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false); - conf.setTimeDuration(HddsConfigKeys.HDDS_PIPELINE_REPORT_INTERVAL, 5, - TimeUnit.SECONDS); // Override the default Node Manager in SCM with this Mock Node Manager. nodeManager = new MockNodeManager(true, 10); - eventQueue = new EventQueue(); pipelineManager = - new SCMPipelineManager(conf, nodeManager, eventQueue); + new SCMPipelineManager(conf, nodeManager, new EventQueue(), null); PipelineProvider mockRatisProvider = new MockRatisPipelineProvider(nodeManager, - pipelineManager.getStateManager(), conf, eventQueue); + pipelineManager.getStateManager(), conf); pipelineManager.setPipelineProvider(HddsProtos.ReplicationType.RATIS, mockRatisProvider); SCMConfigurator configurator = new SCMConfigurator(); @@ -123,10 +112,12 @@ public void setUp() throws Exception { // Initialize these fields so that the tests can pass. mapping = (SCMContainerManager) scm.getContainerManager(); blockManager = (BlockManagerImpl) scm.getScmBlockManager(); + + eventQueue = new EventQueue(); + eventQueue.addHandler(SCMEvents.SAFE_MODE_STATUS, + scm.getSafeModeHandler()); eventQueue.addHandler(SCMEvents.SAFE_MODE_STATUS, scm.getSafeModeHandler()); - DatanodeCommandHandler handler = new DatanodeCommandHandler(); - eventQueue.addHandler(SCMEvents.DATANODE_COMMAND, handler); CloseContainerEventHandler closeContainerHandler = new CloseContainerEventHandler(pipelineManager, mapping); eventQueue.addHandler(SCMEvents.CLOSE_CONTAINER, closeContainerHandler); @@ -145,8 +136,6 @@ public void testAllocateBlock() throws Exception { GenericTestUtils.waitFor(() -> { return !blockManager.isScmInSafeMode(); }, 10, 1000 * 5); - pipelineManager.createPipeline(type, factor); - TestUtils.openAllRatisPipelines(pipelineManager); AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, OzoneConsts.OZONE, new ExcludeList()); Assert.assertNotNull(block); @@ -164,7 +153,6 @@ public void testAllocateBlockWithExclusion() throws Exception { } } catch (IOException e) { } - TestUtils.openAllRatisPipelines(pipelineManager); ExcludeList excludeList = new ExcludeList(); excludeList .addPipeline(pipelineManager.getPipelines(type, factor).get(0).getId()); @@ -271,7 +259,6 @@ public void testMultipleBlockAllocation() pipelineManager.createPipeline(type, factor); pipelineManager.createPipeline(type, factor); - TestUtils.openAllRatisPipelines(pipelineManager); AllocatedBlock allocatedBlock = blockManager .allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, OzoneConsts.OZONE, @@ -318,7 +305,6 @@ public void testMultipleBlockAllocationWithClosedContainer() .getNumber(); i++) { pipelineManager.createPipeline(type, factor); } - TestUtils.openAllRatisPipelines(pipelineManager); // wait till each pipeline has the configured number of containers. // After this each pipeline has numContainerPerOwnerInPipeline containers @@ -373,23 +359,7 @@ public void testBlockAllocationWithNoAvailablePipelines() Assert.assertNotNull(blockManager .allocateBlock(DEFAULT_BLOCK_SIZE, type, factor, OzoneConsts.OZONE, new ExcludeList())); + Assert.assertEquals(1, pipelineManager.getPipelines(type, factor).size()); } - private class DatanodeCommandHandler implements - EventHandler { - - @Override - public void onMessage(final CommandForDatanode command, - final EventPublisher publisher) { - final SCMCommandProto.Type commandType = command.getCommand().getType(); - if (commandType == SCMCommandProto.Type.createPipelineCommand) { - CreatePipelineCommand createCommand = - (CreatePipelineCommand) command.getCommand(); - try { - pipelineManager.openPipeline(createCommand.getPipelineID()); - } catch (IOException e) { - } - } - } - } } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java index 4f503e4f537d..b022fd9ac023 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java @@ -24,7 +24,6 @@ import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.hdds.scm.TestUtils; import org.apache.hadoop.hdds.scm.pipeline.MockRatisPipelineProvider; import org.apache.hadoop.hdds.scm.pipeline.PipelineProvider; import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager; @@ -68,9 +67,8 @@ public static void setUp() throws Exception { configuration .set(HddsConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath()); nodeManager = new MockNodeManager(true, 10); - eventQueue = new EventQueue(); pipelineManager = - new SCMPipelineManager(configuration, nodeManager, eventQueue); + new SCMPipelineManager(configuration, nodeManager, eventQueue, null); PipelineProvider mockRatisProvider = new MockRatisPipelineProvider(nodeManager, pipelineManager.getStateManager(), configuration); @@ -79,13 +77,10 @@ public static void setUp() throws Exception { containerManager = new SCMContainerManager(configuration, nodeManager, pipelineManager, new EventQueue()); - pipelineManager.triggerPipelineCreation(); + eventQueue = new EventQueue(); eventQueue.addHandler(CLOSE_CONTAINER, new CloseContainerEventHandler(pipelineManager, containerManager)); eventQueue.addHandler(DATANODE_COMMAND, nodeManager); - // Move all pipelines created by background from ALLOCATED to OPEN state - Thread.sleep(2000); - TestUtils.openAllRatisPipelines(pipelineManager); } @AfterClass @@ -121,6 +116,7 @@ public void testCloseContainerEventWithInvalidContainer() { @Test public void testCloseContainerEventWithValidContainers() throws IOException { + ContainerInfo container = containerManager .allocateContainer(HddsProtos.ReplicationType.RATIS, HddsProtos.ReplicationFactor.ONE, OzoneConsts.OZONE); @@ -138,6 +134,7 @@ public void testCloseContainerEventWithValidContainers() throws IOException { @Test public void testCloseContainerEventWithRatis() throws IOException { + GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer .captureLogs(CloseContainerEventHandler.LOG); ContainerInfo container = containerManager diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestSCMContainerManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestSCMContainerManager.java index fde94d741c77..6436af0542a4 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestSCMContainerManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestSCMContainerManager.java @@ -96,7 +96,7 @@ public static void setUp() throws Exception { } nodeManager = new MockNodeManager(true, 10); pipelineManager = - new SCMPipelineManager(conf, nodeManager, new EventQueue()); + new SCMPipelineManager(conf, nodeManager, new EventQueue(), null); containerManager = new SCMContainerManager(conf, nodeManager, pipelineManager, new EventQueue()); xceiverClientManager = new XceiverClientManager(conf); @@ -147,7 +147,7 @@ public void testallocateContainerDistributesAllocation() throws Exception { containerInfo.getPipelineID()).getFirstNode() .getUuid()); } - Assert.assertTrue(pipelineList.size() >= 1); + Assert.assertTrue(pipelineList.size() > 5); } @Test diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java index f786060bc1aa..3e4508dbf7d3 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java @@ -108,7 +108,7 @@ SCMContainerManager createContainerManager(Configuration config, final int cacheSize = config.getInt(OZONE_SCM_DB_CACHE_SIZE_MB, OZONE_SCM_DB_CACHE_SIZE_DEFAULT); PipelineManager pipelineManager = - new SCMPipelineManager(config, scmNodeManager, eventQueue); + new SCMPipelineManager(config, scmNodeManager, eventQueue, null); return new SCMContainerManager(config, scmNodeManager, pipelineManager, eventQueue); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDeadNodeHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDeadNodeHandler.java index 14c24e0d64e8..7657b54373f3 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDeadNodeHandler.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDeadNodeHandler.java @@ -73,7 +73,6 @@ public class TestDeadNodeHandler { private SCMNodeManager nodeManager; private ContainerManager containerManager; private NodeReportHandler nodeReportHandler; - private SCMPipelineManager pipelineManager; private DeadNodeHandler deadNodeHandler; private EventPublisher publisher; private EventQueue eventQueue; @@ -88,12 +87,12 @@ public void setup() throws IOException, AuthenticationException { eventQueue = new EventQueue(); scm = HddsTestUtils.getScm(conf); nodeManager = (SCMNodeManager) scm.getScmNodeManager(); - pipelineManager = + SCMPipelineManager manager = (SCMPipelineManager)scm.getPipelineManager(); PipelineProvider mockRatisProvider = - new MockRatisPipelineProvider(nodeManager, - pipelineManager.getStateManager(), conf); - pipelineManager.setPipelineProvider(HddsProtos.ReplicationType.RATIS, + new MockRatisPipelineProvider(nodeManager, manager.getStateManager(), + conf); + manager.setPipelineProvider(HddsProtos.ReplicationType.RATIS, mockRatisProvider); containerManager = scm.getContainerManager(); deadNodeHandler = new DeadNodeHandler(nodeManager, @@ -148,8 +147,6 @@ public void testOnMessage() throws IOException, NodeNotFoundException { nodeManager.register(TestUtils.randomDatanodeDetails(), TestUtils.createNodeReport(storageOne), null); - TestUtils.openAllRatisPipelines(pipelineManager); - ContainerInfo container1 = TestUtils.allocateContainer(containerManager); ContainerInfo container2 = diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java index f4eb797331a0..db76d6678789 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java @@ -121,7 +121,6 @@ OzoneConfiguration getConf() { testDir.getAbsolutePath()); conf.setTimeDuration(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL, 100, TimeUnit.MILLISECONDS); - conf.setBoolean(HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false); return conf; } @@ -1036,11 +1035,9 @@ public void testHandlingSCMCommandEvent() eq.processAll(1000L); List command = nodemanager.processHeartbeat(datanodeDetails); - // With dh registered, SCM will send create pipeline command to dn - Assert.assertTrue(command.size() >= 1); - Assert.assertTrue(command.get(0).getClass().equals( - CloseContainerCommand.class) || - command.get(1).getClass().equals(CloseContainerCommand.class)); + Assert.assertEquals(1, command.size()); + Assert + .assertEquals(command.get(0).getClass(), CloseContainerCommand.class); } catch (IOException e) { e.printStackTrace(); throw e; diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockRatisPipelineProvider.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockRatisPipelineProvider.java index 25b0adc32651..342ee5bea7a2 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockRatisPipelineProvider.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockRatisPipelineProvider.java @@ -22,8 +22,6 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.node.NodeManager; -import org.apache.hadoop.hdds.server.events.EventPublisher; -import org.apache.hadoop.hdds.server.events.EventQueue; import java.io.IOException; import java.util.List; @@ -36,13 +34,7 @@ public class MockRatisPipelineProvider extends RatisPipelineProvider { public MockRatisPipelineProvider(NodeManager nodeManager, PipelineStateManager stateManager, Configuration conf) { - super(nodeManager, stateManager, conf, new EventQueue()); - } - - public MockRatisPipelineProvider(NodeManager nodeManager, - PipelineStateManager stateManager, Configuration conf, - EventPublisher eventPublisher) { - super(nodeManager, stateManager, conf, eventPublisher); + super(nodeManager, stateManager, conf, null); } protected void initializePipeline(Pipeline pipeline) throws IOException { diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java index 065b08b5ddc0..6f0425dcd8a0 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java @@ -61,13 +61,11 @@ public void init() throws Exception { private void createPipelineAndAssertions( HddsProtos.ReplicationFactor factor) throws IOException { Pipeline pipeline = provider.create(factor); - assertPipelineProperties(pipeline, factor, REPLICATION_TYPE, - Pipeline.PipelineState.ALLOCATED); + assertPipelineProperties(pipeline, factor, REPLICATION_TYPE); stateManager.addPipeline(pipeline); Pipeline pipeline1 = provider.create(factor); - assertPipelineProperties(pipeline1, factor, REPLICATION_TYPE, - Pipeline.PipelineState.ALLOCATED); + assertPipelineProperties(pipeline1, factor, REPLICATION_TYPE); // New pipeline should not overlap with the previous created pipeline assertTrue( intersection(pipeline.getNodes(), pipeline1.getNodes()) @@ -79,14 +77,12 @@ private void createPipelineAndAssertions( public void testCreatePipelineWithFactor() throws IOException { HddsProtos.ReplicationFactor factor = HddsProtos.ReplicationFactor.THREE; Pipeline pipeline = provider.create(factor); - assertPipelineProperties(pipeline, factor, REPLICATION_TYPE, - Pipeline.PipelineState.ALLOCATED); + assertPipelineProperties(pipeline, factor, REPLICATION_TYPE); stateManager.addPipeline(pipeline); factor = HddsProtos.ReplicationFactor.ONE; Pipeline pipeline1 = provider.create(factor); - assertPipelineProperties(pipeline1, factor, REPLICATION_TYPE, - Pipeline.PipelineState.ALLOCATED); + assertPipelineProperties(pipeline1, factor, REPLICATION_TYPE); stateManager.addPipeline(pipeline1); // New pipeline should overlap with the previous created pipeline, // and one datanode should overlap between the two types. @@ -117,13 +113,11 @@ public void testCreatePipelineWithNodes() { HddsProtos.ReplicationFactor factor = HddsProtos.ReplicationFactor.THREE; Pipeline pipeline = provider.create(factor, createListOfNodes(factor.getNumber())); - assertPipelineProperties(pipeline, factor, REPLICATION_TYPE, - Pipeline.PipelineState.OPEN); + assertPipelineProperties(pipeline, factor, REPLICATION_TYPE); factor = HddsProtos.ReplicationFactor.ONE; pipeline = provider.create(factor, createListOfNodes(factor.getNumber())); - assertPipelineProperties(pipeline, factor, REPLICATION_TYPE, - Pipeline.PipelineState.OPEN); + assertPipelineProperties(pipeline, factor, REPLICATION_TYPE); } @Test @@ -147,8 +141,7 @@ public void testCreatePipelinesDnExclude() throws IOException { // only 2 healthy DNs left that are not part of any pipeline Pipeline pipeline = provider.create(factor); - assertPipelineProperties(pipeline, factor, REPLICATION_TYPE, - Pipeline.PipelineState.ALLOCATED); + assertPipelineProperties(pipeline, factor, REPLICATION_TYPE); List nodes = pipeline.getNodes(); @@ -163,9 +156,8 @@ public void testCreatePipelinesDnExclude() throws IOException { private static void assertPipelineProperties( Pipeline pipeline, HddsProtos.ReplicationFactor expectedFactor, - HddsProtos.ReplicationType expectedReplicationType, - Pipeline.PipelineState expectedState) { - assertEquals(expectedState, pipeline.getPipelineState()); + HddsProtos.ReplicationType expectedReplicationType) { + assertEquals(Pipeline.PipelineState.OPEN, pipeline.getPipelineState()); assertEquals(expectedReplicationType, pipeline.getType()); assertEquals(expectedFactor, pipeline.getFactor()); assertEquals(expectedFactor.getNumber(), pipeline.getNodes().size()); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestHealthyPipelineSafeModeRule.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestHealthyPipelineSafeModeRule.java index 6ea1bfe18bb8..f6d9b0e7c8f3 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestHealthyPipelineSafeModeRule.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestHealthyPipelineSafeModeRule.java @@ -22,6 +22,10 @@ import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.protocol.proto. + StorageContainerDatanodeProtocolProtos.PipelineReport; +import org.apache.hadoop.hdds.protocol.proto. + StorageContainerDatanodeProtocolProtos.PipelineReportsProto; import org.apache.hadoop.hdds.scm.HddsTestUtils; import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.MockNodeManager; @@ -30,6 +34,7 @@ import org.apache.hadoop.hdds.scm.pipeline.PipelineProvider; import org.apache.hadoop.hdds.scm.pipeline.MockRatisPipelineProvider; import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager; +import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher; import org.apache.hadoop.hdds.server.events.EventQueue; import org.apache.hadoop.test.GenericTestUtils; import org.junit.Assert; @@ -63,11 +68,10 @@ public void testHealthyPipelineSafeModeRuleWithNoPipelines() // enable pipeline check config.setBoolean( HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK, true); - config.setBoolean( - HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false); + SCMPipelineManager pipelineManager = new SCMPipelineManager(config, - nodeManager, eventQueue); + nodeManager, eventQueue, null); PipelineProvider mockRatisProvider = new MockRatisPipelineProvider(nodeManager, pipelineManager.getStateManager(), config); @@ -84,8 +88,10 @@ public void testHealthyPipelineSafeModeRuleWithNoPipelines() } finally { FileUtil.fullyDelete(new File(storageDir)); } + } + @Test public void testHealthyPipelineSafeModeRuleWithPipelines() throws Exception { @@ -107,11 +113,10 @@ public void testHealthyPipelineSafeModeRuleWithPipelines() throws Exception { // enable pipeline check config.setBoolean( HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK, true); - config.setBoolean( - HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false); + SCMPipelineManager pipelineManager = new SCMPipelineManager(config, - nodeManager, eventQueue); + nodeManager, eventQueue, null); PipelineProvider mockRatisProvider = new MockRatisPipelineProvider(nodeManager, @@ -157,6 +162,7 @@ public void testHealthyPipelineSafeModeRuleWithPipelines() throws Exception { } finally { FileUtil.fullyDelete(new File(storageDir)); } + } @@ -182,11 +188,10 @@ public void testHealthyPipelineSafeModeRuleWithMixedPipelines() // enable pipeline check config.setBoolean( HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK, true); - config.setBoolean( - HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false); + SCMPipelineManager pipelineManager = new SCMPipelineManager(config, - nodeManager, eventQueue); + nodeManager, eventQueue, null); PipelineProvider mockRatisProvider = new MockRatisPipelineProvider(nodeManager, pipelineManager.getStateManager(), config); @@ -212,7 +217,7 @@ public void testHealthyPipelineSafeModeRuleWithMixedPipelines() scmSafeModeManager.getHealthyPipelineSafeModeRule(); - // No pipeline event have sent to SCMSafemodeManager + // No datanodes have sent pipelinereport from datanode Assert.assertFalse(healthyPipelineSafeModeRule.validate()); @@ -220,12 +225,12 @@ public void testHealthyPipelineSafeModeRuleWithMixedPipelines() GenericTestUtils.LogCapturer.captureLogs(LoggerFactory.getLogger( SCMSafeModeManager.class)); - // fire event with pipeline create status with ratis type and factor 1 + // fire event with pipeline report with ratis type and factor 1 // pipeline, validate() should return false firePipelineEvent(pipeline1, eventQueue); GenericTestUtils.waitFor(() -> logCapturer.getOutput().contains( - "reported count is 1"), + "reported count is 0"), 1000, 5000); Assert.assertFalse(healthyPipelineSafeModeRule.validate()); @@ -241,7 +246,20 @@ public void testHealthyPipelineSafeModeRuleWithMixedPipelines() } + private void firePipelineEvent(Pipeline pipeline, EventQueue eventQueue) { - eventQueue.fireEvent(SCMEvents.OPEN_PIPELINE, pipeline); + PipelineReportsProto.Builder reportBuilder = PipelineReportsProto + .newBuilder(); + + reportBuilder.addPipelineReport(PipelineReport.newBuilder() + .setPipelineID(pipeline.getId().getProtobuf()) + .setIsLeader(Boolean.TRUE)); + + // Here no need to fire event from 3 nodes, as already pipeline is in + // open state, but doing it. + eventQueue.fireEvent(SCMEvents.PROCESSED_PIPELINE_REPORT, + new SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode( + pipeline.getNodes().get(0), reportBuilder.build())); } + } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestOneReplicaPipelineSafeModeRule.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestOneReplicaPipelineSafeModeRule.java index 0fa5eae64f0f..7a099774e2bb 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestOneReplicaPipelineSafeModeRule.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestOneReplicaPipelineSafeModeRule.java @@ -20,6 +20,10 @@ import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.protocol.proto + .StorageContainerDatanodeProtocolProtos.PipelineReport; +import org.apache.hadoop.hdds.protocol.proto + .StorageContainerDatanodeProtocolProtos.PipelineReportsProto; import org.apache.hadoop.hdds.scm.HddsTestUtils; import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.MockNodeManager; @@ -28,6 +32,7 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.pipeline.PipelineProvider; import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager; +import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher; import org.apache.hadoop.hdds.server.events.EventQueue; import org.apache.hadoop.test.GenericTestUtils; import org.junit.Assert; @@ -58,8 +63,6 @@ private void setup(int nodes, int pipelineFactorThreeCount, HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK, true); ozoneConfiguration.set(HddsConfigKeys.OZONE_METADATA_DIRS, folder.newFolder().toString()); - ozoneConfiguration.setBoolean( - HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false); List containers = new ArrayList<>(); containers.addAll(HddsTestUtils.getContainerInfo(1)); @@ -68,7 +71,7 @@ private void setup(int nodes, int pipelineFactorThreeCount, eventQueue = new EventQueue(); pipelineManager = new SCMPipelineManager(ozoneConfiguration, mockNodeManager, - eventQueue); + eventQueue, null); PipelineProvider mockRatisProvider = new MockRatisPipelineProvider(mockNodeManager, @@ -120,6 +123,7 @@ public void testOneReplicaPipelineRule() throws Exception { firePipelineEvent(pipelines.get(pipelineFactorThreeCount - 1)); GenericTestUtils.waitFor(() -> rule.validate(), 1000, 5000); + } @@ -166,8 +170,11 @@ public void testOneReplicaPipelineRuleMixedPipelines() throws Exception { firePipelineEvent(pipelines.get(pipelineCountThree - 1)); GenericTestUtils.waitFor(() -> rule.validate(), 1000, 5000); + } + + private void createPipelines(int count, HddsProtos.ReplicationFactor factor) throws Exception { for (int i = 0; i < count; i++) { @@ -177,6 +184,27 @@ private void createPipelines(int count, } private void firePipelineEvent(Pipeline pipeline) { - eventQueue.fireEvent(SCMEvents.OPEN_PIPELINE, pipeline); + PipelineReportsProto.Builder reportBuilder = + PipelineReportsProto.newBuilder(); + + reportBuilder.addPipelineReport(PipelineReport.newBuilder() + .setPipelineID(pipeline.getId().getProtobuf()) + .setIsLeader(Boolean.TRUE)); + + if (pipeline.getFactor() == HddsProtos.ReplicationFactor.THREE) { + eventQueue.fireEvent(SCMEvents.PROCESSED_PIPELINE_REPORT, + new SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode( + pipeline.getNodes().get(0), reportBuilder.build())); + eventQueue.fireEvent(SCMEvents.PROCESSED_PIPELINE_REPORT, + new SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode( + pipeline.getNodes().get(1), reportBuilder.build())); + eventQueue.fireEvent(SCMEvents.PROCESSED_PIPELINE_REPORT, + new SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode( + pipeline.getNodes().get(2), reportBuilder.build())); + } else { + eventQueue.fireEvent(SCMEvents.PROCESSED_PIPELINE_REPORT, + new SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode( + pipeline.getNodes().get(0), reportBuilder.build())); + } } } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java index b5839bc905ce..1e608b338168 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java @@ -32,6 +32,8 @@ import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReport; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto; import org.apache.hadoop.hdds.scm.HddsTestUtils; import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.MockNodeManager; @@ -41,6 +43,7 @@ import org.apache.hadoop.hdds.scm.pipeline.PipelineManager; import org.apache.hadoop.hdds.scm.pipeline.PipelineProvider; import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager; +import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode; import org.apache.hadoop.hdds.server.events.EventQueue; import org.apache.hadoop.test.GenericTestUtils; import org.junit.Assert; @@ -70,8 +73,6 @@ public class TestSCMSafeModeManager { public static void setUp() { queue = new EventQueue(); config = new OzoneConfiguration(); - config.setBoolean(HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, - false); } @Test @@ -176,7 +177,7 @@ private OzoneConfiguration createConf(double healthyPercent, HDDS_SCM_SAFEMODE_HEALTHY_PIPELINE_THRESHOLD_PCT, healthyPercent); conf.setDouble(HddsConfigKeys. HDDS_SCM_SAFEMODE_ONE_NODE_REPORTED_PIPELINE_PCT, oneReplicaPercent); - conf.setBoolean(HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false); + return conf; } @@ -198,7 +199,7 @@ public void testFailWithIncorrectValueForHealthyPipelinePercent() 0.9); MockNodeManager mockNodeManager = new MockNodeManager(true, 10); PipelineManager pipelineManager = new SCMPipelineManager(conf, - mockNodeManager, queue); + mockNodeManager, queue, null); scmSafeModeManager = new SCMSafeModeManager( conf, containers, pipelineManager, queue); fail("testFailWithIncorrectValueForHealthyPipelinePercent"); @@ -216,7 +217,7 @@ public void testFailWithIncorrectValueForOneReplicaPipelinePercent() 200); MockNodeManager mockNodeManager = new MockNodeManager(true, 10); PipelineManager pipelineManager = new SCMPipelineManager(conf, - mockNodeManager, queue); + mockNodeManager, queue, null); scmSafeModeManager = new SCMSafeModeManager( conf, containers, pipelineManager, queue); fail("testFailWithIncorrectValueForOneReplicaPipelinePercent"); @@ -233,7 +234,7 @@ public void testFailWithIncorrectValueForSafeModePercent() throws Exception { conf.setDouble(HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT, -1.0); MockNodeManager mockNodeManager = new MockNodeManager(true, 10); PipelineManager pipelineManager = new SCMPipelineManager(conf, - mockNodeManager, queue); + mockNodeManager, queue, null); scmSafeModeManager = new SCMSafeModeManager( conf, containers, pipelineManager, queue); fail("testFailWithIncorrectValueForSafeModePercent"); @@ -257,7 +258,7 @@ public void testSafeModeExitRuleWithPipelineAvailabilityCheck( MockNodeManager mockNodeManager = new MockNodeManager(true, nodeCount); SCMPipelineManager pipelineManager = new SCMPipelineManager(conf, - mockNodeManager, queue); + mockNodeManager, queue, null); PipelineProvider mockRatisProvider = new MockRatisPipelineProvider(mockNodeManager, pipelineManager.getStateManager(), config); @@ -301,12 +302,12 @@ public void testSafeModeExitRuleWithPipelineAvailabilityCheck( // we shall a get an event when datanode is registered. In that case, // validate will return true, and add this to validatedRules. if (Math.max(healthyPipelinePercent, oneReplicaThresholdCount) == 0) { - firePipelineEvent(pipelineManager, pipelines.get(0)); + firePipelineEvent(pipelines.get(0)); } for (int i = 0; i < Math.max(healthyPipelineThresholdCount, - Math.min(oneReplicaThresholdCount, pipelines.size())); i++) { - firePipelineEvent(pipelineManager, pipelines.get(i)); + oneReplicaThresholdCount); i++) { + firePipelineEvent(pipelines.get(i)); if (i < healthyPipelineThresholdCount) { checkHealthy(i + 1); @@ -351,11 +352,16 @@ private void checkOpen(int expectedCount) throws Exception { 1000, 5000); } - private void firePipelineEvent(SCMPipelineManager pipelineManager, - Pipeline pipeline) throws Exception { - pipelineManager.openPipeline(pipeline.getId()); - queue.fireEvent(SCMEvents.OPEN_PIPELINE, - pipelineManager.getPipeline(pipeline.getId())); + private void firePipelineEvent(Pipeline pipeline) throws Exception { + PipelineReportsProto.Builder reportBuilder = + PipelineReportsProto.newBuilder(); + + reportBuilder.addPipelineReport(PipelineReport.newBuilder() + .setPipelineID(pipeline.getId().getProtobuf()) + .setIsLeader(Boolean.TRUE)); + queue.fireEvent(SCMEvents.PROCESSED_PIPELINE_REPORT, + new PipelineReportFromDatanode(pipeline.getNodes().get(0), + reportBuilder.build())); } @@ -474,7 +480,7 @@ public void testSafeModePipelineExitRule() throws Exception { HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK, true); SCMPipelineManager pipelineManager = new SCMPipelineManager(config, - nodeManager, queue); + nodeManager, queue, null); PipelineProvider mockRatisProvider = new MockRatisPipelineProvider(nodeManager, @@ -485,6 +491,11 @@ public void testSafeModePipelineExitRule() throws Exception { Pipeline pipeline = pipelineManager.createPipeline( HddsProtos.ReplicationType.RATIS, HddsProtos.ReplicationFactor.THREE); + PipelineReportsProto.Builder reportBuilder = PipelineReportsProto + .newBuilder(); + reportBuilder.addPipelineReport(PipelineReport.newBuilder() + .setPipelineID(pipeline.getId().getProtobuf()) + .setIsLeader(Boolean.TRUE)); scmSafeModeManager = new SCMSafeModeManager( config, containers, pipelineManager, queue); @@ -493,9 +504,10 @@ public void testSafeModePipelineExitRule() throws Exception { HddsTestUtils.createNodeRegistrationContainerReport(containers)); assertTrue(scmSafeModeManager.getInSafeMode()); - - - firePipelineEvent(pipelineManager, pipeline); + // Trigger the processed pipeline report event + queue.fireEvent(SCMEvents.PROCESSED_PIPELINE_REPORT, + new PipelineReportFromDatanode(pipeline.getNodes().get(0), + reportBuilder.build())); GenericTestUtils.waitFor(() -> { return !scmSafeModeManager.getInSafeMode(); diff --git a/hadoop-ozone/dist/src/main/compose/ozone-om-ha/test.sh b/hadoop-ozone/dist/src/main/compose/ozone-om-ha/run.sh similarity index 100% rename from hadoop-ozone/dist/src/main/compose/ozone-om-ha/test.sh rename to hadoop-ozone/dist/src/main/compose/ozone-om-ha/run.sh diff --git a/hadoop-ozone/dist/src/main/compose/testlib.sh b/hadoop-ozone/dist/src/main/compose/testlib.sh index 684f2f5a1c7e..b20dca894ae2 100755 --- a/hadoop-ozone/dist/src/main/compose/testlib.sh +++ b/hadoop-ozone/dist/src/main/compose/testlib.sh @@ -80,40 +80,6 @@ wait_for_datanodes(){ return 1 } -## @description wait until safemode exit (or 30 seconds) -## @param the docker-compose file -wait_for_safemode_exit(){ - local compose_file=$1 - - #Reset the timer - SECONDS=0 - - #Don't give it up until 30 seconds - while [[ $SECONDS -lt 90 ]]; do - - #This line checks the safemode status in scm - local command="ozone scmcli safemode status" - if [[ "${SECURITY_ENABLED}" == 'true' ]]; then - status=`docker-compose -f "${compose_file}" exec -T scm bash -c "kinit -k HTTP/scm@EXAMPLE.COM -t /etc/security/keytabs/HTTP.keytab && $command'"` - else - status=`docker-compose -f "${compose_file}" exec -T scm bash -c "$command"` - fi - - echo $status - if [[ "$status" ]]; then - if [[ ${status} == "SCM is out of safe mode." ]]; then - #Safemode exits. Let's return from the function. - echo "Safe mode is off" - return - fi - fi - - sleep 2 - done - echo "WARNING! Safemode is still on. Please check the docker-compose files" - return 1 -} - ## @description Starts a docker-compose based test environment ## @param number of datanodes to start and wait for (default: 3) start_docker_env(){ @@ -124,7 +90,6 @@ start_docker_env(){ docker-compose -f "$COMPOSE_FILE" --no-ansi down docker-compose -f "$COMPOSE_FILE" --no-ansi up -d --scale datanode="${datanode_count}" \ && wait_for_datanodes "$COMPOSE_FILE" "${datanode_count}" \ - && wait_for_safemode_exit "$COMPOSE_FILE" \ && sleep 10 if [[ $? -gt 0 ]]; then diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManagerIntegration.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManagerIntegration.java index b4f8c370ef13..fe612a00ebac 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManagerIntegration.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManagerIntegration.java @@ -73,7 +73,7 @@ public void setup() throws Exception { numContainerPerOwnerInPipeline = conf.getInt(ScmConfigKeys.OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT, ScmConfigKeys.OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT_DEFAULT); - cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(3).build(); + cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build(); cluster.waitForClusterToBeReady(); cluster.waitTobeOutOfSafeMode(); xceiverClientManager = new XceiverClientManager(conf); @@ -165,9 +165,7 @@ public void testContainerStateManagerRestart() throws IOException, } } - // Restart SCM will not trigger container report to satisfy the safe mode - // exit rule. - cluster.restartStorageContainerManager(false); + cluster.restartStorageContainerManager(true); List result = cluster.getStorageContainerManager() .getContainerManager().listContainer(null, 100); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/metrics/TestSCMContainerManagerMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/metrics/TestSCMContainerManagerMetrics.java index 26c8c0129bff..f2c31d1fa3a9 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/metrics/TestSCMContainerManagerMetrics.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/metrics/TestSCMContainerManagerMetrics.java @@ -40,8 +40,6 @@ import java.util.HashMap; import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL; -import static org.apache.hadoop.hdds.HddsConfigKeys - .HDDS_SCM_SAFEMODE_PIPELINE_CREATION; import static org.apache.hadoop.test.MetricsAsserts.getLongCounter; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; import static org.junit.Assert.fail; @@ -58,7 +56,6 @@ public class TestSCMContainerManagerMetrics { public void setup() throws Exception { OzoneConfiguration conf = new OzoneConfiguration(); conf.set(HDDS_CONTAINER_REPORT_INTERVAL, "3000s"); - conf.setBoolean(HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false); cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build(); cluster.waitForClusterToBeReady(); scm = cluster.getStorageContainerManager(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java index 21fa7bdca8b2..c583559fd3a5 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java @@ -81,7 +81,7 @@ public void init() throws Exception { cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(3).build(); conf.setTimeDuration(HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL, 1000, TimeUnit.MILLISECONDS); - pipelineDestroyTimeoutInMillis = 1000; + pipelineDestroyTimeoutInMillis = 5000; conf.setTimeDuration(ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT, pipelineDestroyTimeoutInMillis, TimeUnit.MILLISECONDS); cluster.waitForClusterToBeReady(); @@ -169,7 +169,7 @@ public void testPipelineCloseWithPipelineAction() throws Exception { new PipelineActionHandler(pipelineManager, conf); pipelineActionHandler .onMessage(pipelineActionsFromDatanode, new EventQueue()); - Thread.sleep(5000); + Thread.sleep((int) (pipelineDestroyTimeoutInMillis * 1.2)); OzoneContainer ozoneContainer = cluster.getHddsDatanodes().get(0).getDatanodeStateMachine() .getContainer(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java deleted file mode 100644 index 8b8c64fb768f..000000000000 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java +++ /dev/null @@ -1,210 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hdds.scm.pipeline; - -import org.apache.commons.collections.CollectionUtils; -import org.apache.hadoop.hdds.HddsConfigKeys; -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.scm.TestUtils; -import org.apache.hadoop.hdds.scm.container.MockNodeManager; -import org.apache.hadoop.hdds.scm.node.NodeManager; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -/** - * Test for RatisPipelineProvider. - */ -public class TestRatisPipelineProvider { - - private NodeManager nodeManager; - private PipelineProvider provider; - private PipelineStateManager stateManager; - - @Before - public void init() throws Exception { - nodeManager = new MockNodeManager(true, 10); - OzoneConfiguration conf = new OzoneConfiguration(); - conf.setBoolean(HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false); - stateManager = new PipelineStateManager(conf); - provider = new MockRatisPipelineProvider(nodeManager, - stateManager, new OzoneConfiguration()); - } - - private void createPipelineAndAssertions( - HddsProtos.ReplicationFactor factor) throws IOException { - Pipeline pipeline = provider.create(factor); - stateManager.addPipeline(pipeline); - Assert.assertEquals(pipeline.getType(), HddsProtos.ReplicationType.RATIS); - Assert.assertEquals(pipeline.getFactor(), factor); - Assert.assertEquals(Pipeline.PipelineState.ALLOCATED, - pipeline.getPipelineState()); - Assert.assertEquals(pipeline.getNodes().size(), factor.getNumber()); - Pipeline pipeline1 = provider.create(factor); - stateManager.addPipeline(pipeline1); - // New pipeline should not overlap with the previous created pipeline - Assert.assertTrue( - CollectionUtils.intersection(pipeline.getNodes(), pipeline1.getNodes()) - .isEmpty()); - Assert.assertEquals(pipeline1.getType(), HddsProtos.ReplicationType.RATIS); - Assert.assertEquals(pipeline1.getFactor(), factor); - Assert.assertEquals(Pipeline.PipelineState.ALLOCATED, - pipeline1.getPipelineState()); - Assert.assertEquals(pipeline1.getNodes().size(), factor.getNumber()); - } - - @Test - public void testCreatePipelineWithFactor() throws IOException { - HddsProtos.ReplicationFactor factor = HddsProtos.ReplicationFactor.THREE; - Pipeline pipeline = provider.create(factor); - stateManager.addPipeline(pipeline); - Assert.assertEquals(pipeline.getType(), HddsProtos.ReplicationType.RATIS); - Assert.assertEquals(pipeline.getFactor(), factor); - Assert.assertEquals(Pipeline.PipelineState.ALLOCATED, - pipeline.getPipelineState()); - Assert.assertEquals(pipeline.getNodes().size(), factor.getNumber()); - - factor = HddsProtos.ReplicationFactor.ONE; - Pipeline pipeline1 = provider.create(factor); - stateManager.addPipeline(pipeline1); - // New pipeline should overlap with the previous created pipeline, - // and one datanode should overlap between the two types. - Assert.assertEquals( - CollectionUtils.intersection(pipeline.getNodes(), - pipeline1.getNodes()).size(), 1); - Assert.assertEquals(pipeline1.getType(), HddsProtos.ReplicationType.RATIS); - Assert.assertEquals(pipeline1.getFactor(), factor); - Assert.assertEquals(Pipeline.PipelineState.ALLOCATED, - pipeline1.getPipelineState()); - Assert.assertEquals(pipeline1.getNodes().size(), factor.getNumber()); - } - - @Test - public void testCreatePipelineWithFactorThree() throws IOException { - createPipelineAndAssertions(HddsProtos.ReplicationFactor.THREE); - } - - @Test - public void testCreatePipelineWithFactorOne() throws IOException { - createPipelineAndAssertions(HddsProtos.ReplicationFactor.ONE); - } - - private List createListOfNodes(int nodeCount) { - List nodes = new ArrayList<>(); - for (int i = 0; i < nodeCount; i++) { - nodes.add(TestUtils.randomDatanodeDetails()); - } - return nodes; - } - - @Test - public void testCreatePipelineWithNodes() { - HddsProtos.ReplicationFactor factor = HddsProtos.ReplicationFactor.THREE; - Pipeline pipeline = - provider.create(factor, createListOfNodes(factor.getNumber())); - Assert.assertEquals(pipeline.getType(), HddsProtos.ReplicationType.RATIS); - Assert.assertEquals(pipeline.getFactor(), factor); - Assert.assertEquals( - pipeline.getPipelineState(), Pipeline.PipelineState.OPEN); - Assert.assertEquals(pipeline.getNodes().size(), factor.getNumber()); - - factor = HddsProtos.ReplicationFactor.ONE; - pipeline = provider.create(factor, createListOfNodes(factor.getNumber())); - Assert.assertEquals(pipeline.getType(), HddsProtos.ReplicationType.RATIS); - Assert.assertEquals(pipeline.getFactor(), factor); - Assert.assertEquals(pipeline.getPipelineState(), - Pipeline.PipelineState.OPEN); - Assert.assertEquals(pipeline.getNodes().size(), factor.getNumber()); - } - - @Test - public void testCreatePipelinesDnExclude() throws IOException { - - // We need 9 Healthy DNs in MockNodeManager. - NodeManager mockNodeManager = new MockNodeManager(true, 12); - PipelineStateManager stateManagerMock = - new PipelineStateManager(new OzoneConfiguration()); - PipelineProvider providerMock = new MockRatisPipelineProvider( - mockNodeManager, stateManagerMock, new OzoneConfiguration()); - - // Use up first 3 DNs for an open pipeline. - List openPiplineDns = mockNodeManager.getAllNodes() - .subList(0, 3); - HddsProtos.ReplicationFactor factor = HddsProtos.ReplicationFactor.THREE; - - Pipeline openPipeline = Pipeline.newBuilder() - .setType(HddsProtos.ReplicationType.RATIS) - .setFactor(factor) - .setNodes(openPiplineDns) - .setState(Pipeline.PipelineState.OPEN) - .setId(PipelineID.randomId()) - .build(); - - stateManagerMock.addPipeline(openPipeline); - - // Use up next 3 DNs also for an open pipeline. - List moreOpenPiplineDns = mockNodeManager.getAllNodes() - .subList(3, 6); - Pipeline anotherOpenPipeline = Pipeline.newBuilder() - .setType(HddsProtos.ReplicationType.RATIS) - .setFactor(factor) - .setNodes(moreOpenPiplineDns) - .setState(Pipeline.PipelineState.OPEN) - .setId(PipelineID.randomId()) - .build(); - stateManagerMock.addPipeline(anotherOpenPipeline); - - // Use up next 3 DNs also for a closed pipeline. - List closedPiplineDns = mockNodeManager.getAllNodes() - .subList(6, 9); - Pipeline anotherClosedPipeline = Pipeline.newBuilder() - .setType(HddsProtos.ReplicationType.RATIS) - .setFactor(factor) - .setNodes(closedPiplineDns) - .setState(Pipeline.PipelineState.CLOSED) - .setId(PipelineID.randomId()) - .build(); - stateManagerMock.addPipeline(anotherClosedPipeline); - - Pipeline pipeline = providerMock.create(factor); - Assert.assertEquals(pipeline.getType(), HddsProtos.ReplicationType.RATIS); - Assert.assertEquals(pipeline.getFactor(), factor); - Assert.assertEquals(Pipeline.PipelineState.ALLOCATED, - pipeline.getPipelineState()); - Assert.assertEquals(pipeline.getNodes().size(), factor.getNumber()); - List pipelineNodes = pipeline.getNodes(); - - // Pipline nodes cannot be from open pipelines. - Assert.assertTrue( - pipelineNodes.parallelStream().filter(dn -> - (openPiplineDns.contains(dn) || moreOpenPiplineDns.contains(dn))) - .count() == 0); - - // Since we have only 9 Healthy DNs, at least 1 pipeline node should have - // been from the closed pipeline DN list. - Assert.assertTrue(pipelineNodes.parallelStream().filter( - closedPiplineDns::contains).count() > 0); - } -} \ No newline at end of file diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMPipelineManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMPipelineManager.java index 69227021a694..c65385f262d0 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMPipelineManager.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMPipelineManager.java @@ -40,7 +40,6 @@ import org.apache.hadoop.hdds.scm.container.MockNodeManager; import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager; import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode; -import org.apache.hadoop.hdds.scm.TestUtils; import org.apache.hadoop.hdds.server.events.EventQueue; import org.apache.hadoop.metrics2.MetricsRecordBuilder; import org.apache.hadoop.test.GenericTestUtils; @@ -63,7 +62,6 @@ public void setUp() throws Exception { testDir = GenericTestUtils .getTestDir(TestSCMPipelineManager.class.getSimpleName()); conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath()); - conf.setBoolean(HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false); boolean folderExisted = testDir.exists() || testDir.mkdirs(); if (!folderExisted) { throw new IOException("Unable to create test directory path"); @@ -79,7 +77,7 @@ public void cleanup() { @Test public void testPipelineReload() throws IOException { SCMPipelineManager pipelineManager = - new SCMPipelineManager(conf, nodeManager, new EventQueue()); + new SCMPipelineManager(conf, nodeManager, new EventQueue(), null); PipelineProvider mockRatisProvider = new MockRatisPipelineProvider(nodeManager, pipelineManager.getStateManager(), conf); @@ -96,7 +94,7 @@ public void testPipelineReload() throws IOException { // new pipeline manager should be able to load the pipelines from the db pipelineManager = - new SCMPipelineManager(conf, nodeManager, new EventQueue()); + new SCMPipelineManager(conf, nodeManager, new EventQueue(), null); mockRatisProvider = new MockRatisPipelineProvider(nodeManager, pipelineManager.getStateManager(), conf); @@ -119,7 +117,7 @@ public void testPipelineReload() throws IOException { @Test public void testRemovePipeline() throws IOException { SCMPipelineManager pipelineManager = - new SCMPipelineManager(conf, nodeManager, new EventQueue()); + new SCMPipelineManager(conf, nodeManager, new EventQueue(), null); PipelineProvider mockRatisProvider = new MockRatisPipelineProvider(nodeManager, pipelineManager.getStateManager(), conf); @@ -137,74 +135,7 @@ public void testRemovePipeline() throws IOException { // new pipeline manager should not be able to load removed pipelines pipelineManager = - new SCMPipelineManager(conf, nodeManager, new EventQueue()); - try { - pipelineManager.getPipeline(pipeline.getId()); - Assert.fail("Pipeline should not have been retrieved"); - } catch (IOException e) { - Assert.assertTrue(e.getMessage().contains("not found")); - } - - // clean up - pipelineManager.close(); - } - - @Test - public void testPipelineReport() throws IOException { - EventQueue eventQueue = new EventQueue(); - SCMPipelineManager pipelineManager = - new SCMPipelineManager(conf, nodeManager, eventQueue); - PipelineProvider mockRatisProvider = - new MockRatisPipelineProvider(nodeManager, - pipelineManager.getStateManager(), conf); - pipelineManager.setPipelineProvider(HddsProtos.ReplicationType.RATIS, - mockRatisProvider); - - SCMSafeModeManager scmSafeModeManager = - new SCMSafeModeManager(conf, new ArrayList<>(), pipelineManager, - eventQueue); - - // create a pipeline in allocated state with no dns yet reported - Pipeline pipeline = pipelineManager - .createPipeline(HddsProtos.ReplicationType.RATIS, - HddsProtos.ReplicationFactor.THREE); - - Assert - .assertFalse(pipelineManager.getPipeline(pipeline.getId()).isHealthy()); - Assert - .assertFalse(pipelineManager.getPipeline(pipeline.getId()).isOpen()); - - // get pipeline report from each dn in the pipeline - PipelineReportHandler pipelineReportHandler = - new PipelineReportHandler(scmSafeModeManager, pipelineManager, conf); - for (DatanodeDetails dn: pipeline.getNodes()) { - PipelineReportFromDatanode pipelineReportFromDatanode = - TestUtils.getPipelineReportFromDatanode(dn, pipeline.getId()); - // pipeline is not healthy until all dns report - Assert.assertFalse( - pipelineManager.getPipeline(pipeline.getId()).isHealthy()); - pipelineReportHandler - .onMessage(pipelineReportFromDatanode, new EventQueue()); - } - - // pipeline is healthy when all dns report - Assert - .assertTrue(pipelineManager.getPipeline(pipeline.getId()).isHealthy()); - // pipeline should now move to open state - Assert - .assertTrue(pipelineManager.getPipeline(pipeline.getId()).isOpen()); - - // close the pipeline - pipelineManager.finalizeAndDestroyPipeline(pipeline, false); - - for (DatanodeDetails dn: pipeline.getNodes()) { - PipelineReportFromDatanode pipelineReportFromDatanode = - TestUtils.getPipelineReportFromDatanode(dn, pipeline.getId()); - // pipeline report for destroyed pipeline should be ignored - pipelineReportHandler - .onMessage(pipelineReportFromDatanode, new EventQueue()); - } - + new SCMPipelineManager(conf, nodeManager, new EventQueue(), null); try { pipelineManager.getPipeline(pipeline.getId()); Assert.fail("Pipeline should not have been retrieved"); @@ -221,7 +152,7 @@ public void testPipelineCreationFailedMetric() throws Exception { MockNodeManager nodeManagerMock = new MockNodeManager(true, 20); SCMPipelineManager pipelineManager = - new SCMPipelineManager(conf, nodeManagerMock, new EventQueue()); + new SCMPipelineManager(conf, nodeManagerMock, new EventQueue(), null); PipelineProvider mockRatisProvider = new MockRatisPipelineProvider(nodeManagerMock, pipelineManager.getStateManager(), conf); @@ -230,9 +161,9 @@ public void testPipelineCreationFailedMetric() throws Exception { MetricsRecordBuilder metrics = getMetrics( SCMPipelineMetrics.class.getSimpleName()); - long numPipelineAllocated = getLongCounter("NumPipelineAllocated", + long numPipelineCreated = getLongCounter("NumPipelineCreated", metrics); - Assert.assertTrue(numPipelineAllocated == 0); + Assert.assertTrue(numPipelineCreated == 0); // 3 DNs are unhealthy. // Create 5 pipelines (Use up 15 Datanodes) @@ -245,8 +176,8 @@ public void testPipelineCreationFailedMetric() throws Exception { metrics = getMetrics( SCMPipelineMetrics.class.getSimpleName()); - numPipelineAllocated = getLongCounter("NumPipelineAllocated", metrics); - Assert.assertTrue(numPipelineAllocated == 5); + numPipelineCreated = getLongCounter("NumPipelineCreated", metrics); + Assert.assertTrue(numPipelineCreated == 5); long numPipelineCreateFailed = getLongCounter( "NumPipelineCreationFailed", metrics); @@ -265,8 +196,8 @@ public void testPipelineCreationFailedMetric() throws Exception { metrics = getMetrics( SCMPipelineMetrics.class.getSimpleName()); - numPipelineAllocated = getLongCounter("NumPipelineAllocated", metrics); - Assert.assertTrue(numPipelineAllocated == 5); + numPipelineCreated = getLongCounter("NumPipelineCreated", metrics); + Assert.assertTrue(numPipelineCreated == 5); numPipelineCreateFailed = getLongCounter( "NumPipelineCreationFailed", metrics); @@ -279,7 +210,7 @@ public void testPipelineCreationFailedMetric() throws Exception { @Test public void testActivateDeactivatePipeline() throws IOException { final SCMPipelineManager pipelineManager = - new SCMPipelineManager(conf, nodeManager, new EventQueue()); + new SCMPipelineManager(conf, nodeManager, new EventQueue(), null); final PipelineProvider mockRatisProvider = new MockRatisPipelineProvider(nodeManager, pipelineManager.getStateManager(), conf); @@ -326,7 +257,7 @@ public void testActivateDeactivatePipeline() throws IOException { public void testPipelineOpenOnlyWhenLeaderReported() throws Exception { EventQueue eventQueue = new EventQueue(); SCMPipelineManager pipelineManager = - new SCMPipelineManager(conf, nodeManager, eventQueue); + new SCMPipelineManager(conf, nodeManager, eventQueue, null); PipelineProvider mockRatisProvider = new MockRatisPipelineProvider(nodeManager, pipelineManager.getStateManager(), conf); @@ -339,7 +270,7 @@ public void testPipelineOpenOnlyWhenLeaderReported() throws Exception { pipelineManager.close(); // new pipeline manager loads the pipelines from the db in ALLOCATED state pipelineManager = - new SCMPipelineManager(conf, nodeManager, eventQueue); + new SCMPipelineManager(conf, nodeManager, eventQueue, null); mockRatisProvider = new MockRatisPipelineProvider(nodeManager, pipelineManager.getStateManager(), conf); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java index 01022469471c..59cef37d5364 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java @@ -262,7 +262,7 @@ abstract class Builder { // Use relative smaller number of handlers for testing protected int numOfOmHandlers = 20; protected int numOfScmHandlers = 20; - protected int numOfDatanodes = 3; + protected int numOfDatanodes = 1; protected boolean startDataNodes = true; protected CertificateClient certClient; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java index 2ad6c12222be..2813711c2f28 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java @@ -28,6 +28,8 @@ import org.apache.hadoop.conf.StorageUnit; import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.safemode.HealthyPipelineSafeModeRule; import org.apache.hadoop.hdds.scm.server.SCMStorageConfig; import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient; @@ -65,6 +67,7 @@ import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState @@ -95,7 +98,7 @@ public class MiniOzoneClusterImpl implements MiniOzoneCluster { private final List hddsDatanodes; // Timeout for the cluster to be ready - private int waitForClusterToBeReadyTimeout = 120000; // 2 min + private int waitForClusterToBeReadyTimeout = 60000; // 1 min private CertificateClient caClient; /** @@ -144,17 +147,32 @@ public void waitForClusterToBeReady() throws TimeoutException, InterruptedException { GenericTestUtils.waitFor(() -> { final int healthy = scm.getNodeCount(HEALTHY); - final boolean isNodeReady = healthy == hddsDatanodes.size(); - final boolean exitSafeMode = !scm.isInSafeMode(); - - LOG.info("{}. Got {} of {} DN Heartbeats.", - isNodeReady? "Nodes are ready" : "Waiting for nodes to be ready", - healthy, hddsDatanodes.size()); - LOG.info(exitSafeMode? "Cluster exits safe mode" : - "Waiting for cluster to exit safe mode", - healthy, hddsDatanodes.size()); - - return isNodeReady && exitSafeMode; + boolean isReady = healthy == hddsDatanodes.size(); + boolean printIsReadyMsg = true; + List pipelines = scm.getPipelineManager().getPipelines(); + if (!pipelines.isEmpty()) { + List raftPipelines = pipelines.stream().filter(p -> + p.getType() == HddsProtos.ReplicationType.RATIS).collect( + Collectors.toList()); + if (!raftPipelines.isEmpty()) { + List notOpenPipelines = raftPipelines.stream().filter(p -> + p.getPipelineState() != Pipeline.PipelineState.OPEN && + p.getPipelineState() != Pipeline.PipelineState.CLOSED) + .collect(Collectors.toList()); + if (notOpenPipelines.size() > 0) { + LOG.info("Waiting for {} number of pipelines out of {}, to report " + + "a leader.", notOpenPipelines.size(), raftPipelines.size()); + isReady = false; + printIsReadyMsg = false; + } + } + } + if (printIsReadyMsg) { + LOG.info("{}. Got {} of {} DN Heartbeats.", + isReady ? "Cluster is ready" : "Waiting for cluster to be ready", + healthy, hddsDatanodes.size()); + } + return isReady; }, 1000, waitForClusterToBeReadyTimeout); } @@ -642,6 +660,7 @@ private void configureSCMheartbeat() { if (hbInterval.isPresent()) { conf.setTimeDuration(HDDS_HEARTBEAT_INTERVAL, hbInterval.get(), TimeUnit.MILLISECONDS); + } else { conf.setTimeDuration(HDDS_HEARTBEAT_INTERVAL, DEFAULT_HB_INTERVAL_MS, diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java index eadb520b7915..cd975cf4b23a 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java @@ -49,7 +49,7 @@ public static void setup() throws Exception { ozoneConf.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY, SCMContainerPlacementCapacity.class, ContainerPlacementPolicy.class); ozoneConf.setStorageSize(OZONE_SCM_CONTAINER_SIZE, 5, StorageUnit.GB); - cluster = MiniOzoneCluster.newBuilder(ozoneConf).setNumDatanodes(3).build(); + cluster = MiniOzoneCluster.newBuilder(ozoneConf).setNumDatanodes(1).build(); storageClient = new ContainerOperationClient(ozoneConf); cluster.waitForClusterToBeReady(); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerStateMachineIdempotency.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerStateMachineIdempotency.java index 548f9b600d86..76eee6a1ca3e 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerStateMachineIdempotency.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestContainerStateMachineIdempotency.java @@ -61,7 +61,7 @@ public static void init() throws Exception { ozoneConfig.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY, SCMContainerPlacementCapacity.class, ContainerPlacementPolicy.class); cluster = - MiniOzoneCluster.newBuilder(ozoneConfig).setNumDatanodes(3).build(); + MiniOzoneCluster.newBuilder(ozoneConfig).setNumDatanodes(1).build(); cluster.waitForClusterToBeReady(); storageContainerLocationClient = cluster.getStorageContainerLocationClient(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java index 99b40839c7ee..efc2736af71f 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java @@ -282,11 +282,11 @@ private void createMalformedIDFile(File malformedFile) * Test that a DN can register with SCM even if it was started before the SCM. * @throws Exception */ - @Test (timeout = 60000) + @Test (timeout = 300_000) public void testDNstartAfterSCM() throws Exception { - // Start a cluster with 3 DN + // Start a cluster with 1 DN cluster = MiniOzoneCluster.newBuilder(conf) - .setNumDatanodes(3) + .setNumDatanodes(1) .build(); cluster.waitForClusterToBeReady(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java index 706f88010e3a..86dd75abd25d 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java @@ -19,12 +19,8 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic .NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY; -import static org.apache.hadoop.hdds.HddsConfigKeys - .HDDS_COMMAND_STATUS_REPORT_INTERVAL; -import static org.apache.hadoop.hdds.HddsConfigKeys - .HDDS_CONTAINER_REPORT_INTERVAL; -import static org.apache.hadoop.hdds.HddsConfigKeys - .HDDS_SCM_SAFEMODE_PIPELINE_CREATION; +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_COMMAND_STATUS_REPORT_INTERVAL; +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL; import static org.junit.Assert.fail; import static org.mockito.Matchers.argThat; import static org.mockito.Matchers.eq; @@ -332,12 +328,10 @@ public void testBlockDeletingThrottling() throws Exception { 100, TimeUnit.MILLISECONDS); conf.setInt(ScmConfigKeys.OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT, numKeys); - conf.setBoolean(HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false); MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf) .setHbInterval(1000) .setHbProcessorInterval(3000) - .setNumDatanodes(1) .build(); cluster.waitForClusterToBeReady(); @@ -464,7 +458,7 @@ public void testSCMReinitialization() throws Exception { conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, scmPath.toString()); //This will set the cluster id in the version file MiniOzoneCluster cluster = - MiniOzoneCluster.newBuilder(conf).setNumDatanodes(3).build(); + MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build(); cluster.waitForClusterToBeReady(); try { // This will initialize SCM @@ -576,7 +570,6 @@ public void testCloseContainerCommandOnRestart() throws Exception { 100, TimeUnit.MILLISECONDS); conf.setInt(ScmConfigKeys.OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT, numKeys); - conf.setBoolean(HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false); MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf) .setHbInterval(1000) @@ -604,7 +597,7 @@ public void testCloseContainerCommandOnRestart() throws Exception { scm.getContainerManager().updateContainerState(selectedContainer .containerID(), HddsProtos.LifeCycleEvent.FINALIZE); - cluster.restartStorageContainerManager(false); + cluster.restartStorageContainerManager(true); scm = cluster.getStorageContainerManager(); EventPublisher publisher = mock(EventPublisher.class); ReplicationManager replicationManager = scm.getReplicationManager(); @@ -614,7 +607,8 @@ public void testCloseContainerCommandOnRestart() throws Exception { modifiersField.setAccessible(true); modifiersField.setInt(f, f.getModifiers() & ~Modifier.FINAL); f.set(replicationManager, publisher); - Thread.sleep(10000); + scm.getReplicationManager().start(); + Thread.sleep(2000); UUID dnUuid = cluster.getHddsDatanodes().iterator().next() .getDatanodeDetails().getUuid(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBCSID.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBCSID.java index e6ed498bfa28..623b11d22d29 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBCSID.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBCSID.java @@ -45,8 +45,6 @@ HDDS_COMMAND_STATUS_REPORT_INTERVAL; import static org.apache.hadoop.hdds.HddsConfigKeys. HDDS_CONTAINER_REPORT_INTERVAL; -import static org.apache.hadoop.hdds.HddsConfigKeys - .HDDS_SCM_SAFEMODE_PIPELINE_CREATION; import static org.apache.hadoop.hdds.scm.ScmConfigKeys. HDDS_SCM_WATCHER_TIMEOUT; import static org.apache.hadoop.hdds.scm.ScmConfigKeys. @@ -83,7 +81,6 @@ public static void init() throws Exception { conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS); conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS); conf.setQuietMode(false); - conf.setBoolean(HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false); cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).setHbInterval(200) .build(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachine.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachine.java index e4b34fb916fa..82c49107bdc7 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachine.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachine.java @@ -52,8 +52,6 @@ import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_BLOCK_TOKEN_ENABLED; import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_COMMAND_STATUS_REPORT_INTERVAL; import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL; -import static org.apache.hadoop.hdds.HddsConfigKeys - .HDDS_SCM_SAFEMODE_MIN_PIPELINE; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL; @@ -95,7 +93,6 @@ public void setup() throws Exception { OzoneManager.setTestSecureOmFlag(true); conf.setLong(OzoneConfigKeys.DFS_RATIS_SNAPSHOT_THRESHOLD_KEY, 1); // conf.set(HADOOP_SECURITY_AUTHENTICATION, KERBEROS.toString()); - conf.setInt(HDDS_SCM_SAFEMODE_MIN_PIPELINE, 0); cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1) .setHbInterval(200) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java index e6302e368f42..0fb15d0481cc 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestContainerStateMachineFailures.java @@ -73,8 +73,6 @@ HDDS_COMMAND_STATUS_REPORT_INTERVAL; import static org.apache.hadoop.hdds.HddsConfigKeys. HDDS_CONTAINER_REPORT_INTERVAL; -import static org.apache.hadoop.hdds.HddsConfigKeys - .HDDS_PIPELINE_REPORT_INTERVAL; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos. ContainerDataProto.State.UNHEALTHY; import static org.apache.hadoop.hdds.scm.ScmConfigKeys. @@ -120,8 +118,6 @@ public static void init() throws Exception { TimeUnit.MILLISECONDS); conf.setTimeDuration(HDDS_COMMAND_STATUS_REPORT_INTERVAL, 200, TimeUnit.MILLISECONDS); - conf.setTimeDuration(HDDS_PIPELINE_REPORT_INTERVAL, 200, - TimeUnit.MILLISECONDS); conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS); conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS); conf.setTimeDuration(OZONE_SCM_PIPELINE_DESTROY_TIMEOUT, 10, @@ -133,7 +129,7 @@ public static void init() throws Exception { conf.setLong(OzoneConfigKeys.DFS_RATIS_SNAPSHOT_THRESHOLD_KEY, 1); conf.setQuietMode(false); cluster = - MiniOzoneCluster.newBuilder(conf).setNumDatanodes(3).setHbInterval(200) + MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).setHbInterval(200) .build(); cluster.waitForClusterToBeReady(); //the easiest way to create an open container is creating a key @@ -580,7 +576,6 @@ public void testValidateBCSIDOnDnRestart() throws Exception { Assert.assertTrue(!dispatcher.getMissingContainerSet().isEmpty()); Assert .assertTrue(dispatcher.getMissingContainerSet().contains(containerID)); - // write a new key key = objectStore.getVolume(volumeName).getBucket(bucketName) .createKey("ratis", 1024, ReplicationType.RATIS, ReplicationFactor.ONE, @@ -604,7 +599,7 @@ public void testValidateBCSIDOnDnRestart() throws Exception { byte[] blockCommitSequenceIdKey = DFSUtil.string2Bytes(OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID_PREFIX); - // modify the bcsid for the container in the ROCKS DB thereby inducing + // modify the bcsid for the container in the ROCKS DB tereby inducing // corruption db.getStore().put(blockCommitSequenceIdKey, Longs.toByteArray(0)); db.decrementReference(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java index b8e2495332a6..524c3bdb4ad9 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/TestContainerReplication.java @@ -73,7 +73,7 @@ public class TestContainerReplication { @Before public void setup() throws Exception { conf = newOzoneConfiguration(); - cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(3) + cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(2) .setRandomContainerPort(true).build(); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java index ffe5b6f414a9..c7b799264bec 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java @@ -112,7 +112,7 @@ public static void init() throws Exception { 3, TimeUnit.SECONDS); conf.setQuietMode(false); cluster = MiniOzoneCluster.newBuilder(conf) - .setNumDatanodes(3) + .setNumDatanodes(1) .setHbInterval(200) .build(); cluster.waitForClusterToBeReady(); @@ -143,7 +143,7 @@ public void testBlockDeletion() throws Exception { String keyName = UUID.randomUUID().toString(); OzoneOutputStream out = bucket.createKey(keyName, value.getBytes().length, - ReplicationType.RATIS, ReplicationFactor.THREE, new HashMap<>()); + ReplicationType.RATIS, ReplicationFactor.ONE, new HashMap<>()); for (int i = 0; i < 100; i++) { out.write(value.getBytes()); } @@ -152,7 +152,7 @@ public void testBlockDeletion() throws Exception { OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName) .setBucketName(bucketName).setKeyName(keyName).setDataSize(0) .setType(HddsProtos.ReplicationType.RATIS) - .setFactor(HddsProtos.ReplicationFactor.THREE) + .setFactor(HddsProtos.ReplicationFactor.ONE) .setRefreshPipeline(true) .build(); List omKeyLocationInfoGroupList = diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java index 3320f946a4a1..5c7f2c1a9feb 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java @@ -19,7 +19,6 @@ import java.util.HashMap; -import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.client.ReplicationFactor; import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -59,7 +58,6 @@ public void setup() throws Exception { //setup a cluster (1G free space is enough for a unit test) conf = new OzoneConfiguration(); conf.set(OZONE_SCM_CONTAINER_SIZE, "1GB"); - conf.setBoolean(HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false); cluster = MiniOzoneCluster.newBuilder(conf) .setNumDatanodes(1).build(); } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestDeleteContainerHandler.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestDeleteContainerHandler.java index 4df6f69da88a..1cbf69ef6d1c 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestDeleteContainerHandler.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestDeleteContainerHandler.java @@ -17,7 +17,6 @@ package org.apache.hadoop.ozone.container.common.statemachine.commandhandler; -import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.client.ReplicationFactor; import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -67,7 +66,6 @@ public class TestDeleteContainerHandler { public static void setup() throws Exception { conf = new OzoneConfiguration(); conf.set(OZONE_SCM_CONTAINER_SIZE, "1GB"); - conf.setBoolean(HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false); cluster = MiniOzoneCluster.newBuilder(conf) .setNumDatanodes(1).build(); cluster.waitForClusterToBeReady(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/dn/scrubber/TestDataScrubber.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/dn/scrubber/TestDataScrubber.java index 1d0433009997..7fb9825f801a 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/dn/scrubber/TestDataScrubber.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/dn/scrubber/TestDataScrubber.java @@ -85,8 +85,6 @@ public static void init() throws Exception { ozoneConfig.set(HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL, "1s"); ozoneConfig.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY, SCMContainerPlacementCapacity.class, ContainerPlacementPolicy.class); - ozoneConfig.setBoolean(HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, - false); cluster = MiniOzoneCluster.newBuilder(ozoneConfig).setNumDatanodes(1) .build(); cluster.waitForClusterToBeReady(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyPurging.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyPurging.java index e1e0cfaf7182..732fb3445ad9 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyPurging.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyPurging.java @@ -67,7 +67,7 @@ public void setup() throws Exception { conf.setQuietMode(false); cluster = MiniOzoneCluster.newBuilder(conf) - .setNumDatanodes(3) + .setNumDatanodes(1) .setHbInterval(200) .build(); cluster.waitForClusterToBeReady(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmSafeMode.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmSafeMode.java index 1c1f03474681..3614a05a8eb4 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmSafeMode.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmSafeMode.java @@ -341,7 +341,7 @@ public void testSCMSafeModeDisabled() throws Exception { builder = MiniOzoneCluster.newBuilder(conf) .setHbInterval(1000) .setHbProcessorInterval(500) - .setNumDatanodes(3); + .setNumDatanodes(1); cluster = builder.build(); StorageContainerManager scm = cluster.getStorageContainerManager(); assertFalse(scm.isInSafeMode()); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java index 9d187ff7d561..48ce4a65e2f6 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestContainerSmallFile.java @@ -61,7 +61,7 @@ public static void init() throws Exception { ozoneConfig = new OzoneConfiguration(); ozoneConfig.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY, SCMContainerPlacementCapacity.class, ContainerPlacementPolicy.class); - cluster = MiniOzoneCluster.newBuilder(ozoneConfig).setNumDatanodes(3) + cluster = MiniOzoneCluster.newBuilder(ozoneConfig).setNumDatanodes(1) .build(); cluster.waitForClusterToBeReady(); storageContainerLocationClient = cluster diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestGetCommittedBlockLengthAndPutKey.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestGetCommittedBlockLengthAndPutKey.java index 2f8c755f3fa1..b19020f96371 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestGetCommittedBlockLengthAndPutKey.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestGetCommittedBlockLengthAndPutKey.java @@ -65,7 +65,7 @@ public static void init() throws Exception { ozoneConfig.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY, SCMContainerPlacementCapacity.class, ContainerPlacementPolicy.class); cluster = - MiniOzoneCluster.newBuilder(ozoneConfig).setNumDatanodes(3).build(); + MiniOzoneCluster.newBuilder(ozoneConfig).setNumDatanodes(1).build(); cluster.waitForClusterToBeReady(); storageContainerLocationClient = cluster.getStorageContainerLocationClient(); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMXBean.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMXBean.java index 59a28f7ceed2..e700a0e59755 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMXBean.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMXBean.java @@ -57,7 +57,7 @@ public class TestSCMMXBean { public static final Log LOG = LogFactory.getLog(TestSCMMXBean.class); - private static int numOfDatanodes = 3; + private static int numOfDatanodes = 1; private static MiniOzoneCluster cluster; private static OzoneConfiguration conf; private static StorageContainerManager scm; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/node/TestSCMNodeMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/node/TestSCMNodeMetrics.java index 4aa1eae3d1a3..65a6357de9d5 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/node/TestSCMNodeMetrics.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/node/TestSCMNodeMetrics.java @@ -18,7 +18,6 @@ package org.apache.hadoop.ozone.scm.node; -import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto @@ -50,8 +49,7 @@ public class TestSCMNodeMetrics { @Before public void setup() throws Exception { OzoneConfiguration conf = new OzoneConfiguration(); - conf.setBoolean(HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION, false); - cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build(); + cluster = MiniOzoneCluster.newBuilder(conf).build(); cluster.waitForClusterToBeReady(); } diff --git a/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsRenameDir.java b/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsRenameDir.java index 2b95cbd10f00..1d584651bf98 100644 --- a/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsRenameDir.java +++ b/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFsRenameDir.java @@ -52,7 +52,7 @@ public class TestOzoneFsRenameDir { public void init() throws Exception { conf = new OzoneConfiguration(); cluster = MiniOzoneCluster.newBuilder(conf) - .setNumDatanodes(3) + .setNumDatanodes(1) .build(); cluster.waitForClusterToBeReady(); diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java index 2472fe13f29a..9cb0eb16a78c 100644 --- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java +++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -141,23 +141,17 @@ public Response put( String copyHeader = headers.getHeaderString(COPY_SOURCE_HEADER); String storageType = headers.getHeaderString(STORAGE_CLASS_HEADER); - ReplicationType replicationType; - ReplicationFactor replicationFactor; + S3StorageType s3StorageType; boolean storageTypeDefault; if (storageType == null || storageType.equals("")) { - replicationType = S3StorageType.getDefault().getType(); - replicationFactor = S3StorageType.getDefault().getFactor(); + s3StorageType = S3StorageType.getDefault(); storageTypeDefault = true; } else { - try { - replicationType = S3StorageType.valueOf(storageType).getType(); - replicationFactor = S3StorageType.valueOf(storageType).getFactor(); - } catch (IllegalArgumentException ex) { - throw S3ErrorTable.newError(S3ErrorTable.INVALID_ARGUMENT, - storageType); - } + s3StorageType = toS3StorageType(storageType); storageTypeDefault = false; } + ReplicationType replicationType = s3StorageType.getType(); + ReplicationFactor replicationFactor = s3StorageType.getFactor(); if (copyHeader != null) { //Copy object, as copy source available. @@ -214,10 +208,7 @@ public Response get( if (uploadId != null) { // When we have uploadId, this is the request for list Parts. - int partMarker = 0; - if (partNumberMarker != null) { - partMarker = Integer.parseInt(partNumberMarker); - } + int partMarker = parsePartNumberMarker(partNumberMarker); return listParts(bucketName, keyPath, uploadId, partMarker, maxParts); } @@ -233,16 +224,12 @@ public Response get( String rangeHeaderVal = headers.getHeaderString(RANGE_HEADER); RangeHeader rangeHeader = null; - if (LOG.isDebugEnabled()) { - LOG.debug("range Header provided value is {}", rangeHeaderVal); - } + LOG.debug("range Header provided value: {}", rangeHeaderVal); if (rangeHeaderVal != null) { rangeHeader = RangeHeaderParserUtil.parseRangeHeader(rangeHeaderVal, length); - if (LOG.isDebugEnabled()) { - LOG.debug("range Header provided value is {}", rangeHeader); - } + LOG.debug("range Header provided: {}", rangeHeader); if (rangeHeader.isInValidRange()) { throw S3ErrorTable.newError( S3ErrorTable.INVALID_RANGE, rangeHeaderVal); @@ -261,20 +248,13 @@ public Response get( .header(CONTENT_LENGTH, keyDetails.getDataSize()); } else { - LOG.debug("range Header provided value is {}", rangeHeader); OzoneInputStream key = bucket.readKey(keyPath); long startOffset = rangeHeader.getStartOffset(); long endOffset = rangeHeader.getEndOffset(); - long copyLength; - if (startOffset == endOffset) { - // if range header is given as bytes=0-0, then we should return 1 - // byte from start offset - copyLength = 1; - } else { - copyLength = rangeHeader.getEndOffset() - rangeHeader - .getStartOffset() + 1; - } + // eg. if range header is given as bytes=0-0, then we should return 1 + // byte from start offset + long copyLength = endOffset - startOffset + 1; StreamingOutput output = dest -> { try (S3WrapperInputStream s3WrapperInputStream = new S3WrapperInputStream( @@ -334,7 +314,8 @@ private void addLastModifiedDate( @HEAD public Response head( @PathParam("bucket") String bucketName, - @PathParam("path") String keyPath) throws Exception { + @PathParam("path") String keyPath) throws IOException, OS3Exception { + OzoneKeyDetails key; try { @@ -441,20 +422,14 @@ public Response initializeMultipartUpload( OzoneBucket ozoneBucket = getBucket(bucket); String storageType = headers.getHeaderString(STORAGE_CLASS_HEADER); - ReplicationType replicationType; - ReplicationFactor replicationFactor; + S3StorageType s3StorageType; if (storageType == null || storageType.equals("")) { - replicationType = S3StorageType.getDefault().getType(); - replicationFactor = S3StorageType.getDefault().getFactor(); + s3StorageType = S3StorageType.getDefault(); } else { - try { - replicationType = S3StorageType.valueOf(storageType).getType(); - replicationFactor = S3StorageType.valueOf(storageType).getFactor(); - } catch (IllegalArgumentException ex) { - throw S3ErrorTable.newError(S3ErrorTable.INVALID_ARGUMENT, - storageType); - } + s3StorageType = toS3StorageType(storageType); } + ReplicationType replicationType = s3StorageType.getType(); + ReplicationFactor replicationFactor = s3StorageType.getFactor(); OmMultipartInfo multipartInfo = ozoneBucket .initiateMultipartUpload(key, replicationType, replicationFactor); @@ -541,9 +516,10 @@ private Response createMultipartKey(String bucket, String key, long length, try { OzoneBucket ozoneBucket = getBucket(bucket); String copyHeader; - OmMultipartCommitUploadPartInfo omMultipartCommitUploadPartInfo; - try (OzoneOutputStream ozoneOutputStream = ozoneBucket.createMultipartKey( - key, length, partNumber, uploadID)) { + OzoneOutputStream ozoneOutputStream = null; + try { + ozoneOutputStream = ozoneBucket.createMultipartKey( + key, length, partNumber, uploadID); copyHeader = headers.getHeaderString(COPY_SOURCE_HEADER); if (copyHeader != null) { Pair result = parseSourceHeader(copyHeader); @@ -570,9 +546,12 @@ private Response createMultipartKey(String bucket, String key, long length, } else { IOUtils.copy(body, ozoneOutputStream); } - omMultipartCommitUploadPartInfo = ozoneOutputStream - .getCommitUploadPartInfo(); + } finally { + IOUtils.closeQuietly(ozoneOutputStream); } + + OmMultipartCommitUploadPartInfo omMultipartCommitUploadPartInfo = + ozoneOutputStream.getCommitUploadPartInfo(); String eTag = omMultipartCommitUploadPartInfo.getPartName(); if (copyHeader != null) { @@ -672,30 +651,28 @@ private CopyObjectResponse copyObject(String copyHeader, try { // Checking whether we trying to copying to it self. - if (sourceBucket.equals(destBucket)) { - if (sourceKey.equals(destkey)) { - // When copying to same storage type when storage type is provided, - // we should not throw exception, as aws cli checks if any of the - // options like storage type are provided or not when source and - // dest are given same - if (storageTypeDefault) { - OS3Exception ex = S3ErrorTable.newError(S3ErrorTable - .INVALID_REQUEST, copyHeader); - ex.setErrorMessage("This copy request is illegal because it is " + - "trying to copy an object to it self itself without changing " + - "the object's metadata, storage class, website redirect " + - "location or encryption attributes."); - throw ex; - } else { - // TODO: Actually here we should change storage type, as ozone - // still does not support this just returning dummy response - // for now - CopyObjectResponse copyObjectResponse = new CopyObjectResponse(); - copyObjectResponse.setETag(OzoneUtils.getRequestID()); - copyObjectResponse.setLastModified(Instant.ofEpochMilli( - Time.now())); - return copyObjectResponse; - } + if (sourceBucket.equals(destBucket) && sourceKey.equals(destkey)) { + // When copying to same storage type when storage type is provided, + // we should not throw exception, as aws cli checks if any of the + // options like storage type are provided or not when source and + // dest are given same + if (storageTypeDefault) { + OS3Exception ex = S3ErrorTable.newError(S3ErrorTable + .INVALID_REQUEST, copyHeader); + ex.setErrorMessage("This copy request is illegal because it is " + + "trying to copy an object to it self itself without changing " + + "the object's metadata, storage class, website redirect " + + "location or encryption attributes."); + throw ex; + } else { + // TODO: Actually here we should change storage type, as ozone + // still does not support this just returning dummy response + // for now + CopyObjectResponse copyObjectResponse = new CopyObjectResponse(); + copyObjectResponse.setETag(OzoneUtils.getRequestID()); + copyObjectResponse.setLastModified(Instant.ofEpochMilli( + Time.now())); + return copyObjectResponse; } } @@ -766,4 +743,22 @@ public static Pair parseSourceHeader(String copyHeader) return Pair.of(header.substring(0, pos), header.substring(pos + 1)); } + + private static S3StorageType toS3StorageType(String storageType) + throws OS3Exception { + try { + return S3StorageType.valueOf(storageType); + } catch (IllegalArgumentException ex) { + throw S3ErrorTable.newError(S3ErrorTable.INVALID_ARGUMENT, + storageType); + } + } + + private static int parsePartNumberMarker(String partNumberMarker) { + int partMarker = 0; + if (partNumberMarker != null) { + partMarker = Integer.parseInt(partNumberMarker); + } + return partMarker; + } } diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneOutputStreamStub.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneOutputStreamStub.java index 28e377b4282a..83cb90799f1c 100644 --- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneOutputStreamStub.java +++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneOutputStreamStub.java @@ -32,6 +32,7 @@ public class OzoneOutputStreamStub extends OzoneOutputStream { private final String partName; + private boolean closed; /** * Constructs OzoneOutputStreamStub with outputStream and partName. @@ -62,12 +63,15 @@ public synchronized void flush() throws IOException { @Override public synchronized void close() throws IOException { //commitKey can be done here, if needed. - getOutputStream().close(); + if (!closed) { + getOutputStream().close(); + closed = true; + } } @Override public OmMultipartCommitUploadPartInfo getCommitUploadPartInfo() { - return new OmMultipartCommitUploadPartInfo(partName); + return closed ? new OmMultipartCommitUploadPartInfo(partName) : null; } } diff --git a/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/fsck/TestContainerMapper.java b/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/fsck/TestContainerMapper.java index b6a8f189de2b..112674aaa1da 100644 --- a/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/fsck/TestContainerMapper.java +++ b/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/fsck/TestContainerMapper.java @@ -65,7 +65,7 @@ public static void init() throws Exception { conf.set(OZONE_OM_DB_DIRS, dbPath); conf.set(ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE, "100MB"); cluster = MiniOzoneCluster.newBuilder(conf) - .setNumDatanodes(3) + .setNumDatanodes(1) .setScmId(SCM_ID) .build(); cluster.waitForClusterToBeReady();