Browse Source

Revert "HDDS-1596. Create service endpoint to download configuration from SCM."

This reverts commit c0499bd70455e67bef9a1e00da73e25c9e2cc0ff.
Márton Elek 5 năm trước cách đây
mục cha
commit
371c9eb6a6
22 tập tin đã thay đổi với 23 bổ sung504 xóa
  1. 1 29
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/OzoneConfiguration.java
  2. 0 90
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/discovery/DiscoveryUtil.java
  3. 0 22
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/discovery/package-info.java
  4. 1 7
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
  5. 0 13
      hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/BaseHttpServer.java
  6. 1 11
      hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/ServerUtils.java
  7. 0 17
      hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/TestServerUtils.java
  8. 0 21
      hadoop-hdds/pom.xml
  9. 0 18
      hadoop-hdds/server-scm/pom.xml
  10. 0 60
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/discovery/ConfigurationEndpoint.java
  11. 0 44
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/discovery/ConfigurationXml.java
  12. 0 56
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/discovery/ConfigurationXmlEntry.java
  13. 0 35
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/discovery/DiscoveryApplication.java
  14. 0 22
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/discovery/package-info.java
  15. 4 5
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java
  16. 1 14
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManagerHttpServer.java
  17. 0 5
      hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManagerStarter.java
  18. 1 18
      hadoop-ozone/dist/src/main/compose/ozone/docker-compose.yaml
  19. 14 1
      hadoop-ozone/dist/src/main/compose/ozone/docker-config
  20. 0 5
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java
  21. 0 6
      hadoop-ozone/ozonefs/pom.xml
  22. 0 5
      hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/Gateway.java

+ 1 - 29
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/conf/OzoneConfiguration.java

@@ -47,43 +47,19 @@ public class OzoneConfiguration extends Configuration {
   }
 
   public OzoneConfiguration() {
-    this(false);
-  }
-
-  private OzoneConfiguration(boolean justTheDefaults) {
     OzoneConfiguration.activate();
     loadDefaults();
-    if (!justTheDefaults) {
-      loadConfigFiles();
-    }
-  }
-
-  private void loadConfigFiles() {
-    addResource("ozone-global.xml");
-    addResource("ozone-site.xml");
   }
 
   public OzoneConfiguration(Configuration conf) {
-    this(conf, false);
-  }
-
-  private OzoneConfiguration(Configuration conf, boolean justTheDefaults) {
     super(conf);
     //load the configuration from the classloader of the original conf.
     setClassLoader(conf.getClassLoader());
     if (!(conf instanceof OzoneConfiguration)) {
       loadDefaults();
-      //here we load the REAL configuration.
-      if (!justTheDefaults) {
-        loadConfigFiles();
-      }
     }
   }
 
-  public static OzoneConfiguration createWithDefaultsOnly() {
-    return new OzoneConfiguration(true);
-  }
-
   private void loadDefaults() {
     try {
       //there could be multiple ozone-default-generated.xml files on the
@@ -98,6 +74,7 @@ public class OzoneConfiguration extends Configuration {
     } catch (IOException e) {
       e.printStackTrace();
     }
+    addResource("ozone-site.xml");
   }
 
   public List<Property> readPropertyFromXml(URL url) throws JAXBException {
@@ -339,9 +316,4 @@ public class OzoneConfiguration extends Configuration {
     }
     return props;
   }
-
-  @Override
-  public synchronized Properties getProps() {
-    return super.getProps();
-  }
 }

+ 0 - 90
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/discovery/DiscoveryUtil.java

@@ -1,90 +0,0 @@
-/**
- * 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.hdds.discovery;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.net.URL;
-import java.nio.channels.Channels;
-import java.nio.channels.ReadableByteChannel;
-
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Utility to download ozone configuration from SCM.
- */
-public final class DiscoveryUtil {
-
-  private static final Logger LOG =
-      LoggerFactory.getLogger(DiscoveryUtil.class);
-
-  public static final String OZONE_GLOBAL_XML = "ozone-global.xml";
-
-  private DiscoveryUtil() {
-  }
-
-  /**
-   * Download ozone-global.conf from SCM to the local HADOOP_CONF_DIR.
-   */
-  public static boolean loadGlobalConfig(OzoneConfiguration conf) {
-    String hadoopConfDir = System.getenv("HADOOP_CONF_DIR");
-    if (hadoopConfDir == null || hadoopConfDir.isEmpty()) {
-      LOG.warn(
-          "HADOOP_CONF_DIR is not set, can't download ozone-global.xml from "
-              + "SCM.");
-      return false;
-    }
-    if (conf.get("ozone.scm.names") == null) {
-      LOG.warn("ozone.scm.names is not set. Can't download config from scm.");
-      return false;
-    }
-    for (int i = 0; i < 60; i++) {
-      for (String scmHost : conf.getStrings("ozone.scm.names")) {
-        String configOrigin =
-            String.format("http://%s:9876/discovery/config", scmHost);
-        File destinationFile = new File(hadoopConfDir, OZONE_GLOBAL_XML);
-
-        try {
-          LOG.info("Downloading {} to {}", configOrigin,
-              destinationFile.getAbsolutePath());
-          URL confUrl = new URL(configOrigin);
-          ReadableByteChannel rbc = Channels.newChannel(confUrl.openStream());
-          FileOutputStream fos =
-              new FileOutputStream(
-                  destinationFile);
-          fos.getChannel().transferFrom(rbc, 0, Long.MAX_VALUE);
-          return true;
-        } catch (Exception ex) {
-          LOG.error("Can't download config from " + configOrigin, ex);
-        }
-      }
-      LOG.warn(
-          "Configuration download was unsuccessful. Let's wait 5 seconds and"
-              + " retry.");
-      try {
-        Thread.sleep(5000);
-      } catch (InterruptedException e) {
-        LOG.error("Polling the config file upload is interrupted", e);
-      }
-    }
-    return false;
-  }
-}

+ 0 - 22
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/discovery/package-info.java

@@ -1,22 +0,0 @@
-/**
- * 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
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * 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.hdds.discovery;
-
-/*
- * Discovery/config service related classes.
- */

+ 1 - 7
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java

@@ -25,7 +25,6 @@ import org.apache.hadoop.hdds.HddsUtils;
 import org.apache.hadoop.hdds.cli.GenericCli;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.discovery.DiscoveryUtil;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
 import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
@@ -141,12 +140,7 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
       StringUtils
           .startupShutdownMessage(HddsDatanodeService.class, args, LOG);
     }
-    OzoneConfiguration ozoneConfiguration = createOzoneConfiguration();
-    if (DiscoveryUtil.loadGlobalConfig(ozoneConfiguration)) {
-      //reload the configuration with the downloaded new configs.
-      ozoneConfiguration = createOzoneConfiguration();
-    }
-    start(ozoneConfiguration);
+    start(createOzoneConfiguration());
     join();
     return null;
   }

+ 0 - 13
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/BaseHttpServer.java

@@ -33,7 +33,6 @@ import org.slf4j.LoggerFactory;
 import javax.servlet.http.HttpServlet;
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.util.Map;
 import java.util.Optional;
 
 import static org.apache.hadoop.hdds.HddsUtils.getHostNameFromConfigKeys;
@@ -128,18 +127,6 @@ public abstract class BaseHttpServer {
     httpServer.addServlet(servletName, pathSpec, clazz);
   }
 
-  /**
-   * Add a servlet to BaseHttpServer.
-   *
-   * @param servletName The name of the servlet
-   * @param pathSpec    The path spec for the servlet
-   * @param clazz       The servlet class
-   */
-  protected void addInternalServlet(String servletName, String pathSpec,
-      Class<? extends HttpServlet> clazz, Map<String, String> initParams) {
-    httpServer.addInternalServlet(servletName, pathSpec, clazz, initParams);
-  }
-
   /**
    * Returns the WebAppContext associated with this HttpServer.
    *

+ 1 - 11
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/ServerUtils.java

@@ -96,17 +96,7 @@ public final class ServerUtils {
         rpcServer.getListenerAddress());
   }
 
-  public static InetSocketAddress updateRPCListenPort(
-      OzoneConfiguration conf, String rpcAddressKey,
-      InetSocketAddress listenerAddress) {
-    String originalValue = conf.get(rpcAddressKey);
-    //remove existing port
-    originalValue = originalValue.replaceAll(":.*", "");
-    conf.set(rpcAddressKey,
-        originalValue + ":" + listenerAddress.getPort());
-    return new InetSocketAddress(originalValue,
-        listenerAddress.getPort());
-  }
+
   /**
    * After starting an server, updates configuration with the actual
    * listening address of that server. The listening address may be different

+ 0 - 17
hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/TestServerUtils.java

@@ -23,14 +23,11 @@ import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.test.PathUtils;
-
-import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
 import java.io.File;
-import java.net.InetSocketAddress;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -123,18 +120,4 @@ public class TestServerUtils {
     ServerUtils.getOzoneMetaDirPath(conf);
   }
 
-  @Test
-  public void updateRpcListenPort() {
-    OzoneConfiguration conf = new OzoneConfiguration();
-
-    conf.set("test1", "localhost:0");
-    ServerUtils.updateRPCListenPort(conf, "test1",
-        new InetSocketAddress("0.0.0.0", 1234));
-    Assert.assertEquals("localhost:1234", conf.get("test1"));
-
-    conf.set("test2", "localhost");
-    ServerUtils.updateRPCListenPort(conf, "test2",
-        new InetSocketAddress("0.0.0.0", 1234));
-    Assert.assertEquals("localhost:1234", conf.get("test2"));
-  }
 }

+ 0 - 21
hadoop-hdds/pom.xml

@@ -191,27 +191,6 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
         <version>${bouncycastle.version}</version>
       </dependency>
 
-      <dependency>
-        <groupId>com.sun.xml.bind</groupId>
-        <artifactId>jaxb-impl</artifactId>
-        <version>2.3.0.1</version>
-      </dependency>
-      <dependency>
-        <groupId>com.sun.xml.bind</groupId>
-        <artifactId>jaxb-core</artifactId>
-        <version>2.3.0.1</version>
-      </dependency>
-      <dependency>
-        <groupId>javax.xml.bind</groupId>
-        <artifactId>jaxb-api</artifactId>
-        <version>2.3.0</version>
-      </dependency>
-      <dependency>
-        <groupId>javax.activation</groupId>
-        <artifactId>activation</artifactId>
-        <version>1.1.1</version>
-      </dependency>
-
       <dependency>
         <groupId>org.junit.jupiter</groupId>
         <artifactId>junit-jupiter-api</artifactId>

+ 0 - 18
hadoop-hdds/server-scm/pom.xml

@@ -54,24 +54,6 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
       <artifactId>hadoop-hdds-docs</artifactId>
     </dependency>
 
-    <dependency>
-      <groupId>com.sun.xml.bind</groupId>
-      <artifactId>jaxb-impl</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.sun.xml.bind</groupId>
-      <artifactId>jaxb-core</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>javax.xml.bind</groupId>
-      <artifactId>jaxb-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>javax.activation</groupId>
-      <artifactId>activation</artifactId>
-    </dependency>
-
-
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-hdds-container-service</artifactId>

+ 0 - 60
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/discovery/ConfigurationEndpoint.java

@@ -1,60 +0,0 @@
-/**
- * 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.hdds.discovery;
-
-import javax.servlet.ServletContext;
-import javax.ws.rs.GET;
-import javax.ws.rs.Path;
-
-import java.util.Map.Entry;
-import java.util.Properties;
-
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.scm.server.StorageContainerManagerHttpServer;
-
-/**
- * JAXRS endpoint to publish current ozone configuration.
- */
-@Path("/config")
-public class ConfigurationEndpoint {
-
-  private Properties defaults =
-      OzoneConfiguration.createWithDefaultsOnly().getProps();
-
-  @javax.ws.rs.core.Context
-  private ServletContext context;
-
-  /**
-   * Returns with the non-default configuration.
-   */
-  @GET
-  public ConfigurationXml getConfiguration() {
-    OzoneConfiguration conf = (OzoneConfiguration) context.getAttribute(
-        StorageContainerManagerHttpServer.CONFIG_CONTEXT_ATTRIBUTE);
-    ConfigurationXml configXml = new ConfigurationXml();
-    for (Entry<Object, Object> entry : conf.getProps().entrySet()) {
-      //return only the non-defaults
-      if (defaults.get(entry.getKey()) != entry.getValue()) {
-        configXml.addConfiguration(entry.getKey().toString(),
-            entry.getValue().toString());
-      }
-    }
-    return configXml;
-  }
-
-}

+ 0 - 44
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/discovery/ConfigurationXml.java

@@ -1,44 +0,0 @@
-/**
- * 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.hdds.discovery;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * JAXB representation of Hadoop Configuration.
- */
-@XmlRootElement(name = "configuration")
-public class ConfigurationXml {
-
-  private List<ConfigurationXmlEntry> property = new ArrayList<>();
-
-  public List<ConfigurationXmlEntry> getProperty() {
-    return property;
-  }
-
-  public void setProperty(
-      List<ConfigurationXmlEntry> property) {
-    this.property = property;
-  }
-
-  public void addConfiguration(String key, String name) {
-    property.add(new ConfigurationXmlEntry(key, name));
-  }
-}

+ 0 - 56
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/discovery/ConfigurationXmlEntry.java

@@ -1,56 +0,0 @@
-/**
- * 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.hdds.discovery;
-
-import javax.xml.bind.annotation.XmlElement;
-
-/**
- * JAXB representation of one property of a hadoop configuration XML.
- */
-public class ConfigurationXmlEntry {
-
-  @XmlElement
-  private String name;
-
-  @XmlElement
-  private String value;
-
-  public ConfigurationXmlEntry() {
-  }
-
-  public ConfigurationXmlEntry(String name, String value) {
-    this.name = name;
-    this.value = value;
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public void setValue(String value) {
-    this.value = value;
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public String getValue() {
-    return value;
-  }
-}

+ 0 - 35
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/discovery/DiscoveryApplication.java

@@ -1,35 +0,0 @@
-/**
- * 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.hdds.discovery;
-
-import javax.ws.rs.core.Application;
-import java.util.HashSet;
-import java.util.Set;
-
-/**
- * JAXRS resource file registration.
- */
-public class DiscoveryApplication extends Application {
-
-  @Override
-  public Set<Class<?>> getClasses() {
-    Set<Class<?>> classes = new HashSet<>();
-    classes.add(DiscoveryApplication.class);
-    return classes;
-  }
-}

+ 0 - 22
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/discovery/package-info.java

@@ -1,22 +0,0 @@
-/**
- * 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
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * 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.hdds.discovery;
-
-/*
- * Discovery/config service related classes.
- */

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

@@ -54,8 +54,6 @@ import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.ozone.common.DeleteBlockGroupResult;
 import org.apache.hadoop.ozone.protocolPB
     .ScmBlockLocationProtocolServerSideTranslatorPB;
-
-import static org.apache.hadoop.hdds.server.ServerUtils.updateRPCListenPort;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -71,6 +69,7 @@ import static org.apache.hadoop.hdds.scm.ScmConfigKeys
     .OZONE_SCM_HANDLER_COUNT_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
     .OZONE_SCM_HANDLER_COUNT_KEY;
+import static org.apache.hadoop.hdds.server.ServerUtils.updateRPCListenAddress;
 import static org.apache.hadoop.hdds.scm.server.StorageContainerManager
     .startRpcServer;
 
@@ -120,9 +119,9 @@ public class SCMBlockProtocolServer implements
             blockProtoPbService,
             handlerCount);
     blockRpcAddress =
-        updateRPCListenPort(
-            conf, OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY,
-            blockRpcServer.getListenerAddress());
+        updateRPCListenAddress(
+            conf, OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY, scmBlockAddress,
+            blockRpcServer);
     if (conf.getBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
         false)) {
       blockRpcServer.refreshServiceAcl(conf, SCMPolicyProvider.getInstance());

+ 1 - 14
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManagerHttpServer.java

@@ -17,33 +17,20 @@
 
 package org.apache.hadoop.hdds.scm.server;
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.server.BaseHttpServer;
 
-import com.sun.jersey.spi.container.servlet.ServletContainer;
+import java.io.IOException;
 
 /**
  * HttpServer2 wrapper for the Ozone Storage Container Manager.
  */
 public class StorageContainerManagerHttpServer extends BaseHttpServer {
 
-  public static final String CONFIG_CONTEXT_ATTRIBUTE = "ozone.configuration";
-
   public StorageContainerManagerHttpServer(Configuration conf)
       throws IOException {
     super(conf, "scm");
-    Map<String, String> initParameters = new HashMap<>();
-    initParameters.put("com.sun.jersey.config.property.packages",
-        "org.apache.hadoop.hdds.discovery");
-    addInternalServlet("rest", "/discovery/*", ServletContainer.class,
-        initParameters);
-    getWebAppContext().setAttribute(CONFIG_CONTEXT_ATTRIBUTE, conf);
-
   }
 
   @Override protected String getHttpAddressKey() {

+ 0 - 5
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManagerStarter.java

@@ -24,7 +24,6 @@ package org.apache.hadoop.hdds.scm.server;
 import org.apache.hadoop.hdds.cli.GenericCli;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.discovery.DiscoveryUtil;
 import org.apache.hadoop.hdds.tracing.TracingUtil;
 import org.apache.hadoop.ozone.common.StorageInfo;
 import org.apache.hadoop.util.StringUtils;
@@ -122,10 +121,6 @@ public class StorageContainerManagerStarter extends GenericCli {
    */
   private void commonInit() {
     conf = createOzoneConfiguration();
-    if (DiscoveryUtil.loadGlobalConfig(conf)) {
-      //reload the configuration with the downloaded new configs.
-      conf = createOzoneConfiguration();
-    }
 
     String[] originalArgs = getCmd().getParseResult().originalArgs()
         .toArray(new String[0]);

+ 1 - 18
hadoop-ozone/dist/src/main/compose/ozone/docker-compose.yaml

@@ -25,8 +25,6 @@ services:
         - 9864
         - 9882
       command: ["/opt/hadoop/bin/ozone","datanode"]
-      environment:
-        OZONE-SITE.XML_ozone.scm.names: scm
       env_file:
         - ./docker-config
    om:
@@ -38,7 +36,6 @@ services:
          - 9874:9874
       environment:
          ENSURE_OM_INITIALIZED: /data/metadata/om/current/VERSION
-         OZONE-SITE.XML_ozone.scm.names: scm
       env_file:
           - ./docker-config
       command: ["/opt/hadoop/bin/ozone","om"]
@@ -52,19 +49,5 @@ services:
       env_file:
           - ./docker-config
       environment:
-        ENSURE_SCM_INITIALIZED: /data/metadata/scm/current/VERSION
-        OZONE-SITE.XML_ozone.scm.names: scm
-        OZONE-SITE.XML_ozone.om.address: om
-        OZONE-SITE.XML_ozone.om.http-address: om:9874
-        OZONE-SITE.XML_ozone.enabled: "true"
-        OZONE-SITE.XML_ozone.scm.datanode.id.dir: /data
-        OZONE-SITE.XML_ozone.scm.block.client.address: scm
-        OZONE-SITE.XML_ozone.metadata.dirs: /data/metadata
-        OZONE-SITE.XML_ozone.handler.type: distributed
-        OZONE-SITE.XML_ozone.scm.client.address: scm
-        OZONE-SITE.XML_ozone.replication: 1
-        OZONE-SITE.XML_hdds.datanode.dir: /data/hdds
-        OZONE-SITE.XML_hdds.profiler.endpoint.enabled: "true"
-        HDFS-SITE.XML_rpc.metrics.quantile.enable: "true"
-        HDFS-SITE.XML_rpc.metrics.percentiles.intervals: 60,300
+          ENSURE_SCM_INITIALIZED: /data/metadata/scm/current/VERSION
       command: ["/opt/hadoop/bin/ozone","scm"]

+ 14 - 1
hadoop-ozone/dist/src/main/compose/ozone/docker-config

@@ -14,7 +14,20 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-
+OZONE-SITE.XML_ozone.om.address=om
+OZONE-SITE.XML_ozone.om.http-address=om:9874
+OZONE-SITE.XML_ozone.scm.names=scm
+OZONE-SITE.XML_ozone.enabled=True
+OZONE-SITE.XML_ozone.scm.datanode.id.dir=/data
+OZONE-SITE.XML_ozone.scm.block.client.address=scm
+OZONE-SITE.XML_ozone.metadata.dirs=/data/metadata
+OZONE-SITE.XML_ozone.handler.type=distributed
+OZONE-SITE.XML_ozone.scm.client.address=scm
+OZONE-SITE.XML_ozone.replication=1
+OZONE-SITE.XML_hdds.datanode.dir=/data/hdds
+OZONE-SITE.XML_hdds.profiler.endpoint.enabled=true
+HDFS-SITE.XML_rpc.metrics.quantile.enable=true
+HDFS-SITE.XML_rpc.metrics.percentiles.intervals=60,300
 ASYNC_PROFILER_HOME=/opt/profiler
 LOG4J.PROPERTIES_log4j.rootLogger=INFO, stdout
 LOG4J.PROPERTIES_log4j.appender.stdout=org.apache.log4j.ConsoleAppender

+ 0 - 5
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.ozone.om;
 import org.apache.hadoop.hdds.cli.GenericCli;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.discovery.DiscoveryUtil;
 import org.apache.hadoop.hdds.tracing.TracingUtil;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.util.StringUtils;
@@ -101,10 +100,6 @@ public class OzoneManagerStarter extends GenericCli {
    */
   private void commonInit() {
     conf = createOzoneConfiguration();
-    if (DiscoveryUtil.loadGlobalConfig(conf)) {
-      //reload the configuration with the downloaded new configs.
-      conf = createOzoneConfiguration();
-    }
 
     String[] originalArgs = getCmd().getParseResult().originalArgs()
         .toArray(new String[0]);

+ 0 - 6
hadoop-ozone/ozonefs/pom.xml

@@ -130,12 +130,6 @@
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-ozone-common</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-hdds-server-scm</artifactId>
-        </exclusion>
-      </exclusions>
     </dependency>
     <dependency>
       <groupId>com.google.code.findbugs</groupId>

+ 0 - 5
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/Gateway.java

@@ -22,7 +22,6 @@ import java.io.IOException;
 import org.apache.hadoop.hdds.cli.GenericCli;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.discovery.DiscoveryUtil;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -48,10 +47,6 @@ public class Gateway extends GenericCli {
   @Override
   public Void call() throws Exception {
     OzoneConfiguration ozoneConfiguration = createOzoneConfiguration();
-    if (DiscoveryUtil.loadGlobalConfig(ozoneConfiguration)) {
-      //reload the configuration with the downloaded new configs.
-      ozoneConfiguration = createOzoneConfiguration();
-    }
     OzoneConfigurationHolder.setConfiguration(ozoneConfiguration);
     httpServer = new S3GatewayHttpServer(ozoneConfiguration, "s3gateway");
     start();