-
Notifications
You must be signed in to change notification settings - Fork 588
HDDS-6806. EC: Implement the EC Reconstruction coordinator. #3504
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from 8 commits
Commits
Show all changes
11 commits
Select commit
Hold shift + click to select a range
3bc2612
HDDS-6806: EC: Implement the EC Reconstruction coordinator
umamaheswararao c6eafa9
Adding the ECCoordinator implementation class.
umamaheswararao edb0dde
Fixed a warning.
umamaheswararao 2c85a35
Got few changes from https://github.com/apache/ozone/pull/3467 and ad…
umamaheswararao cece6b2
To make SCCleintConfig inited. TODO: revert this commit
umamaheswararao 99e3b1f
Added XceiverClientManager config
umamaheswararao beeb451
Some additional cleanups to simplify the logic.
umamaheswararao 4407fcc
Passing the certificate client to XceiverClientManager
umamaheswararao 253de95
Added replica index in createRecoveringContainer
umamaheswararao eecddbb
Fixed the review comments
umamaheswararao e2ea67b
Fixed few left overs
umamaheswararao File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
145 changes: 145 additions & 0 deletions
145
.../java/org/apache/hadoop/ozone/container/ec/reconstruction/ECContainerOperationClient.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| 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, | ||
guihecheng marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| 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()); | ||
umamaheswararao marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| 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(); | ||
| } | ||
| } | ||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.