ソースを参照

HDDS-490. Improve om and scm start up options . Contributed by Namit Maheshwari.

Arpit Agarwal 6 年 前
コミット
2614078b21

+ 4 - 4
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java

@@ -927,10 +927,10 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
    * Startup options.
    */
   public enum StartupOption {
-    INIT("-init"),
-    CLUSTERID("-clusterid"),
-    GENCLUSTERID("-genclusterid"),
-    REGULAR("-regular"),
+    INIT("--init"),
+    CLUSTERID("--clusterid"),
+    GENCLUSTERID("--genclusterid"),
+    REGULAR("--regular"),
     HELP("-help");
 
     private final String name;

+ 2 - 2
hadoop-ozone/dist/src/main/compose/ozonescripts/start.sh

@@ -15,10 +15,10 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 docker-compose ps | grep datanode | awk '{print $1}' | xargs -n1  docker inspect --format '{{ .Config.Hostname }}' > ../../etc/hadoop/workers
-docker-compose exec scm /opt/hadoop/bin/ozone scm -init
+docker-compose exec scm /opt/hadoop/bin/ozone scm --init
 docker-compose exec scm /opt/hadoop/sbin/start-ozone.sh
 #We need a running SCM for om objectstore creation
 #TODO create a utility to wait for the startup
 sleep 10
-docker-compose exec om /opt/hadoop/bin/ozone om -createObjectStore
+docker-compose exec om /opt/hadoop/bin/ozone om --init
 docker-compose exec scm /opt/hadoop/sbin/start-ozone.sh

+ 5 - 5
hadoop-ozone/docs/content/RealCluster.md

@@ -26,7 +26,7 @@ menu:
 Before we boot up the Ozone cluster, we need to initialize both SCM and Ozone Manager.
 
 {{< highlight bash >}}
-ozone scm -init
+ozone scm --init
 {{< /highlight >}}
 This allows SCM to create the cluster Identity and initialize its state.
 The ```init``` command is similar to Namenode format. Init command is executed only once, that allows SCM to create all the required on-disk structures to work correctly.
@@ -37,7 +37,7 @@ ozone --daemon start scm
 Once we know SCM is up and running, we can create an Object Store for our use. This is done by running the following command.
 
 {{< highlight bash >}}
-ozone om -createObjectStore
+ozone om --init
 {{< /highlight >}}
 
 
@@ -50,7 +50,7 @@ ozone --daemon start om
 
 At this point Ozone's name services, the Ozone manager, and the block service  SCM is both running.
 **Please note**: If SCM is not running
-```createObjectStore``` command will fail. SCM start will fail if on-disk data structures are missing. So please make sure you have done both ```init``` and ```createObjectStore``` commands.
+```om --init``` command will fail. SCM start will fail if on-disk data structures are missing. So please make sure you have done both ```scm --init``` and ```om --init``` commands.
 
 Now we need to start the data nodes. Please run the following command on each datanode.
 {{< highlight bash >}}
@@ -64,8 +64,8 @@ At this point SCM, Ozone Manager and data nodes are up and running.
 -------
 If you want to make your life simpler, you can just run
 {{< highlight bash >}}
-ozone scm -init
-ozone om -createObjectStore
+ozone scm --init
+ozone om --init
 start-ozone.sh
 {{< /highlight >}}
 This assumes that you have set up the slaves file correctly and ssh

+ 103 - 0
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmInit.java

@@ -0,0 +1,103 @@
+/**
+ * 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.om;
+
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.web.handlers.UserArgs;
+import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
+import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.UUID;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS;
+
+/**
+ * Test Ozone Manager Init.
+ */
+public class TestOmInit {
+  private static MiniOzoneCluster cluster = null;
+  private static StorageHandler storageHandler;
+  private static UserArgs userArgs;
+  private static OMMetrics omMetrics;
+  private static OzoneConfiguration conf;
+  private static String clusterId;
+  private static String scmId;
+  private static String omId;
+
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
+  /**
+   * Create a MiniDFSCluster for testing.
+   * <p>
+   * Ozone is made active by setting OZONE_ENABLED = true
+   *
+   * @throws IOException
+   */
+  @BeforeClass
+  public static void init() throws Exception {
+    conf = new OzoneConfiguration();
+    clusterId = UUID.randomUUID().toString();
+    scmId = UUID.randomUUID().toString();
+    omId = UUID.randomUUID().toString();
+    conf.setInt(OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS, 2);
+    cluster =  MiniOzoneCluster.newBuilder(conf)
+        .setClusterId(clusterId)
+        .setScmId(scmId)
+        .setOmId(omId)
+        .build();
+    cluster.waitForClusterToBeReady();
+    storageHandler = new ObjectStoreHandler(conf).getStorageHandler();
+    userArgs = new UserArgs(null, OzoneUtils.getRequestID(),
+        null, null, null, null);
+    omMetrics = cluster.getOzoneManager().getMetrics();
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterClass
+  public static void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+
+  /**
+   * Tests the OM Initialization.
+   * @throws IOException
+   */
+  @Test
+  public void testOmInitAgain() throws IOException {
+    // Stop the Ozone Manager
+    cluster.getOzoneManager().stop();
+    // Now try to init the OM again. It should succeed
+    Assert.assertTrue(OzoneManager.omInit(conf));
+  }
+
+}

+ 7 - 7
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java

@@ -107,7 +107,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
 
   private static final String USAGE =
       "Usage: \n ozone om [genericOptions] " + "[ "
-          + StartupOption.CREATEOBJECTSTORE.getName() + " ]\n " + "ozone om [ "
+          + StartupOption.INIT.getName() + " ]\n " + "ozone om [ "
           + StartupOption.HELP.getName() + " ]\n";
   private final OzoneConfiguration configuration;
   private final RPC.Server omRpcServer;
@@ -318,7 +318,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
       return null;
     }
     switch (startOpt) {
-    case CREATEOBJECTSTORE:
+    case INIT:
       if (printBanner) {
         StringUtils.startupShutdownMessage(OzoneManager.class, argv, LOG);
       }
@@ -347,8 +347,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    * @throws IOException in case ozone metadata directory path is not
    *                     accessible
    */
-
-  private static boolean omInit(OzoneConfiguration conf) throws IOException {
+  @VisibleForTesting
+  static boolean omInit(OzoneConfiguration conf) throws IOException {
     OMStorage omStorage = new OMStorage(conf);
     StorageState state = omStorage.getState();
     if (state != StorageState.INITIALIZED) {
@@ -1164,9 +1164,9 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    * Startup options.
    */
   public enum StartupOption {
-    CREATEOBJECTSTORE("-createObjectStore"),
-    HELP("-help"),
-    REGULAR("-regular");
+    INIT("--init"),
+    HELP("--help"),
+    REGULAR("--regular");
 
     private final String name;