Skip to content
Merged
Show file tree
Hide file tree
Changes from 8 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@ public class XceiverClientManager implements Closeable, XceiverClientFactory {
LoggerFactory.getLogger(XceiverClientManager.class);
//TODO : change this to SCM configuration class
private final ConfigurationSource conf;
private final ScmClientConfig clientConfig;
private final Cache<String, XceiverClientSpi> clientCache;
private List<X509Certificate> caCerts;

Expand All @@ -88,6 +89,7 @@ public XceiverClientManager(ConfigurationSource conf,
List<X509Certificate> caCerts) throws IOException {
Preconditions.checkNotNull(clientConf);
Preconditions.checkNotNull(conf);
this.clientConfig = clientConf;
long staleThresholdMs = clientConf.getStaleThreshold(MILLISECONDS);
this.conf = conf;
this.isSecurityEnabled = OzoneSecurityUtil.isSecurityEnabled(conf);
Expand Down Expand Up @@ -347,6 +349,37 @@ public void setMaxSize(int maxSize) {
this.maxSize = maxSize;
}

public void setStaleThreshold(long threshold) {
this.staleThreshold = threshold;
}

}

/**
* Builder of XceiverClientManagerConfig.
*/
public static class XceiverClientManagerConfigBuilder {

private int maxCacheSize;
private long staleThresholdMs;

public XceiverClientManagerConfigBuilder setMaxCacheSize(int maxCacheSize) {
this.maxCacheSize = maxCacheSize;
return this;
}

public XceiverClientManagerConfigBuilder setStaleThresholdMs(
long staleThresholdMs) {
this.staleThresholdMs = staleThresholdMs;
return this;
}

public ScmClientConfig build() {
ScmClientConfig clientConfig = new ScmClientConfig();
clientConfig.setMaxSize(this.maxCacheSize);
clientConfig.setStaleThreshold(this.staleThresholdMs);
return clientConfig;
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,10 @@ public void write(byte[] b, int off, int len) throws IOException {
writeChunkToContainer(ChunkBuffer.wrap(ByteBuffer.wrap(b, off, len)));
}

public void write(ByteBuffer buff) throws IOException {
this.currentChunkRspFuture = writeChunkToContainer(ChunkBuffer.wrap(buff));
}

public CompletableFuture<ContainerProtos.
ContainerCommandResponseProto> executePutBlock(boolean close,
boolean force, long blockGroupLength) throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.RefreshVolumeUsageCommandHandler;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.ReplicateContainerCommandHandler;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.SetNodeOperationalStateCommandHandler;
import org.apache.hadoop.ozone.container.ec.reconstruction.ECReconstructionCoordinator;
import org.apache.hadoop.ozone.container.ec.reconstruction.ECReconstructionSupervisor;
import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
Expand Down Expand Up @@ -181,9 +182,12 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails,
replicationSupervisorMetrics =
ReplicationSupervisorMetrics.create(supervisor);

ECReconstructionCoordinator ecReconstructionCoordinator =
new ECReconstructionCoordinator(conf, certClient);
ecReconstructionSupervisor =
new ECReconstructionSupervisor(container.getContainerSet(), context,
replicationConfig.getReplicationMaxStreams());
replicationConfig.getReplicationMaxStreams(),
ecReconstructionCoordinator);


// When we add new handlers just adding a new handler here should do the
Expand Down Expand Up @@ -389,6 +393,10 @@ public void close() throws IOException {
Thread.currentThread().interrupt();
}

if (ecReconstructionSupervisor != null) {
ecReconstructionSupervisor.close();
}

if (connectionManager != null) {
connectionManager.close();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,8 +53,9 @@ public void handle(SCMCommand command, OzoneContainer container,
ecContainersCommand.getMissingContainerIndexes(),
ecContainersCommand.getSources(),
ecContainersCommand.getTargetDatanodes());
this.supervisor.addTask(
new ECReconstructionCoordinatorTask(reconstructionCommandInfo));
this.supervisor.addTask(new ECReconstructionCoordinatorTask(
this.supervisor.getReconstructionCoordinator(),
reconstructionCommandInfo));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,145 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.ec.reconstruction;

import com.google.common.collect.ImmutableList;
import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls;
import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
import org.apache.hadoop.hdds.utils.HAUtils;
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.jetbrains.annotations.NotNull;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import java.util.stream.Collectors;

/**
* This class wraps necessary container-level rpc calls
* during ec offline reconstruction.
* - ListBlock
* - CloseContainer
*/
public class ECContainerOperationClient implements Closeable {
private static final Logger LOG =
LoggerFactory.getLogger(ECContainerOperationClient.class);
private final XceiverClientManager xceiverClientManager;

public ECContainerOperationClient(XceiverClientManager clientManager) {
this.xceiverClientManager = clientManager;
}

public ECContainerOperationClient(ConfigurationSource conf,
CertificateClient certificateClient) throws IOException {
this(createClientManager(conf, certificateClient));
}

@NotNull
private static XceiverClientManager createClientManager(
ConfigurationSource conf, CertificateClient certificateClient)
throws IOException {
return new XceiverClientManager(conf,
new XceiverClientManager.XceiverClientManagerConfigBuilder()
.setMaxCacheSize(256).setStaleThresholdMs(10 * 1000).build(),
certificateClient != null ?
HAUtils.buildCAX509List(certificateClient, conf) :
null);
}

public BlockData[] listBlock(long containerId, DatanodeDetails dn,
ECReplicationConfig repConfig, Token<? extends TokenIdentifier> token)
throws IOException {
XceiverClientSpi xceiverClient = this.xceiverClientManager.acquireClient(
Pipeline.newBuilder().setId(PipelineID.valueOf(dn.getUuid()))
.setReplicationConfig(repConfig).setNodes(ImmutableList.of(dn))
.setState(Pipeline.PipelineState.CLOSED).build());
try {
List<ContainerProtos.BlockData> blockDataList = ContainerProtocolCalls
.listBlock(xceiverClient, containerId, null, Integer.MAX_VALUE, token)
.getBlockDataList();
return blockDataList.stream().map(i -> {
try {
return BlockData.getFromProtoBuf(i);
} catch (IOException e) {
LOG.debug("Failed while converting to protobuf BlockData. Returning"
+ " null for listBlock from DN: " + dn,
e);
// TODO: revisit here.
return null;
}
}).collect(Collectors.toList())
.toArray(new BlockData[blockDataList.size()]);
} finally {
this.xceiverClientManager.releaseClient(xceiverClient, false);
}
}

public void closeContainer(long containerID, DatanodeDetails dn,
ECReplicationConfig repConfig, String encodedToken) throws IOException {
XceiverClientSpi xceiverClient = this.xceiverClientManager.acquireClient(
Pipeline.newBuilder().setId(PipelineID.valueOf(dn.getUuid()))
.setReplicationConfig(repConfig).setNodes(ImmutableList.of(dn))
.setState(Pipeline.PipelineState.CLOSED).build());
try {
ContainerProtocolCalls
.closeContainer(xceiverClient, containerID, encodedToken);
} finally {
this.xceiverClientManager.releaseClient(xceiverClient, false);
}
}

public void createRecoveringContainer(long containerID, DatanodeDetails dn,
ECReplicationConfig repConfig, String encodedToken) throws IOException {
XceiverClientSpi xceiverClient = this.xceiverClientManager.acquireClient(
// To get the same client from cache, we try to use the DN UUID as
// pipelineID for uniqueness. Please note, pipeline does not have any
// significance after it's close. So, we are ok to use any ID.
Pipeline.newBuilder().setId(PipelineID.valueOf(dn.getUuid()))
.setReplicationConfig(repConfig).setNodes(ImmutableList.of(dn))
.setState(Pipeline.PipelineState.CLOSED).build());
try {
ContainerProtocolCalls
.createRecoveringContainer(xceiverClient, containerID, encodedToken);
} finally {
this.xceiverClientManager.releaseClient(xceiverClient, false);
}
}

public XceiverClientManager getXceiverClientManager() {
return xceiverClientManager;
}

@Override
public void close() throws IOException {
if (xceiverClientManager != null) {
xceiverClientManager.close();
}
}
}
Loading