Skip to content

Commit

Permalink
HDDS-12080. move ratis metadata direcotry cleaner to OzoneContainer i…
Browse files Browse the repository at this point in the history
…nitilizer
  • Loading branch information
Slava Tutrinov committed Jan 17, 2025
1 parent 1507d3f commit f1af721
Show file tree
Hide file tree
Showing 11 changed files with 83 additions and 55 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -181,7 +181,7 @@ public DatanodeStateMachine(HddsDatanodeService hddsDatanodeService,
constructionLock.writeLock().lock();
try {
container = new OzoneContainer(hddsDatanodeService, this.datanodeDetails,
conf, context, certClient, secretKeyClient);
conf, context, certClient, secretKeyClient, connectionManager);
} finally {
constructionLock.writeLock().unlock();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,16 +16,11 @@
*/
package org.apache.hadoop.ozone.container.common.states.endpoint;

import java.io.File;
import java.io.IOException;
import java.net.BindException;
import java.util.Arrays;
import java.util.Objects;
import java.util.concurrent.Callable;

import org.apache.commons.io.FileUtils;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.DatanodeLayoutStorage;
Expand Down Expand Up @@ -99,20 +94,6 @@ public EndpointStateMachine.EndPointStates call() throws Exception {
layoutStorage.setClusterId(clusterId);
layoutStorage.persistCurrentState();

HddsProtos.NodeState nodePreviousState = rpcEndPoint.getEndPoint()
.getNodePreviousState(ozoneContainer.getDatanodeDetails().getUuid());

if (nodePreviousState != null && nodePreviousState.equals(HddsProtos.NodeState.DEAD)) {
ozoneContainer.getMetaVolumeSet().getVolumeMap().forEach((key, value) ->
Arrays.asList(Objects.requireNonNull(value.getStorageDir().listFiles())).stream().filter(File::isDirectory).forEach(f -> {
try {
FileUtils.deleteDirectory(f);
} catch (IOException e) {
LOG.warn("Failed to delete directory {}", f.getAbsolutePath(), e);
}
}));
}

// Start the container services after getting the version information
ozoneContainer.start(clusterId);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,13 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Maps;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.hdds.HddsUtils;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.DatanodeDetails.Port.Name;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerType;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.IncrementalContainerReportProto;
Expand All @@ -49,6 +52,7 @@
import org.apache.hadoop.ozone.container.common.interfaces.Handler;
import org.apache.hadoop.ozone.container.common.report.IncrementalReportSender;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
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.XceiverServerGrpc;
import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi;
Expand All @@ -73,14 +77,20 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.File;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.nio.file.Files;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
Expand Down Expand Up @@ -135,6 +145,7 @@ public class OzoneContainer {
private ScheduledExecutorService dbCompactionExecutorService;

private final ContainerMetrics metrics;
private final SCMConnectionManager scmConnectionManager;
private WitnessedContainerMetadataStore witnessedContainerMetadataStore;

enum InitializingStatus {
Expand All @@ -151,12 +162,14 @@ enum InitializingStatus {
* @throws IOException
*/
public OzoneContainer(HddsDatanodeService hddsDatanodeService,
DatanodeDetails datanodeDetails, ConfigurationSource conf,
StateContext context, CertificateClient certClient,
SecretKeyVerifierClient secretKeyClient) throws IOException {
DatanodeDetails datanodeDetails, ConfigurationSource conf,
StateContext context, CertificateClient certClient,
SecretKeyVerifierClient secretKeyClient,
SCMConnectionManager scmConnectionManager) throws IOException {
config = conf;
this.datanodeDetails = datanodeDetails;
this.context = context;
this.scmConnectionManager = scmConnectionManager;
this.volumeChecker = new StorageVolumeChecker(conf, new Timer(),
datanodeDetails.threadNamePrefix());

Expand Down Expand Up @@ -229,6 +242,42 @@ public OzoneContainer(HddsDatanodeService hddsDatanodeService,
*/
controller = new ContainerController(containerSet, handlers);

if (scmConnectionManager != null) {
Collection<InetSocketAddress> scmAddressesForDatanodes = HddsUtils.getSCMAddressForDatanodes(conf);
for (InetSocketAddress scmAddress : scmAddressesForDatanodes) {
scmConnectionManager.addSCMServer(scmAddress, context.getThreadNamePrefix());
context.addEndpoint(scmAddress);
}

scmConnectionManager.getValues().stream()
.filter(endPoint -> !endPoint.isPassive())
.findFirst()
.ifPresent(rpcEndPoint -> {
try {
HddsProtos.NodeState nodePreviousState = rpcEndPoint.getEndPoint()
.getNodeState(datanodeDetails.getUuid());

if (nodePreviousState != null && nodePreviousState.equals(HddsProtos.NodeState.DEAD)) {
this.getMetaVolumeSet().getVolumeMap().forEach((key, value) -> {
Arrays.stream(Objects.requireNonNull(value.getStorageDir()
.listFiles((dir, name) -> !name.equals("tmp"))))
.filter(File::isDirectory)
.forEach(directory -> {
try {
FileUtils.deleteDirectory(directory);
} catch (IOException e) {
LOG.warn("Failed to delete directory: {}", directory);
}
});
});
}

} catch (IOException e) {
LOG.error(String.format("Failed to get datanode previous state with SCM: %s", e.getMessage()), e);
}
});
}

writeChannel = XceiverServerRatis.newXceiverServerRatis(hddsDatanodeService,
datanodeDetails, config, hddsDispatcher, controller, certClient,
context);
Expand Down Expand Up @@ -300,7 +349,7 @@ public OzoneContainer(HddsDatanodeService hddsDatanodeService,
public OzoneContainer(
DatanodeDetails datanodeDetails, ConfigurationSource conf,
StateContext context) throws IOException {
this(null, datanodeDetails, conf, context, null, null);
this(null, datanodeDetails, conf, context, null, null, null);
}

public GrpcTlsConfig getTlsClientConfig() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,8 @@

import org.apache.hadoop.hdds.annotation.InterfaceAudience;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ExtendedDatanodeDetailsProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodePreviousStateResponseProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeStateRequestProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeStateResponseProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
import org.apache.hadoop.hdds.protocol.proto
Expand Down Expand Up @@ -91,9 +91,6 @@ SCMRegisteredResponseProto register(
PipelineReportsProto pipelineReports,
LayoutVersionProto layoutInfo) throws IOException;

NodePreviousStateResponseProto
getNodePreviousState(
StorageContainerDatanodeProtocolProtos.NodePreviousStateRequestProto
request) throws IOException;
NodeStateResponseProto getNodeState(NodeStateRequestProto request) throws IOException;

}
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,9 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos
.ExtendedDatanodeDetailsProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.LayoutVersionProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodePreviousStateRequestProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodePreviousStateResponseProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeStateRequestProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeStateResponseProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
Expand Down Expand Up @@ -184,17 +183,17 @@ public SCMRegisteredResponseProto register(
.getRegisterResponse();
}

public HddsProtos.NodeState getNodePreviousState(UUID datanodeUuid) throws IOException {
NodePreviousStateRequestProto request = NodePreviousStateRequestProto.newBuilder()
public HddsProtos.NodeState getNodeState(UUID datanodeUuid) throws IOException {
NodeStateRequestProto request = NodeStateRequestProto.newBuilder()
.setDatanodeUUID(datanodeUuid.toString())
.build();
return getNodePreviousState(request).getPreviousState();
return getNodeState(request).getNodeState();

}

@Override
public NodePreviousStateResponseProto getNodePreviousState(NodePreviousStateRequestProto request) throws IOException {
return submitRequest(Type.NodePreviousState, builder -> builder.setNodePreviousStateRequest(request))
.getNodePreviousStateResponse();
public NodeStateResponseProto getNodeState(NodeStateRequestProto request) throws IOException {
return submitRequest(Type.NodePreviousState, builder -> builder.setNodeStateRequest(request))
.getNodeStateResponse();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -122,8 +122,8 @@ public SCMDatanodeResponse processMessage(SCMDatanodeRequest request)
return SCMDatanodeResponse.newBuilder()
.setCmdType(cmdType)
.setStatus(Status.OK)
.setNodePreviousStateResponse(
impl.getNodePreviousState(request.getNodePreviousStateRequest()))
.setNodeStateResponse(
impl.getNodeState(request.getNodeStateRequest()))
.build();
default:
throw new ServiceException("Unknown command type: " + cmdType);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@
.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeStateRequestProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeStateResponseProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
import org.apache.hadoop.hdds.protocol.proto
Expand Down Expand Up @@ -256,8 +258,8 @@ private void sleepIfNeeded() {
}

@Override
public StorageContainerDatanodeProtocolProtos.NodePreviousStateResponseProto getNodePreviousState(
StorageContainerDatanodeProtocolProtos.NodePreviousStateRequestProto request) throws IOException {
public NodeStateResponseProto getNodeState(
NodeStateRequestProto request) throws IOException {
return null;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ message SCMDatanodeRequest {
optional SCMVersionRequestProto getVersionRequest = 3;
optional SCMRegisterRequestProto registerRequest = 4;
optional SCMHeartbeatRequestProto sendHeartbeatRequest = 5;
optional NodePreviousStateRequestProto nodePreviousStateRequest = 6;
optional NodeStateRequestProto nodeStateRequest = 6;
}

message SCMDatanodeResponse {
Expand All @@ -61,7 +61,7 @@ message SCMDatanodeResponse {
optional SCMVersionResponseProto getVersionResponse = 6;
optional SCMRegisteredResponseProto registerResponse = 7;
optional SCMHeartbeatResponseProto sendHeartbeatResponse = 8;
optional NodePreviousStateResponseProto nodePreviousStateResponse = 9;
optional NodeStateResponseProto nodeStateResponse = 9;

}

Expand Down Expand Up @@ -127,7 +127,7 @@ message SCMRegisteredResponseProto {
optional string networkLocation = 8;
}

message NodePreviousStateRequestProto {
message NodeStateRequestProto {
required string datanodeUUID = 1;
}

Expand Down Expand Up @@ -164,8 +164,8 @@ message SCMHeartbeatResponseProto {
optional int64 term = 3;
}

message NodePreviousStateResponseProto {
optional NodeState previousState = 1;
message NodeStateResponseProto {
optional NodeState nodeState = 1;
}


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,9 +37,9 @@
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.LayoutVersionProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodePreviousStateRequestProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodePreviousStateResponseProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeStateRequestProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeStateResponseProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ReconstructECContainersCommandProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ReregisterCommandProto;
Expand Down Expand Up @@ -279,7 +279,7 @@ public SCMRegisteredResponseProto register(
}

@Override
public NodePreviousStateResponseProto getNodePreviousState(NodePreviousStateRequestProto request) throws IOException {
public NodeStateResponseProto getNodeState(NodeStateRequestProto request) throws IOException {
NodeStatus nodeStatus = null;
try {
DatanodeDetails nodeByUuid = scm.getScmNodeManager().getNodeByUuid(request.getDatanodeUUID());
Expand All @@ -289,9 +289,9 @@ public NodePreviousStateResponseProto getNodePreviousState(NodePreviousStateRequ
} catch (NodeNotFoundException e) {
LOG.warn("Node not found for UUID: {}", request.getDatanodeUUID());
}
NodePreviousStateResponseProto.Builder builder = NodePreviousStateResponseProto.newBuilder();
NodeStateResponseProto.Builder builder = NodeStateResponseProto.newBuilder();
if (nodeStatus != null) {
builder.setPreviousState(nodeStatus.getHealth());
builder.setNodeState(nodeStatus.getHealth());
}
return builder.build();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -311,7 +311,7 @@ private OzoneContainer createAndStartOzoneContainerInstance() {
try {
StateContext stateContext = ContainerTestUtils.getMockContext(dn, conf);
container = new OzoneContainer(
null, dn, conf, stateContext, caClient, keyClient);
null, dn, conf, stateContext, caClient, keyClient, null);
MutableVolumeSet volumeSet = container.getVolumeSet();
StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())
.forEach(hddsVolume -> hddsVolume.setDbParentDir(tempFolder.toFile()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,7 @@ void testCreateOzoneContainer(boolean requireToken, boolean hasToken,

DatanodeDetails dn = MockDatanodeDetails.randomDatanodeDetails();
container = new OzoneContainer(null, dn, conf, ContainerTestUtils
.getMockContext(dn, conf), caClient, secretKeyClient);
.getMockContext(dn, conf), caClient, secretKeyClient, null);
MutableVolumeSet volumeSet = container.getVolumeSet();
StorageVolumeUtil.getHddsVolumesList(volumeSet.getVolumesList())
.forEach(hddsVolume -> hddsVolume.setDbParentDir(tempFolder.toFile()));
Expand Down

0 comments on commit f1af721

Please sign in to comment.