You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2022/06/15 09:39:00 UTC

[GitHub] [ozone] adoroszlai commented on a diff in pull request #3504: HDDS-6806. EC: Implement the EC Reconstruction coordinator.

adoroszlai commented on code in PR #3504:
URL: https://github.com/apache/ozone/pull/3504#discussion_r897696957


##########
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java:
##########
@@ -347,6 +349,37 @@ public void setMaxSize(int maxSize) {
       this.maxSize = maxSize;
     }
 
+    public void setStaleThreshold(long threshold) {
+      this.staleThreshold = threshold;
+    }
+
+  }
+
+  /**
+   * Builder of XceiverClientManagerConfig.

Review Comment:
   Nit: isn't it `ScmClientConfig`?



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.client.BlockID;
+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.ByteStringConversion;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo;
+import org.apache.hadoop.hdds.scm.storage.BufferPool;
+import org.apache.hadoop.hdds.scm.storage.ECBlockOutputStream;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.io.ElasticByteBufferPool;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactoryImpl;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStreamProxy;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * The Coordinator implements the main flow of reconstructing
+ * missing container replicas.
+ * <p>
+ * For a container reconstruction task, the main flow is:
+ * - ListBlock from all healthy replicas
+ * - calculate effective block group len for all blocks
+ * - for each block
+ * - build a ReconstructInputStream to read healthy chunks
+ * - build a ECBlockOutputStream to write out decoded chunks
+ * - for each stripe
+ * - use ReconstructInputStream.readStripe to decode missing chunks
+ * - use ECBlockOutputStream.write to write decoded chunks to TargetDNs
+ * - PutBlock
+ * - CloseContainer

Review Comment:
   Nit: seems like some indentation is missing (e.g. to clarify which steps are part of the for loops).



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.client.BlockID;
+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.ByteStringConversion;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo;
+import org.apache.hadoop.hdds.scm.storage.BufferPool;
+import org.apache.hadoop.hdds.scm.storage.ECBlockOutputStream;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.io.ElasticByteBufferPool;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactoryImpl;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStreamProxy;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * The Coordinator implements the main flow of reconstructing
+ * missing container replicas.
+ * <p>
+ * For a container reconstruction task, the main flow is:
+ * - ListBlock from all healthy replicas
+ * - calculate effective block group len for all blocks
+ * - for each block
+ * - build a ReconstructInputStream to read healthy chunks
+ * - build a ECBlockOutputStream to write out decoded chunks
+ * - for each stripe
+ * - use ReconstructInputStream.readStripe to decode missing chunks
+ * - use ECBlockOutputStream.write to write decoded chunks to TargetDNs
+ * - PutBlock
+ * - CloseContainer
+ */
+public class ECReconstructionCoordinator implements Closeable {
+
+  static final Logger LOG =
+      LoggerFactory.getLogger(ECReconstructionCoordinator.class);
+
+  private static final int EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE = 3;
+
+  private final ECContainerOperationClient containerOperationClient;
+
+  private final ConfigurationSource config;
+
+  private final ByteBufferPool byteBufferPool;
+
+  private ExecutorService ecReconstructExecutor;
+
+  private BlockInputStreamFactory blockInputStreamFactory;
+
+  public ECReconstructionCoordinator(ECContainerOperationClient containerClient,
+      ConfigurationSource conf, ByteBufferPool byteBufferPool,
+      ExecutorService reconstructExecutor,
+      BlockInputStreamFactory streamFactory) {
+    this.containerOperationClient = containerClient;
+    this.config = conf;
+    this.byteBufferPool = byteBufferPool;
+    this.blockInputStreamFactory = streamFactory;
+    this.ecReconstructExecutor = reconstructExecutor;
+  }
+
+  public ECReconstructionCoordinator(ConfigurationSource conf,
+      CertificateClient certificateClient) throws IOException {
+    this(new ECContainerOperationClient(conf, certificateClient), conf,
+        new ElasticByteBufferPool(), null, null);
+    this.ecReconstructExecutor =
+        new ThreadPoolExecutor(EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE,
+            config.getObject(OzoneClientConfig.class)
+                .getEcReconstructStripeReadPoolLimit(), 60, TimeUnit.SECONDS,
+            new SynchronousQueue<>(), new ThreadFactoryBuilder()
+            .setNameFormat("ec-reconstruct-reader-TID-%d").build(),
+            new ThreadPoolExecutor.CallerRunsPolicy());
+    this.blockInputStreamFactory = BlockInputStreamFactoryImpl
+        .getInstance(byteBufferPool, () -> ecReconstructExecutor);
+  }
+
+  public void reconstructECContainerGroup(long containerID,
+      ECReplicationConfig repConfig,
+      SortedMap<Integer, DatanodeDetails> sourceNodeMap,
+      SortedMap<Integer, DatanodeDetails> targetNodeMap) throws IOException {
+
+    Pipeline pipeline = rebuildInputPipeline(repConfig, sourceNodeMap);
+
+    SortedMap<Long, BlockData[]> blockDataMap =
+        getBlockDataMap(containerID, repConfig, sourceNodeMap);
+
+    SortedMap<Long, BlockLocationInfo> blockLocationInfoMap =
+        calcBlockLocationInfoMap(containerID, blockDataMap, pipeline);
+
+    // 1. create target recovering containers.
+    Set<Map.Entry<Integer, DatanodeDetails>> targetIndexDns =
+        targetNodeMap.entrySet();
+    Iterator<Map.Entry<Integer, DatanodeDetails>> iterator =
+        targetIndexDns.iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<Integer, DatanodeDetails> indexDnPair = iterator.next();

Review Comment:
   Nit: why not `for (Map.Entry<Integer, DatanodeDetails> indexDnPair : targetNodeMap.entrySet())`?



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.client.BlockID;
+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.ByteStringConversion;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo;
+import org.apache.hadoop.hdds.scm.storage.BufferPool;
+import org.apache.hadoop.hdds.scm.storage.ECBlockOutputStream;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.io.ElasticByteBufferPool;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactoryImpl;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStreamProxy;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * The Coordinator implements the main flow of reconstructing
+ * missing container replicas.
+ * <p>
+ * For a container reconstruction task, the main flow is:
+ * - ListBlock from all healthy replicas
+ * - calculate effective block group len for all blocks
+ * - for each block
+ * - build a ReconstructInputStream to read healthy chunks
+ * - build a ECBlockOutputStream to write out decoded chunks
+ * - for each stripe
+ * - use ReconstructInputStream.readStripe to decode missing chunks
+ * - use ECBlockOutputStream.write to write decoded chunks to TargetDNs
+ * - PutBlock
+ * - CloseContainer
+ */
+public class ECReconstructionCoordinator implements Closeable {
+
+  static final Logger LOG =
+      LoggerFactory.getLogger(ECReconstructionCoordinator.class);
+
+  private static final int EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE = 3;
+
+  private final ECContainerOperationClient containerOperationClient;
+
+  private final ConfigurationSource config;
+
+  private final ByteBufferPool byteBufferPool;
+
+  private ExecutorService ecReconstructExecutor;
+
+  private BlockInputStreamFactory blockInputStreamFactory;
+
+  public ECReconstructionCoordinator(ECContainerOperationClient containerClient,
+      ConfigurationSource conf, ByteBufferPool byteBufferPool,
+      ExecutorService reconstructExecutor,
+      BlockInputStreamFactory streamFactory) {
+    this.containerOperationClient = containerClient;
+    this.config = conf;
+    this.byteBufferPool = byteBufferPool;
+    this.blockInputStreamFactory = streamFactory;
+    this.ecReconstructExecutor = reconstructExecutor;
+  }
+
+  public ECReconstructionCoordinator(ConfigurationSource conf,
+      CertificateClient certificateClient) throws IOException {
+    this(new ECContainerOperationClient(conf, certificateClient), conf,
+        new ElasticByteBufferPool(), null, null);
+    this.ecReconstructExecutor =
+        new ThreadPoolExecutor(EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE,
+            config.getObject(OzoneClientConfig.class)
+                .getEcReconstructStripeReadPoolLimit(), 60, TimeUnit.SECONDS,
+            new SynchronousQueue<>(), new ThreadFactoryBuilder()
+            .setNameFormat("ec-reconstruct-reader-TID-%d").build(),
+            new ThreadPoolExecutor.CallerRunsPolicy());
+    this.blockInputStreamFactory = BlockInputStreamFactoryImpl
+        .getInstance(byteBufferPool, () -> ecReconstructExecutor);
+  }
+
+  public void reconstructECContainerGroup(long containerID,
+      ECReplicationConfig repConfig,
+      SortedMap<Integer, DatanodeDetails> sourceNodeMap,
+      SortedMap<Integer, DatanodeDetails> targetNodeMap) throws IOException {
+
+    Pipeline pipeline = rebuildInputPipeline(repConfig, sourceNodeMap);
+
+    SortedMap<Long, BlockData[]> blockDataMap =
+        getBlockDataMap(containerID, repConfig, sourceNodeMap);
+
+    SortedMap<Long, BlockLocationInfo> blockLocationInfoMap =
+        calcBlockLocationInfoMap(containerID, blockDataMap, pipeline);
+
+    // 1. create target recovering containers.
+    Set<Map.Entry<Integer, DatanodeDetails>> targetIndexDns =
+        targetNodeMap.entrySet();
+    Iterator<Map.Entry<Integer, DatanodeDetails>> iterator =
+        targetIndexDns.iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<Integer, DatanodeDetails> indexDnPair = iterator.next();
+      this.containerOperationClient
+          .createRecoveringContainer(containerID, indexDnPair.getValue(),
+              repConfig, null, indexDnPair.getKey());
+    }
+
+    // 2. Reconstruct and transfer to targets
+    for (BlockLocationInfo blockLocationInfo : blockLocationInfoMap.values()) {
+      reconstructECBlockGroup(blockLocationInfo, repConfig, targetNodeMap);
+    }
+
+    // 3. Close containers
+    iterator = targetIndexDns.iterator();
+    while (iterator.hasNext()) {
+      DatanodeDetails dn = iterator.next().getValue();
+      this.containerOperationClient
+          .closeContainer(containerID, dn, repConfig, null);
+    }
+
+  }
+
+  void reconstructECBlockGroup(BlockLocationInfo blockLocationInfo,
+      ECReplicationConfig repConfig,
+      SortedMap<Integer, DatanodeDetails> targetMap)
+      throws IOException {
+    long safeBlockGroupLength = blockLocationInfo.getLength();
+    List<Integer> missingContainerIndexes =
+        targetMap.keySet().stream().collect(Collectors.toList());
+
+    // calculate the real missing block indexes
+    int dataLocs = ECBlockInputStreamProxy
+        .expectedDataLocations(repConfig, safeBlockGroupLength);
+    List<Integer> toReconstructIndexes = new ArrayList<>();
+    for (int i = 0; i < missingContainerIndexes.size(); i++) {
+      Integer index = missingContainerIndexes.get(i);

Review Comment:
   Nit:
   
   ```suggestion
       for (Integer index : missingContainerIndexes) {
   ```



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.client.BlockID;
+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.ByteStringConversion;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo;
+import org.apache.hadoop.hdds.scm.storage.BufferPool;
+import org.apache.hadoop.hdds.scm.storage.ECBlockOutputStream;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.io.ElasticByteBufferPool;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactoryImpl;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStreamProxy;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * The Coordinator implements the main flow of reconstructing
+ * missing container replicas.
+ * <p>
+ * For a container reconstruction task, the main flow is:
+ * - ListBlock from all healthy replicas
+ * - calculate effective block group len for all blocks
+ * - for each block
+ * - build a ReconstructInputStream to read healthy chunks
+ * - build a ECBlockOutputStream to write out decoded chunks
+ * - for each stripe
+ * - use ReconstructInputStream.readStripe to decode missing chunks
+ * - use ECBlockOutputStream.write to write decoded chunks to TargetDNs
+ * - PutBlock
+ * - CloseContainer
+ */
+public class ECReconstructionCoordinator implements Closeable {
+
+  static final Logger LOG =
+      LoggerFactory.getLogger(ECReconstructionCoordinator.class);
+
+  private static final int EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE = 3;
+
+  private final ECContainerOperationClient containerOperationClient;
+
+  private final ConfigurationSource config;
+
+  private final ByteBufferPool byteBufferPool;
+
+  private ExecutorService ecReconstructExecutor;
+
+  private BlockInputStreamFactory blockInputStreamFactory;
+
+  public ECReconstructionCoordinator(ECContainerOperationClient containerClient,
+      ConfigurationSource conf, ByteBufferPool byteBufferPool,
+      ExecutorService reconstructExecutor,
+      BlockInputStreamFactory streamFactory) {
+    this.containerOperationClient = containerClient;
+    this.config = conf;
+    this.byteBufferPool = byteBufferPool;
+    this.blockInputStreamFactory = streamFactory;
+    this.ecReconstructExecutor = reconstructExecutor;
+  }
+
+  public ECReconstructionCoordinator(ConfigurationSource conf,
+      CertificateClient certificateClient) throws IOException {
+    this(new ECContainerOperationClient(conf, certificateClient), conf,
+        new ElasticByteBufferPool(), null, null);
+    this.ecReconstructExecutor =
+        new ThreadPoolExecutor(EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE,
+            config.getObject(OzoneClientConfig.class)
+                .getEcReconstructStripeReadPoolLimit(), 60, TimeUnit.SECONDS,
+            new SynchronousQueue<>(), new ThreadFactoryBuilder()
+            .setNameFormat("ec-reconstruct-reader-TID-%d").build(),
+            new ThreadPoolExecutor.CallerRunsPolicy());
+    this.blockInputStreamFactory = BlockInputStreamFactoryImpl
+        .getInstance(byteBufferPool, () -> ecReconstructExecutor);
+  }
+
+  public void reconstructECContainerGroup(long containerID,
+      ECReplicationConfig repConfig,
+      SortedMap<Integer, DatanodeDetails> sourceNodeMap,
+      SortedMap<Integer, DatanodeDetails> targetNodeMap) throws IOException {
+
+    Pipeline pipeline = rebuildInputPipeline(repConfig, sourceNodeMap);
+
+    SortedMap<Long, BlockData[]> blockDataMap =
+        getBlockDataMap(containerID, repConfig, sourceNodeMap);
+
+    SortedMap<Long, BlockLocationInfo> blockLocationInfoMap =
+        calcBlockLocationInfoMap(containerID, blockDataMap, pipeline);
+
+    // 1. create target recovering containers.
+    Set<Map.Entry<Integer, DatanodeDetails>> targetIndexDns =
+        targetNodeMap.entrySet();
+    Iterator<Map.Entry<Integer, DatanodeDetails>> iterator =
+        targetIndexDns.iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<Integer, DatanodeDetails> indexDnPair = iterator.next();
+      this.containerOperationClient
+          .createRecoveringContainer(containerID, indexDnPair.getValue(),
+              repConfig, null, indexDnPair.getKey());
+    }
+
+    // 2. Reconstruct and transfer to targets
+    for (BlockLocationInfo blockLocationInfo : blockLocationInfoMap.values()) {
+      reconstructECBlockGroup(blockLocationInfo, repConfig, targetNodeMap);
+    }
+
+    // 3. Close containers
+    iterator = targetIndexDns.iterator();
+    while (iterator.hasNext()) {
+      DatanodeDetails dn = iterator.next().getValue();
+      this.containerOperationClient
+          .closeContainer(containerID, dn, repConfig, null);
+    }
+
+  }
+
+  void reconstructECBlockGroup(BlockLocationInfo blockLocationInfo,
+      ECReplicationConfig repConfig,
+      SortedMap<Integer, DatanodeDetails> targetMap)
+      throws IOException {
+    long safeBlockGroupLength = blockLocationInfo.getLength();
+    List<Integer> missingContainerIndexes =
+        targetMap.keySet().stream().collect(Collectors.toList());

Review Comment:
   Can be simplified to `new ArrayList<>(targetMap.keySet())` or is some filter missing here?



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECContainerOperationClient.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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, int replicaIndex)
+      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());

Review Comment:
   Can you please extract a method for this pipeline creation?  It is repeated in a few places.



##########
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ECBlockOutputStream.java:
##########
@@ -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));

Review Comment:
   Nit: should it return the future and let coordinator use it directly instead of `getCurrent...`?



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.client.BlockID;
+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.ByteStringConversion;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo;
+import org.apache.hadoop.hdds.scm.storage.BufferPool;
+import org.apache.hadoop.hdds.scm.storage.ECBlockOutputStream;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.io.ElasticByteBufferPool;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactoryImpl;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStreamProxy;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * The Coordinator implements the main flow of reconstructing
+ * missing container replicas.
+ * <p>
+ * For a container reconstruction task, the main flow is:
+ * - ListBlock from all healthy replicas
+ * - calculate effective block group len for all blocks
+ * - for each block
+ * - build a ReconstructInputStream to read healthy chunks
+ * - build a ECBlockOutputStream to write out decoded chunks
+ * - for each stripe
+ * - use ReconstructInputStream.readStripe to decode missing chunks
+ * - use ECBlockOutputStream.write to write decoded chunks to TargetDNs
+ * - PutBlock
+ * - CloseContainer
+ */
+public class ECReconstructionCoordinator implements Closeable {
+
+  static final Logger LOG =
+      LoggerFactory.getLogger(ECReconstructionCoordinator.class);
+
+  private static final int EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE = 3;
+
+  private final ECContainerOperationClient containerOperationClient;
+
+  private final ConfigurationSource config;
+
+  private final ByteBufferPool byteBufferPool;
+
+  private ExecutorService ecReconstructExecutor;
+
+  private BlockInputStreamFactory blockInputStreamFactory;
+
+  public ECReconstructionCoordinator(ECContainerOperationClient containerClient,
+      ConfigurationSource conf, ByteBufferPool byteBufferPool,
+      ExecutorService reconstructExecutor,
+      BlockInputStreamFactory streamFactory) {
+    this.containerOperationClient = containerClient;
+    this.config = conf;
+    this.byteBufferPool = byteBufferPool;
+    this.blockInputStreamFactory = streamFactory;
+    this.ecReconstructExecutor = reconstructExecutor;
+  }
+
+  public ECReconstructionCoordinator(ConfigurationSource conf,
+      CertificateClient certificateClient) throws IOException {
+    this(new ECContainerOperationClient(conf, certificateClient), conf,
+        new ElasticByteBufferPool(), null, null);
+    this.ecReconstructExecutor =
+        new ThreadPoolExecutor(EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE,
+            config.getObject(OzoneClientConfig.class)
+                .getEcReconstructStripeReadPoolLimit(), 60, TimeUnit.SECONDS,
+            new SynchronousQueue<>(), new ThreadFactoryBuilder()
+            .setNameFormat("ec-reconstruct-reader-TID-%d").build(),
+            new ThreadPoolExecutor.CallerRunsPolicy());
+    this.blockInputStreamFactory = BlockInputStreamFactoryImpl
+        .getInstance(byteBufferPool, () -> ecReconstructExecutor);
+  }
+
+  public void reconstructECContainerGroup(long containerID,
+      ECReplicationConfig repConfig,
+      SortedMap<Integer, DatanodeDetails> sourceNodeMap,
+      SortedMap<Integer, DatanodeDetails> targetNodeMap) throws IOException {
+
+    Pipeline pipeline = rebuildInputPipeline(repConfig, sourceNodeMap);
+
+    SortedMap<Long, BlockData[]> blockDataMap =
+        getBlockDataMap(containerID, repConfig, sourceNodeMap);
+
+    SortedMap<Long, BlockLocationInfo> blockLocationInfoMap =
+        calcBlockLocationInfoMap(containerID, blockDataMap, pipeline);
+
+    // 1. create target recovering containers.
+    Set<Map.Entry<Integer, DatanodeDetails>> targetIndexDns =
+        targetNodeMap.entrySet();
+    Iterator<Map.Entry<Integer, DatanodeDetails>> iterator =
+        targetIndexDns.iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<Integer, DatanodeDetails> indexDnPair = iterator.next();
+      this.containerOperationClient
+          .createRecoveringContainer(containerID, indexDnPair.getValue(),
+              repConfig, null, indexDnPair.getKey());
+    }
+
+    // 2. Reconstruct and transfer to targets
+    for (BlockLocationInfo blockLocationInfo : blockLocationInfoMap.values()) {
+      reconstructECBlockGroup(blockLocationInfo, repConfig, targetNodeMap);
+    }
+
+    // 3. Close containers
+    iterator = targetIndexDns.iterator();
+    while (iterator.hasNext()) {
+      DatanodeDetails dn = iterator.next().getValue();
+      this.containerOperationClient
+          .closeContainer(containerID, dn, repConfig, null);
+    }
+
+  }
+
+  void reconstructECBlockGroup(BlockLocationInfo blockLocationInfo,
+      ECReplicationConfig repConfig,
+      SortedMap<Integer, DatanodeDetails> targetMap)
+      throws IOException {
+    long safeBlockGroupLength = blockLocationInfo.getLength();
+    List<Integer> missingContainerIndexes =
+        targetMap.keySet().stream().collect(Collectors.toList());
+
+    // calculate the real missing block indexes
+    int dataLocs = ECBlockInputStreamProxy
+        .expectedDataLocations(repConfig, safeBlockGroupLength);
+    List<Integer> toReconstructIndexes = new ArrayList<>();
+    for (int i = 0; i < missingContainerIndexes.size(); i++) {
+      Integer index = missingContainerIndexes.get(i);
+      if (index <= dataLocs || index > repConfig.getData()) {
+        toReconstructIndexes.add(index);
+      }
+      // else padded indexes.
+    }
+
+    // Looks like we don't need to reconstruct any missing blocks in this block
+    // group. The reason for this should be block group had only padding blocks
+    // in the missing locations.
+    if (toReconstructIndexes.size() == 0) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Skipping the reconstruction for the block: "
+            + blockLocationInfo.getBlockID() + ". In the missing locations: "
+            + missingContainerIndexes
+            + ", this block group has only padded blocks.");
+      }
+      return;
+    }
+
+    try (ECBlockReconstructedStripeInputStream sis
+        = new ECBlockReconstructedStripeInputStream(
+        repConfig, blockLocationInfo, true,
+        this.containerOperationClient.getXceiverClientManager(), null,
+        this.blockInputStreamFactory, byteBufferPool,
+        this.ecReconstructExecutor)) {
+
+      ECBlockOutputStream[] targetBlockStreams =
+          new ECBlockOutputStream[toReconstructIndexes.size()];
+      ByteBuffer[] bufs = new ByteBuffer[toReconstructIndexes.size()];
+      for (int i = 0; i < toReconstructIndexes.size(); i++) {
+        OzoneClientConfig configuration = new OzoneClientConfig();
+        // TODO: Let's avoid unnecessary bufferPool creation for
+        BufferPool bufferPool =
+            new BufferPool(configuration.getStreamBufferSize(),
+                (int) (configuration.getStreamBufferMaxSize() / configuration
+                    .getStreamBufferSize()),
+                ByteStringConversion.createByteBufferConversion(false));

Review Comment:
   Nit: not sure what the TODO is about, but at least this can be moved outside the loop.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.client.BlockID;
+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.ByteStringConversion;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo;
+import org.apache.hadoop.hdds.scm.storage.BufferPool;
+import org.apache.hadoop.hdds.scm.storage.ECBlockOutputStream;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.io.ElasticByteBufferPool;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactoryImpl;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStreamProxy;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * The Coordinator implements the main flow of reconstructing
+ * missing container replicas.
+ * <p>
+ * For a container reconstruction task, the main flow is:
+ * - ListBlock from all healthy replicas
+ * - calculate effective block group len for all blocks
+ * - for each block
+ * - build a ReconstructInputStream to read healthy chunks
+ * - build a ECBlockOutputStream to write out decoded chunks
+ * - for each stripe
+ * - use ReconstructInputStream.readStripe to decode missing chunks
+ * - use ECBlockOutputStream.write to write decoded chunks to TargetDNs
+ * - PutBlock
+ * - CloseContainer
+ */
+public class ECReconstructionCoordinator implements Closeable {
+
+  static final Logger LOG =
+      LoggerFactory.getLogger(ECReconstructionCoordinator.class);
+
+  private static final int EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE = 3;
+
+  private final ECContainerOperationClient containerOperationClient;
+
+  private final ConfigurationSource config;
+
+  private final ByteBufferPool byteBufferPool;
+
+  private ExecutorService ecReconstructExecutor;
+
+  private BlockInputStreamFactory blockInputStreamFactory;
+
+  public ECReconstructionCoordinator(ECContainerOperationClient containerClient,
+      ConfigurationSource conf, ByteBufferPool byteBufferPool,
+      ExecutorService reconstructExecutor,
+      BlockInputStreamFactory streamFactory) {
+    this.containerOperationClient = containerClient;
+    this.config = conf;
+    this.byteBufferPool = byteBufferPool;
+    this.blockInputStreamFactory = streamFactory;
+    this.ecReconstructExecutor = reconstructExecutor;
+  }
+
+  public ECReconstructionCoordinator(ConfigurationSource conf,
+      CertificateClient certificateClient) throws IOException {
+    this(new ECContainerOperationClient(conf, certificateClient), conf,
+        new ElasticByteBufferPool(), null, null);
+    this.ecReconstructExecutor =
+        new ThreadPoolExecutor(EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE,
+            config.getObject(OzoneClientConfig.class)
+                .getEcReconstructStripeReadPoolLimit(), 60, TimeUnit.SECONDS,
+            new SynchronousQueue<>(), new ThreadFactoryBuilder()
+            .setNameFormat("ec-reconstruct-reader-TID-%d").build(),
+            new ThreadPoolExecutor.CallerRunsPolicy());
+    this.blockInputStreamFactory = BlockInputStreamFactoryImpl
+        .getInstance(byteBufferPool, () -> ecReconstructExecutor);
+  }
+
+  public void reconstructECContainerGroup(long containerID,
+      ECReplicationConfig repConfig,
+      SortedMap<Integer, DatanodeDetails> sourceNodeMap,
+      SortedMap<Integer, DatanodeDetails> targetNodeMap) throws IOException {
+
+    Pipeline pipeline = rebuildInputPipeline(repConfig, sourceNodeMap);
+
+    SortedMap<Long, BlockData[]> blockDataMap =
+        getBlockDataMap(containerID, repConfig, sourceNodeMap);
+
+    SortedMap<Long, BlockLocationInfo> blockLocationInfoMap =
+        calcBlockLocationInfoMap(containerID, blockDataMap, pipeline);
+
+    // 1. create target recovering containers.
+    Set<Map.Entry<Integer, DatanodeDetails>> targetIndexDns =
+        targetNodeMap.entrySet();
+    Iterator<Map.Entry<Integer, DatanodeDetails>> iterator =
+        targetIndexDns.iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<Integer, DatanodeDetails> indexDnPair = iterator.next();
+      this.containerOperationClient
+          .createRecoveringContainer(containerID, indexDnPair.getValue(),
+              repConfig, null, indexDnPair.getKey());
+    }
+
+    // 2. Reconstruct and transfer to targets
+    for (BlockLocationInfo blockLocationInfo : blockLocationInfoMap.values()) {
+      reconstructECBlockGroup(blockLocationInfo, repConfig, targetNodeMap);
+    }
+
+    // 3. Close containers
+    iterator = targetIndexDns.iterator();
+    while (iterator.hasNext()) {
+      DatanodeDetails dn = iterator.next().getValue();
+      this.containerOperationClient
+          .closeContainer(containerID, dn, repConfig, null);
+    }
+
+  }
+
+  void reconstructECBlockGroup(BlockLocationInfo blockLocationInfo,
+      ECReplicationConfig repConfig,
+      SortedMap<Integer, DatanodeDetails> targetMap)
+      throws IOException {
+    long safeBlockGroupLength = blockLocationInfo.getLength();
+    List<Integer> missingContainerIndexes =
+        targetMap.keySet().stream().collect(Collectors.toList());
+
+    // calculate the real missing block indexes
+    int dataLocs = ECBlockInputStreamProxy
+        .expectedDataLocations(repConfig, safeBlockGroupLength);
+    List<Integer> toReconstructIndexes = new ArrayList<>();
+    for (int i = 0; i < missingContainerIndexes.size(); i++) {
+      Integer index = missingContainerIndexes.get(i);
+      if (index <= dataLocs || index > repConfig.getData()) {
+        toReconstructIndexes.add(index);
+      }
+      // else padded indexes.
+    }
+
+    // Looks like we don't need to reconstruct any missing blocks in this block
+    // group. The reason for this should be block group had only padding blocks
+    // in the missing locations.
+    if (toReconstructIndexes.size() == 0) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Skipping the reconstruction for the block: "
+            + blockLocationInfo.getBlockID() + ". In the missing locations: "
+            + missingContainerIndexes
+            + ", this block group has only padded blocks.");
+      }
+      return;
+    }
+
+    try (ECBlockReconstructedStripeInputStream sis
+        = new ECBlockReconstructedStripeInputStream(
+        repConfig, blockLocationInfo, true,
+        this.containerOperationClient.getXceiverClientManager(), null,
+        this.blockInputStreamFactory, byteBufferPool,
+        this.ecReconstructExecutor)) {
+
+      ECBlockOutputStream[] targetBlockStreams =
+          new ECBlockOutputStream[toReconstructIndexes.size()];
+      ByteBuffer[] bufs = new ByteBuffer[toReconstructIndexes.size()];
+      for (int i = 0; i < toReconstructIndexes.size(); i++) {
+        OzoneClientConfig configuration = new OzoneClientConfig();
+        // TODO: Let's avoid unnecessary bufferPool creation for
+        BufferPool bufferPool =
+            new BufferPool(configuration.getStreamBufferSize(),
+                (int) (configuration.getStreamBufferMaxSize() / configuration
+                    .getStreamBufferSize()),
+                ByteStringConversion.createByteBufferConversion(false));
+        targetBlockStreams[i] =
+            new ECBlockOutputStream(blockLocationInfo.getBlockID(),
+                this.containerOperationClient.getXceiverClientManager(),
+                Pipeline.newBuilder().setId(PipelineID.valueOf(
+                    targetMap.get(toReconstructIndexes.get(i)).getUuid()))
+                    .setReplicationConfig(repConfig).setNodes(ImmutableList
+                    .of(targetMap.get(toReconstructIndexes.get(i))))
+                    .setState(Pipeline.PipelineState.CLOSED).build(),

Review Comment:
   Nit: please assign `targetMap.get(toReconstructIndexes.get(i))` to local var, and reuse suggested helper method from `ECContainerOperationClient`.



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.client.BlockID;
+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.ByteStringConversion;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo;
+import org.apache.hadoop.hdds.scm.storage.BufferPool;
+import org.apache.hadoop.hdds.scm.storage.ECBlockOutputStream;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.io.ElasticByteBufferPool;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactoryImpl;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStreamProxy;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * The Coordinator implements the main flow of reconstructing
+ * missing container replicas.
+ * <p>
+ * For a container reconstruction task, the main flow is:
+ * - ListBlock from all healthy replicas
+ * - calculate effective block group len for all blocks
+ * - for each block
+ * - build a ReconstructInputStream to read healthy chunks
+ * - build a ECBlockOutputStream to write out decoded chunks
+ * - for each stripe
+ * - use ReconstructInputStream.readStripe to decode missing chunks
+ * - use ECBlockOutputStream.write to write decoded chunks to TargetDNs
+ * - PutBlock
+ * - CloseContainer
+ */
+public class ECReconstructionCoordinator implements Closeable {
+
+  static final Logger LOG =
+      LoggerFactory.getLogger(ECReconstructionCoordinator.class);
+
+  private static final int EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE = 3;
+
+  private final ECContainerOperationClient containerOperationClient;
+
+  private final ConfigurationSource config;
+
+  private final ByteBufferPool byteBufferPool;
+
+  private ExecutorService ecReconstructExecutor;
+
+  private BlockInputStreamFactory blockInputStreamFactory;
+
+  public ECReconstructionCoordinator(ECContainerOperationClient containerClient,
+      ConfigurationSource conf, ByteBufferPool byteBufferPool,
+      ExecutorService reconstructExecutor,
+      BlockInputStreamFactory streamFactory) {
+    this.containerOperationClient = containerClient;
+    this.config = conf;
+    this.byteBufferPool = byteBufferPool;
+    this.blockInputStreamFactory = streamFactory;
+    this.ecReconstructExecutor = reconstructExecutor;
+  }
+
+  public ECReconstructionCoordinator(ConfigurationSource conf,
+      CertificateClient certificateClient) throws IOException {
+    this(new ECContainerOperationClient(conf, certificateClient), conf,
+        new ElasticByteBufferPool(), null, null);
+    this.ecReconstructExecutor =
+        new ThreadPoolExecutor(EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE,
+            config.getObject(OzoneClientConfig.class)
+                .getEcReconstructStripeReadPoolLimit(), 60, TimeUnit.SECONDS,
+            new SynchronousQueue<>(), new ThreadFactoryBuilder()
+            .setNameFormat("ec-reconstruct-reader-TID-%d").build(),
+            new ThreadPoolExecutor.CallerRunsPolicy());
+    this.blockInputStreamFactory = BlockInputStreamFactoryImpl
+        .getInstance(byteBufferPool, () -> ecReconstructExecutor);
+  }
+
+  public void reconstructECContainerGroup(long containerID,
+      ECReplicationConfig repConfig,
+      SortedMap<Integer, DatanodeDetails> sourceNodeMap,
+      SortedMap<Integer, DatanodeDetails> targetNodeMap) throws IOException {
+
+    Pipeline pipeline = rebuildInputPipeline(repConfig, sourceNodeMap);
+
+    SortedMap<Long, BlockData[]> blockDataMap =
+        getBlockDataMap(containerID, repConfig, sourceNodeMap);
+
+    SortedMap<Long, BlockLocationInfo> blockLocationInfoMap =
+        calcBlockLocationInfoMap(containerID, blockDataMap, pipeline);
+
+    // 1. create target recovering containers.
+    Set<Map.Entry<Integer, DatanodeDetails>> targetIndexDns =
+        targetNodeMap.entrySet();
+    Iterator<Map.Entry<Integer, DatanodeDetails>> iterator =
+        targetIndexDns.iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<Integer, DatanodeDetails> indexDnPair = iterator.next();
+      this.containerOperationClient
+          .createRecoveringContainer(containerID, indexDnPair.getValue(),
+              repConfig, null, indexDnPair.getKey());
+    }
+
+    // 2. Reconstruct and transfer to targets
+    for (BlockLocationInfo blockLocationInfo : blockLocationInfoMap.values()) {
+      reconstructECBlockGroup(blockLocationInfo, repConfig, targetNodeMap);
+    }
+
+    // 3. Close containers
+    iterator = targetIndexDns.iterator();
+    while (iterator.hasNext()) {
+      DatanodeDetails dn = iterator.next().getValue();
+      this.containerOperationClient
+          .closeContainer(containerID, dn, repConfig, null);
+    }
+
+  }
+
+  void reconstructECBlockGroup(BlockLocationInfo blockLocationInfo,
+      ECReplicationConfig repConfig,
+      SortedMap<Integer, DatanodeDetails> targetMap)
+      throws IOException {
+    long safeBlockGroupLength = blockLocationInfo.getLength();
+    List<Integer> missingContainerIndexes =
+        targetMap.keySet().stream().collect(Collectors.toList());
+
+    // calculate the real missing block indexes
+    int dataLocs = ECBlockInputStreamProxy
+        .expectedDataLocations(repConfig, safeBlockGroupLength);
+    List<Integer> toReconstructIndexes = new ArrayList<>();
+    for (int i = 0; i < missingContainerIndexes.size(); i++) {
+      Integer index = missingContainerIndexes.get(i);
+      if (index <= dataLocs || index > repConfig.getData()) {
+        toReconstructIndexes.add(index);
+      }
+      // else padded indexes.
+    }
+
+    // Looks like we don't need to reconstruct any missing blocks in this block
+    // group. The reason for this should be block group had only padding blocks
+    // in the missing locations.
+    if (toReconstructIndexes.size() == 0) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Skipping the reconstruction for the block: "
+            + blockLocationInfo.getBlockID() + ". In the missing locations: "
+            + missingContainerIndexes
+            + ", this block group has only padded blocks.");
+      }
+      return;
+    }
+
+    try (ECBlockReconstructedStripeInputStream sis
+        = new ECBlockReconstructedStripeInputStream(
+        repConfig, blockLocationInfo, true,
+        this.containerOperationClient.getXceiverClientManager(), null,
+        this.blockInputStreamFactory, byteBufferPool,
+        this.ecReconstructExecutor)) {
+
+      ECBlockOutputStream[] targetBlockStreams =
+          new ECBlockOutputStream[toReconstructIndexes.size()];
+      ByteBuffer[] bufs = new ByteBuffer[toReconstructIndexes.size()];
+      for (int i = 0; i < toReconstructIndexes.size(); i++) {
+        OzoneClientConfig configuration = new OzoneClientConfig();
+        // TODO: Let's avoid unnecessary bufferPool creation for
+        BufferPool bufferPool =
+            new BufferPool(configuration.getStreamBufferSize(),
+                (int) (configuration.getStreamBufferMaxSize() / configuration
+                    .getStreamBufferSize()),
+                ByteStringConversion.createByteBufferConversion(false));
+        targetBlockStreams[i] =
+            new ECBlockOutputStream(blockLocationInfo.getBlockID(),
+                this.containerOperationClient.getXceiverClientManager(),
+                Pipeline.newBuilder().setId(PipelineID.valueOf(
+                    targetMap.get(toReconstructIndexes.get(i)).getUuid()))
+                    .setReplicationConfig(repConfig).setNodes(ImmutableList
+                    .of(targetMap.get(toReconstructIndexes.get(i))))
+                    .setState(Pipeline.PipelineState.CLOSED).build(),
+                bufferPool, configuration, blockLocationInfo.getToken());
+        bufs[i] = byteBufferPool.getBuffer(false, repConfig.getEcChunkSize());
+        // Make sure it's clean. Don't want to reuse the erroneously returned
+        // buffers from the pool.
+        bufs[i].clear();
+      }
+
+      sis.setRecoveryIndexes(toReconstructIndexes.stream().map(i -> (i - 1))
+          .collect(Collectors.toSet()));
+      long length = safeBlockGroupLength;
+      while (length > 0) {
+        int readLen = sis.recoverChunks(bufs);
+        // TODO: can be submitted in parallel
+        for (int i = 0; i < bufs.length; i++) {
+          targetBlockStreams[i].write(bufs[i]);
+          if (isFailed(targetBlockStreams[i],
+              targetBlockStreams[i].getCurrentChunkResponseFuture())) {
+            // If one chunk response failed, we should retry.
+            // Even after retries if it failed, we should declare the
+            // reconstruction as failed.
+            // For now, let's throw the exception.
+            throw new IOException(
+                "Chunk write failed at the new target node: "
+                    + targetBlockStreams[i].getDatanodeDetails()
+                    + ". Aborting the reconstruction process.");
+          }
+          bufs[i].clear();
+        }
+        length -= readLen;
+      }
+
+      for (int i = 0; i < targetBlockStreams.length; i++) {
+        try {
+          targetBlockStreams[i]
+              .executePutBlock(true, true, blockLocationInfo.getLength());
+          if (isFailed(targetBlockStreams[i],
+              targetBlockStreams[i].getCurrentPutBlkResponseFuture())) {
+            // If one chunk response failed, we should retry.
+            // Even after retries if it failed, we should declare the
+            // reconstruction as failed.
+            // For now, let's throw the exception.
+            throw new IOException(
+                "Chunk write failed at the new target node: "
+                    + targetBlockStreams[i].getDatanodeDetails()
+                    + ". Aborting the reconstruction process.");
+          }
+        } finally {
+          byteBufferPool.putBuffer(bufs[i]);
+          targetBlockStreams[i].close();

Review Comment:
   Loop would exit on first failed stream.  Seems like cleanup is not performed on the remaining ones.



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/storage/TestContainerCommandsEC.java:
##########
@@ -278,6 +292,274 @@ public void testCreateRecoveryContainer() throws Exception {
     }
   }
 
+  private static byte[] getBytesWith(int singleDigitNumber, int total) {
+    StringBuilder builder = new StringBuilder(singleDigitNumber);
+    for (int i = 1; i <= total; i++) {
+      builder.append(singleDigitNumber);
+    }
+    return builder.toString().getBytes(UTF_8);
+  }
+
+  /**
+   * Tests the reconstruction of single missed data index.
+   */
+  @Test
+  public void testECReconstructionCoordinatorWithMissingIndexes1()
+      throws Exception {
+    testECReconstructionCoordinator(ImmutableList.of(1));
+  }
+
+  /**
+   * Tests the reconstruction of single missed data index.
+   */
+  @Test
+  public void testECReconstructionCoordinatorWithMissingIndexes2()
+      throws Exception {
+    testECReconstructionCoordinator(ImmutableList.of(2));
+  }
+
+  /**
+   * Tests the reconstruction of single missed data index.
+   */
+  @Test
+  public void testECReconstructionCoordinatorWithMissingIndexes3()
+      throws Exception {
+    testECReconstructionCoordinator(ImmutableList.of(3));
+  }
+
+  /**
+   * Tests the reconstruction of single missed parity index.
+   */
+  @Test
+  public void testECReconstructionCoordinatorWithMissingIndexes4()
+      throws Exception {
+    testECReconstructionCoordinator(ImmutableList.of(4));
+  }
+
+  /**
+   * Tests the reconstruction of single missed parity index.
+   */
+  @Test
+  public void testECReconstructionCoordinatorWithMissingIndexes5()
+      throws Exception {
+    testECReconstructionCoordinator(ImmutableList.of(5));
+  }
+
+  /**
+   * Tests the reconstruction of all missed parity indexes.
+   */
+  @Test
+  public void testECReconstructionCoordinatorWithMissingIndexes45()
+      throws Exception {
+    testECReconstructionCoordinator(ImmutableList.of(4, 5));
+  }
+
+  /**
+   * Tests the reconstruction of multiple data indexes missed.
+   */
+  @Test
+  public void testECReconstructionCoordinatorWithMissingIndexes23()
+      throws Exception {
+    testECReconstructionCoordinator(ImmutableList.of(2, 3));
+  }
+
+  /**
+   * Tests the reconstruction of data and parity missing indexes.
+   */
+  @Test
+  public void testECReconstructionCoordinatorWithMissingIndexes24()
+      throws Exception {
+    testECReconstructionCoordinator(ImmutableList.of(2, 4));
+  }
+
+  /**
+   * Tests the reconstruction of data and parity missing indexes.
+   */
+  @Test
+  public void testECReconstructionCoordinatorWithMissingIndexes35()
+      throws Exception {
+    testECReconstructionCoordinator(ImmutableList.of(3, 5));
+  }

Review Comment:
   Can be done with single parameterized test:
   
   ```suggestion
     @ParameterizedTest
     @MethodSource("recoverableMissingIndexes")
     void testECReconstructionCoordinatorWith(List<Integer> missingIndexes)
         throws Exception {
       testECReconstructionCoordinator(missingIndexes);
     }
   
     static Stream<List<Integer>> recoverableMissingIndexes() {
       return Stream.concat(
           IntStream.rangeClosed(1, 5).mapToObj(ImmutableList::of),
           Stream.of(
               ImmutableList.of(2, 3),
               ImmutableList.of(2, 4),
               ImmutableList.of(3, 5),
               ImmutableList.of(4, 5)
       );
     }
   ```
   
   (+ needs imports)



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ec/reconstruction/ECReconstructionCoordinator.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.client.BlockID;
+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.ByteStringConversion;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.BlockLocationInfo;
+import org.apache.hadoop.hdds.scm.storage.BufferPool;
+import org.apache.hadoop.hdds.scm.storage.ECBlockOutputStream;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.io.ElasticByteBufferPool;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactory;
+import org.apache.hadoop.ozone.client.io.BlockInputStreamFactoryImpl;
+import org.apache.hadoop.ozone.client.io.ECBlockInputStreamProxy;
+import org.apache.hadoop.ozone.client.io.ECBlockReconstructedStripeInputStream;
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * The Coordinator implements the main flow of reconstructing
+ * missing container replicas.
+ * <p>
+ * For a container reconstruction task, the main flow is:
+ * - ListBlock from all healthy replicas
+ * - calculate effective block group len for all blocks
+ * - for each block
+ * - build a ReconstructInputStream to read healthy chunks
+ * - build a ECBlockOutputStream to write out decoded chunks
+ * - for each stripe
+ * - use ReconstructInputStream.readStripe to decode missing chunks
+ * - use ECBlockOutputStream.write to write decoded chunks to TargetDNs
+ * - PutBlock
+ * - CloseContainer
+ */
+public class ECReconstructionCoordinator implements Closeable {
+
+  static final Logger LOG =
+      LoggerFactory.getLogger(ECReconstructionCoordinator.class);
+
+  private static final int EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE = 3;
+
+  private final ECContainerOperationClient containerOperationClient;
+
+  private final ConfigurationSource config;
+
+  private final ByteBufferPool byteBufferPool;
+
+  private ExecutorService ecReconstructExecutor;
+
+  private BlockInputStreamFactory blockInputStreamFactory;
+
+  public ECReconstructionCoordinator(ECContainerOperationClient containerClient,
+      ConfigurationSource conf, ByteBufferPool byteBufferPool,
+      ExecutorService reconstructExecutor,
+      BlockInputStreamFactory streamFactory) {
+    this.containerOperationClient = containerClient;
+    this.config = conf;
+    this.byteBufferPool = byteBufferPool;
+    this.blockInputStreamFactory = streamFactory;
+    this.ecReconstructExecutor = reconstructExecutor;
+  }
+
+  public ECReconstructionCoordinator(ConfigurationSource conf,
+      CertificateClient certificateClient) throws IOException {
+    this(new ECContainerOperationClient(conf, certificateClient), conf,
+        new ElasticByteBufferPool(), null, null);
+    this.ecReconstructExecutor =
+        new ThreadPoolExecutor(EC_RECONSTRUCT_STRIPE_READ_POOL_MIN_SIZE,
+            config.getObject(OzoneClientConfig.class)
+                .getEcReconstructStripeReadPoolLimit(), 60, TimeUnit.SECONDS,
+            new SynchronousQueue<>(), new ThreadFactoryBuilder()
+            .setNameFormat("ec-reconstruct-reader-TID-%d").build(),
+            new ThreadPoolExecutor.CallerRunsPolicy());
+    this.blockInputStreamFactory = BlockInputStreamFactoryImpl
+        .getInstance(byteBufferPool, () -> ecReconstructExecutor);
+  }
+
+  public void reconstructECContainerGroup(long containerID,
+      ECReplicationConfig repConfig,
+      SortedMap<Integer, DatanodeDetails> sourceNodeMap,
+      SortedMap<Integer, DatanodeDetails> targetNodeMap) throws IOException {
+
+    Pipeline pipeline = rebuildInputPipeline(repConfig, sourceNodeMap);
+
+    SortedMap<Long, BlockData[]> blockDataMap =
+        getBlockDataMap(containerID, repConfig, sourceNodeMap);
+
+    SortedMap<Long, BlockLocationInfo> blockLocationInfoMap =
+        calcBlockLocationInfoMap(containerID, blockDataMap, pipeline);
+
+    // 1. create target recovering containers.
+    Set<Map.Entry<Integer, DatanodeDetails>> targetIndexDns =
+        targetNodeMap.entrySet();
+    Iterator<Map.Entry<Integer, DatanodeDetails>> iterator =
+        targetIndexDns.iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<Integer, DatanodeDetails> indexDnPair = iterator.next();
+      this.containerOperationClient
+          .createRecoveringContainer(containerID, indexDnPair.getValue(),
+              repConfig, null, indexDnPair.getKey());
+    }
+
+    // 2. Reconstruct and transfer to targets
+    for (BlockLocationInfo blockLocationInfo : blockLocationInfoMap.values()) {
+      reconstructECBlockGroup(blockLocationInfo, repConfig, targetNodeMap);
+    }
+
+    // 3. Close containers
+    iterator = targetIndexDns.iterator();
+    while (iterator.hasNext()) {
+      DatanodeDetails dn = iterator.next().getValue();
+      this.containerOperationClient
+          .closeContainer(containerID, dn, repConfig, null);
+    }
+
+  }
+
+  void reconstructECBlockGroup(BlockLocationInfo blockLocationInfo,
+      ECReplicationConfig repConfig,
+      SortedMap<Integer, DatanodeDetails> targetMap)
+      throws IOException {
+    long safeBlockGroupLength = blockLocationInfo.getLength();
+    List<Integer> missingContainerIndexes =
+        targetMap.keySet().stream().collect(Collectors.toList());
+
+    // calculate the real missing block indexes
+    int dataLocs = ECBlockInputStreamProxy
+        .expectedDataLocations(repConfig, safeBlockGroupLength);
+    List<Integer> toReconstructIndexes = new ArrayList<>();
+    for (int i = 0; i < missingContainerIndexes.size(); i++) {
+      Integer index = missingContainerIndexes.get(i);
+      if (index <= dataLocs || index > repConfig.getData()) {
+        toReconstructIndexes.add(index);
+      }
+      // else padded indexes.
+    }
+
+    // Looks like we don't need to reconstruct any missing blocks in this block
+    // group. The reason for this should be block group had only padding blocks
+    // in the missing locations.
+    if (toReconstructIndexes.size() == 0) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Skipping the reconstruction for the block: "
+            + blockLocationInfo.getBlockID() + ". In the missing locations: "
+            + missingContainerIndexes
+            + ", this block group has only padded blocks.");
+      }
+      return;
+    }
+
+    try (ECBlockReconstructedStripeInputStream sis
+        = new ECBlockReconstructedStripeInputStream(
+        repConfig, blockLocationInfo, true,
+        this.containerOperationClient.getXceiverClientManager(), null,
+        this.blockInputStreamFactory, byteBufferPool,
+        this.ecReconstructExecutor)) {
+
+      ECBlockOutputStream[] targetBlockStreams =
+          new ECBlockOutputStream[toReconstructIndexes.size()];
+      ByteBuffer[] bufs = new ByteBuffer[toReconstructIndexes.size()];
+      for (int i = 0; i < toReconstructIndexes.size(); i++) {
+        OzoneClientConfig configuration = new OzoneClientConfig();
+        // TODO: Let's avoid unnecessary bufferPool creation for
+        BufferPool bufferPool =
+            new BufferPool(configuration.getStreamBufferSize(),
+                (int) (configuration.getStreamBufferMaxSize() / configuration
+                    .getStreamBufferSize()),
+                ByteStringConversion.createByteBufferConversion(false));
+        targetBlockStreams[i] =
+            new ECBlockOutputStream(blockLocationInfo.getBlockID(),
+                this.containerOperationClient.getXceiverClientManager(),
+                Pipeline.newBuilder().setId(PipelineID.valueOf(
+                    targetMap.get(toReconstructIndexes.get(i)).getUuid()))
+                    .setReplicationConfig(repConfig).setNodes(ImmutableList
+                    .of(targetMap.get(toReconstructIndexes.get(i))))
+                    .setState(Pipeline.PipelineState.CLOSED).build(),
+                bufferPool, configuration, blockLocationInfo.getToken());
+        bufs[i] = byteBufferPool.getBuffer(false, repConfig.getEcChunkSize());
+        // Make sure it's clean. Don't want to reuse the erroneously returned
+        // buffers from the pool.
+        bufs[i].clear();
+      }
+
+      sis.setRecoveryIndexes(toReconstructIndexes.stream().map(i -> (i - 1))
+          .collect(Collectors.toSet()));
+      long length = safeBlockGroupLength;
+      while (length > 0) {
+        int readLen = sis.recoverChunks(bufs);
+        // TODO: can be submitted in parallel
+        for (int i = 0; i < bufs.length; i++) {
+          targetBlockStreams[i].write(bufs[i]);
+          if (isFailed(targetBlockStreams[i],
+              targetBlockStreams[i].getCurrentChunkResponseFuture())) {
+            // If one chunk response failed, we should retry.
+            // Even after retries if it failed, we should declare the
+            // reconstruction as failed.
+            // For now, let's throw the exception.
+            throw new IOException(
+                "Chunk write failed at the new target node: "
+                    + targetBlockStreams[i].getDatanodeDetails()
+                    + ". Aborting the reconstruction process.");
+          }

Review Comment:
   Nit: block can be extracted to method.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org