Browse Source

HDFS-7240 git commit fixup

Signed-off-by: Owen O'Malley <omalley@apache.org>
Owen O'Malley 7 years ago
parent
commit
2adda92de1
25 changed files with 303 additions and 277 deletions
  1. 21 1
      dev-support/bin/dist-layout-stitching
  2. 17 0
      hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh
  3. 34 0
      hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
  4. 0 26
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java
  5. 0 9
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecRegistry.java
  6. 13 0
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  7. 3 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
  8. 0 58
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java
  9. 150 0
      hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerLocationManagerImpl.java
  10. 0 9
      hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
  11. 5 73
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  12. 0 20
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  13. 1 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  14. 0 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  15. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  16. 0 29
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
  17. 6 1
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java
  18. 3 3
      hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/netty/ObjectStoreChannelHandler.java
  19. 4 4
      hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/netty/ObjectStoreJerseyContainer.java
  20. 2 2
      hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/netty/RequestContentObjectStoreChannelHandler.java
  21. 3 3
      hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/netty/RequestDispatchObjectStoreChannelHandler.java
  22. 39 0
      hadoop-project/pom.xml
  23. 1 1
      hadoop-tools/hadoop-ozone/pom.xml
  24. 0 7
      hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
  25. 0 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestUserGroupMappingPlacementRule.java

+ 21 - 1
dev-support/bin/dist-layout-stitching

@@ -21,6 +21,9 @@ VERSION=$1
 # project.build.directory
 # project.build.directory
 BASEDIR=$2
 BASEDIR=$2
 
 
+#hdds.version
+HDDS_VERSION=$3
+
 function run()
 function run()
 {
 {
   declare res
   declare res
@@ -132,7 +135,6 @@ run copy "${ROOT}/hadoop-hdfs-project/hadoop-hdfs-native-client/target/hadoop-hd
 run copy "${ROOT}/hadoop-hdfs-project/hadoop-hdfs-rbf/target/hadoop-hdfs-rbf-${VERSION}" .
 run copy "${ROOT}/hadoop-hdfs-project/hadoop-hdfs-rbf/target/hadoop-hdfs-rbf-${VERSION}" .
 run copy "${ROOT}/hadoop-yarn-project/target/hadoop-yarn-project-${VERSION}" .
 run copy "${ROOT}/hadoop-yarn-project/target/hadoop-yarn-project-${VERSION}" .
 run copy "${ROOT}/hadoop-mapreduce-project/target/hadoop-mapreduce-${VERSION}" .
 run copy "${ROOT}/hadoop-mapreduce-project/target/hadoop-mapreduce-${VERSION}" .
-run copy "${ROOT}/hadoop-tools/hadoop-tools-dist/target/hadoop-tools-dist-${VERSION}" .
 
 
 #copy httpfs and kms as is
 #copy httpfs and kms as is
 run cp -pr "${ROOT}/hadoop-hdfs-project/hadoop-hdfs-httpfs/target/hadoop-hdfs-httpfs-${VERSION}"/* .
 run cp -pr "${ROOT}/hadoop-hdfs-project/hadoop-hdfs-httpfs/target/hadoop-hdfs-httpfs-${VERSION}"/* .
@@ -144,6 +146,24 @@ run cp -p "${ROOT}/hadoop-client-modules/hadoop-client-api/target/hadoop-client-
 run cp -p "${ROOT}/hadoop-client-modules/hadoop-client-runtime/target/hadoop-client-runtime-${VERSION}.jar" share/hadoop/client/
 run cp -p "${ROOT}/hadoop-client-modules/hadoop-client-runtime/target/hadoop-client-runtime-${VERSION}.jar" share/hadoop/client/
 run cp -p "${ROOT}/hadoop-client-modules/hadoop-client-minicluster/target/hadoop-client-minicluster-${VERSION}.jar" share/hadoop/client/
 run cp -p "${ROOT}/hadoop-client-modules/hadoop-client-minicluster/target/hadoop-client-minicluster-${VERSION}.jar" share/hadoop/client/
 
 
+# HDDS
+run copy "${ROOT}/hadoop-hdds/common/target/hadoop-hdds-common-${HDDS_VERSION}" .
+run copy "${ROOT}/hadoop-hdds/framework/target/hadoop-hdds-server-framework-${HDDS_VERSION}" .
+run copy "${ROOT}/hadoop-hdds/server-scm/target/hadoop-hdds-server-scm-${HDDS_VERSION}" .
+run copy "${ROOT}/hadoop-hdds/container-service/target/hadoop-hdds-container-service-${HDDS_VERSION}" .
+run copy "${ROOT}/hadoop-hdds/client/target/hadoop-hdds-client-${HDDS_VERSION}" .
+run copy "${ROOT}/hadoop-hdds/tools/target/hadoop-hdds-tools-${HDDS_VERSION}" .
+
+# Ozone
+run copy "${ROOT}/hadoop-ozone/common/target/hadoop-ozone-common-${HDDS_VERSION}" .
+run copy "${ROOT}/hadoop-ozone/ozone-manager/target/hadoop-ozone-ozone-manager-${HDDS_VERSION}" .
+run copy "${ROOT}/hadoop-ozone/objectstore-service/target/hadoop-ozone-objectstore-service-${HDDS_VERSION}" .
+run copy "${ROOT}/hadoop-ozone/client/target/hadoop-ozone-client-${HDDS_VERSION}" .
+run copy "${ROOT}/hadoop-ozone/tools/target/hadoop-ozone-tools-${HDDS_VERSION}" .
+
+run copy "${ROOT}/hadoop-tools/hadoop-tools-dist/target/hadoop-tools-dist-${VERSION}" .
+
+
 echo
 echo
 echo "Hadoop dist layout available at: ${BASEDIR}/hadoop-${VERSION}"
 echo "Hadoop dist layout available at: ${BASEDIR}/hadoop-${VERSION}"
 echo
 echo

+ 17 - 0
hadoop-common-project/hadoop-common/src/main/conf/hadoop-env.sh

@@ -402,7 +402,24 @@ esac
 # and therefore may override any similar flags set in HADOOP_OPTS
 # and therefore may override any similar flags set in HADOOP_OPTS
 #
 #
 # export HDFS_DFSROUTER_OPTS=""
 # export HDFS_DFSROUTER_OPTS=""
+
+###
+# HDFS Key Space Manager specific parameters
 ###
 ###
+# Specify the JVM options to be used when starting the HDFS Key Space Manager.
+# These options will be appended to the options specified as HADOOP_OPTS
+# and therefore may override any similar flags set in HADOOP_OPTS
+#
+# export HDFS_KSM_OPTS=""
+
+###
+# HDFS StorageContainerManager specific parameters
+###
+# Specify the JVM options to be used when starting the HDFS Storage Container Manager.
+# These options will be appended to the options specified as HADOOP_OPTS
+# and therefore may override any similar flags set in HADOOP_OPTS
+#
+# export HDFS_STORAGECONTAINERMANAGER_OPTS=""
 
 
 ###
 ###
 # Advanced Users Only!
 # Advanced Users Only!

+ 34 - 0
hadoop-common-project/hadoop-common/src/main/conf/log4j.properties

@@ -293,6 +293,40 @@ log4j.appender.EWMA.cleanupInterval=${yarn.ewma.cleanupInterval}
 log4j.appender.EWMA.messageAgeLimitSeconds=${yarn.ewma.messageAgeLimitSeconds}
 log4j.appender.EWMA.messageAgeLimitSeconds=${yarn.ewma.messageAgeLimitSeconds}
 log4j.appender.EWMA.maxUniqueMessages=${yarn.ewma.maxUniqueMessages}
 log4j.appender.EWMA.maxUniqueMessages=${yarn.ewma.maxUniqueMessages}
 
 
+
+# Fair scheduler requests log on state dump
+log4j.logger.org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler.statedump=DEBUG,FSLOGGER
+log4j.additivity.org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler.statedump=false
+log4j.appender.FSLOGGER=org.apache.log4j.RollingFileAppender
+log4j.appender.FSLOGGER.File=${hadoop.log.dir}/fairscheduler-statedump.log
+log4j.appender.FSLOGGER.layout=org.apache.log4j.PatternLayout
+log4j.appender.FSLOGGER.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+log4j.appender.FSLOGGER.MaxFileSize=${hadoop.log.maxfilesize}
+log4j.appender.FSLOGGER.MaxBackupIndex=${hadoop.log.maxbackupindex}
+
+#
+# Add a logger for ozone that is separate from the Datanode.
+#
+log4j.logger.org.apache.hadoop.ozone=DEBUG,OZONE,FILE
+
+# Do not log into datanode logs. Remove this line to have single log.
+log4j.additivity.org.apache.hadoop.ozone=false
+
+# For development purposes, log both to console and log file.
+log4j.appender.OZONE=org.apache.log4j.ConsoleAppender
+log4j.appender.OZONE.Threshold=info
+log4j.appender.OZONE.layout=org.apache.log4j.PatternLayout
+log4j.appender.OZONE.layout.ConversionPattern=%d{ISO8601} [%t] %-5p \
+ %X{component} %X{function} %X{resource} %X{user} %X{request} - %m%n
+
+# Real ozone logger that writes to ozone.log
+log4j.appender.FILE=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.FILE.File=${hadoop.log.dir}/ozone.log
+log4j.appender.FILE.Threshold=debug
+log4j.appender.FILE.layout=org.apache.log4j.PatternLayout
+log4j.appender.FILE.layout.ConversionPattern=%d{ISO8601} [%t] %-5p \
+(%F:%L) %X{function} %X{resource} %X{user} %X{request} - \
+%m%n
 #
 #
 # Fair scheduler state dump
 # Fair scheduler state dump
 #
 #

+ 0 - 26
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java

@@ -161,32 +161,6 @@ public class ConfigUtil {
         targets);
         targets);
   }
   }
 
 
-  /**
-   *
-   * @param conf
-   * @param mountTableName
-   * @param src
-   * @param settings
-   * @param targets
-   */
-  public static void addLinkNfly(Configuration conf, String mountTableName,
-      String src, String settings, final URI ... targets) {
-
-    settings = settings == null
-        ? "minReplication=2,repairOnRead=true"
-        : settings;
-
-    conf.set(getConfigViewFsPrefix(mountTableName) + "." +
-            Constants.CONFIG_VIEWFS_LINK_NFLY + "." + settings + "." + src,
-        StringUtils.uriToString(targets));
-  }
-
-  public static void addLinkNfly(final Configuration conf, final String src,
-      final URI ... targets) {
-    addLinkNfly(conf, Constants.CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE, src, null,
-        targets);
-  }
-
   /**
   /**
    * Add config variable for homedir for default mount table
    * Add config variable for homedir for default mount table
    * @param conf - add to this conf
    * @param conf - add to this conf

+ 0 - 9
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecRegistry.java

@@ -179,13 +179,4 @@ public final class CodecRegistry {
   public Map<String, String> getCodec2CoderCompactMap() {
   public Map<String, String> getCodec2CoderCompactMap() {
     return coderNameCompactMap;
     return coderNameCompactMap;
   }
   }
-
-  /**
-   * Get all codec names and their corresponding coder list.
-   * @return a map of all codec names, and their corresponding code list
-   * separated by ','.
-   */
-  public HashMap<String, String> getCodec2CoderCompactMap() {
-    return coderNameCompactMap;
-  }
 }
 }

+ 13 - 0
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -1685,6 +1685,19 @@
 </property>
 </property>
 
 
 
 
+<!-- Ozone file system properties -->
+<property>
+  <name>fs.o3.impl</name>
+  <value>org.apache.hadoop.fs.ozone.OzoneFileSystem</value>
+  <description>The implementation class of the Ozone FileSystem.</description>
+</property>
+
+<property>
+  <name>fs.AbstractFileSystem.o3.impl</name>
+  <value>org.apache.hadoop.fs.ozone.OzFs</value>
+  <description>The implementation class of the OzFs AbstractFileSystem.</description>
+</property>
+
 <!-- ipc properties -->
 <!-- ipc properties -->
 
 
 <property>
 <property>

+ 3 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java

@@ -101,6 +101,9 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase {
     // S3A properties are in a different subtree.
     // S3A properties are in a different subtree.
     xmlPrefixToSkipCompare.add("fs.s3a.");
     xmlPrefixToSkipCompare.add("fs.s3a.");
 
 
+    // O3 properties are in a different subtree.
+    xmlPrefixToSkipCompare.add("fs.o3.");
+
     //ftp properties are in a different subtree.
     //ftp properties are in a different subtree.
     // - org.apache.hadoop.fs.ftp.FTPFileSystem.
     // - org.apache.hadoop.fs.ftp.FTPFileSystem.
     xmlPrefixToSkipCompare.add("fs.ftp.impl");
     xmlPrefixToSkipCompare.add("fs.ftp.impl");

+ 0 - 58
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfServlet.java

@@ -247,62 +247,4 @@ public class TestConfServlet {
     }
     }
     assertEquals("", sw.toString());
     assertEquals("", sw.toString());
   }
   }
-
-  @Test
-  public void testOzoneConfigTagCommands() throws Exception {
-    StringWriter sw = null;
-    PrintWriter pw = null;
-    ConfServlet service = null;
-    try {
-      service = new ConfServlet();
-      ServletContext context = mock(ServletContext.class);
-      ServletConfig servletConf = mock(ServletConfig.class);
-      service.init(servletConf);
-      Configuration config = mock(OzoneConfiguration.class);
-      when(context.getAttribute(HttpServer2.CONF_CONTEXT_ATTRIBUTE))
-          .thenReturn(new Configuration());
-      when(service.getServletContext())
-          .thenReturn(context);
-
-      HttpServletRequest request = mock(HttpServletRequest.class);
-      when(request.getParameter("cmd"))
-          .thenReturn("getOzoneTags");
-      HttpServletResponse response = mock(HttpServletResponse.class);
-      sw = new StringWriter();
-      pw = new PrintWriter(sw);
-      when(response.getWriter()).thenReturn(pw);
-
-      // response request
-      service.doGet(request, response);
-      String result = sw.toString().trim();
-
-      for (OzonePropertyTag ozoneTag : OzonePropertyTag.values()) {
-        assertTrue(result.contains(ozoneTag.toString()));
-      }
-
-      when(request.getParameter("cmd"))
-          .thenReturn("badCommand");
-      when(request.getParameter("tags"))
-          .thenReturn("OZONE,CLIENT");
-      when(request.getParameter("group"))
-          .thenReturn("ozone");
-
-      service.doGet(request, response);
-      Mockito.verify(response).sendError(
-          Mockito.eq(HttpServletResponse.SC_NOT_FOUND),
-          Mockito.eq("badCommand is not a valid command."));
-
-    } finally {
-      if (sw != null) {
-        sw.close();
-      }
-      if (pw != null) {
-        pw.close();
-      }
-      if (service != null) {
-        service.destroy();
-      }
-    }
-
-  }
 }
 }

+ 150 - 0
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerLocationManagerImpl.java

@@ -0,0 +1,150 @@
+/*
+ * 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.ozone.container.common.impl;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.metrics2.util.MBeans;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.container.common.interfaces
+    .ContainerLocationManager;
+import org.apache.hadoop.ozone.container.common.interfaces
+    .ContainerLocationManagerMXBean;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.management.ObjectName;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * A class that tells the ContainerManager where to place the containers.
+ * Please note : There is *no* one-to-one correlation between metadata
+ * Locations and data Locations.
+ *
+ *  For example : A user could map all container files to a
+ *  SSD but leave data/metadata on bunch of other disks.
+ */
+public class ContainerLocationManagerImpl implements ContainerLocationManager,
+    ContainerLocationManagerMXBean {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContainerLocationManagerImpl.class);
+
+  private final List<ContainerStorageLocation> dataLocations;
+  private int currentIndex;
+  private final List<StorageLocation> metadataLocations;
+  private final ObjectName jmxbean;
+
+  /**
+   * Constructs a Location Manager.
+   * @param metadataLocations  - Refers to the metadataLocations
+   * where we store the container metadata.
+   * @param dataDirs - metadataLocations where we store the actual
+   * data or chunk files.
+   * @param conf - configuration.
+   * @throws IOException
+   */
+  public ContainerLocationManagerImpl(List<StorageLocation> metadataLocations,
+      List<StorageLocation> dataDirs, Configuration conf)
+      throws IOException {
+    dataLocations = new LinkedList<>();
+    for (StorageLocation dataDir : dataDirs) {
+      dataLocations.add(new ContainerStorageLocation(dataDir, conf));
+    }
+    this.metadataLocations = metadataLocations;
+    jmxbean = MBeans.register("OzoneDataNode",
+        ContainerLocationManager.class.getSimpleName(), this);
+  }
+
+  /**
+   * Returns the path where the container should be placed from a set of
+   * metadataLocations.
+   *
+   * @return A path where we should place this container and metadata.
+   * @throws IOException
+   */
+  @Override
+  public Path getContainerPath()
+      throws IOException {
+    Preconditions.checkState(metadataLocations.size() > 0);
+    int index = currentIndex % metadataLocations.size();
+    return Paths.get(metadataLocations.get(index).getNormalizedUri());
+  }
+
+  /**
+   * Returns the path where the container Data file are stored.
+   *
+   * @return  a path where we place the LevelDB and data files of a container.
+   * @throws IOException
+   */
+  @Override
+  public Path getDataPath(String containerName) throws IOException {
+    Path currentPath = Paths.get(
+        dataLocations.get(currentIndex++ % dataLocations.size())
+            .getNormalizedUri());
+    currentPath = currentPath.resolve(OzoneConsts.CONTAINER_PREFIX);
+    return currentPath.resolve(containerName);
+  }
+
+  @Override
+  public StorageLocationReport[] getLocationReport() throws IOException {
+    StorageLocationReport[] reports =
+        new StorageLocationReport[dataLocations.size()];
+    for (int idx = 0; idx < dataLocations.size(); idx++) {
+      ContainerStorageLocation loc = dataLocations.get(idx);
+      long scmUsed = 0;
+      long remaining = 0;
+      try {
+        scmUsed = loc.getScmUsed();
+        remaining = loc.getAvailable();
+      } catch (IOException ex) {
+        LOG.warn("Failed to get scmUsed and remaining for container " +
+            "storage location {}", loc.getNormalizedUri());
+        // reset scmUsed and remaining if df/du failed.
+        scmUsed = 0;
+        remaining = 0;
+      }
+
+      // TODO: handle failed storage
+      // For now, include storage report for location that failed to get df/du.
+      StorageLocationReport r = new StorageLocationReport(
+          loc.getStorageUuId(), false, loc.getCapacity(),
+          scmUsed, remaining);
+      reports[idx] = r;
+    }
+    return reports;
+  }
+
+  /**
+   * Supports clean shutdown of container location du threads.
+   *
+   * @throws IOException
+   */
+  @Override
+  public void shutdown() throws IOException {
+    for (ContainerStorageLocation loc: dataLocations) {
+      loc.shutdown();
+    }
+    MBeans.unregister(jmxbean);
+  }
+}

+ 0 - 9
hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml

@@ -2,9 +2,6 @@
      <Match>
      <Match>
        <Package name="org.apache.hadoop.record.compiler.generated" />
        <Package name="org.apache.hadoop.record.compiler.generated" />
      </Match>
      </Match>
-     <Match>
-       <Package name="org.apache.hadoop.hdfs.ozone.protocol.proto" />
-     </Match>
      <Match>
      <Match>
        <Package name="org.apache.hadoop.hdfs.protocol.proto" />
        <Package name="org.apache.hadoop.hdfs.protocol.proto" />
      </Match>
      </Match>
@@ -17,12 +14,6 @@
      <Match>
      <Match>
        <Package name="org.apache.hadoop.hdfs.qjournal.protocol" />
        <Package name="org.apache.hadoop.hdfs.qjournal.protocol" />
      </Match>
      </Match>
-     <Match>
-       <Package name="org.apache.hadoop.ozone.protocol.proto" />
-     </Match>
-     <Match>
-       <Package name ="org.apache.hadoop.cblock.protocol.proto" />
-     </Match>
      <Match>
      <Match>
        <Bug pattern="EI_EXPOSE_REP" />
        <Bug pattern="EI_EXPOSE_REP" />
      </Match>
      </Match>

+ 5 - 73
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -168,6 +168,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <artifactId>netty</artifactId>
       <artifactId>netty</artifactId>
       <scope>compile</scope>
       <scope>compile</scope>
     </dependency>
     </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-all</artifactId>
+      <scope>compile</scope>
+    </dependency>
     <dependency>
     <dependency>
       <groupId>org.apache.htrace</groupId>
       <groupId>org.apache.htrace</groupId>
       <artifactId>htrace-core4</artifactId>
       <artifactId>htrace-core4</artifactId>
@@ -187,17 +192,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <groupId>org.fusesource.leveldbjni</groupId>
       <groupId>org.fusesource.leveldbjni</groupId>
       <artifactId>leveldbjni-all</artifactId>
       <artifactId>leveldbjni-all</artifactId>
     </dependency>
     </dependency>
-    <dependency>
-      <groupId>org.rocksdb</groupId>
-      <artifactId>rocksdbjni</artifactId>
-      <version>5.8.0</version>
-    </dependency>
-    <dependency>
-      <groupId>io.swagger</groupId>
-      <artifactId>swagger-annotations</artifactId>
-      <version>1.5.9</version>
-      <scope>provided</scope>
-    </dependency>
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
     <dependency>
     <dependency>
       <groupId>org.bouncycastle</groupId>
       <groupId>org.bouncycastle</groupId>
@@ -218,16 +212,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
         <artifactId>assertj-core</artifactId>
         <artifactId>assertj-core</artifactId>
         <scope>test</scope>
         <scope>test</scope>
     </dependency>
     </dependency>
-    <dependency>
-      <groupId>org.jctools</groupId>
-      <artifactId>jctools-core</artifactId>
-      <optional>true</optional>
-    </dependency>
-    <dependency>
-      <groupId>org.xerial</groupId>
-      <artifactId>sqlite-jdbc</artifactId>
-      <version>3.8.7</version>
-    </dependency>
   </dependencies>
   </dependencies>
 
 
   <build>
   <build>
@@ -320,7 +304,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <configuration>
             <configuration>
               <tasks>
               <tasks>
                 <copy file="src/main/resources/hdfs-default.xml" todir="src/site/resources"/>
                 <copy file="src/main/resources/hdfs-default.xml" todir="src/site/resources"/>
-                <copy file="src/main/resources/ozone-default.xml" todir="src/site/resources"/>
                 <copy file="src/main/xsl/configuration.xsl" todir="src/site/resources"/>
                 <copy file="src/main/xsl/configuration.xsl" todir="src/site/resources"/>
               </tasks>
               </tasks>
             </configuration>
             </configuration>
@@ -420,14 +403,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <exclude>src/main/webapps/static/dataTables.bootstrap.js</exclude>
             <exclude>src/main/webapps/static/dataTables.bootstrap.js</exclude>
             <exclude>src/main/webapps/static/d3-v4.1.1.min.js</exclude>
             <exclude>src/main/webapps/static/d3-v4.1.1.min.js</exclude>
             <exclude>src/test/resources/diskBalancer/data-cluster-3node-3disk.json</exclude>
             <exclude>src/test/resources/diskBalancer/data-cluster-3node-3disk.json</exclude>
-            <exclude>src/main/webapps/static/nvd3-1.8.5.min.css.map</exclude>
-            <exclude>src/main/webapps/static/nvd3-1.8.5.min.js</exclude>
-            <exclude>src/main/webapps/static/angular-route-1.6.4.min.js</exclude>
-            <exclude>src/main/webapps/static/nvd3-1.8.5.min.css</exclude>
-            <exclude>src/main/webapps/static/angular-nvd3-1.0.9.min.js</exclude>
-            <exclude>src/main/webapps/static/nvd3-1.8.5.min.js.map</exclude>
-            <exclude>src/main/webapps/static/angular-1.6.4.min.js</exclude>
-            <exclude>src/main/webapps/static/d3-3.5.17.min.js</exclude>
           </excludes>
           </excludes>
         </configuration>
         </configuration>
       </plugin>
       </plugin>
@@ -440,55 +415,12 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
               <includes>
               <includes>
                 <include>configuration.xsl</include>
                 <include>configuration.xsl</include>
                 <include>hdfs-default.xml</include>
                 <include>hdfs-default.xml</include>
-                <include>ozone-default.xml</include>
               </includes>
               </includes>
               <followSymlinks>false</followSymlinks>
               <followSymlinks>false</followSymlinks>
             </fileset>
             </fileset>
           </filesets>
           </filesets>
         </configuration>
         </configuration>
       </plugin>
       </plugin>
-      <plugin>
-        <groupId>com.github.kongchen</groupId>
-        <artifactId>swagger-maven-plugin</artifactId>
-        <version>3.1.5</version>
-        <executions>
-          <execution>
-            <phase>compile</phase>
-            <goals>
-              <goal>generate</goal>
-            </goals>
-          </execution>
-        </executions>
-        <configuration>
-          <apiSources>
-            <apiSource>
-              <springmvc>false</springmvc>
-              <swaggerDirectory>target/webapps/static</swaggerDirectory>
-              <swaggerFileName>ozone.swagger</swaggerFileName>
-              <schemes>
-                <scheme>http</scheme>
-              </schemes>
-              <host>localhost:9864</host>
-              <basePath>/</basePath>
-              <locations>
-                <location>org.apache.hadoop.ozone.web.interfaces</location>
-              </locations>
-              <info>
-                <title>HDFS Ozone REST Api</title>
-                <version>${project.version}</version>
-                <contact>
-                  <name>Apache Hadoop project</name>
-                  <url>https://hadoop.apache.org</url>
-                </contact>
-                <license>
-                  <url>http://www.apache.org/licenses/LICENSE-2.0.html</url>
-                  <name>Apache 2.0</name>
-                </license>
-              </info>
-            </apiSource>
-          </apiSources>
-        </configuration>
-      </plugin>
     </plugins>
     </plugins>
   </build>
   </build>
 
 

+ 0 - 20
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.stream.Collectors;
 import java.util.stream.Collectors;
@@ -1762,25 +1761,6 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     }
     }
   }
   }
 
 
-  @Override
-  public GetErasureCodingCodecsResponseProto getErasureCodingCodecs(
-      RpcController controller, GetErasureCodingCodecsRequestProto request)
-      throws ServiceException {
-    try {
-      HashMap<String, String> codecs = server.getErasureCodingCodecs();
-      GetErasureCodingCodecsResponseProto.Builder resBuilder =
-          GetErasureCodingCodecsResponseProto.newBuilder();
-      for (Map.Entry<String, String> codec : codecs.entrySet()) {
-        resBuilder.addCodec(
-            PBHelperClient.convertErasureCodingCodec(
-                codec.getKey(), codec.getValue()));
-      }
-      return resBuilder.build();
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
-  }
-
   @Override
   @Override
   public AddErasureCodingPoliciesResponseProto addErasureCodingPolicies(
   public AddErasureCodingPoliciesResponseProto addErasureCodingPolicies(
       RpcController controller, AddErasureCodingPoliciesRequestProto request)
       RpcController controller, AddErasureCodingPoliciesRequestProto request)

+ 1 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -111,7 +111,6 @@ import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.datanode.checker.DatasetVolumeChecker;
 import org.apache.hadoop.hdfs.server.datanode.checker.DatasetVolumeChecker;
 import org.apache.hadoop.hdfs.server.datanode.checker.StorageLocationChecker;
 import org.apache.hadoop.hdfs.server.datanode.checker.StorageLocationChecker;
-import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.hdfs.client.BlockReportOptions;
 import org.apache.hadoop.hdfs.client.BlockReportOptions;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@@ -1575,21 +1574,7 @@ public class DataNode extends ReconfigurableBase
     
     
     registerBlockPoolWithSecretManager(bpRegistration, blockPoolId);
     registerBlockPoolWithSecretManager(bpRegistration, blockPoolId);
   }
   }
-
-  @VisibleForTesting
-  public OzoneContainer getOzoneContainerManager() {
-    return this.datanodeStateMachine.getContainer();
-  }
-
-  @VisibleForTesting
-  public DatanodeStateMachine.DatanodeStates getOzoneStateMachineState() {
-    if (this.datanodeStateMachine != null) {
-      return this.datanodeStateMachine.getContext().getState();
-    }
-    // if the state machine doesn't exist then DN initialization is in progress
-    return DatanodeStateMachine.DatanodeStates.INIT;
-  }
-
+  
   /**
   /**
    * After the block pool has contacted the NN, registers that block pool
    * After the block pool has contacted the NN, registers that block pool
    * with the secret manager, updating it with the secrets provided by the NN.
    * with the secret manager, updating it with the secrets provided by the NN.

+ 0 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -7655,20 +7655,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
     }
   }
   }
 
 
-  /**
-   * Get available erasure coding codecs and corresponding coders.
-   */
-  HashMap<String, String> getErasureCodingCodecs() throws IOException {
-    checkOperation(OperationCategory.READ);
-    readLock();
-    try {
-      checkOperation(OperationCategory.READ);
-      return FSDirErasureCodingOp.getErasureCodingCodecs(this);
-    } finally {
-      readUnlock("getErasureCodingCodecs");
-    }
-  }
-
   void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag,
   void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag,
                 boolean logRetryCache)
                 boolean logRetryCache)
       throws IOException {
       throws IOException {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -4018,7 +4018,7 @@
 
 
 <property>
 <property>
   <name>dfs.journalnode.enable.sync</name>
   <name>dfs.journalnode.enable.sync</name>
-  <value>false</value>
+  <value>true</value>
   <description>
   <description>
     If true, the journal nodes wil sync with each other. The journal nodes
     If true, the journal nodes wil sync with each other. The journal nodes
     will periodically gossip with other journal nodes to compare edit log
     will periodically gossip with other journal nodes to compare edit log

+ 0 - 29
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java

@@ -894,9 +894,6 @@ public class TestDataNodeHotSwapVolumes {
           1, fsVolumeReferences.size());
           1, fsVolumeReferences.size());
     }
     }
 
 
-    // Add a new DataNode to help with the pipeline recover.
-    cluster.startDataNodes(conf, 1, true, null, null, null);
-
     // Verify the file has sufficient replications.
     // Verify the file has sufficient replications.
     DFSTestUtil.waitReplication(fs, testFile, REPLICATION);
     DFSTestUtil.waitReplication(fs, testFile, REPLICATION);
     // Read the content back
     // Read the content back
@@ -928,32 +925,6 @@ public class TestDataNodeHotSwapVolumes {
       assertTrue(String.format("DataNode(%d) should have more than 1 blocks",
       assertTrue(String.format("DataNode(%d) should have more than 1 blocks",
           dataNodeIdx), blockCount > 1);
           dataNodeIdx), blockCount > 1);
     }
     }
-
-    // Write more files to make sure that the DataNode that has removed volume
-    // is still alive to receive data.
-    for (int i = 0; i < 10; i++) {
-      final Path file = new Path("/after-" + i);
-      try (FSDataOutputStream fout = fs.create(file, REPLICATION)) {
-        rb.nextBytes(writeBuf);
-        fout.write(writeBuf);
-      }
-    }
-
-    try (FsDatasetSpi.FsVolumeReferences fsVolumeReferences = fsDatasetSpi
-        .getFsVolumeReferences()) {
-      assertEquals("Volume remove wasn't successful.",
-          1, fsVolumeReferences.size());
-      FsVolumeSpi volume = fsVolumeReferences.get(0);
-      String bpid = cluster.getNamesystem().getBlockPoolId();
-      FsVolumeSpi.BlockIterator blkIter = volume.newBlockIterator(bpid, "test");
-      int blockCount = 0;
-      while (!blkIter.atEnd()) {
-        blkIter.nextBlock();
-        blockCount++;
-      }
-      assertTrue(String.format("DataNode(%d) should have more than 1 blocks",
-          dataNodeIdx), blockCount > 1);
-    }
   }
   }
 
 
   @Test(timeout=60000)
   @Test(timeout=60000)

+ 6 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/server/TestContainerServer.java

@@ -81,7 +81,12 @@ public class TestContainerServer {
               ContainerTestHelper.createSingleNodePipeline(containerName));
               ContainerTestHelper.createSingleNodePipeline(containerName));
       channel.writeInbound(request);
       channel.writeInbound(request);
       Assert.assertTrue(channel.finish());
       Assert.assertTrue(channel.finish());
-      ContainerCommandResponseProto response = channel.readOutbound();
+
+      Object responseObject = channel.readOutbound();
+      Assert.assertTrue(responseObject instanceof
+          ContainerCommandResponseProto);
+      ContainerCommandResponseProto  response =
+          (ContainerCommandResponseProto) responseObject;
       Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
       Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
     } finally {
     } finally {
       if (channel != null) {
       if (channel != null) {

+ 3 - 3
hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/netty/ObjectStoreChannelHandler.java

@@ -26,9 +26,9 @@ import io.netty.handler.codec.http.HttpResponseStatus;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
-import static io.netty.handler.codec.http.HttpHeaderNames.CONNECTION;
-import static io.netty.handler.codec.http.HttpHeaderNames.CONTENT_LENGTH;
-import static io.netty.handler.codec.http.HttpHeaderValues.CLOSE;
+import static io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
+import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_LENGTH;
+import static io.netty.handler.codec.http.HttpHeaders.Values.CLOSE;
 import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
 import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
 import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
 import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
 
 

+ 4 - 4
hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/netty/ObjectStoreJerseyContainer.java

@@ -46,7 +46,7 @@ import com.sun.jersey.spi.container.ContainerResponseWriter;
 import com.sun.jersey.spi.container.WebApplication;
 import com.sun.jersey.spi.container.WebApplication;
 
 
 import io.netty.handler.codec.http.DefaultHttpResponse;
 import io.netty.handler.codec.http.DefaultHttpResponse;
-import io.netty.handler.codec.http.HttpHeaderUtil;
+//import io.netty.handler.codec.http.HttpUtil;
 import io.netty.handler.codec.http.HttpHeaders;
 import io.netty.handler.codec.http.HttpHeaders;
 import io.netty.handler.codec.http.HttpRequest;
 import io.netty.handler.codec.http.HttpRequest;
 import io.netty.handler.codec.http.HttpResponse;
 import io.netty.handler.codec.http.HttpResponse;
@@ -263,7 +263,7 @@ public final class ObjectStoreJerseyContainer {
       this.nettyResp = jerseyResponseToNettyResponse(jerseyResp);
       this.nettyResp = jerseyResponseToNettyResponse(jerseyResp);
       this.nettyResp.headers().set(CONTENT_LENGTH, Math.max(0, contentLength));
       this.nettyResp.headers().set(CONTENT_LENGTH, Math.max(0, contentLength));
       this.nettyResp.headers().set(CONNECTION,
       this.nettyResp.headers().set(CONNECTION,
-          HttpHeaderUtil.isKeepAlive(this.nettyReq) ? KEEP_ALIVE : CLOSE);
+          HttpHeaders.isKeepAlive(this.nettyReq) ? KEEP_ALIVE : CLOSE);
       this.latch.countDown();
       this.latch.countDown();
       LOG.trace(
       LOG.trace(
           "end writeStatusAndHeaders, contentLength = {}, jerseyResp = {}.",
           "end writeStatusAndHeaders, contentLength = {}, jerseyResp = {}.",
@@ -340,9 +340,9 @@ public final class ObjectStoreJerseyContainer {
     String host = nettyHeaders.get(HOST);
     String host = nettyHeaders.get(HOST);
     String scheme = host.startsWith("https") ? "https://" : "http://";
     String scheme = host.startsWith("https") ? "https://" : "http://";
     String baseUri = scheme + host + "/";
     String baseUri = scheme + host + "/";
-    String reqUri = scheme + host + nettyReq.uri();
+    String reqUri = scheme + host + nettyReq.getUri();
     LOG.trace("baseUri = {}, reqUri = {}", baseUri, reqUri);
     LOG.trace("baseUri = {}, reqUri = {}", baseUri, reqUri);
-    return new ContainerRequest(webapp, nettyReq.method().name(),
+    return new ContainerRequest(webapp, nettyReq.getMethod().name(),
         new URI(baseUri), new URI(reqUri), jerseyHeaders, reqIn);
         new URI(baseUri), new URI(reqUri), jerseyHeaders, reqIn);
   }
   }
 }
 }

+ 2 - 2
hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/netty/RequestContentObjectStoreChannelHandler.java

@@ -21,7 +21,7 @@ import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelFutureListener;
 import io.netty.channel.ChannelFutureListener;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.handler.codec.http.HttpContent;
 import io.netty.handler.codec.http.HttpContent;
-import io.netty.handler.codec.http.HttpHeaderUtil;
+import io.netty.handler.codec.http.HttpHeaders;
 import io.netty.handler.codec.http.HttpRequest;
 import io.netty.handler.codec.http.HttpRequest;
 import io.netty.handler.codec.http.HttpResponse;
 import io.netty.handler.codec.http.HttpResponse;
 import io.netty.handler.codec.http.LastHttpContent;
 import io.netty.handler.codec.http.LastHttpContent;
@@ -85,7 +85,7 @@ public final class RequestContentObjectStoreChannelHandler
       ChannelFuture respFuture = ctx.writeAndFlush(new ChunkedStream(
       ChannelFuture respFuture = ctx.writeAndFlush(new ChunkedStream(
           this.respIn));
           this.respIn));
       respFuture.addListener(new CloseableCleanupListener(this.respIn));
       respFuture.addListener(new CloseableCleanupListener(this.respIn));
-      if (!HttpHeaderUtil.isKeepAlive(this.nettyReq)) {
+      if (!HttpHeaders.isKeepAlive(this.nettyReq)) {
         respFuture.addListener(ChannelFutureListener.CLOSE);
         respFuture.addListener(ChannelFutureListener.CLOSE);
       } else {
       } else {
         respFuture.addListener(new ChannelFutureListener() {
         respFuture.addListener(new ChannelFutureListener() {

+ 3 - 3
hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/netty/RequestDispatchObjectStoreChannelHandler.java

@@ -19,7 +19,7 @@ package org.apache.hadoop.ozone.web.netty;
 
 
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.handler.codec.http.DefaultFullHttpResponse;
 import io.netty.handler.codec.http.DefaultFullHttpResponse;
-import io.netty.handler.codec.http.HttpHeaderUtil;
+import io.netty.handler.codec.http.HttpHeaders;
 import io.netty.handler.codec.http.HttpRequest;
 import io.netty.handler.codec.http.HttpRequest;
 import io.netty.handler.codec.http.HttpResponse;
 import io.netty.handler.codec.http.HttpResponse;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
@@ -67,7 +67,7 @@ public final class RequestDispatchObjectStoreChannelHandler
       throws Exception {
       throws Exception {
     LOG.trace("begin RequestDispatchObjectStoreChannelHandler channelRead0, " +
     LOG.trace("begin RequestDispatchObjectStoreChannelHandler channelRead0, " +
         "ctx = {}, nettyReq = {}", ctx, nettyReq);
         "ctx = {}, nettyReq = {}", ctx, nettyReq);
-    if (!nettyReq.decoderResult().isSuccess()) {
+    if (!nettyReq.getDecoderResult().isSuccess()) {
       sendErrorResponse(ctx, BAD_REQUEST);
       sendErrorResponse(ctx, BAD_REQUEST);
       return;
       return;
     }
     }
@@ -77,7 +77,7 @@ public final class RequestDispatchObjectStoreChannelHandler
     this.respIn = new PipedInputStream();
     this.respIn = new PipedInputStream();
     this.respOut = new PipedOutputStream(respIn);
     this.respOut = new PipedOutputStream(respIn);
 
 
-    if (HttpHeaderUtil.is100ContinueExpected(nettyReq)) {
+    if (HttpHeaders.is100ContinueExpected(nettyReq)) {
       LOG.trace("Sending continue response.");
       LOG.trace("Sending continue response.");
       ctx.writeAndFlush(new DefaultFullHttpResponse(HTTP_1_1, CONTINUE));
       ctx.writeAndFlush(new DefaultFullHttpResponse(HTTP_1_1, CONTINUE));
     }
     }

+ 39 - 0
hadoop-project/pom.xml

@@ -96,6 +96,8 @@
     <apacheds.version>2.0.0-M21</apacheds.version>
     <apacheds.version>2.0.0-M21</apacheds.version>
     <ldap-api.version>1.0.0-M33</ldap-api.version>
     <ldap-api.version>1.0.0-M33</ldap-api.version>
 
 
+    <!-- Apache Ratis version -->
+    <ratis.version>0.1.1-alpha-8fd74ed-SNAPSHOT</ratis.version>
     <jcache.version>1.0-alpha-1</jcache.version>
     <jcache.version>1.0-alpha-1</jcache.version>
     <ehcache.version>3.3.1</ehcache.version>
     <ehcache.version>3.3.1</ehcache.version>
     <hikari.version>2.4.12</hikari.version>
     <hikari.version>2.4.12</hikari.version>
@@ -873,6 +875,43 @@
         </exclusions>
         </exclusions>
       </dependency>
       </dependency>
 
 
+      <dependency>
+        <groupId>org.jctools</groupId>
+        <artifactId>jctools-core</artifactId>
+        <version>1.2.1</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.ratis</groupId>
+        <artifactId>ratis-proto-shaded</artifactId>
+        <version>${ratis.version}</version>
+      </dependency>
+      <dependency>
+        <artifactId>ratis-common</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${ratis.version}</version>
+      </dependency>
+      <dependency>
+        <artifactId>ratis-client</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${ratis.version}</version>
+      </dependency>
+      <dependency>
+        <artifactId>ratis-server</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${ratis.version}</version>
+      </dependency>
+      <dependency>
+        <artifactId>ratis-netty</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${ratis.version}</version>
+      </dependency>
+      <dependency>
+        <artifactId>ratis-grpc</artifactId>
+        <groupId>org.apache.ratis</groupId>
+        <version>${ratis.version}</version>
+      </dependency>
+
       <dependency>
       <dependency>
         <groupId>io.netty</groupId>
         <groupId>io.netty</groupId>
         <artifactId>netty</artifactId>
         <artifactId>netty</artifactId>

+ 1 - 1
hadoop-tools/hadoop-ozone/pom.xml

@@ -19,7 +19,7 @@
   <parent>
   <parent>
     <groupId>org.apache.hadoop</groupId>
     <groupId>org.apache.hadoop</groupId>
     <artifactId>hadoop-project</artifactId>
     <artifactId>hadoop-project</artifactId>
-    <version>3.1.0-SNAPSHOT</version>
+    <version>3.2.0-SNAPSHOT</version>
     <relativePath>../../hadoop-project</relativePath>
     <relativePath>../../hadoop-project</relativePath>
   </parent>
   </parent>
   <artifactId>hadoop-ozone-filesystem</artifactId>
   <artifactId>hadoop-ozone-filesystem</artifactId>

+ 0 - 7
hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java

@@ -216,13 +216,6 @@ public class SLSRunner extends Configured implements Tool {
     return Collections.unmodifiableMap(simulateInfoMap);
     return Collections.unmodifiableMap(simulateInfoMap);
   }
   }
 
 
-  /**
-   * @return an unmodifiable view of the simulated info map.
-   */
-  public static Map<String, Object> getSimulateInfoMap() {
-    return Collections.unmodifiableMap(simulateInfoMap);
-  }
-
   public void setSimulationParams(TraceType inType, String[] inTraces,
   public void setSimulationParams(TraceType inType, String[] inTraces,
       String nodes, String outDir, Set<String> trackApps,
       String nodes, String outDir, Set<String> trackApps,
       boolean printsimulation) throws IOException, ClassNotFoundException {
       boolean printsimulation) throws IOException, ClassNotFoundException {

+ 0 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/placement/TestUserGroupMappingPlacementRule.java