This is an automated email from the ASF dual-hosted git repository.

siyao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new 05773af  HDDS-6135. SCM Container DB bootstrap on Recon startup for 
SCM HA. (#2972)
05773af is described below

commit 05773afc893dadc8f4bc5e7c4560e586727f6531
Author: Aswin Shakil Balasubramanian <[email protected]>
AuthorDate: Thu Jan 20 05:01:29 2022 -0800

    HDDS-6135. SCM Container DB bootstrap on Recon startup for SCM HA. (#2972)
---
 .../hadoop/hdds/scm/ha/SCMRatisServerImpl.java     |   3 +-
 .../dist/src/main/compose/ozone-ha/docker-config   |   4 +
 .../hadoop/ozone/recon/TestReconScmHASnapshot.java |  70 +++++++++++
 .../hadoop/ozone/recon/TestReconScmSnapshot.java   | 133 +++++++++++++++++++++
 .../hadoop/ozone/recon/scm/ReconNodeManager.java   |   6 +
 .../impl/StorageContainerServiceProviderImpl.java  |  48 ++++++--
 6 files changed, 255 insertions(+), 9 deletions(-)

diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMRatisServerImpl.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMRatisServerImpl.java
index f639300..b48dfb6 100644
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMRatisServerImpl.java
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMRatisServerImpl.java
@@ -258,7 +258,8 @@ public class SCMRatisServerImpl implements SCMRatisServer {
               peer.getAddress().concat(isLocal ?
                   ":".concat(RaftProtos.RaftPeerRole.LEADER.toString()) :
                   ":".concat(RaftProtos.RaftPeerRole.FOLLOWER.toString()))
-                  .concat(":".concat(peer.getId().toString()))));
+                  .concat(":".concat(peer.getId().toString()))
+                  .concat(":".concat(peerInetAddress.getHostAddress()))));
     }
     return ratisRoles;
   }
diff --git a/hadoop-ozone/dist/src/main/compose/ozone-ha/docker-config 
b/hadoop-ozone/dist/src/main/compose/ozone-ha/docker-config
index 5b2632d..fa38aad 100644
--- a/hadoop-ozone/dist/src/main/compose/ozone-ha/docker-config
+++ b/hadoop-ozone/dist/src/main/compose/ozone-ha/docker-config
@@ -38,6 +38,10 @@ OZONE-SITE.XML_ozone.datanode.pipeline.limit=1
 OZONE-SITE.XML_hdds.scmclient.max.retry.timeout=30s
 OZONE-SITE.XML_ozone.scm.primordial.node.id=scm1
 OZONE-SITE.XML_hdds.container.report.interval=60s
+OZONE-SITE.XML_ozone.recon.db.dir=/data/metadata/recon
+OZONE-SITE.XML_ozone.recon.address=recon:9891
+OZONE-SITE.XML_ozone.recon.http-address=0.0.0.0:9888
+OZONE-SITE.XML_ozone.recon.https-address=0.0.0.0:9889
 
 OZONE_CONF_DIR=/etc/hadoop
 OZONE_LOG_DIR=/var/log/hadoop
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/recon/TestReconScmHASnapshot.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/recon/TestReconScmHASnapshot.java
new file mode 100644
index 0000000..2937770b
--- /dev/null
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/recon/TestReconScmHASnapshot.java
@@ -0,0 +1,70 @@
+/**
+ * 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.recon;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HA_ENABLE_KEY;
+
+/**
+ * Test Recon SCM HA Snapshot Download implementation.
+ */
+public class TestReconScmHASnapshot {
+
+  /**
+   * Set a timeout for each test.
+   */
+  @Rule
+  public Timeout timeout = Timeout.seconds(100);
+  private OzoneConfiguration conf;
+  private MiniOzoneCluster ozoneCluster = null;
+
+  @Before
+  public void setup() throws Exception {
+    conf = new OzoneConfiguration();
+    conf.setBoolean(OZONE_SCM_HA_ENABLE_KEY, true);
+    conf.setBoolean(
+        ReconServerConfigKeys.OZONE_RECON_SCM_SNAPSHOT_ENABLED, true);
+    conf.setInt(ReconServerConfigKeys.OZONE_RECON_SCM_CONTAINER_THRESHOLD, 0);
+    conf.setInt(ScmConfigKeys.OZONE_DATANODE_PIPELINE_LIMIT, 5);
+    ozoneCluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(4)
+        .includeRecon(true)
+        .build();
+    ozoneCluster.waitForClusterToBeReady();
+  }
+
+  @Test
+  public void testScmHASnapshot() throws Exception {
+    TestReconScmSnapshot.testSnapshot(ozoneCluster);
+  }
+
+  @After
+  public void shutdown() throws Exception {
+    if (ozoneCluster != null) {
+      ozoneCluster.shutdown();
+    }
+  }
+}
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/recon/TestReconScmSnapshot.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/recon/TestReconScmSnapshot.java
new file mode 100644
index 0000000..08f465f
--- /dev/null
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/recon/TestReconScmSnapshot.java
@@ -0,0 +1,133 @@
+/**
+ * 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.recon;
+
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.recon.scm.ReconNodeManager;
+import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test Recon SCM Snapshot Download implementation.
+ */
+public class TestReconScmSnapshot {
+  /**
+   * Set a timeout for each test.
+   */
+  @Rule
+  public Timeout timeout = Timeout.seconds(100);
+  private OzoneConfiguration conf;
+  private MiniOzoneCluster ozoneCluster = null;
+
+  @Before
+  public void setup() throws Exception {
+    conf = new OzoneConfiguration();
+    conf.setBoolean(
+        ReconServerConfigKeys.OZONE_RECON_SCM_SNAPSHOT_ENABLED, true);
+    conf.setInt(ReconServerConfigKeys.OZONE_RECON_SCM_CONTAINER_THRESHOLD, 0);
+    ozoneCluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(4)
+        .includeRecon(true)
+        .build();
+    ozoneCluster.waitForClusterToBeReady();
+  }
+
+  @Test
+  public void testScmSnapshot() throws Exception {
+    testSnapshot(ozoneCluster);
+  }
+
+  public static void testSnapshot(MiniOzoneCluster cluster) throws Exception{
+    GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
+        .captureLogs(LoggerFactory.getLogger(
+        ReconStorageContainerManagerFacade.class));
+
+    List<ContainerInfo> reconContainers = cluster.getReconServer()
+        .getReconStorageContainerManager().getContainerManager()
+        .getContainers();
+    assertEquals(0, reconContainers.size());
+
+    ReconNodeManager nodeManager;
+    nodeManager = (ReconNodeManager) cluster.getReconServer()
+        .getReconStorageContainerManager().getScmNodeManager();
+    long keyCountBefore = nodeManager.getNodeDBKeyCount();
+
+    //Stopping Recon to add Containers in SCM
+    cluster.stopRecon();
+
+    ContainerManager containerManager;
+    containerManager = cluster.getStorageContainerManager()
+        .getContainerManager();
+
+    for (int i = 0; i < 10; i++) {
+      containerManager.allocateContainer(new RatisReplicationConfig(
+          HddsProtos.ReplicationFactor.ONE), "testOwner");
+    }
+
+    cluster.startRecon();
+
+    //ContainerCount after Recon DB is updated with SCM DB
+    containerManager = cluster.getStorageContainerManager()
+        .getContainerManager();
+    ContainerManager reconContainerManager = cluster.getReconServer()
+        .getReconStorageContainerManager().getContainerManager();
+    assertTrue(logCapturer.getOutput()
+        .contains("Recon Container Count: " + reconContainers.size() +
+        ", SCM Container Count: " + containerManager.getContainers().size()));
+    assertEquals(containerManager.getContainers().size(),
+        reconContainerManager.getContainers().size());
+
+    //PipelineCount after Recon DB is updated with SCM DB
+    PipelineManager scmPipelineManager = cluster.getStorageContainerManager()
+        .getPipelineManager();
+    PipelineManager reconPipelineManager = cluster.getReconServer()
+        .getReconStorageContainerManager().getPipelineManager();
+    assertEquals(scmPipelineManager.getPipelines().size(),
+        reconPipelineManager.getPipelines().size());
+
+    //NodeCount after Recon DB updated with SCM DB
+    nodeManager = (ReconNodeManager) cluster.getReconServer()
+        .getReconStorageContainerManager().getScmNodeManager();
+    long keyCountAfter = nodeManager.getNodeDBKeyCount();
+    assertEquals(keyCountAfter, keyCountBefore);
+  }
+
+  @After
+  public void shutdown() throws Exception {
+    if (ozoneCluster != null) {
+      ozoneCluster.shutdown();
+    }
+  }
+}
diff --git 
a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconNodeManager.java
 
b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconNodeManager.java
index 74029e0..41cdc7a 100644
--- 
a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconNodeManager.java
+++ 
b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconNodeManager.java
@@ -26,6 +26,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import 
org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.LayoutVersionProto;
@@ -299,4 +300,9 @@ public class ReconNodeManager extends SCMNodeManager {
     this.nodeDB = nodeTable;
     loadExistingNodes();
   }
+
+  @VisibleForTesting
+  public long getNodeDBKeyCount() throws IOException {
+    return nodeDB.getEstimatedKeyCount();
+  }
 }
diff --git 
a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/StorageContainerServiceProviderImpl.java
 
b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/StorageContainerServiceProviderImpl.java
index 2fe423a..8f83c66 100644
--- 
a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/StorageContainerServiceProviderImpl.java
+++ 
b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/StorageContainerServiceProviderImpl.java
@@ -33,6 +33,7 @@ import java.io.InputStream;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.List;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
 import javax.inject.Inject;
@@ -42,8 +43,13 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import 
org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
+import org.apache.hadoop.hdds.scm.ha.InterSCMGrpcClient;
+import org.apache.hadoop.hdds.scm.ha.SCMHAUtils;
+import org.apache.hadoop.hdds.scm.ha.SCMSnapshotDownloader;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import 
org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
 import org.apache.hadoop.hdds.server.http.HttpConfig;
 import org.apache.hadoop.hdds.utils.db.DBCheckpoint;
 import org.apache.hadoop.hdds.utils.db.RocksDBCheckpoint;
@@ -51,6 +57,7 @@ import org.apache.hadoop.hdfs.web.URLConnectionFactory;
 import org.apache.hadoop.ozone.recon.ReconUtils;
 import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider;
 import org.apache.hadoop.security.SecurityUtil;
+import org.apache.ratis.proto.RaftProtos;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -160,15 +167,40 @@ public class StorageContainerServiceProviderImpl
             ".tar.gz");
 
     try {
-      SecurityUtil.doAsLoginUser(() -> {
-        try (InputStream inputStream = reconUtils.makeHttpCall(
-            connectionFactory, getScmDBSnapshotUrl(),
-            isOmSpnegoEnabled()).getInputStream()) {
-          FileUtils.copyInputStreamToFile(inputStream, targetFile);
+      if (!SCMHAUtils.isSCMHAEnabled(configuration)) {
+        SecurityUtil.doAsLoginUser(() -> {
+          try (InputStream inputStream = reconUtils.makeHttpCall(
+              connectionFactory, getScmDBSnapshotUrl(),
+              isOmSpnegoEnabled()).getInputStream()) {
+            FileUtils.copyInputStreamToFile(inputStream, targetFile);
+          }
+          return null;
+        });
+        LOG.info("Downloaded SCM Snapshot from SCM");
+      } else {
+        List<String> ratisRoles = scmClient.getScmInfo().getRatisPeerRoles();
+        for (String ratisRole: ratisRoles) {
+          String[] role = ratisRole.split(":");
+          if (role[2].equals(RaftProtos.RaftPeerRole.LEADER.toString())) {
+            String hostAddress = role[4].trim();
+            int grpcPort = configuration.getInt(
+                ScmConfigKeys.OZONE_SCM_GRPC_PORT_KEY,
+                ScmConfigKeys.OZONE_SCM_GRPC_PORT_DEFAULT);
+
+            try (SCMSnapshotDownloader downloadClient =
+                 new InterSCMGrpcClient(hostAddress, grpcPort,
+                 configuration, new SCMCertificateClient(
+                 new SecurityConfig(configuration)))) {
+              downloadClient.download(targetFile.toPath()).get();
+            } catch (ExecutionException | InterruptedException e) {
+              LOG.error("Rocks DB checkpoint downloading failed", e);
+              throw new IOException(e);
+            }
+            LOG.info("Downloaded SCM Snapshot from Leader SCM");
+            break;
+          }
         }
-        return null;
-      });
-
+      }
       Path untarredDbDir = Paths.get(scmSnapshotDBParentDir.getAbsolutePath(),
           snapshotFileName);
       reconUtils.untarCheckpointFile(targetFile, untarredDbDir);

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to