Browse Source

HDDS-1168.Use random ports in TestBlockManager and TestDeletedBlockLog.
Contributed by Nandakumar.

Anu Engineer 6 năm trước cách đây
mục cha
commit
021f8deef0

+ 33 - 1
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.hdds.protocol.proto
 import org.apache.hadoop.hdds.protocol.proto
         .StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.server.SCMConfigurator;
 import org.apache.hadoop.hdds.scm.server
     .SCMDatanodeHeartbeatDispatcher.PipelineActionsFromDatanode;
 import org.apache.hadoop.hdds.scm.server
@@ -474,9 +475,40 @@ public final class TestUtils {
 
   }
 
+  /**
+   * Construct and returns StorageContainerManager instance using the given
+   * configuration. The ports used by this StorageContainerManager are
+   * randomly selected from free ports available.
+   *
+   * @param conf OzoneConfiguration
+   * @return StorageContainerManager instance
+   * @throws IOException
+   * @throws AuthenticationException
+   */
   public static StorageContainerManager getScm(OzoneConfiguration conf)
       throws IOException, AuthenticationException {
+    return getScm(conf, new SCMConfigurator());
+  }
+
+  /**
+   * Construct and returns StorageContainerManager instance using the given
+   * configuration and the configurator. The ports used by this
+   * StorageContainerManager are randomly selected from free ports available.
+   *
+   * @param conf OzoneConfiguration
+   * @param configurator SCMConfigurator
+   * @return StorageContainerManager instance
+   * @throws IOException
+   * @throws AuthenticationException
+   */
+  public static StorageContainerManager getScm(OzoneConfiguration conf,
+                                               SCMConfigurator configurator)
+      throws IOException, AuthenticationException {
     conf.setBoolean(OZONE_ENABLED, true);
+    conf.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "127.0.0.1:0");
+    conf.set(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY, "127.0.0.1:0");
+    conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY, "127.0.0.1:0");
+    conf.set(ScmConfigKeys.OZONE_SCM_HTTP_ADDRESS_KEY, "127.0.0.1:0");
     SCMStorageConfig scmStore = new SCMStorageConfig(conf);
     if(scmStore.getState() != Storage.StorageState.INITIALIZED) {
       String clusterId = UUID.randomUUID().toString();
@@ -486,7 +518,7 @@ public final class TestUtils {
       // writes the version file properties
       scmStore.initialize();
     }
-    return StorageContainerManager.createSCM(null, conf);
+    return new StorageContainerManager(conf, configurator);
   }
 
 }

+ 2 - 22
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java

@@ -19,12 +19,12 @@ package org.apache.hadoop.hdds.scm.block;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.UUID;
 import java.util.concurrent.TimeoutException;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 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.TestUtils;
 import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.container.MockNodeManager;
@@ -35,14 +35,11 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
 import org.apache.hadoop.hdds.scm.pipeline.RatisPipelineUtils;
 import org.apache.hadoop.hdds.scm.server.SCMConfigurator;
-import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
 import org.apache.hadoop.hdds.server.events.EventHandler;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
 import org.apache.hadoop.hdds.server.events.EventQueue;
-import org.apache.hadoop.ozone.common.Storage.StorageState;
 import org.apache.hadoop.ozone.container.common.SCMTestUtils;
-import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Assert;
@@ -52,7 +49,6 @@ import org.junit.Test;
 import org.junit.rules.ExpectedException;
 import org.junit.rules.TemporaryFolder;
 
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED;
 import static org.apache.hadoop.ozone.OzoneConsts.GB;
 import static org.apache.hadoop.ozone.OzoneConsts.MB;
 
@@ -95,7 +91,7 @@ public class TestBlockManager implements EventHandler<Boolean> {
     nodeManager = new MockNodeManager(true, 10);
     SCMConfigurator configurator = new SCMConfigurator();
     configurator.setScmNodeManager(nodeManager);
-    scm = getScm(conf, configurator);
+    scm = TestUtils.getScm(conf, configurator);
 
     // Initialize these fields so that the tests can pass.
     mapping = (SCMContainerManager) scm.getContainerManager();
@@ -124,22 +120,6 @@ public class TestBlockManager implements EventHandler<Boolean> {
     scm.stop();
   }
 
-  private static StorageContainerManager getScm(OzoneConfiguration conf,
-                                                SCMConfigurator configurator)
-      throws IOException, AuthenticationException {
-    conf.setBoolean(OZONE_ENABLED, true);
-    SCMStorageConfig scmStore = new SCMStorageConfig(conf);
-    if(scmStore.getState() != StorageState.INITIALIZED) {
-      String clusterId = UUID.randomUUID().toString();
-      String scmId = UUID.randomUUID().toString();
-      scmStore.setClusterId(clusterId);
-      scmStore.setScmId(scmId);
-      // writes the version file properties
-      scmStore.initialize();
-    }
-    return new StorageContainerManager(conf, configurator);
-  }
-
   @Test
   public void testAllocateBlock() throws Exception {
     eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, false);