You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by eh...@apache.org on 2018/09/04 09:08:44 UTC

[01/50] [abbrv] hadoop git commit: YARN-8705. Refactor the UAM heartbeat thread in preparation for YARN-8696. Contributed by Botong Huang. [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-12090 959f49b48 -> 06477abcd (forced update)


YARN-8705. Refactor the UAM heartbeat thread in preparation for YARN-8696. Contributed by Botong Huang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f1525825
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f1525825
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f1525825

Branch: refs/heads/HDFS-12090
Commit: f1525825623a1307b5aa55c456b6afa3e0c61135
Parents: 7b1fa56
Author: Giovanni Matteo Fumarola <gi...@microsoft.com>
Authored: Mon Aug 27 10:32:22 2018 -0700
Committer: Giovanni Matteo Fumarola <gi...@microsoft.com>
Committed: Mon Aug 27 10:32:22 2018 -0700

----------------------------------------------------------------------
 .../yarn/server/AMHeartbeatRequestHandler.java  | 227 +++++++++++++++++
 .../server/uam/UnmanagedApplicationManager.java | 170 ++-----------
 .../amrmproxy/FederationInterceptor.java        | 245 +++++++++----------
 3 files changed, 358 insertions(+), 284 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f1525825/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMHeartbeatRequestHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMHeartbeatRequestHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMHeartbeatRequestHandler.java
new file mode 100644
index 0000000..42227bb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/AMHeartbeatRequestHandler.java
@@ -0,0 +1,227 @@
+/**
+* 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.yarn.server;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
+import org.apache.hadoop.yarn.util.AsyncCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+/**
+ * Extends Thread and provides an implementation that is used for processing the
+ * AM heart beat request asynchronously and sending back the response using the
+ * callback method registered with the system.
+ */
+public class AMHeartbeatRequestHandler extends Thread {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(AMHeartbeatRequestHandler.class);
+
+  // Indication flag for the thread to keep running
+  private volatile boolean keepRunning;
+
+  private Configuration conf;
+  private ApplicationId applicationId;
+
+  private BlockingQueue<AsyncAllocateRequestInfo> requestQueue;
+  private AMRMClientRelayer rmProxyRelayer;
+  private UserGroupInformation userUgi;
+  private int lastResponseId;
+
+  public AMHeartbeatRequestHandler(Configuration conf,
+      ApplicationId applicationId) {
+    super("AMHeartbeatRequestHandler Heartbeat Handler Thread");
+    this.setUncaughtExceptionHandler(
+        new HeartBeatThreadUncaughtExceptionHandler());
+    this.keepRunning = true;
+
+    this.conf = conf;
+    this.applicationId = applicationId;
+    this.requestQueue = new LinkedBlockingQueue<>();
+
+    resetLastResponseId();
+  }
+
+  /**
+   * Shutdown the thread.
+   */
+  public void shutdown() {
+    this.keepRunning = false;
+    this.interrupt();
+  }
+
+  @Override
+  public void run() {
+    while (keepRunning) {
+      AsyncAllocateRequestInfo requestInfo;
+      try {
+        requestInfo = requestQueue.take();
+        if (requestInfo == null) {
+          throw new YarnException(
+              "Null requestInfo taken from request queue");
+        }
+        if (!keepRunning) {
+          break;
+        }
+
+        // change the response id before forwarding the allocate request as we
+        // could have different values for each UAM
+        AllocateRequest request = requestInfo.getRequest();
+        if (request == null) {
+          throw new YarnException("Null allocateRequest from requestInfo");
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Sending Heartbeat to Unmanaged AM. AskList:"
+              + ((request.getAskList() == null) ? " empty"
+                  : request.getAskList().size()));
+        }
+
+        request.setResponseId(lastResponseId);
+        AllocateResponse response = rmProxyRelayer.allocate(request);
+        if (response == null) {
+          throw new YarnException("Null allocateResponse from allocate");
+        }
+
+        lastResponseId = response.getResponseId();
+        // update token if RM has reissued/renewed
+        if (response.getAMRMToken() != null) {
+          LOG.debug("Received new AMRMToken");
+          YarnServerSecurityUtils.updateAMRMToken(response.getAMRMToken(),
+              userUgi, conf);
+        }
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Received Heartbeat reply from RM. Allocated Containers:"
+              + ((response.getAllocatedContainers() == null) ? " empty"
+                  : response.getAllocatedContainers().size()));
+        }
+
+        if (requestInfo.getCallback() == null) {
+          throw new YarnException("Null callback from requestInfo");
+        }
+        requestInfo.getCallback().callback(response);
+      } catch (InterruptedException ex) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Interrupted while waiting for queue", ex);
+        }
+      } catch (Throwable ex) {
+        LOG.warn(
+            "Error occurred while processing heart beat for " + applicationId,
+            ex);
+      }
+    }
+
+    LOG.info("AMHeartbeatRequestHandler thread for {} is exiting",
+        applicationId);
+  }
+
+  /**
+   * Reset the lastResponseId to zero.
+   */
+  public void resetLastResponseId() {
+    this.lastResponseId = 0;
+  }
+
+  /**
+   * Set the AMRMClientRelayer for RM connection.
+   */
+  public void setAMRMClientRelayer(AMRMClientRelayer relayer) {
+    this.rmProxyRelayer = relayer;
+  }
+
+  /**
+   * Set the UGI for RM connection.
+   */
+  public void setUGI(UserGroupInformation ugi) {
+    this.userUgi = ugi;
+  }
+
+  /**
+   * Sends the specified heart beat request to the resource manager and invokes
+   * the callback asynchronously with the response.
+   *
+   * @param request the allocate request
+   * @param callback the callback method for the request
+   * @throws YarnException if registerAM is not called yet
+   */
+  public void allocateAsync(AllocateRequest request,
+      AsyncCallback<AllocateResponse> callback) throws YarnException {
+    try {
+      this.requestQueue.put(new AsyncAllocateRequestInfo(request, callback));
+    } catch (InterruptedException ex) {
+      // Should not happen as we have MAX_INT queue length
+      LOG.debug("Interrupted while waiting to put on response queue", ex);
+    }
+  }
+
+  @VisibleForTesting
+  public int getRequestQueueSize() {
+    return this.requestQueue.size();
+  }
+
+  /**
+   * Data structure that encapsulates AllocateRequest and AsyncCallback
+   * instance.
+   */
+  public static class AsyncAllocateRequestInfo {
+    private AllocateRequest request;
+    private AsyncCallback<AllocateResponse> callback;
+
+    public AsyncAllocateRequestInfo(AllocateRequest request,
+        AsyncCallback<AllocateResponse> callback) {
+      Preconditions.checkArgument(request != null,
+          "AllocateRequest cannot be null");
+      Preconditions.checkArgument(callback != null, "Callback cannot be null");
+
+      this.request = request;
+      this.callback = callback;
+    }
+
+    public AsyncCallback<AllocateResponse> getCallback() {
+      return this.callback;
+    }
+
+    public AllocateRequest getRequest() {
+      return this.request;
+    }
+  }
+
+  /**
+   * Uncaught exception handler for the background heartbeat thread.
+   */
+  public class HeartBeatThreadUncaughtExceptionHandler
+      implements UncaughtExceptionHandler {
+    @Override
+    public void uncaughtException(Thread t, Throwable e) {
+      LOG.error("Heartbeat thread {} for application {} crashed!", t.getName(),
+          applicationId, e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f1525825/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java
index abdec19..78dcfb6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java
@@ -19,11 +19,8 @@
 package org.apache.hadoop.yarn.server.uam;
 
 import java.io.IOException;
-import java.lang.Thread.UncaughtExceptionHandler;
 import java.util.EnumSet;
 import java.util.Set;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
@@ -63,9 +60,9 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
+import org.apache.hadoop.yarn.server.AMHeartbeatRequestHandler;
 import org.apache.hadoop.yarn.server.AMRMClientRelayer;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
 import org.apache.hadoop.yarn.util.AsyncCallback;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.slf4j.Logger;
@@ -89,8 +86,7 @@ public class UnmanagedApplicationManager {
   public static final String APP_NAME = "UnmanagedAM";
   private static final String DEFAULT_QUEUE_CONFIG = "uam.default.queue.name";
 
-  private BlockingQueue<AsyncAllocateRequestInfo> requestQueue;
-  private AMRequestHandlerThread handlerThread;
+  private AMHeartbeatRequestHandler heartbeatHandler;
   private AMRMClientRelayer rmProxyRelayer;
   private ApplicationId applicationId;
   private String submitter;
@@ -99,7 +95,6 @@ public class UnmanagedApplicationManager {
   private String queueName;
   private UserGroupInformation userUgi;
   private RegisterApplicationMasterRequest registerRequest;
-  private int lastResponseId;
   private ApplicationClientProtocol rmClient;
   private long asyncApiPollIntervalMillis;
   private RecordFactory recordFactory;
@@ -137,8 +132,8 @@ public class UnmanagedApplicationManager {
     this.queueName = queueName;
     this.submitter = submitter;
     this.appNameSuffix = appNameSuffix;
-    this.handlerThread = new AMRequestHandlerThread();
-    this.requestQueue = new LinkedBlockingQueue<>();
+    this.heartbeatHandler =
+        new AMHeartbeatRequestHandler(this.conf, this.applicationId);
     this.rmProxyRelayer = null;
     this.connectionInitiated = false;
     this.registerRequest = null;
@@ -194,6 +189,9 @@ public class UnmanagedApplicationManager {
     this.rmProxyRelayer =
         new AMRMClientRelayer(createRMProxy(ApplicationMasterProtocol.class,
             this.conf, this.userUgi, amrmToken), this.applicationId);
+
+    this.heartbeatHandler.setAMRMClientRelayer(this.rmProxyRelayer);
+    this.heartbeatHandler.setUGI(this.userUgi);
   }
 
   /**
@@ -215,7 +213,7 @@ public class UnmanagedApplicationManager {
         this.applicationId);
     RegisterApplicationMasterResponse response =
         this.rmProxyRelayer.registerApplicationMaster(this.registerRequest);
-    this.lastResponseId = 0;
+    this.heartbeatHandler.resetLastResponseId();
 
     for (Container container : response.getContainersFromPreviousAttempts()) {
       LOG.debug("RegisterUAM returned existing running container "
@@ -227,12 +225,9 @@ public class UnmanagedApplicationManager {
     }
 
     // Only when register succeed that we start the heartbeat thread
-    this.handlerThread.setUncaughtExceptionHandler(
-        new HeartBeatThreadUncaughtExceptionHandler());
-    this.handlerThread.setDaemon(true);
-    this.handlerThread.start();
+    this.heartbeatHandler.setDaemon(true);
+    this.heartbeatHandler.start();
 
-    this.lastResponseId = 0;
     return response;
   }
 
@@ -248,7 +243,7 @@ public class UnmanagedApplicationManager {
       FinishApplicationMasterRequest request)
       throws YarnException, IOException {
 
-    this.handlerThread.shutdown();
+    this.heartbeatHandler.shutdown();
 
     if (this.rmProxyRelayer == null) {
       if (this.connectionInitiated) {
@@ -277,7 +272,7 @@ public class UnmanagedApplicationManager {
     KillApplicationRequest request =
         KillApplicationRequest.newInstance(this.applicationId);
 
-    this.handlerThread.shutdown();
+    this.heartbeatHandler.shutdown();
 
     if (this.rmClient == null) {
       this.rmClient = createRMProxy(ApplicationClientProtocol.class, this.conf,
@@ -296,12 +291,8 @@ public class UnmanagedApplicationManager {
    */
   public void allocateAsync(AllocateRequest request,
       AsyncCallback<AllocateResponse> callback) throws YarnException {
-    try {
-      this.requestQueue.put(new AsyncAllocateRequestInfo(request, callback));
-    } catch (InterruptedException ex) {
-      // Should not happen as we have MAX_INT queue length
-      LOG.debug("Interrupted while waiting to put on response queue", ex);
-    }
+    this.heartbeatHandler.allocateAsync(request, callback);
+
     // Two possible cases why the UAM is not successfully registered yet:
     // 1. launchUAM is not called at all. Should throw here.
     // 2. launchUAM is called but hasn't successfully returned.
@@ -519,139 +510,8 @@ public class UnmanagedApplicationManager {
     return this.rmClient.getApplicationReport(request).getApplicationReport();
   }
 
-  /**
-   * Data structure that encapsulates AllocateRequest and AsyncCallback
-   * instance.
-   */
-  public static class AsyncAllocateRequestInfo {
-    private AllocateRequest request;
-    private AsyncCallback<AllocateResponse> callback;
-
-    public AsyncAllocateRequestInfo(AllocateRequest request,
-        AsyncCallback<AllocateResponse> callback) {
-      Preconditions.checkArgument(request != null,
-          "AllocateRequest cannot be null");
-      Preconditions.checkArgument(callback != null, "Callback cannot be null");
-
-      this.request = request;
-      this.callback = callback;
-    }
-
-    public AsyncCallback<AllocateResponse> getCallback() {
-      return this.callback;
-    }
-
-    public AllocateRequest getRequest() {
-      return this.request;
-    }
-  }
-
   @VisibleForTesting
   public int getRequestQueueSize() {
-    return this.requestQueue.size();
-  }
-
-  /**
-   * Extends Thread and provides an implementation that is used for processing
-   * the AM heart beat request asynchronously and sending back the response
-   * using the callback method registered with the system.
-   */
-  public class AMRequestHandlerThread extends Thread {
-
-    // Indication flag for the thread to keep running
-    private volatile boolean keepRunning;
-
-    public AMRequestHandlerThread() {
-      super("UnmanagedApplicationManager Heartbeat Handler Thread");
-      this.keepRunning = true;
-    }
-
-    /**
-     * Shutdown the thread.
-     */
-    public void shutdown() {
-      this.keepRunning = false;
-      this.interrupt();
-    }
-
-    @Override
-    public void run() {
-      while (keepRunning) {
-        AsyncAllocateRequestInfo requestInfo;
-        try {
-          requestInfo = requestQueue.take();
-          if (requestInfo == null) {
-            throw new YarnException(
-                "Null requestInfo taken from request queue");
-          }
-          if (!keepRunning) {
-            break;
-          }
-
-          // change the response id before forwarding the allocate request as we
-          // could have different values for each UAM
-          AllocateRequest request = requestInfo.getRequest();
-          if (request == null) {
-            throw new YarnException("Null allocateRequest from requestInfo");
-          }
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Sending Heartbeat to Unmanaged AM. AskList:"
-                + ((request.getAskList() == null) ? " empty"
-                    : request.getAskList().size()));
-          }
-
-          request.setResponseId(lastResponseId);
-          AllocateResponse response = rmProxyRelayer.allocate(request);
-          if (response == null) {
-            throw new YarnException("Null allocateResponse from allocate");
-          }
-
-          lastResponseId = response.getResponseId();
-          // update token if RM has reissued/renewed
-          if (response.getAMRMToken() != null) {
-            LOG.debug("Received new AMRMToken");
-            YarnServerSecurityUtils.updateAMRMToken(response.getAMRMToken(),
-                userUgi, conf);
-          }
-
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Received Heartbeat reply from RM. Allocated Containers:"
-                + ((response.getAllocatedContainers() == null) ? " empty"
-                    : response.getAllocatedContainers().size()));
-          }
-
-          if (requestInfo.getCallback() == null) {
-            throw new YarnException("Null callback from requestInfo");
-          }
-          requestInfo.getCallback().callback(response);
-        } catch (InterruptedException ex) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Interrupted while waiting for queue", ex);
-          }
-        } catch (IOException ex) {
-          LOG.warn("IO Error occurred while processing heart beat for "
-              + applicationId, ex);
-        } catch (Throwable ex) {
-          LOG.warn(
-              "Error occurred while processing heart beat for " + applicationId,
-              ex);
-        }
-      }
-
-      LOG.info("UnmanagedApplicationManager has been stopped for {}. "
-          + "AMRequestHandlerThread thread is exiting", applicationId);
-    }
-  }
-
-  /**
-   * Uncaught exception handler for the background heartbeat thread.
-   */
-  protected class HeartBeatThreadUncaughtExceptionHandler
-      implements UncaughtExceptionHandler {
-    @Override
-    public void uncaughtException(Thread t, Throwable e) {
-      LOG.error("Heartbeat thread {} for application {} crashed!",
-          t.getName(), applicationId, e);
-    }
+    return this.heartbeatHandler.getRequestQueueSize();
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f1525825/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
index 65a2277..eb818f1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
@@ -116,6 +116,8 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
       NMSS_CLASS_PREFIX + "secondarySC/";
   public static final String STRING_TO_BYTE_FORMAT = "UTF-8";
 
+  private ApplicationAttemptId attemptId;
+
   /**
    * The home sub-cluster is the sub-cluster where the AM container is running
    * in.
@@ -125,20 +127,6 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   private volatile int lastHomeResponseId;
 
   /**
-   * A flag for work preserving NM restart. If we just recovered, we need to
-   * generate an {@link ApplicationMasterNotRegisteredException} exception back
-   * to AM (similar to what RM will do after its restart/fail-over) in its next
-   * allocate to trigger AM re-register (which we will shield from RM and just
-   * return our saved register response) and a full pending requests re-send, so
-   * that all the {@link AMRMClientRelayer} will be re-populated with all
-   * pending requests.
-   *
-   * TODO: When split-merge is not idempotent, this can lead to some
-   * over-allocation without a full cancel to RM.
-   */
-  private volatile boolean justRecovered;
-
-  /**
    * UAM pool for secondary sub-clusters (ones other than home sub-cluster),
    * using subClusterId as uamId. One UAM is created per sub-cluster RM except
    * the home RM.
@@ -156,15 +144,29 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
    */
   private Map<String, AMRMClientRelayer> secondaryRelayers;
 
-  /** Thread pool used for asynchronous operations. */
-  private ExecutorService threadpool;
-
   /**
    * Stores the AllocateResponses that are received asynchronously from all the
    * sub-cluster resource managers except the home RM.
    */
   private Map<SubClusterId, List<AllocateResponse>> asyncResponseSink;
 
+  /** Thread pool used for asynchronous operations. */
+  private ExecutorService threadpool;
+
+  /**
+   * A flag for work preserving NM restart. If we just recovered, we need to
+   * generate an {@link ApplicationMasterNotRegisteredException} exception back
+   * to AM (similar to what RM will do after its restart/fail-over) in its next
+   * allocate to trigger AM re-register (which we will shield from RM and just
+   * return our saved register response) and a full pending requests re-send, so
+   * that all the {@link AMRMClientRelayer} will be re-populated with all
+   * pending requests.
+   *
+   * TODO: When split-merge is not idempotent, this can lead to some
+   * over-allocation without a full cancel to RM.
+   */
+  private volatile boolean justRecovered;
+
   /**
    * Used to keep track of the container Id and the sub cluster RM that created
    * the container, so that we know which sub-cluster to forward later requests
@@ -179,7 +181,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   private RegisterApplicationMasterRequest amRegistrationRequest;
 
   /**
-   * The original registration response from home RM. This instance is reused
+   * The original registration response returned to AM. This instance is reused
    * for duplicate register request from AM, triggered by timeout between AM and
    * AMRMProxy.
    */
@@ -247,12 +249,12 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
       }
     }
 
+    this.attemptId = appContext.getApplicationAttemptId();
+    ApplicationId appId = this.attemptId.getApplicationId();
     this.homeSubClusterId =
         SubClusterId.newInstance(YarnConfiguration.getClusterId(conf));
-    this.homeRMRelayer = new AMRMClientRelayer(
-        createHomeRMProxy(appContext, ApplicationMasterProtocol.class,
-            this.appOwner),
-        getApplicationContext().getApplicationAttemptId().getApplicationId());
+    this.homeRMRelayer = new AMRMClientRelayer(createHomeRMProxy(appContext,
+        ApplicationMasterProtocol.class, this.appOwner), appId);
 
     this.federationFacade = FederationStateStoreFacade.getInstance();
     this.subClusterResolver = this.federationFacade.getSubClusterResolver();
@@ -267,9 +269,8 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   @Override
   public void recover(Map<String, byte[]> recoveredDataMap) {
     super.recover(recoveredDataMap);
-    ApplicationAttemptId attemptId =
-        getApplicationContext().getApplicationAttemptId();
-    LOG.info("Recovering data for FederationInterceptor for {}", attemptId);
+    LOG.info("Recovering data for FederationInterceptor for {}",
+        this.attemptId);
     if (recoveredDataMap == null) {
       return;
     }
@@ -280,7 +281,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
                 .parseFrom(recoveredDataMap.get(NMSS_REG_REQUEST_KEY));
         this.amRegistrationRequest =
             new RegisterApplicationMasterRequestPBImpl(pb);
-        LOG.info("amRegistrationRequest recovered for {}", attemptId);
+        LOG.info("amRegistrationRequest recovered for {}", this.attemptId);
 
         // Give the register request to homeRMRelayer for future re-registration
         this.homeRMRelayer.setAMRegistrationRequest(this.amRegistrationRequest);
@@ -291,7 +292,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
                 .parseFrom(recoveredDataMap.get(NMSS_REG_RESPONSE_KEY));
         this.amRegistrationResponse =
             new RegisterApplicationMasterResponsePBImpl(pb);
-        LOG.info("amRegistrationResponse recovered for {}", attemptId);
+        LOG.info("amRegistrationResponse recovered for {}", this.attemptId);
         // Trigger re-register and full pending re-send only if we have a
         // saved register response. This should always be true though.
         this.justRecovered = true;
@@ -301,9 +302,9 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
       Map<String, Token<AMRMTokenIdentifier>> uamMap;
       if (this.registryClient != null) {
         uamMap = this.registryClient
-            .loadStateFromRegistry(attemptId.getApplicationId());
+            .loadStateFromRegistry(this.attemptId.getApplicationId());
         LOG.info("Found {} existing UAMs for application {} in Yarn Registry",
-            uamMap.size(), attemptId.getApplicationId());
+            uamMap.size(), this.attemptId.getApplicationId());
       } else {
         uamMap = new HashMap<>();
         for (Entry<String, byte[]> entry : recoveredDataMap.entrySet()) {
@@ -319,7 +320,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
           }
         }
         LOG.info("Found {} existing UAMs for application {} in NMStateStore",
-            uamMap.size(), attemptId.getApplicationId());
+            uamMap.size(), this.attemptId.getApplicationId());
       }
 
       // Re-attach the UAMs
@@ -336,7 +337,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
 
         try {
           this.uamPool.reAttachUAM(subClusterId.getId(), config,
-              attemptId.getApplicationId(),
+              this.attemptId.getApplicationId(),
               this.amRegistrationResponse.getQueue(),
               getApplicationContext().getUser(), this.homeSubClusterId.getId(),
               entry.getValue());
@@ -359,9 +360,8 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
               subClusterId);
 
         } catch (Exception e) {
-          LOG.error(
-              "Error reattaching UAM to " + subClusterId + " for " + attemptId,
-              e);
+          LOG.error("Error reattaching UAM to " + subClusterId + " for "
+              + this.attemptId, e);
         }
       }
 
@@ -374,8 +374,8 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
           createHomeRMProxy(getApplicationContext(),
               ApplicationClientProtocol.class, appSubmitter);
 
-      GetContainersResponse response =
-          rmClient.getContainers(GetContainersRequest.newInstance(attemptId));
+      GetContainersResponse response = rmClient
+          .getContainers(GetContainersRequest.newInstance(this.attemptId));
       for (ContainerReport container : response.getContainerList()) {
         containerIdToSubClusterIdMap.put(container.getContainerId(),
             this.homeSubClusterId);
@@ -388,7 +388,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
 
       LOG.info(
           "In all {} UAMs {} running containers including AM recovered for {}",
-          uamMap.size(), containers, attemptId);
+          uamMap.size(), containers, this.attemptId);
 
       if (this.amRegistrationResponse != null) {
         // Initialize the AMRMProxyPolicy
@@ -439,12 +439,11 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
           RegisterApplicationMasterRequestPBImpl pb =
               (RegisterApplicationMasterRequestPBImpl)
                   this.amRegistrationRequest;
-          getNMStateStore().storeAMRMProxyAppContextEntry(
-              getApplicationContext().getApplicationAttemptId(),
+          getNMStateStore().storeAMRMProxyAppContextEntry(this.attemptId,
               NMSS_REG_REQUEST_KEY, pb.getProto().toByteArray());
         } catch (Exception e) {
           LOG.error("Error storing AMRMProxy application context entry for "
-              + getApplicationContext().getApplicationAttemptId(), e);
+              + this.attemptId, e);
         }
       }
     }
@@ -479,8 +478,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
           this.homeSubClusterId);
     }
 
-    ApplicationId appId =
-        getApplicationContext().getApplicationAttemptId().getApplicationId();
+    ApplicationId appId = this.attemptId.getApplicationId();
     reAttachUAMAndMergeRegisterResponse(this.amRegistrationResponse, appId);
 
     if (getNMStateStore() != null) {
@@ -488,12 +486,11 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
         RegisterApplicationMasterResponsePBImpl pb =
             (RegisterApplicationMasterResponsePBImpl)
                 this.amRegistrationResponse;
-        getNMStateStore().storeAMRMProxyAppContextEntry(
-            getApplicationContext().getApplicationAttemptId(),
+        getNMStateStore().storeAMRMProxyAppContextEntry(this.attemptId,
             NMSS_REG_RESPONSE_KEY, pb.getProto().toByteArray());
       } catch (Exception e) {
         LOG.error("Error storing AMRMProxy application context entry for "
-            + getApplicationContext().getApplicationAttemptId(), e);
+            + this.attemptId, e);
       }
     }
 
@@ -535,8 +532,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
       this.lastHomeResponseId = request.getResponseId();
 
       throw new ApplicationMasterNotRegisteredException(
-          "AMRMProxy just restarted and recovered for "
-              + getApplicationContext().getApplicationAttemptId()
+          "AMRMProxy just restarted and recovered for " + this.attemptId
               + ". AM should re-register and full re-send pending requests.");
     }
 
@@ -553,8 +549,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
     if (this.justRecovered
         || request.getResponseId() > this.lastHomeResponseId) {
       LOG.warn("Setting allocate responseId for {} from {} to {}",
-          getApplicationContext().getApplicationAttemptId(),
-          request.getResponseId(), this.lastHomeResponseId);
+          this.attemptId, request.getResponseId(), this.lastHomeResponseId);
       request.setResponseId(this.lastHomeResponseId);
     }
 
@@ -573,8 +568,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
 
       // Send the request to the home RM and get the response
       AllocateRequest homeRequest = requests.get(this.homeSubClusterId);
-      LOG.info("{} heartbeating to home RM with responseId {}",
-          getApplicationContext().getApplicationAttemptId(),
+      LOG.info("{} heartbeating to home RM with responseId {}", this.attemptId,
           homeRequest.getResponseId());
 
       AllocateResponse homeResponse = this.homeRMRelayer.allocate(homeRequest);
@@ -612,8 +606,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
       }
 
       LOG.info("{} heartbeat response from home RM with responseId {}",
-          getApplicationContext().getApplicationAttemptId(),
-          homeResponse.getResponseId());
+          this.attemptId, homeResponse.getResponseId());
 
       // Update lastHomeResponseId in three cases:
       // 1. The normal responseId increments
@@ -676,15 +669,14 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
                 secondaryRelayers.remove(subClusterId);
 
                 if (getNMStateStore() != null) {
-                  getNMStateStore().removeAMRMProxyAppContextEntry(
-                      getApplicationContext().getApplicationAttemptId(),
+                  getNMStateStore().removeAMRMProxyAppContextEntry(attemptId,
                       NMSS_SECONDARY_SC_PREFIX + subClusterId);
                 }
               }
             } catch (Throwable e) {
               LOG.warn("Failed to finish unmanaged application master: "
                   + "RM address: " + subClusterId + " ApplicationId: "
-                  + getApplicationContext().getApplicationAttemptId(), e);
+                  + attemptId, e);
             }
             return new FinishApplicationMasterResponseInfo(uamResponse,
                 subClusterId);
@@ -720,8 +712,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
         } catch (Throwable e) {
           failedToUnRegister = true;
           LOG.warn("Failed to finish unmanaged application master: "
-              + " ApplicationId: "
-              + getApplicationContext().getApplicationAttemptId(), e);
+              + " ApplicationId: " + this.attemptId, e);
         }
       }
     }
@@ -733,8 +724,8 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
       // attempt will be launched.
       this.uamPool.stop();
       if (this.registryClient != null) {
-        this.registryClient.removeAppFromRegistry(getApplicationContext()
-            .getApplicationAttemptId().getApplicationId());
+        this.registryClient
+            .removeAppFromRegistry(this.attemptId.getApplicationId());
       }
     }
     return homeResponse;
@@ -755,12 +746,12 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   public void shutdown() {
     // Do not stop uamPool service and kill UAMs here because of possible second
     // app attempt
-    if (threadpool != null) {
+    if (this.threadpool != null) {
       try {
-        threadpool.shutdown();
+        this.threadpool.shutdown();
       } catch (Throwable ex) {
       }
-      threadpool = null;
+      this.threadpool = null;
     }
     super.shutdown();
   }
@@ -1090,59 +1081,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
       }
 
       this.uamPool.allocateAsync(subClusterId.getId(), entry.getValue(),
-          new AsyncCallback<AllocateResponse>() {
-            @Override
-            public void callback(AllocateResponse response) {
-              synchronized (asyncResponseSink) {
-                List<AllocateResponse> responses = null;
-                if (asyncResponseSink.containsKey(subClusterId)) {
-                  responses = asyncResponseSink.get(subClusterId);
-                } else {
-                  responses = new ArrayList<>();
-                  asyncResponseSink.put(subClusterId, responses);
-                }
-                responses.add(response);
-              }
-
-              // Save the new AMRMToken for the UAM if present
-              if (response.getAMRMToken() != null) {
-                Token<AMRMTokenIdentifier> newToken = ConverterUtils
-                    .convertFromYarn(response.getAMRMToken(), (Text) null);
-                // Update the token in registry or NMSS
-                if (registryClient != null) {
-                  registryClient
-                      .writeAMRMTokenForUAM(
-                          getApplicationContext().getApplicationAttemptId()
-                              .getApplicationId(),
-                          subClusterId.getId(), newToken);
-                } else if (getNMStateStore() != null) {
-                  try {
-                    getNMStateStore().storeAMRMProxyAppContextEntry(
-                        getApplicationContext().getApplicationAttemptId(),
-                        NMSS_SECONDARY_SC_PREFIX + subClusterId.getId(),
-                        newToken.encodeToUrlString()
-                            .getBytes(STRING_TO_BYTE_FORMAT));
-                  } catch (IOException e) {
-                    LOG.error(
-                        "Error storing UAM token as AMRMProxy "
-                            + "context entry in NMSS for "
-                            + getApplicationContext().getApplicationAttemptId(),
-                        e);
-                  }
-                }
-              }
-
-              // Notify policy of secondary sub-cluster responses
-              try {
-                policyInterpreter.notifyOfResponse(subClusterId, response);
-              } catch (YarnException e) {
-                LOG.warn(
-                    "notifyOfResponse for policy failed for home sub-cluster "
-                        + subClusterId,
-                    e);
-              }
-            }
-          });
+          new HeartbeatCallBack(subClusterId));
     }
 
     return registrations;
@@ -1195,7 +1134,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
                 try {
                   // For appNameSuffix, use subClusterId of the home sub-cluster
                   token = uamPool.launchUAM(subClusterId, config,
-                      appContext.getApplicationAttemptId().getApplicationId(),
+                      attemptId.getApplicationId(),
                       amRegistrationResponse.getQueue(), appContext.getUser(),
                       homeSubClusterId.toString(), true);
 
@@ -1206,8 +1145,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
                       registerRequest);
                 } catch (Throwable e) {
                   LOG.error("Failed to register application master: "
-                      + subClusterId + " Application: "
-                      + appContext.getApplicationAttemptId(), e);
+                      + subClusterId + " Application: " + attemptId, e);
                 }
                 return new RegisterApplicationMasterResponseInfo(uamResponse,
                     SubClusterId.newInstance(subClusterId), token);
@@ -1232,20 +1170,18 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
           } else {
             LOG.info("Successfully registered unmanaged application master: "
                 + uamResponse.getSubClusterId() + " ApplicationId: "
-                + getApplicationContext().getApplicationAttemptId());
+                + this.attemptId);
             successfulRegistrations.put(uamResponse.getSubClusterId(),
                 uamResponse.getResponse());
 
             // Save the UAM token in registry or NMSS
             if (registryClient != null) {
               registryClient.writeAMRMTokenForUAM(
-                  getApplicationContext().getApplicationAttemptId()
-                      .getApplicationId(),
+                  this.attemptId.getApplicationId(),
                   uamResponse.getSubClusterId().getId(),
                   uamResponse.getUamToken());
             } else if (getNMStateStore() != null) {
-              getNMStateStore().storeAMRMProxyAppContextEntry(
-                  getApplicationContext().getApplicationAttemptId(),
+              getNMStateStore().storeAMRMProxyAppContextEntry(this.attemptId,
                   NMSS_SECONDARY_SC_PREFIX
                       + uamResponse.getSubClusterId().getId(),
                   uamResponse.getUamToken().encodeToUrlString()
@@ -1254,8 +1190,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
           }
         } catch (Exception e) {
           LOG.warn("Failed to register unmanaged application master: "
-              + " ApplicationId: "
-              + getApplicationContext().getApplicationAttemptId(), e);
+              + " ApplicationId: " + this.attemptId, e);
         }
       }
     }
@@ -1490,9 +1425,8 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
               "Duplicate containerID found in the allocated containers. This"
                   + " can happen if the RM epoch is not configured properly."
                   + " ContainerId: " + container.getId().toString()
-                  + " ApplicationId: "
-                  + getApplicationContext().getApplicationAttemptId()
-                  + " From RM: " + subClusterId
+                  + " ApplicationId: " + this.attemptId + " From RM: "
+                  + subClusterId
                   + " . Previous container was from sub-cluster: "
                   + existingSubClusterId);
         }
@@ -1588,6 +1522,59 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   }
 
   /**
+   * Async callback handler for heart beat response from all sub-clusters.
+   */
+  private class HeartbeatCallBack implements AsyncCallback<AllocateResponse> {
+    private SubClusterId subClusterId;
+
+    HeartbeatCallBack(SubClusterId subClusterId) {
+      this.subClusterId = subClusterId;
+    }
+
+    @Override
+    public void callback(AllocateResponse response) {
+      synchronized (asyncResponseSink) {
+        List<AllocateResponse> responses = null;
+        if (asyncResponseSink.containsKey(subClusterId)) {
+          responses = asyncResponseSink.get(subClusterId);
+        } else {
+          responses = new ArrayList<>();
+          asyncResponseSink.put(subClusterId, responses);
+        }
+        responses.add(response);
+      }
+
+      // Save the new AMRMToken for the UAM if present
+      if (response.getAMRMToken() != null) {
+        Token<AMRMTokenIdentifier> newToken = ConverterUtils
+            .convertFromYarn(response.getAMRMToken(), (Text) null);
+        // Update the token in registry or NMSS
+        if (registryClient != null) {
+          registryClient.writeAMRMTokenForUAM(attemptId.getApplicationId(),
+              subClusterId.getId(), newToken);
+        } else if (getNMStateStore() != null) {
+          try {
+            getNMStateStore().storeAMRMProxyAppContextEntry(attemptId,
+                NMSS_SECONDARY_SC_PREFIX + subClusterId.getId(),
+                newToken.encodeToUrlString().getBytes(STRING_TO_BYTE_FORMAT));
+          } catch (IOException e) {
+            LOG.error("Error storing UAM token as AMRMProxy "
+                + "context entry in NMSS for " + attemptId, e);
+          }
+        }
+      }
+
+      // Notify policy of secondary sub-cluster responses
+      try {
+        policyInterpreter.notifyOfResponse(subClusterId, response);
+      } catch (YarnException e) {
+        LOG.warn("notifyOfResponse for policy failed for home sub-cluster "
+            + subClusterId, e);
+      }
+    }
+  }
+
+  /**
    * Private structure for encapsulating SubClusterId and
    * RegisterApplicationMasterResponse instances.
    */


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[31/50] [abbrv] hadoop git commit: HADOOP-15680. ITestNativeAzureFileSystemConcurrencyLive times out. Contributed by Andras Bokor.

Posted by eh...@apache.org.
HADOOP-15680. ITestNativeAzureFileSystemConcurrencyLive times out.
Contributed by Andras Bokor.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e8d138ca
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e8d138ca
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e8d138ca

Branch: refs/heads/HDFS-12090
Commit: e8d138ca7c1b695688515d816ac693437c87df62
Parents: 2e6c110
Author: Steve Loughran <st...@apache.org>
Authored: Thu Aug 30 14:36:00 2018 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Aug 30 14:36:00 2018 +0100

----------------------------------------------------------------------
 .../hadoop/fs/azure/ITestNativeAzureFileSystemConcurrencyLive.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e8d138ca/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemConcurrencyLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemConcurrencyLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemConcurrencyLive.java
index 87cac15..1c868ea 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemConcurrencyLive.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemConcurrencyLive.java
@@ -39,7 +39,7 @@ public class ITestNativeAzureFileSystemConcurrencyLive
     extends AbstractWasbTestBase {
 
   private static final int THREAD_COUNT = 102;
-  private static final int TEST_EXECUTION_TIMEOUT = 5000;
+  private static final int TEST_EXECUTION_TIMEOUT = 30000;
 
   @Override
   protected AzureBlobStorageTestAccount createTestAccount() throws Exception {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[06/50] [abbrv] hadoop git commit: HDDS-247. Handle CLOSED_CONTAINER_IO exception in ozoneClient. Contributed by Shashikant Banerjee.

Posted by eh...@apache.org.
HDDS-247. Handle CLOSED_CONTAINER_IO exception in ozoneClient. Contributed by Shashikant Banerjee.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3974427f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3974427f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3974427f

Branch: refs/heads/HDFS-12090
Commit: 3974427f67299496e13b04f0d006d367b705fcb5
Parents: 26c2a97
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Tue Aug 28 07:11:36 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Tue Aug 28 07:12:07 2018 +0530

----------------------------------------------------------------------
 .../hdds/scm/storage/ChunkOutputStream.java     |  28 +-
 .../ozone/client/io/ChunkGroupOutputStream.java | 195 +++++++--
 .../hadoop/ozone/om/helpers/OmKeyInfo.java      |  23 +-
 .../rpc/TestCloseContainerHandlingByClient.java | 408 +++++++++++++++++++
 .../ozone/container/ContainerTestHelper.java    |  21 +
 .../hadoop/ozone/om/TestOmBlockVersioning.java  |  16 +-
 6 files changed, 630 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3974427f/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java
index 779e636..7309434 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java
@@ -94,6 +94,10 @@ public class ChunkOutputStream extends OutputStream {
     this.chunkIndex = 0;
   }
 
+  public ByteBuffer getBuffer() {
+    return buffer;
+  }
+
   @Override
   public synchronized void write(int b) throws IOException {
     checkOpen();
@@ -106,7 +110,8 @@ public class ChunkOutputStream extends OutputStream {
   }
 
   @Override
-  public void write(byte[] b, int off, int len) throws IOException {
+  public synchronized void write(byte[] b, int off, int len)
+      throws IOException {
     if (b == null) {
       throw new NullPointerException();
     }
@@ -143,24 +148,27 @@ public class ChunkOutputStream extends OutputStream {
 
   @Override
   public synchronized void close() throws IOException {
-    if (xceiverClientManager != null && xceiverClient != null &&
-        buffer != null) {
+    if (xceiverClientManager != null && xceiverClient != null
+        && buffer != null) {
+      if (buffer.position() > 0) {
+        writeChunkToContainer();
+      }
       try {
-        if (buffer.position() > 0) {
-          writeChunkToContainer();
-        }
         putKey(xceiverClient, containerKeyData.build(), traceID);
       } catch (IOException e) {
         throw new IOException(
             "Unexpected Storage Container Exception: " + e.toString(), e);
       } finally {
-        xceiverClientManager.releaseClient(xceiverClient);
-        xceiverClientManager = null;
-        xceiverClient = null;
-        buffer = null;
+        cleanup();
       }
     }
+  }
 
+  public synchronized void cleanup() {
+    xceiverClientManager.releaseClient(xceiverClient);
+    xceiverClientManager = null;
+    xceiverClient = null;
+    buffer = null;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3974427f/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
index 83b4dfd..988af07 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ObjectStageChangeRequestProto;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
@@ -46,8 +47,10 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Optional;
 
 /**
  * Maintaining a list of ChunkInputStream. Write based on offset.
@@ -111,6 +114,11 @@ public class ChunkGroupOutputStream extends OutputStream {
     return streamEntries;
   }
 
+  @VisibleForTesting
+  public int getOpenID() {
+    return openID;
+  }
+
   public ChunkGroupOutputStream(
       OpenKeySession handler, XceiverClientManager xceiverClientManager,
       StorageContainerLocationProtocolClientSideTranslatorPB scmClient,
@@ -220,26 +228,9 @@ public class ChunkGroupOutputStream extends OutputStream {
 
   @Override
   public synchronized void write(int b) throws IOException {
-    checkNotClosed();
-
-    if (streamEntries.size() <= currentStreamIndex) {
-      Preconditions.checkNotNull(omClient);
-      // allocate a new block, if a exception happens, log an error and
-      // throw exception to the caller directly, and the write fails.
-      try {
-        allocateNewBlock(currentStreamIndex);
-      } catch (IOException ioe) {
-        LOG.error("Allocate block fail when writing.");
-        throw ioe;
-      }
-    }
-    ChunkOutputStreamEntry entry = streamEntries.get(currentStreamIndex);
-    entry.write(b);
-    incrementBlockLength(currentStreamIndex, 1);
-    if (entry.getRemaining() <= 0) {
-      currentStreamIndex += 1;
-    }
-    byteOffset += 1;
+    byte[] buf = new byte[1];
+    buf[0] = (byte) b;
+    write(buf, 0, 1);
   }
 
   /**
@@ -258,7 +249,10 @@ public class ChunkGroupOutputStream extends OutputStream {
   public synchronized void write(byte[] b, int off, int len)
       throws IOException {
     checkNotClosed();
+    handleWrite(b, off, len);
+  }
 
+  private void handleWrite(byte[] b, int off, int len) throws IOException {
     if (b == null) {
       throw new NullPointerException();
     }
@@ -288,10 +282,21 @@ public class ChunkGroupOutputStream extends OutputStream {
       // still do a sanity check.
       Preconditions.checkArgument(currentStreamIndex < streamEntries.size());
       ChunkOutputStreamEntry current = streamEntries.get(currentStreamIndex);
-      int writeLen = Math.min(len, (int)current.getRemaining());
-      current.write(b, off, writeLen);
+      int writeLen = Math.min(len, (int) current.getRemaining());
+      try {
+        current.write(b, off, writeLen);
+      } catch (IOException ioe) {
+        if (checkIfContainerIsClosed(ioe)) {
+          handleCloseContainerException(current, currentStreamIndex);
+          continue;
+        } else {
+          throw ioe;
+        }
+      }
       incrementBlockLength(currentStreamIndex, writeLen);
       if (current.getRemaining() <= 0) {
+        // since the current block is already written close the stream.
+        handleFlushOrClose(true);
         currentStreamIndex += 1;
       }
       len -= writeLen;
@@ -301,6 +306,90 @@ public class ChunkGroupOutputStream extends OutputStream {
   }
 
   /**
+   * It performs following actions :
+   * a. Updates the committed length at datanode for the current stream in
+   *    datanode.
+   * b. Reads the data from the underlying buffer and writes it the next stream.
+   *
+   * @param streamEntry StreamEntry
+   * @param streamIndex Index of the entry
+   * @throws IOException Throws IOexception if Write fails
+   */
+  private void handleCloseContainerException(ChunkOutputStreamEntry streamEntry,
+      int streamIndex) throws IOException {
+    // TODO : If the block is still not committed and is in the
+    // pending openBlock Map, it will return BLOCK_NOT_COMMITTED
+    // exception. We should handle this by retrying the same operation
+    // n times and update the OzoneManager with the actual block length
+    // written. At this point of time, we also need to allocate new blocks
+    // from a different container and may need to nullify
+    // all the remaining pre-allocated blocks in case they were
+    // pre-allocated on the same container which got closed now.This needs
+    // caching the closed container list on the client itself.
+    long committedLength = 0;
+    ByteBuffer buffer = streamEntry.getBuffer();
+    if (buffer == null) {
+      // the buffer here will be null only when closeContainerException is
+      // hit while calling putKey during close on chunkOutputStream.
+      // Since closeContainer auto commit pending keys, no need to do
+      // anything here.
+      return;
+    }
+
+    // In case where not a single chunk of data has been written to the Datanode
+    // yet. This block does not yet exist on the datanode but cached on the
+    // outputStream buffer. No need to call GetCommittedBlockLength here
+    // for this block associated with the stream here.
+    if (streamEntry.currentPosition >= chunkSize
+        || streamEntry.currentPosition != buffer.position()) {
+      ContainerProtos.GetCommittedBlockLengthResponseProto responseProto =
+          ContainerProtocolCalls
+              .getCommittedBlockLength(streamEntry.xceiverClient,
+                  streamEntry.blockID, requestID);
+      committedLength = responseProto.getBlockLength();
+      // update the length of the current stream
+      locationInfoList.get(streamIndex).setLength(committedLength);
+    }
+
+    if (buffer.position() > 0) {
+      // If the data is still cached in the underlying stream, we need to
+      // allocate new block and write this data in the datanode. The cached
+      // data in the buffer does not exceed chunkSize.
+      Preconditions.checkState(buffer.position() < chunkSize);
+      currentStreamIndex += 1;
+      // readjust the byteOffset value to the length actually been written.
+      byteOffset -= buffer.position();
+      handleWrite(buffer.array(), 0, buffer.position());
+    }
+
+    // just clean up the current stream. Since the container is already closed,
+    // it will be auto committed. No need to call close again here.
+    streamEntry.cleanup();
+    // This case will arise when while writing the first chunk itself fails.
+    // In such case, the current block associated with the stream has no data
+    // written. Remove it from the current stream list.
+    if (committedLength == 0) {
+      streamEntries.remove(streamIndex);
+      locationInfoList.remove(streamIndex);
+      Preconditions.checkArgument(currentStreamIndex != 0);
+      currentStreamIndex -= 1;
+    }
+  }
+
+  private boolean checkIfContainerIsClosed(IOException ioe) {
+    return Optional.of(ioe.getCause())
+        .filter(e -> e instanceof StorageContainerException)
+        .map(e -> (StorageContainerException) e)
+        .filter(sce -> sce.getResult() == Result.CLOSED_CONTAINER_IO)
+        .isPresent();
+  }
+
+  private long getKeyLength() {
+    return locationInfoList.parallelStream().mapToLong(e -> e.getLength())
+        .sum();
+  }
+
+  /**
    * Contact OM to get a new block. Set the new block with the index (e.g.
    * first block has index = 0, second has index = 1 etc.)
    *
@@ -317,11 +406,41 @@ public class ChunkGroupOutputStream extends OutputStream {
   @Override
   public synchronized void flush() throws IOException {
     checkNotClosed();
+    handleFlushOrClose(false);
+  }
+
+  /**
+   * Close or Flush the latest outputStream.
+   * @param close Flag which decides whether to call close or flush on the
+   *              outputStream.
+   * @throws IOException In case, flush or close fails with exception.
+   */
+  private void handleFlushOrClose(boolean close) throws IOException {
     if (streamEntries.size() == 0) {
       return;
     }
-    for (int i = 0; i <= currentStreamIndex; i++) {
-      streamEntries.get(i).flush();
+    int size = streamEntries.size();
+    int streamIndex =
+        currentStreamIndex >= size ? size - 1 : currentStreamIndex;
+    ChunkOutputStreamEntry entry = streamEntries.get(streamIndex);
+    if (entry != null) {
+      try {
+        if (close) {
+          entry.close();
+        } else {
+          entry.flush();
+        }
+      } catch (IOException ioe) {
+        if (checkIfContainerIsClosed(ioe)) {
+          // This call will allocate a new streamEntry and write the Data.
+          // Close needs to be retried on the newly allocated streamEntry as
+          // as well.
+          handleCloseContainerException(entry, streamIndex);
+          handleFlushOrClose(close);
+        } else {
+          throw ioe;
+        }
+      }
     }
   }
 
@@ -336,16 +455,11 @@ public class ChunkGroupOutputStream extends OutputStream {
       return;
     }
     closed = true;
-    for (ChunkOutputStreamEntry entry : streamEntries) {
-      if (entry != null) {
-        entry.close();
-      }
-    }
+    handleFlushOrClose(true);
     if (keyArgs != null) {
       // in test, this could be null
-      long length =
-          locationInfoList.parallelStream().mapToLong(e -> e.getLength()).sum();
-      Preconditions.checkState(byteOffset == length);
+      Preconditions.checkState(streamEntries.size() == locationInfoList.size());
+      Preconditions.checkState(byteOffset == getKeyLength());
       keyArgs.setDataSize(byteOffset);
       keyArgs.setLocationInfoList(locationInfoList);
       omClient.commitKey(keyArgs, openID);
@@ -506,6 +620,23 @@ public class ChunkGroupOutputStream extends OutputStream {
         this.outputStream.close();
       }
     }
+
+    ByteBuffer getBuffer() throws IOException {
+      if (this.outputStream instanceof ChunkOutputStream) {
+        ChunkOutputStream out = (ChunkOutputStream) this.outputStream;
+        return out.getBuffer();
+      }
+      throw new IOException("Invalid Output Stream for Key: " + key);
+    }
+
+    public void cleanup() {
+      checkStream();
+      if (this.outputStream instanceof ChunkOutputStream) {
+        ChunkOutputStream out = (ChunkOutputStream) this.outputStream;
+        out.cleanup();
+      }
+    }
+
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3974427f/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
index 3603964..f6e4265 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyInfo.java
@@ -125,19 +125,16 @@ public final class OmKeyInfo {
     OmKeyLocationInfoGroup keyLocationInfoGroup = getLatestVersionLocations();
     List<OmKeyLocationInfo> currentList =
         keyLocationInfoGroup.getLocationList();
-    Preconditions.checkNotNull(keyLocationInfoGroup);
-    Preconditions.checkState(locationInfoList.size() <= currentList.size());
-    for (OmKeyLocationInfo current : currentList) {
-      // For Versioning, while committing the key for the newer version,
-      // we just need to update the lengths for new blocks. Need to iterate over
-      // and find the new blocks added in the latest version.
-      for (OmKeyLocationInfo info : locationInfoList) {
-        if (info.getBlockID().equals(current.getBlockID())) {
-          current.setLength(info.getLength());
-          break;
-        }
-      }
-    }
+    List<OmKeyLocationInfo> latestVersionList =
+        keyLocationInfoGroup.getBlocksLatestVersionOnly();
+    // Updates the latest locationList in the latest version only with
+    // given locationInfoList here.
+    // TODO : The original allocated list and the updated list here may vary
+    // as the containers on the Datanode on which the blocks were pre allocated
+    // might get closed. The diff of blocks between these two lists here
+    // need to be garbage collected in case the ozone client dies.
+    currentList.removeAll(latestVersionList);
+    currentList.addAll(locationInfoList);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3974427f/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
new file mode 100644
index 0000000..e5ecd81
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
@@ -0,0 +1,408 @@
+/**
+ * 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.client.rpc;
+
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientFactory;
+import org.apache.hadoop.ozone.client.io.ChunkGroupOutputStream;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.container.ContainerTestHelper;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+
+/**
+ * Tests Close Container Exception handling by Ozone Client.
+ */
+public class TestCloseContainerHandlingByClient {
+
+  private static MiniOzoneCluster cluster;
+  private static OzoneConfiguration conf;
+  private static OzoneClient client;
+  private static ObjectStore objectStore;
+  private static int chunkSize;
+  private static int blockSize;
+  private static String volumeName;
+  private static String bucketName;
+  private static String keyString;
+
+
+  /**
+   * Create a MiniDFSCluster for testing.
+   * <p>
+   * Ozone is made active by setting OZONE_ENABLED = true and
+   * OZONE_HANDLER_TYPE_KEY = "distributed"
+   *
+   * @throws IOException
+   */
+  @BeforeClass
+  public static void init() throws Exception {
+    conf = new OzoneConfiguration();
+    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
+        OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
+    chunkSize = (int)OzoneConsts.MB;
+    blockSize = 4 * chunkSize;
+    conf.setInt(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY, chunkSize);
+    conf.setLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB, (4));
+    cluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(3).build();
+    cluster.waitForClusterToBeReady();
+    //the easiest way to create an open container is creating a key
+    client = OzoneClientFactory.getClient(conf);
+    objectStore = client.getObjectStore();
+    keyString = UUID.randomUUID().toString();
+    volumeName = "closecontainerexceptionhandlingtest";
+    bucketName = volumeName;
+    objectStore.createVolume(volumeName);
+    objectStore.getVolume(volumeName).createBucket(bucketName);
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterClass
+  public static void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  private static String fixedLengthString(String string, int length) {
+    return String.format("%1$"+length+ "s", string);
+  }
+
+  @Test
+  public void testBlockWritesWithFlushAndClose() throws Exception {
+    String keyName = "standalone";
+    OzoneOutputStream key =
+        createKey(keyName, ReplicationType.STAND_ALONE, 0);
+    // write data more than 1 chunk
+    byte[] data =
+        fixedLengthString(keyString, chunkSize + chunkSize / 2).getBytes();
+    key.write(data);
+
+    Assert.assertTrue(key.getOutputStream() instanceof ChunkGroupOutputStream);
+    //get the name of a valid container
+    OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setType(HddsProtos.ReplicationType.STAND_ALONE)
+        .setFactor(HddsProtos.ReplicationFactor.ONE).setKeyName(keyName)
+        .build();
+
+    waitForContainerClose(keyName, key, HddsProtos.ReplicationType.STAND_ALONE);
+    key.write(data);
+    key.flush();
+    key.close();
+    // read the key from OM again and match the length.The length will still
+    // be the equal to the original data size.
+    OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
+    List<OmKeyLocationInfo> keyLocationInfos =
+        keyInfo.getKeyLocationVersions().get(0).getBlocksLatestVersionOnly();
+    //we have written two blocks
+    Assert.assertEquals(2, keyLocationInfos.size());
+    OmKeyLocationInfo omKeyLocationInfo = keyLocationInfos.get(0);
+    Assert.assertEquals(data.length - (data.length % chunkSize),
+        omKeyLocationInfo.getLength());
+    Assert.assertEquals(data.length + (data.length % chunkSize),
+        keyLocationInfos.get(1).getLength());
+    Assert.assertEquals(2 * data.length, keyInfo.getDataSize());
+
+    // Written the same data twice
+    String dataString = new String(data);
+    dataString.concat(dataString);
+    validateData(keyName, dataString.getBytes());
+  }
+
+  @Test
+  public void testBlockWritesCloseConsistency() throws Exception {
+    String keyName = "standalone2";
+    OzoneOutputStream key = createKey(keyName, ReplicationType.STAND_ALONE, 0);
+    // write data more than 1 chunk
+    byte[] data =
+        fixedLengthString(keyString, chunkSize + chunkSize / 2).getBytes();
+    key.write(data);
+
+    Assert.assertTrue(key.getOutputStream() instanceof ChunkGroupOutputStream);
+    //get the name of a valid container
+    OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setType(HddsProtos.ReplicationType.STAND_ALONE)
+        .setFactor(HddsProtos.ReplicationFactor.ONE).setKeyName(keyName)
+        .build();
+
+    waitForContainerClose(keyName, key, HddsProtos.ReplicationType.STAND_ALONE);
+    key.close();
+    // read the key from OM again and match the length.The length will still
+    // be the equal to the original data size.
+    OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
+    List<OmKeyLocationInfo> keyLocationInfos =
+        keyInfo.getKeyLocationVersions().get(0).getBlocksLatestVersionOnly();
+    // Though we have written only block initially, the close will hit
+    // closeContainerException and remaining data in the chunkOutputStream
+    // buffer will be copied into a different allocated block and will be
+    // committed.
+    Assert.assertEquals(2, keyLocationInfos.size());
+    OmKeyLocationInfo omKeyLocationInfo = keyLocationInfos.get(0);
+    Assert.assertEquals(data.length - (data.length % chunkSize),
+        omKeyLocationInfo.getLength());
+    Assert.assertEquals(data.length % chunkSize,
+        keyLocationInfos.get(1).getLength());
+    Assert.assertEquals(data.length, keyInfo.getDataSize());
+    validateData(keyName, data);
+  }
+
+  @Test
+  public void testMultiBlockWrites() throws Exception {
+
+    String keyName = "standalone3";
+    OzoneOutputStream key =
+        createKey(keyName, ReplicationType.STAND_ALONE, (4 * blockSize));
+    ChunkGroupOutputStream groupOutputStream =
+        (ChunkGroupOutputStream) key.getOutputStream();
+    // With the initial size provided, it should have preallocated 3 blocks
+    Assert.assertEquals(4, groupOutputStream.getStreamEntries().size());
+    // write data more than 1 chunk
+    byte[] data = fixedLengthString(keyString, (3 * blockSize)).getBytes();
+    Assert.assertEquals(data.length, 3 * blockSize);
+    key.write(data);
+
+    Assert.assertTrue(key.getOutputStream() instanceof ChunkGroupOutputStream);
+    //get the name of a valid container
+    OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setType(HddsProtos.ReplicationType.STAND_ALONE)
+        .setFactor(HddsProtos.ReplicationFactor.ONE).setKeyName(keyName)
+        .build();
+
+    waitForContainerClose(keyName, key,
+        HddsProtos.ReplicationType.STAND_ALONE);
+    // write 1 more block worth of data. It will fail and new block will be
+    // allocated
+    key.write(fixedLengthString(keyString, blockSize).getBytes());
+
+    key.close();
+    // read the key from OM again and match the length.The length will still
+    // be the equal to the original data size.
+    OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
+    List<OmKeyLocationInfo> keyLocationInfos =
+        keyInfo.getKeyLocationVersions().get(0).getBlocksLatestVersionOnly();
+    // Though we have written only block initially, the close will hit
+    // closeContainerException and remaining data in the chunkOutputStream
+    // buffer will be copied into a different allocated block and will be
+    // committed.
+    Assert.assertEquals(4, keyLocationInfos.size());
+    Assert.assertEquals(4 * blockSize, keyInfo.getDataSize());
+    for (OmKeyLocationInfo locationInfo : keyLocationInfos) {
+      Assert.assertEquals(blockSize, locationInfo.getLength());
+    }
+  }
+
+  @Test
+  public void testMultiBlockWrites2() throws Exception {
+
+    String keyName = "standalone4";
+    long dataLength = 0;
+    OzoneOutputStream key =
+        createKey(keyName, ReplicationType.STAND_ALONE, 4 * blockSize);
+    ChunkGroupOutputStream groupOutputStream =
+        (ChunkGroupOutputStream) key.getOutputStream();
+
+    Assert.assertTrue(key.getOutputStream() instanceof ChunkGroupOutputStream);
+    // With the initial size provided, it should have pre allocated 4 blocks
+    Assert.assertEquals(4, groupOutputStream.getStreamEntries().size());
+    String dataString = fixedLengthString(keyString, (3 * blockSize));
+    byte[] data = dataString.getBytes();
+    key.write(data);
+    // 3 block are completely written to the DataNode in 3 blocks.
+    // Data of length half of chunkSize resides in the chunkOutput stream buffer
+    String dataString2 = fixedLengthString(keyString, chunkSize * 1 / 2);
+    key.write(dataString2.getBytes());
+    //get the name of a valid container
+    OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setType(HddsProtos.ReplicationType.STAND_ALONE)
+        .setFactor(HddsProtos.ReplicationFactor.ONE).setKeyName(keyName)
+        .build();
+
+    waitForContainerClose(keyName, key, HddsProtos.ReplicationType.STAND_ALONE);
+
+    key.close();
+    // read the key from OM again and match the length.The length will still
+    // be the equal to the original data size.
+    OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
+    List<OmKeyLocationInfo> keyLocationInfos =
+        keyInfo.getKeyLocationVersions().get(0).getBlocksLatestVersionOnly();
+    // Though we have written only block initially, the close will hit
+    // closeContainerException and remaining data in the chunkOutputStream
+    // buffer will be copied into a different allocated block and will be
+    // committed.
+    Assert.assertEquals(4, keyLocationInfos.size());
+    dataLength = 3 * blockSize + (long) (0.5 * chunkSize);
+    Assert.assertEquals(dataLength, keyInfo.getDataSize());
+    validateData(keyName, dataString.concat(dataString2).getBytes());
+  }
+
+  private void waitForContainerClose(String keyName,
+      OzoneOutputStream outputStream, HddsProtos.ReplicationType type)
+      throws Exception {
+
+    ChunkGroupOutputStream groupOutputStream =
+        (ChunkGroupOutputStream) outputStream.getOutputStream();
+    int clientId = groupOutputStream.getOpenID();
+    OMMetadataManager metadataManager =
+        cluster.getOzoneManager().getMetadataManager();
+    String objectKey =
+        metadataManager.getKeyWithDBPrefix(volumeName, bucketName, keyName);
+    byte[] openKey = metadataManager.getOpenKeyNameBytes(objectKey, clientId);
+    byte[] openKeyData = metadataManager.get(openKey);
+    OmKeyInfo keyInfo = OmKeyInfo.getFromProtobuf(
+        OzoneManagerProtocolProtos.KeyInfo.parseFrom(openKeyData));
+    List<OmKeyLocationInfo> locationInfoList =
+        keyInfo.getKeyLocationVersions().get(0).getBlocksLatestVersionOnly();
+    List<Long> containerIdList = new ArrayList<>();
+    List<Pipeline> pipelineList = new ArrayList<>();
+    for (OmKeyLocationInfo info : locationInfoList) {
+      containerIdList.add(info.getContainerID());
+    }
+    Assert.assertTrue(!containerIdList.isEmpty());
+    for (long containerID : containerIdList) {
+      Pipeline pipeline =
+          cluster.getStorageContainerManager().getScmContainerManager()
+              .getContainerWithPipeline(containerID).getPipeline();
+      pipelineList.add(pipeline);
+      List<DatanodeDetails> datanodes = pipeline.getMachines();
+      for (DatanodeDetails details : datanodes) {
+        Assert.assertFalse(ContainerTestHelper
+            .isContainerClosed(cluster, containerID, details));
+        // send the order to close the container
+        cluster.getStorageContainerManager().getScmNodeManager()
+            .addDatanodeCommand(details.getUuid(),
+                new CloseContainerCommand(containerID, type, pipeline.getId()));
+      }
+    }
+
+    int index = 0;
+    for (long containerID : containerIdList) {
+      Pipeline pipeline = pipelineList.get(index);
+      List<DatanodeDetails> datanodes = pipeline.getMachines();
+      for (DatanodeDetails datanodeDetails : datanodes) {
+        GenericTestUtils.waitFor(() -> ContainerTestHelper
+                .isContainerClosed(cluster, containerID, datanodeDetails), 500,
+            15 * 1000);
+        //double check if it's really closed (waitFor also throws an exception)
+        Assert.assertTrue(ContainerTestHelper
+            .isContainerClosed(cluster, containerID, datanodeDetails));
+      }
+      index++;
+    }
+
+  }
+
+  private OzoneOutputStream createKey(String keyName, ReplicationType type,
+      long size) throws Exception {
+    ReplicationFactor factor =
+        type == ReplicationType.STAND_ALONE ? ReplicationFactor.ONE :
+            ReplicationFactor.THREE;
+    return objectStore.getVolume(volumeName).getBucket(bucketName)
+        .createKey(keyName, size, type, factor);
+  }
+
+  private void validateData(String keyName, byte[] data) throws Exception {
+    byte[] readData = new byte[data.length];
+    OzoneInputStream is =
+        objectStore.getVolume(volumeName).getBucket(bucketName)
+            .readKey(keyName);
+    is.read(readData);
+    MessageDigest sha1 = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
+    sha1.update(data);
+    MessageDigest sha2 = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
+    sha2.update(readData);
+    Assert.assertTrue(Arrays.equals(sha1.digest(), sha2.digest()));
+    is.close();
+  }
+
+
+  @Test
+  public void testBlockWriteViaRatis() throws Exception {
+    String keyName = "ratis";
+    OzoneOutputStream key = createKey(keyName, ReplicationType.RATIS, 0);
+    byte[] data =
+        fixedLengthString(keyString, chunkSize + chunkSize / 2).getBytes();
+    key.write(data);
+
+    //get the name of a valid container
+    OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName).
+        setBucketName(bucketName).setType(HddsProtos.ReplicationType.RATIS)
+        .setFactor(HddsProtos.ReplicationFactor.THREE)
+        .setKeyName(keyName).build();
+
+    Assert.assertTrue(key.getOutputStream() instanceof ChunkGroupOutputStream);
+    waitForContainerClose(keyName, key, HddsProtos.ReplicationType.RATIS);
+    // Again Write the Data. This will throw an exception which will be handled
+    // and new blocks will be allocated
+    key.write(data);
+    key.flush();
+    // The write will fail but exception will be handled and length will be
+    // updated correctly in OzoneManager once the steam is closed
+    key.close();
+    // read the key from OM again and match the length.The length will still
+    // be the equal to the original data size.
+    OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
+    List<OmKeyLocationInfo> keyLocationInfos =
+        keyInfo.getKeyLocationVersions().get(0).getBlocksLatestVersionOnly();
+    //we have written two blocks
+    Assert.assertEquals(2, keyLocationInfos.size());
+    OmKeyLocationInfo omKeyLocationInfo = keyLocationInfos.get(0);
+    Assert.assertEquals(data.length - (data.length % chunkSize),
+        omKeyLocationInfo.getLength());
+    Assert.assertEquals(data.length + (data.length % chunkSize),
+        keyLocationInfos.get(1).getLength());
+    Assert.assertEquals(2 * data.length, keyInfo.getDataSize());
+    String dataString = new String(data);
+    dataString.concat(dataString);
+    validateData(keyName, dataString.getBytes());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3974427f/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
index ca92110..dc166b5 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
@@ -21,6 +21,10 @@ package org.apache.hadoop.ozone.container;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
+import org.apache.hadoop.ozone.HddsDatanodeService;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.container.common.impl.ContainerData;
+import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.ratis.shaded.com.google.protobuf.ByteString;
 import org.apache.commons.codec.binary.Hex;
 import org.apache.hadoop.hdds.client.BlockID;
@@ -604,4 +608,21 @@ public final class ContainerTestHelper {
   public static long getTestContainerID() {
     return Time.getUtcTime();
   }
+
+  public static boolean isContainerClosed(MiniOzoneCluster cluster,
+      long containerID, DatanodeDetails datanode) {
+    ContainerData containerData;
+    for (HddsDatanodeService datanodeService : cluster.getHddsDatanodes()) {
+      if (datanode.equals(datanodeService.getDatanodeDetails())) {
+        Container container =
+            datanodeService.getDatanodeStateMachine().getContainer()
+                .getContainerSet().getContainer(containerID);
+        if (container != null) {
+          containerData = container.getContainerData();
+          return containerData.isClosed();
+        }
+      }
+    }
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3974427f/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmBlockVersioning.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmBlockVersioning.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmBlockVersioning.java
index f5dddee..0eb1677 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmBlockVersioning.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmBlockVersioning.java
@@ -39,12 +39,12 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
+import org.junit.Assert;
 import org.junit.rules.ExpectedException;
 
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
 
@@ -124,8 +124,8 @@ public class TestOmBlockVersioning {
     // 1st update, version 0
     OpenKeySession openKey = ozoneManager.openKey(keyArgs);
     // explicitly set the keyLocation list before committing the key.
-    keyArgs.setLocationInfoList(
-        openKey.getKeyInfo().getLatestVersionLocations().getLocationList());
+    keyArgs.setLocationInfoList(openKey.getKeyInfo().getLatestVersionLocations()
+        .getBlocksLatestVersionOnly());
     ozoneManager.commitKey(keyArgs, openKey.getId());
 
     OmKeyInfo keyInfo = ozoneManager.lookupKey(keyArgs);
@@ -139,8 +139,8 @@ public class TestOmBlockVersioning {
     //OmKeyLocationInfo locationInfo =
     //    ozoneManager.allocateBlock(keyArgs, openKey.getId());
     // explicitly set the keyLocation list before committing the key.
-    keyArgs.setLocationInfoList(
-        openKey.getKeyInfo().getLatestVersionLocations().getLocationList());
+    keyArgs.setLocationInfoList(openKey.getKeyInfo().getLatestVersionLocations()
+        .getBlocksLatestVersionOnly());
     ozoneManager.commitKey(keyArgs, openKey.getId());
 
     keyInfo = ozoneManager.lookupKey(keyArgs);
@@ -150,10 +150,14 @@ public class TestOmBlockVersioning {
 
     // 3rd update, version 2
     openKey = ozoneManager.openKey(keyArgs);
+
     // this block will be appended to the latest version of version 2.
     OmKeyLocationInfo locationInfo =
         ozoneManager.allocateBlock(keyArgs, openKey.getId());
-    List<OmKeyLocationInfo> locationInfoList = new ArrayList<>();
+    List<OmKeyLocationInfo> locationInfoList =
+        openKey.getKeyInfo().getLatestVersionLocations()
+            .getBlocksLatestVersionOnly();
+    Assert.assertTrue(locationInfoList.size() == 1);
     locationInfoList.add(locationInfo);
     keyArgs.setLocationInfoList(locationInfoList);
     ozoneManager.commitKey(keyArgs, openKey.getId());


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[05/50] [abbrv] hadoop git commit: HDFS-13838. WebHdfsFileSystem.getFileStatus() won't return correct "snapshot enabled" status. Contributed by Siyao Meng.

Posted by eh...@apache.org.
HDFS-13838. WebHdfsFileSystem.getFileStatus() won't return correct "snapshot enabled" status. Contributed by Siyao Meng.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/26c2a97c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/26c2a97c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/26c2a97c

Branch: refs/heads/HDFS-12090
Commit: 26c2a97c566969f50eb8e8432009724c51152a98
Parents: 602d138
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Mon Aug 27 16:02:35 2018 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Mon Aug 27 16:02:35 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/web/JsonUtilClient.java |  4 ++++
 .../java/org/apache/hadoop/hdfs/web/TestWebHDFS.java    | 12 ++++++++++++
 2 files changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/26c2a97c/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
index 9bb1846..a685573 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
@@ -133,6 +133,7 @@ class JsonUtilClient {
     Boolean aclBit = (Boolean) m.get("aclBit");
     Boolean encBit = (Boolean) m.get("encBit");
     Boolean erasureBit  = (Boolean) m.get("ecBit");
+    Boolean snapshotEnabledBit  = (Boolean) m.get("snapshotEnabled");
     EnumSet<HdfsFileStatus.Flags> f =
         EnumSet.noneOf(HdfsFileStatus.Flags.class);
     if (aclBit != null && aclBit) {
@@ -144,6 +145,9 @@ class JsonUtilClient {
     if (erasureBit != null && erasureBit) {
       f.add(HdfsFileStatus.Flags.HAS_EC);
     }
+    if (snapshotEnabledBit != null && snapshotEnabledBit) {
+      f.add(HdfsFileStatus.Flags.SNAPSHOT_ENABLED);
+    }
 
     Map<String, Object> ecPolicyObj = (Map) m.get("ecPolicyObj");
     ErasureCodingPolicy ecPolicy = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/26c2a97c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
index cbc428a..9152636 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
@@ -482,6 +482,9 @@ public class TestWebHDFS {
 
       // allow snapshots on /bar using webhdfs
       webHdfs.allowSnapshot(bar);
+      // check if snapshot status is enabled
+      assertTrue(dfs.getFileStatus(bar).isSnapshotEnabled());
+      assertTrue(webHdfs.getFileStatus(bar).isSnapshotEnabled());
       webHdfs.createSnapshot(bar, "s1");
       final Path s1path = SnapshotTestHelper.getSnapshotRoot(bar, "s1");
       Assert.assertTrue(webHdfs.exists(s1path));
@@ -491,15 +494,24 @@ public class TestWebHDFS {
       assertEquals(bar, snapshottableDirs[0].getFullPath());
       dfs.deleteSnapshot(bar, "s1");
       dfs.disallowSnapshot(bar);
+      // check if snapshot status is disabled
+      assertFalse(dfs.getFileStatus(bar).isSnapshotEnabled());
+      assertFalse(webHdfs.getFileStatus(bar).isSnapshotEnabled());
       snapshottableDirs = dfs.getSnapshottableDirListing();
       assertNull(snapshottableDirs);
 
       // disallow snapshots on /bar using webhdfs
       dfs.allowSnapshot(bar);
+      // check if snapshot status is enabled, again
+      assertTrue(dfs.getFileStatus(bar).isSnapshotEnabled());
+      assertTrue(webHdfs.getFileStatus(bar).isSnapshotEnabled());
       snapshottableDirs = dfs.getSnapshottableDirListing();
       assertEquals(1, snapshottableDirs.length);
       assertEquals(bar, snapshottableDirs[0].getFullPath());
       webHdfs.disallowSnapshot(bar);
+      // check if snapshot status is disabled, again
+      assertFalse(dfs.getFileStatus(bar).isSnapshotEnabled());
+      assertFalse(webHdfs.getFileStatus(bar).isSnapshotEnabled());
       snapshottableDirs = dfs.getSnapshottableDirListing();
       assertNull(snapshottableDirs);
       try {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[41/50] [abbrv] hadoop git commit: Revert "HDDS-98. Adding Ozone Manager Audit Log."

Posted by eh...@apache.org.
Revert "HDDS-98. Adding Ozone Manager Audit Log."

This reverts commit 630b64ec7e963968a5bdcd1d625fc78746950137.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6edf3d2e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6edf3d2e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6edf3d2e

Branch: refs/heads/HDFS-12090
Commit: 6edf3d2ea3de3629629c21cec3b4435bd71753ba
Parents: 19abaac
Author: Nanda kumar <na...@apache.org>
Authored: Sun Sep 2 00:18:13 2018 +0530
Committer: Nanda kumar <na...@apache.org>
Committed: Sun Sep 2 00:18:13 2018 +0530

----------------------------------------------------------------------
 .../src/main/compose/ozone/docker-config        |  37 ----
 .../org/apache/hadoop/ozone/OzoneConsts.java    |  32 ---
 hadoop-ozone/common/src/main/bin/ozone          |   2 -
 .../src/main/conf/om-audit-log4j2.properties    |  86 --------
 .../org/apache/hadoop/ozone/audit/OMAction.java |  25 +--
 .../hadoop/ozone/om/helpers/OmBucketArgs.java   |  25 +--
 .../hadoop/ozone/om/helpers/OmBucketInfo.java   |  21 +-
 .../hadoop/ozone/om/helpers/OmKeyArgs.java      |  22 +-
 .../hadoop/ozone/om/helpers/OmVolumeArgs.java   |  16 +-
 .../apache/hadoop/ozone/om/OzoneManager.java    | 218 +------------------
 10 files changed, 18 insertions(+), 466 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6edf3d2e/hadoop-dist/src/main/compose/ozone/docker-config
----------------------------------------------------------------------
diff --git a/hadoop-dist/src/main/compose/ozone/docker-config b/hadoop-dist/src/main/compose/ozone/docker-config
index 21127f8..a1828a3 100644
--- a/hadoop-dist/src/main/compose/ozone/docker-config
+++ b/hadoop-dist/src/main/compose/ozone/docker-config
@@ -31,40 +31,3 @@ LOG4J.PROPERTIES_log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
 LOG4J.PROPERTIES_log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n
 #Enable this variable to print out all hadoop rpc traffic to the stdout. See http://byteman.jboss.org/ to define your own instrumentation.
 #BYTEMAN_SCRIPT_URL=https://raw.githubusercontent.com/apache/hadoop/trunk/dev-support/byteman/hadooprpc.btm
-
-#LOG4J2.PROPERTIES_* are for Ozone Audit Logging
-LOG4J2.PROPERTIES_monitorInterval=30
-LOG4J2.PROPERTIES_filter=read,write
-LOG4J2.PROPERTIES_filter.read.type=MarkerFilter
-LOG4J2.PROPERTIES_filter.read.marker=READ
-LOG4J2.PROPERTIES_filter.read.onMatch=DENY
-LOG4J2.PROPERTIES_filter.read.onMismatch=NEUTRAL
-LOG4J2.PROPERTIES_filter.write.type=MarkerFilter
-LOG4J2.PROPERTIES_filter.write.marker=WRITE
-LOG4J2.PROPERTIES_filter.write.onMatch=NEUTRAL
-LOG4J2.PROPERTIES_filter.write.onMismatch=NEUTRAL
-LOG4J2.PROPERTIES_appenders=console, rolling
-LOG4J2.PROPERTIES_appender.console.type=Console
-LOG4J2.PROPERTIES_appender.console.name=STDOUT
-LOG4J2.PROPERTIES_appender.console.layout.type=PatternLayout
-LOG4J2.PROPERTIES_appender.console.layout.pattern=%d{DEFAULT} | %-5level | %c{1} | %msg | %throwable{3} %n
-LOG4J2.PROPERTIES_appender.rolling.type=RollingFile
-LOG4J2.PROPERTIES_appender.rolling.name=RollingFile
-LOG4J2.PROPERTIES_appender.rolling.fileName =${sys:hadoop.log.dir}/om-audit-${hostName}.log
-LOG4J2.PROPERTIES_appender.rolling.filePattern=${sys:hadoop.log.dir}/om-audit-${hostName}-%d{yyyy-MM-dd-HH-mm-ss}-%i.log.gz
-LOG4J2.PROPERTIES_appender.rolling.layout.type=PatternLayout
-LOG4J2.PROPERTIES_appender.rolling.layout.pattern=%d{DEFAULT} | %-5level | %c{1} | %msg | %throwable{3} %n
-LOG4J2.PROPERTIES_appender.rolling.policies.type=Policies
-LOG4J2.PROPERTIES_appender.rolling.policies.time.type=TimeBasedTriggeringPolicy
-LOG4J2.PROPERTIES_appender.rolling.policies.time.interval=86400
-LOG4J2.PROPERTIES_appender.rolling.policies.size.type=SizeBasedTriggeringPolicy
-LOG4J2.PROPERTIES_appender.rolling.policies.size.size=64MB
-LOG4J2.PROPERTIES_loggers=audit
-LOG4J2.PROPERTIES_logger.audit.type=AsyncLogger
-LOG4J2.PROPERTIES_logger.audit.name=OMAudit
-LOG4J2.PROPERTIES_logger.audit.level=INFO
-LOG4J2.PROPERTIES_logger.audit.appenderRefs=rolling
-LOG4J2.PROPERTIES_logger.audit.appenderRef.file.ref=RollingFile
-LOG4J2.PROPERTIES_rootLogger.level=INFO
-LOG4J2.PROPERTIES_rootLogger.appenderRefs=stdout
-LOG4J2.PROPERTIES_rootLogger.appenderRef.stdout.ref=STDOUT

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6edf3d2e/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index 9645c02..15366fb 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -184,36 +184,4 @@ public final class OzoneConsts {
   public static final String CHUNKS_PATH = "chunksPath";
   public static final String CONTAINER_DB_TYPE = "containerDBType";
   public static final String CHECKSUM = "checksum";
-
-  // For OM Audit usage
-  public static final String VOLUME = "volume";
-  public static final String BUCKET = "bucket";
-  public static final String KEY = "key";
-  public static final String QUOTA = "quota";
-  public static final String QUOTA_IN_BYTES = "quotaInBytes";
-  public static final String CLIENT_ID = "clientID";
-  public static final String OWNER = "owner";
-  public static final String ADMIN = "admin";
-  public static final String USERNAME = "username";
-  public static final String PREV_KEY = "prevKey";
-  public static final String START_KEY = "startKey";
-  public static final String MAX_KEYS = "maxKeys";
-  public static final String PREFIX = "prefix";
-  public static final String KEY_PREFIX = "keyPrefix";
-  public static final String ACLS = "acls";
-  public static final String USER_ACL = "userAcl";
-  public static final String ADD_ACLS = "addAcls";
-  public static final String REMOVE_ACLS = "removeAcls";
-  public static final String MAX_NUM_OF_BUCKETS = "maxNumOfBuckets";
-  public static final String TO_KEY_NAME = "toKeyName";
-  public static final String STORAGE_TYPE = "storageType";
-  public static final String IS_VERSION_ENABLED = "isVersionEnabled";
-  public static final String CREATION_TIME = "creationTime";
-  public static final String DATA_SIZE = "dataSize";
-  public static final String REPLICATION_TYPE = "replicationType";
-  public static final String REPLICATION_FACTOR = "replicationFactor";
-  public static final String KEY_LOCATION_INFO = "keyLocationInfo";
-
-
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6edf3d2e/hadoop-ozone/common/src/main/bin/ozone
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/bin/ozone b/hadoop-ozone/common/src/main/bin/ozone
index 17b47a9..db37ea4 100755
--- a/hadoop-ozone/common/src/main/bin/ozone
+++ b/hadoop-ozone/common/src/main/bin/ozone
@@ -97,8 +97,6 @@ function ozonecmd_case
     om)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
       HADOOP_CLASSNAME=org.apache.hadoop.ozone.om.OzoneManager
-      HDFS_OM_OPTS="${HDFS_OM_OPTS} -Dlog4j.configurationFile=${HADOOP_CONF_DIR}/om-audit-log4j2.properties"
-      HADOOP_OPTS="${HADOOP_OPTS} ${HDFS_OM_OPTS}"
     ;;
     oz)
       HADOOP_CLASSNAME=org.apache.hadoop.ozone.web.ozShell.Shell

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6edf3d2e/hadoop-ozone/common/src/main/conf/om-audit-log4j2.properties
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/conf/om-audit-log4j2.properties b/hadoop-ozone/common/src/main/conf/om-audit-log4j2.properties
deleted file mode 100644
index 7d097a0..0000000
--- a/hadoop-ozone/common/src/main/conf/om-audit-log4j2.properties
+++ /dev/null
@@ -1,86 +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.
-#
-name=PropertiesConfig
-
-# Checks for config change periodically and reloads
-monitorInterval=30
-
-filter=read,write
-# filter.read.onMatch=DENY avoids logging all READ events
-# filter.read.onMatch=ACCEPT permits logging all READ events
-# The above two settings ignore the log levels in configuration
-# filter.read.onMatch=NEUTRAL permits logging of only those READ events
-# which are attempted at log level equal or greater than log level specified
-# in the configuration
-filter.read.type=MarkerFilter
-filter.read.marker=READ
-filter.read.onMatch=DENY
-filter.read.onMismatch=NEUTRAL
-
-# filter.write.onMatch=DENY avoids logging all WRITE events
-# filter.write.onMatch=ACCEPT permits logging all WRITE events
-# The above two settings ignore the log levels in configuration
-# filter.write.onMatch=NEUTRAL permits logging of only those WRITE events
-# which are attempted at log level equal or greater than log level specified
-# in the configuration
-filter.write.type=MarkerFilter
-filter.write.marker=WRITE
-filter.write.onMatch=NEUTRAL
-filter.write.onMismatch=NEUTRAL
-
-# Log Levels are organized from most specific to least:
-# OFF (most specific, no logging)
-# FATAL (most specific, little data)
-# ERROR
-# WARN
-# INFO
-# DEBUG
-# TRACE (least specific, a lot of data)
-# ALL (least specific, all data)
-
-appenders=console, rolling
-appender.console.type=Console
-appender.console.name=STDOUT
-appender.console.layout.type=PatternLayout
-appender.console.layout.pattern=%d{DEFAULT} | %-5level | %c{1} | %msg | %throwable{3} %n
-
-#Rolling File Appender with size & time thresholds.
-#Rolling is triggered when either threshold is breached.
-#The rolled over file is compressed by default
-#Time interval is specified in seconds 86400s=1 day
-appender.rolling.type=RollingFile
-appender.rolling.name=RollingFile
-appender.rolling.fileName =${sys:hadoop.log.dir}/om-audit-${hostName}.log
-appender.rolling.filePattern=${sys:hadoop.log.dir}/om-audit-${hostName}-%d{yyyy-MM-dd-HH-mm-ss}-%i.log.gz
-appender.rolling.layout.type=PatternLayout
-appender.rolling.layout.pattern=%d{DEFAULT} | %-5level | %c{1} | %msg | %throwable{3} %n
-appender.rolling.policies.type=Policies
-appender.rolling.policies.time.type=TimeBasedTriggeringPolicy
-appender.rolling.policies.time.interval=86400
-appender.rolling.policies.size.type=SizeBasedTriggeringPolicy
-appender.rolling.policies.size.size=64MB
-
-loggers=audit
-logger.audit.type=AsyncLogger
-logger.audit.name=OMAudit
-logger.audit.level=INFO
-logger.audit.appenderRefs=rolling
-logger.audit.appenderRef.file.ref=RollingFile
-
-rootLogger.level=INFO
-rootLogger.appenderRefs=stdout
-rootLogger.appenderRef.stdout.ref=STDOUT

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6edf3d2e/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
index 6488f5e..d780ea2 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
@@ -18,33 +18,24 @@
 package org.apache.hadoop.ozone.audit;
 
 /**
- * Enum to define Audit Action types for OzoneManager.
+ * Enum to define OM Action types for Audit.
  */
 public enum OMAction implements AuditAction {
 
-  // WRITE Actions
-  ALLOCATE_BLOCK("ALLOCATE_BLOCK"),
-  ALLOCATE_KEY("ALLOCATE_KEY"),
-  COMMIT_KEY("COMMIT_KEY"),
   CREATE_VOLUME("CREATE_VOLUME"),
   CREATE_BUCKET("CREATE_BUCKET"),
   CREATE_KEY("CREATE_KEY"),
+  READ_VOLUME("READ_VOLUME"),
+  READ_BUCKET("READ_BUCKET"),
+  READ_KEY("READ_BUCKET"),
+  UPDATE_VOLUME("UPDATE_VOLUME"),
+  UPDATE_BUCKET("UPDATE_BUCKET"),
+  UPDATE_KEY("UPDATE_KEY"),
   DELETE_VOLUME("DELETE_VOLUME"),
   DELETE_BUCKET("DELETE_BUCKET"),
   DELETE_KEY("DELETE_KEY"),
-  RENAME_KEY("RENAME_KEY"),
   SET_OWNER("SET_OWNER"),
-  SET_QUOTA("SET_QUOTA"),
-  UPDATE_VOLUME("UPDATE_VOLUME"),
-  UPDATE_BUCKET("UPDATE_BUCKET"),
-  UPDATE_KEY("UPDATE_KEY"),
-  // READ Actions
-  LIST_BUCKETS("LIST_BUCKETS"),
-  LIST_VOLUMES("LIST_VOLUMES"),
-  LIST_KEYS("LIST_KEYS"),
-  READ_VOLUME("READ_VOLUME"),
-  READ_BUCKET("READ_BUCKET"),
-  READ_KEY("READ_BUCKET");
+  SET_QUOTA("SET_QUOTA");
 
   private String action;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6edf3d2e/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
index 1bd258e..6aabfef 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
@@ -17,17 +17,13 @@
  */
 package org.apache.hadoop.ozone.om.helpers;
 
-import java.util.LinkedHashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.stream.Collectors;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.audit.Auditable;
 import org.apache.hadoop.ozone.protocol.proto
     .OzoneManagerProtocolProtos.BucketArgs;
 import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
@@ -35,7 +31,7 @@ import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
 /**
  * A class that encapsulates Bucket Arguments.
  */
-public final class OmBucketArgs implements Auditable {
+public final class OmBucketArgs {
   /**
    * Name of the volume in which the bucket belongs to.
    */
@@ -139,25 +135,6 @@ public final class OmBucketArgs implements Auditable {
     return new Builder();
   }
 
-  @Override
-  public Map<String, String> toAuditMap() {
-    Map<String, String> auditMap = new LinkedHashMap<>();
-    auditMap.put(OzoneConsts.VOLUME, this.volumeName);
-    auditMap.put(OzoneConsts.BUCKET, this.bucketName);
-    if(this.addAcls != null){
-      auditMap.put(OzoneConsts.ADD_ACLS, this.addAcls.toString());
-    }
-    if(this.removeAcls != null){
-      auditMap.put(OzoneConsts.REMOVE_ACLS, this.removeAcls.toString());
-    }
-    auditMap.put(OzoneConsts.IS_VERSION_ENABLED,
-                String.valueOf(this.isVersionEnabled));
-    if(this.storageType != null){
-      auditMap.put(OzoneConsts.STORAGE_TYPE, this.storageType.name());
-    }
-    return auditMap;
-  }
-
   /**
    * Builder for OmBucketArgs.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6edf3d2e/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
index 0a136a7..bf5abdd 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
@@ -21,22 +21,18 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.audit.Auditable;
 import org.apache.hadoop.ozone.protocol.proto
     .OzoneManagerProtocolProtos.BucketInfo;
 import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
 
-import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
 import java.util.stream.Collectors;
 
 /**
  * A class that encapsulates Bucket Info.
  */
-public final class OmBucketInfo implements Auditable {
+public final class OmBucketInfo {
   /**
    * Name of the volume in which the bucket belongs to.
    */
@@ -141,21 +137,6 @@ public final class OmBucketInfo implements Auditable {
     return new Builder();
   }
 
-  @Override
-  public Map<String, String> toAuditMap() {
-    Map<String, String> auditMap = new LinkedHashMap<>();
-    auditMap.put(OzoneConsts.VOLUME, this.volumeName);
-    auditMap.put(OzoneConsts.BUCKET, this.bucketName);
-    auditMap.put(OzoneConsts.ACLS,
-        (this.acls != null) ? this.acls.toString() : null);
-    auditMap.put(OzoneConsts.IS_VERSION_ENABLED,
-        String.valueOf(this.isVersionEnabled));
-    auditMap.put(OzoneConsts.STORAGE_TYPE,
-        (this.storageType != null) ? this.storageType.name() : null);
-    auditMap.put(OzoneConsts.CREATION_TIME, String.valueOf(this.creationTime));
-    return auditMap;
-  }
-
   /**
    * Builder for OmBucketInfo.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6edf3d2e/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java
index d8d41d5..aab35c5 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java
@@ -18,18 +18,14 @@
 package org.apache.hadoop.ozone.om.helpers;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.audit.Auditable;
 
-import java.util.LinkedHashMap;
 import java.util.List;
-import java.util.Map;
 
 /**
  * Args for key. Client use this to specify key's attributes on  key creation
  * (putKey()).
  */
-public final class OmKeyArgs implements Auditable {
+public final class OmKeyArgs {
   private final String volumeName;
   private final String bucketName;
   private final String keyName;
@@ -86,22 +82,6 @@ public final class OmKeyArgs implements Auditable {
     return locationInfoList;
   }
 
-  @Override
-  public Map<String, String> toAuditMap() {
-    Map<String, String> auditMap = new LinkedHashMap<>();
-    auditMap.put(OzoneConsts.VOLUME, this.volumeName);
-    auditMap.put(OzoneConsts.BUCKET, this.bucketName);
-    auditMap.put(OzoneConsts.KEY, this.keyName);
-    auditMap.put(OzoneConsts.DATA_SIZE, String.valueOf(this.dataSize));
-    auditMap.put(OzoneConsts.REPLICATION_TYPE,
-        (this.type != null) ? this.type.name() : null);
-    auditMap.put(OzoneConsts.REPLICATION_FACTOR,
-        (this.factor != null) ? this.factor.name() : null);
-    auditMap.put(OzoneConsts.KEY_LOCATION_INFO,
-        (this.locationInfoList != null) ? locationInfoList.toString() : null);
-    return auditMap;
-  }
-
   /**
    * Builder class of OmKeyArgs.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6edf3d2e/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
index 27e25f9..c8b59b6 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
@@ -18,8 +18,6 @@
 package org.apache.hadoop.ozone.om.helpers;
 
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.audit.Auditable;
 import org.apache.hadoop.ozone.protocol.proto
     .OzoneManagerProtocolProtos.OzoneAclInfo;
 import org.apache.hadoop.ozone.protocol.proto
@@ -28,7 +26,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.KeyValue;
 
 import java.io.IOException;
 import java.util.HashMap;
-import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -38,7 +35,7 @@ import java.util.stream.Collectors;
 /**
  * A class that encapsulates the OmVolumeArgs Args.
  */
-public final class OmVolumeArgs implements Auditable{
+public final class OmVolumeArgs {
   private final String adminName;
   private final String ownerName;
   private final String volume;
@@ -125,17 +122,6 @@ public final class OmVolumeArgs implements Auditable{
     return new Builder();
   }
 
-  @Override
-  public Map<String, String> toAuditMap() {
-    Map<String, String> auditMap = new LinkedHashMap<>();
-    auditMap.put(OzoneConsts.ADMIN, this.adminName);
-    auditMap.put(OzoneConsts.OWNER, this.ownerName);
-    auditMap.put(OzoneConsts.VOLUME, this.volume);
-    auditMap.put(OzoneConsts.CREATION_TIME, String.valueOf(this.creationTime));
-    auditMap.put(OzoneConsts.QUOTA_IN_BYTES, String.valueOf(this.quotaInBytes));
-    return auditMap;
-  }
-
   /**
    * Builder for OmVolumeArgs.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6edf3d2e/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
index 887ddd0..71fa921 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
@@ -27,14 +27,6 @@ import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.hdds.server.ServiceRuntimeInfoImpl;
-import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.audit.AuditAction;
-import org.apache.hadoop.ozone.audit.AuditEventStatus;
-import org.apache.hadoop.ozone.audit.AuditLogger;
-import org.apache.hadoop.ozone.audit.AuditLoggerType;
-import org.apache.hadoop.ozone.audit.AuditMessage;
-import org.apache.hadoop.ozone.audit.OMAction;
 import org.apache.hadoop.ozone.common.Storage.StorageState;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
@@ -77,8 +69,6 @@ import static org.apache.hadoop.hdds.HddsUtils.isHddsEnabled;
 import static org.apache.hadoop.ozone.OmUtils.getOmAddress;
 import static org.apache.hadoop.hdds.server.ServerUtils
     .updateRPCListenAddress;
-
-import org.apache.logging.log4j.Level;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -88,7 +78,6 @@ import java.io.PrintStream;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -115,9 +104,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   private static final Logger LOG =
       LoggerFactory.getLogger(OzoneManager.class);
 
-  private static final AuditLogger AUDIT =
-      new AuditLogger(AuditLoggerType.OMLOGGER);
-
   private static final String USAGE =
       "Usage: \n ozone om [genericOptions] " + "[ "
           + StartupOption.CREATEOBJECTSTORE.getName() + " ]\n " + "ozone om [ "
@@ -349,8 +335,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    * @throws IOException in case OM instance creation fails.
    */
 
-  public static OzoneManager createOm(String[] argv, OzoneConfiguration conf)
-      throws IOException {
+  public static OzoneManager createOm(String[] argv,
+                                      OzoneConfiguration conf) throws IOException {
     if (!isHddsEnabled(conf)) {
       System.err.println("OM cannot be started in secure mode or when " +
           OZONE_ENABLED + " is set to false");
@@ -500,13 +486,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     try {
       metrics.incNumVolumeCreates();
       volumeManager.createVolume(args);
-      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.CREATE_VOLUME,
-          args.toAuditMap()));
     } catch (Exception ex) {
       metrics.incNumVolumeCreateFails();
-      AUDIT.logWriteFailure(Level.ERROR,
-          buildAuditMessageForFailure(OMAction.CREATE_VOLUME,
-          args.toAuditMap()), ex);
       throw ex;
     }
   }
@@ -520,17 +501,11 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    */
   @Override
   public void setOwner(String volume, String owner) throws IOException {
-    Map<String, String> auditMap = buildAuditMap(volume);
-    auditMap.put(OzoneConsts.OWNER, owner);
     try {
       metrics.incNumVolumeUpdates();
       volumeManager.setOwner(volume, owner);
-      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.SET_OWNER,
-          auditMap));
     } catch (Exception ex) {
       metrics.incNumVolumeUpdateFails();
-      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.SET_OWNER,
-          auditMap), ex);
       throw ex;
     }
   }
@@ -544,22 +519,15 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    */
   @Override
   public void setQuota(String volume, long quota) throws IOException {
-    Map<String, String> auditMap = buildAuditMap(volume);
-    auditMap.put(OzoneConsts.QUOTA, String.valueOf(quota));
     try {
       metrics.incNumVolumeUpdates();
       volumeManager.setQuota(volume, quota);
-      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.SET_QUOTA,
-          auditMap));
     } catch (Exception ex) {
       metrics.incNumVolumeUpdateFails();
-      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.SET_QUOTA,
-          auditMap), ex);
       throw ex;
     }
   }
 
-  //TODO: Should define new OMAction type? How to log OzoneAclInfo ?
   /**
    * Checks if the specified user can access this volume.
    *
@@ -572,23 +540,12 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   @Override
   public boolean checkVolumeAccess(String volume, OzoneAclInfo userAcl)
       throws IOException {
-    boolean auditSuccess = true;
-    Map<String, String> auditMap = buildAuditMap(volume);
-    auditMap.put(OzoneConsts.USER_ACL, userAcl.getName());
     try {
       metrics.incNumVolumeCheckAccesses();
       return volumeManager.checkVolumeAccess(volume, userAcl);
     } catch (Exception ex) {
       metrics.incNumVolumeCheckAccessFails();
-      auditSuccess = false;
-      AUDIT.logReadFailure(buildAuditMessageForFailure(OMAction.READ_VOLUME,
-          auditMap), ex);
       throw ex;
-    } finally {
-      if(auditSuccess){
-        AUDIT.logReadSuccess(buildAuditMessageForSuccess(OMAction.READ_VOLUME,
-            auditMap));
-      }
     }
   }
 
@@ -601,22 +558,12 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    */
   @Override
   public OmVolumeArgs getVolumeInfo(String volume) throws IOException {
-    boolean auditSuccess = true;
-    Map<String, String> auditMap = buildAuditMap(volume);
     try {
       metrics.incNumVolumeInfos();
       return volumeManager.getVolumeInfo(volume);
     } catch (Exception ex) {
       metrics.incNumVolumeInfoFails();
-      auditSuccess = false;
-      AUDIT.logReadFailure(buildAuditMessageForFailure(OMAction.READ_VOLUME,
-          auditMap), ex);
       throw ex;
-    } finally {
-      if(auditSuccess){
-        AUDIT.logReadSuccess(buildAuditMessageForSuccess(OMAction.READ_VOLUME,
-            auditMap));
-      }
     }
   }
 
@@ -631,12 +578,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     try {
       metrics.incNumVolumeDeletes();
       volumeManager.deleteVolume(volume);
-      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.DELETE_VOLUME,
-          buildAuditMap(volume)));
     } catch (Exception ex) {
       metrics.incNumVolumeDeleteFails();
-      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.DELETE_VOLUME,
-          buildAuditMap(volume)), ex);
       throw ex;
     }
   }
@@ -654,27 +597,13 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    */
   @Override
   public List<OmVolumeArgs> listVolumeByUser(String userName, String prefix,
-      String prevKey, int maxKeys) throws IOException {
-    boolean auditSuccess = true;
-    Map<String, String> auditMap = new LinkedHashMap<>();
-    auditMap.put(OzoneConsts.PREV_KEY, prevKey);
-    auditMap.put(OzoneConsts.PREFIX, prefix);
-    auditMap.put(OzoneConsts.MAX_KEYS, String.valueOf(maxKeys));
-    auditMap.put(OzoneConsts.USERNAME, userName);
+                                             String prevKey, int maxKeys) throws IOException {
     try {
       metrics.incNumVolumeLists();
       return volumeManager.listVolumes(userName, prefix, prevKey, maxKeys);
     } catch (Exception ex) {
       metrics.incNumVolumeListFails();
-      auditSuccess = false;
-      AUDIT.logReadFailure(buildAuditMessageForFailure(OMAction.LIST_VOLUMES,
-          auditMap), ex);
       throw ex;
-    } finally {
-      if(auditSuccess){
-        AUDIT.logReadSuccess(buildAuditMessageForSuccess(OMAction.LIST_VOLUMES,
-            auditMap));
-      }
     }
   }
 
@@ -691,26 +620,12 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   @Override
   public List<OmVolumeArgs> listAllVolumes(String prefix, String prevKey, int
       maxKeys) throws IOException {
-    boolean auditSuccess = true;
-    Map<String, String> auditMap = new LinkedHashMap<>();
-    auditMap.put(OzoneConsts.PREV_KEY, prevKey);
-    auditMap.put(OzoneConsts.PREFIX, prefix);
-    auditMap.put(OzoneConsts.MAX_KEYS, String.valueOf(maxKeys));
-    auditMap.put(OzoneConsts.USERNAME, null);
     try {
       metrics.incNumVolumeLists();
       return volumeManager.listVolumes(null, prefix, prevKey, maxKeys);
     } catch (Exception ex) {
       metrics.incNumVolumeListFails();
-      auditSuccess = false;
-      AUDIT.logReadFailure(buildAuditMessageForFailure(OMAction.LIST_VOLUMES,
-          auditMap), ex);
       throw ex;
-    } finally {
-      if(auditSuccess){
-        AUDIT.logReadSuccess(buildAuditMessageForSuccess(OMAction.LIST_VOLUMES,
-            auditMap));
-      }
     }
   }
 
@@ -725,12 +640,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     try {
       metrics.incNumBucketCreates();
       bucketManager.createBucket(bucketInfo);
-      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.CREATE_BUCKET,
-          bucketInfo.toAuditMap()));
     } catch (Exception ex) {
       metrics.incNumBucketCreateFails();
-      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.CREATE_BUCKET,
-          bucketInfo.toAuditMap()), ex);
       throw ex;
     }
   }
@@ -739,30 +650,16 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    * {@inheritDoc}
    */
   @Override
-  public List<OmBucketInfo> listBuckets(String volumeName, String startKey,
-      String prefix, int maxNumOfBuckets)
+  public List<OmBucketInfo> listBuckets(String volumeName,
+                                        String startKey, String prefix, int maxNumOfBuckets)
       throws IOException {
-    boolean auditSuccess = true;
-    Map<String, String> auditMap = buildAuditMap(volumeName);
-    auditMap.put(OzoneConsts.START_KEY, startKey);
-    auditMap.put(OzoneConsts.PREFIX, prefix);
-    auditMap.put(OzoneConsts.MAX_NUM_OF_BUCKETS,
-                String.valueOf(maxNumOfBuckets));
     try {
       metrics.incNumBucketLists();
       return bucketManager.listBuckets(volumeName,
           startKey, prefix, maxNumOfBuckets);
     } catch (IOException ex) {
       metrics.incNumBucketListFails();
-      auditSuccess = false;
-      AUDIT.logReadFailure(buildAuditMessageForFailure(OMAction.LIST_BUCKETS,
-          auditMap), ex);
       throw ex;
-    } finally {
-      if(auditSuccess){
-        AUDIT.logReadSuccess(buildAuditMessageForSuccess(OMAction.LIST_BUCKETS,
-            auditMap));
-      }
     }
   }
 
@@ -777,23 +674,12 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   @Override
   public OmBucketInfo getBucketInfo(String volume, String bucket)
       throws IOException {
-    boolean auditSuccess = true;
-    Map<String, String> auditMap = buildAuditMap(volume);
-    auditMap.put(OzoneConsts.BUCKET, bucket);
     try {
       metrics.incNumBucketInfos();
       return bucketManager.getBucketInfo(volume, bucket);
     } catch (Exception ex) {
       metrics.incNumBucketInfoFails();
-      auditSuccess = false;
-      AUDIT.logReadFailure(buildAuditMessageForFailure(OMAction.READ_BUCKET,
-          auditMap), ex);
       throw ex;
-    } finally {
-      if(auditSuccess){
-        AUDIT.logReadSuccess(buildAuditMessageForSuccess(OMAction.READ_BUCKET,
-            auditMap));
-      }
     }
   }
 
@@ -806,38 +692,23 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    */
   @Override
   public OpenKeySession openKey(OmKeyArgs args) throws IOException {
-    boolean auditSuccess = true;
     try {
       metrics.incNumKeyAllocates();
       return keyManager.openKey(args);
     } catch (Exception ex) {
       metrics.incNumKeyAllocateFails();
-      auditSuccess = false;
-      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.ALLOCATE_KEY,
-          args.toAuditMap()), ex);
       throw ex;
-    } finally {
-      if(auditSuccess){
-        AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.ALLOCATE_KEY,
-            args.toAuditMap()));
-      }
     }
   }
 
   @Override
   public void commitKey(OmKeyArgs args, int clientID)
       throws IOException {
-    Map<String, String> auditMap = args.toAuditMap();
-    auditMap.put(OzoneConsts.CLIENT_ID, String.valueOf(clientID));
     try {
       metrics.incNumKeyCommits();
       keyManager.commitKey(args, clientID);
-      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.COMMIT_KEY,
-          args.toAuditMap()));
     } catch (Exception ex) {
       metrics.incNumKeyCommitFails();
-      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.COMMIT_KEY,
-          args.toAuditMap()), ex);
       throw ex;
     }
   }
@@ -845,24 +716,12 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   @Override
   public OmKeyLocationInfo allocateBlock(OmKeyArgs args, int clientID)
       throws IOException {
-    boolean auditSuccess = true;
-    Map<String, String> auditMap = args.toAuditMap();
-    auditMap.put(OzoneConsts.CLIENT_ID, String.valueOf(clientID));
     try {
       metrics.incNumBlockAllocateCalls();
       return keyManager.allocateBlock(args, clientID);
     } catch (Exception ex) {
       metrics.incNumBlockAllocateCallFails();
-      auditSuccess = false;
-      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.ALLOCATE_BLOCK,
-          args.toAuditMap()), ex);
       throw ex;
-    } finally {
-      if(auditSuccess){
-        AUDIT.logWriteSuccess(buildAuditMessageForSuccess(
-            OMAction.ALLOCATE_BLOCK,
-            args.toAuditMap()));
-      }
     }
   }
 
@@ -875,37 +734,22 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    */
   @Override
   public OmKeyInfo lookupKey(OmKeyArgs args) throws IOException {
-    boolean auditSuccess = true;
     try {
       metrics.incNumKeyLookups();
       return keyManager.lookupKey(args);
     } catch (Exception ex) {
       metrics.incNumKeyLookupFails();
-      auditSuccess = false;
-      AUDIT.logReadFailure(buildAuditMessageForFailure(OMAction.READ_KEY,
-          args.toAuditMap()), ex);
       throw ex;
-    } finally {
-      if(auditSuccess){
-        AUDIT.logReadSuccess(buildAuditMessageForSuccess(OMAction.READ_KEY,
-            args.toAuditMap()));
-      }
     }
   }
 
   @Override
   public void renameKey(OmKeyArgs args, String toKeyName) throws IOException {
-    Map<String, String> auditMap = args.toAuditMap();
-    auditMap.put(OzoneConsts.TO_KEY_NAME, toKeyName);
     try {
       metrics.incNumKeyRenames();
       keyManager.renameKey(args, toKeyName);
-      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.RENAME_KEY,
-          args.toAuditMap()));
     } catch (IOException e) {
       metrics.incNumKeyRenameFails();
-      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.RENAME_KEY,
-          args.toAuditMap()), e);
       throw e;
     }
   }
@@ -921,40 +765,22 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     try {
       metrics.incNumKeyDeletes();
       keyManager.deleteKey(args);
-      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.DELETE_KEY,
-          args.toAuditMap()));
     } catch (Exception ex) {
       metrics.incNumKeyDeleteFails();
-      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.DELETE_KEY,
-          args.toAuditMap()), ex);
       throw ex;
     }
   }
 
   @Override
   public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
-      String startKey, String keyPrefix, int maxKeys) throws IOException {
-    boolean auditSuccess = true;
-    Map<String, String> auditMap = buildAuditMap(volumeName);
-    auditMap.put(OzoneConsts.BUCKET, bucketName);
-    auditMap.put(OzoneConsts.START_KEY, startKey);
-    auditMap.put(OzoneConsts.MAX_KEYS, String.valueOf(maxKeys));
-    auditMap.put(OzoneConsts.KEY_PREFIX, keyPrefix);
+                                  String startKey, String keyPrefix, int maxKeys) throws IOException {
     try {
       metrics.incNumKeyLists();
       return keyManager.listKeys(volumeName, bucketName,
           startKey, keyPrefix, maxKeys);
     } catch (IOException ex) {
       metrics.incNumKeyListFails();
-      auditSuccess = false;
-      AUDIT.logReadFailure(buildAuditMessageForFailure(OMAction.LIST_KEYS,
-          auditMap), ex);
       throw ex;
-    } finally {
-      if(auditSuccess){
-        AUDIT.logReadSuccess(buildAuditMessageForSuccess(OMAction.LIST_KEYS,
-            auditMap));
-      }
     }
   }
 
@@ -969,12 +795,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     try {
       metrics.incNumBucketUpdates();
       bucketManager.setBucketProperty(args);
-      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.UPDATE_BUCKET,
-          args.toAuditMap()));
     } catch (Exception ex) {
       metrics.incNumBucketUpdateFails();
-      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.UPDATE_BUCKET,
-          args.toAuditMap()), ex);
       throw ex;
     }
   }
@@ -987,17 +809,11 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    * @throws IOException
    */
   public void deleteBucket(String volume, String bucket) throws IOException {
-    Map<String, String> auditMap = buildAuditMap(volume);
-    auditMap.put(OzoneConsts.BUCKET, bucket);
     try {
       metrics.incNumBucketDeletes();
       bucketManager.deleteBucket(volume, bucket);
-      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.DELETE_BUCKET,
-          auditMap));
     } catch (Exception ex) {
       metrics.incNumBucketDeleteFails();
-      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.DELETE_BUCKET,
-          auditMap), ex);
       throw ex;
     }
   }
@@ -1092,26 +908,4 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     // metrics.incNumGetServiceListFails()
     return services;
   }
-
-  private Map<String, String> buildAuditMap(String volume){
-    Map<String, String> auditMap = new LinkedHashMap<>();
-    auditMap.put(OzoneConsts.VOLUME, volume);
-    return auditMap;
-  }
-
-  // TODO: Temporary method until AuditMessage is simplified
-  private AuditMessage buildAuditMessageForSuccess(AuditAction op,
-      Map<String, String> auditMap) {
-    return new AuditMessage(Server.getRemoteUser().getUserName(),
-        Server.getRemoteIp().getHostAddress(), op.toString(), auditMap,
-        AuditEventStatus.SUCCESS.toString());
-  }
-
-  // TODO: Temporary method until AuditMessage is simplified
-  private AuditMessage buildAuditMessageForFailure(AuditAction op,
-      Map<String, String> auditMap) {
-    return new AuditMessage(Server.getRemoteUser().getUserName(),
-        Server.getRemoteIp().getHostAddress(), op.toString(), auditMap,
-        AuditEventStatus.FAILURE.toString());
-  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[02/50] [abbrv] hadoop git commit: HDSS-375. ContainerReportHandler should not send replication events for open containers. Contributed by Ajay Kumar.

Posted by eh...@apache.org.
HDSS-375. ContainerReportHandler should not send replication events for open containers. Contributed by Ajay Kumar.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c9b63956
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c9b63956
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c9b63956

Branch: refs/heads/HDFS-12090
Commit: c9b63956d97521ec21a051bfcbbf4b79262ea16f
Parents: f152582
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Mon Aug 27 10:39:30 2018 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Mon Aug 27 10:40:33 2018 -0700

----------------------------------------------------------------------
 .../scm/container/ContainerReportHandler.java   |  4 ++
 .../container/TestContainerReportHandler.java   | 40 +++++++++++++++-----
 2 files changed, 34 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9b63956/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
index 5a9e726..5ca2bcb 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReportHandler.java
@@ -129,6 +129,10 @@ public class ContainerReportHandler implements
           "Container is missing from containerStateManager. Can't request "
               + "replication. {}",
           containerID);
+      return;
+    }
+    if (container.isContainerOpen()) {
+      return;
     }
     if (replicationStatus.isReplicationEnabled()) {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9b63956/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
index e7b6cd9..443b4b2 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
@@ -84,6 +84,7 @@ public class TestContainerReportHandler implements EventPublisher {
                 new Builder()
                     .setReplicationFactor(ReplicationFactor.THREE)
                     .setContainerID((Long) invocation.getArguments()[0])
+                    .setState(LifeCycleState.CLOSED)
                     .build()
         );
 
@@ -116,26 +117,45 @@ public class TestContainerReportHandler implements EventPublisher {
     when(pipelineSelector.getReplicationPipeline(ReplicationType.STAND_ALONE,
         ReplicationFactor.THREE)).thenReturn(pipeline);
 
-    long c1 = containerStateManager
+    ContainerInfo cont1 = containerStateManager
         .allocateContainer(pipelineSelector, ReplicationType.STAND_ALONE,
-            ReplicationFactor.THREE, "root").getContainerInfo()
-        .getContainerID();
-
-    long c2 = containerStateManager
+            ReplicationFactor.THREE, "root").getContainerInfo();
+    ContainerInfo cont2 = containerStateManager
         .allocateContainer(pipelineSelector, ReplicationType.STAND_ALONE,
-            ReplicationFactor.THREE, "root").getContainerInfo()
-        .getContainerID();
-
+            ReplicationFactor.THREE, "root").getContainerInfo();
+    // Open Container
+    ContainerInfo cont3 = containerStateManager
+        .allocateContainer(pipelineSelector, ReplicationType.STAND_ALONE,
+            ReplicationFactor.THREE, "root").getContainerInfo();
+
+    long c1 = cont1.getContainerID();
+    long c2 = cont2.getContainerID();
+    long c3 = cont3.getContainerID();
+
+    // Close remaining containers
+    try {
+      containerStateManager.getContainerStateMap()
+          .updateState(cont1, cont1.getState(), LifeCycleState.CLOSING);
+      containerStateManager.getContainerStateMap()
+          .updateState(cont1, cont1.getState(), LifeCycleState.CLOSED);
+      containerStateManager.getContainerStateMap()
+          .updateState(cont2, cont2.getState(), LifeCycleState.CLOSING);
+      containerStateManager.getContainerStateMap()
+          .updateState(cont2, cont2.getState(), LifeCycleState.CLOSED);
+
+    } catch (IOException e) {
+      LOG.info("Failed to change state of open containers.", e);
+    }
     //when
 
     //initial reports before replication is enabled. 2 containers w 3 replicas.
     reportHandler.onMessage(
         new ContainerReportFromDatanode(dn1,
-            createContainerReport(new long[] {c1, c2})), this);
+            createContainerReport(new long[] {c1, c2, c3})), this);
 
     reportHandler.onMessage(
         new ContainerReportFromDatanode(dn2,
-            createContainerReport(new long[] {c1, c2})), this);
+            createContainerReport(new long[] {c1, c2, c3})), this);
 
     reportHandler.onMessage(
         new ContainerReportFromDatanode(dn3,


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[37/50] [abbrv] hadoop git commit: HDDS-388. Fix the name of the db profile configuration key. Contributed by Elek, Marton.

Posted by eh...@apache.org.
HDDS-388. Fix the name of the db profile configuration key.
Contributed by Elek, Marton.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/50d2e3ec
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/50d2e3ec
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/50d2e3ec

Branch: refs/heads/HDFS-12090
Commit: 50d2e3ec41c73f9a0198d4a4e3d6f308d3030b8a
Parents: 630b64e
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Aug 31 14:30:29 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Fri Aug 31 14:30:29 2018 -0700

----------------------------------------------------------------------
 hadoop-hdds/common/src/main/resources/ozone-default.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/50d2e3ec/hadoop-hdds/common/src/main/resources/ozone-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 6d2ee09..d3ec4a5 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -1100,7 +1100,7 @@
   </property>
 
   <property>
-    <name>ozone.db.profile</name>
+    <name>hdds.db.profile</name>
     <value>DBProfile.SSD</value>
     <tag>OZONE, OM, PERFORMANCE, REQUIRED</tag>
     <description>This property allows user to pick a configuration


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[45/50] [abbrv] hadoop git commit: HDDS-263. Add retries in Ozone Client to handle BlockNotCommitted Exception. Contributed by Shashikant Banerjee.

Posted by eh...@apache.org.
HDDS-263. Add retries in Ozone Client to handle BlockNotCommitted Exception. Contributed by Shashikant Banerjee.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/873ef8ae
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/873ef8ae
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/873ef8ae

Branch: refs/heads/HDFS-12090
Commit: 873ef8ae81321325889c9d3a6939163e98fbf5bb
Parents: ff036e4
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Mon Sep 3 12:26:34 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Mon Sep 3 12:26:34 2018 +0530

----------------------------------------------------------------------
 .../helpers/BlockNotCommittedException.java     | 36 ++++++++
 .../scm/storage/ContainerProtocolCalls.java     |  5 ++
 .../apache/hadoop/ozone/OzoneConfigKeys.java    |  8 ++
 .../common/src/main/resources/ozone-default.xml | 16 ++++
 .../hadoop/ozone/client/OzoneClientUtils.java   | 28 ++++++
 .../ozone/client/io/ChunkGroupOutputStream.java | 89 +++++++++++++++----
 .../hadoop/ozone/client/rpc/RpcClient.java      |  5 ++
 .../rpc/TestCloseContainerHandlingByClient.java | 91 +++++++++++++++++---
 .../web/storage/DistributedStorageHandler.java  |  5 ++
 9 files changed, 254 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/873ef8ae/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/BlockNotCommittedException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/BlockNotCommittedException.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/BlockNotCommittedException.java
new file mode 100644
index 0000000..86f5a66
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/BlockNotCommittedException.java
@@ -0,0 +1,36 @@
+/*
+ * 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.scm.container.common.helpers;
+
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+
+/**
+ * Exceptions thrown when a block is yet to be committed on the datanode.
+ */
+public class BlockNotCommittedException extends StorageContainerException {
+
+  /**
+   * Constructs an {@code IOException} with the specified detail message.
+   *
+   * @param message The detail message (which is saved for later retrieval by
+   * the {@link #getMessage()} method)
+   */
+  public BlockNotCommittedException(String message) {
+    super(message, ContainerProtos.Result.BLOCK_NOT_COMMITTED);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/873ef8ae/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
index 1f2fafb..1d6a89d 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hdds.scm.storage;
 
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .BlockNotCommittedException;
 import org.apache.ratis.shaded.com.google.protobuf.ByteString;
 import org.apache.hadoop.hdds.scm.XceiverClientSpi;
 import org.apache.hadoop.hdds.scm.container.common.helpers
@@ -420,6 +422,9 @@ public final class ContainerProtocolCalls  {
   ) throws StorageContainerException {
     if (response.getResult() == ContainerProtos.Result.SUCCESS) {
       return;
+    } else if (response.getResult()
+        == ContainerProtos.Result.BLOCK_NOT_COMMITTED) {
+      throw new BlockNotCommittedException(response.getMessage());
     }
     throw new StorageContainerException(
         response.getMessage(), response.getResult());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/873ef8ae/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
index 6ad9085..8f53da5 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
@@ -194,6 +194,14 @@ public final class OzoneConfigKeys {
   public static final int
       OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL_DEFAULT = 10;
 
+  public static final String OZONE_CLIENT_MAX_RETRIES =
+      "ozone.client.max.retries";
+  public static final int OZONE_CLIENT_MAX_RETRIES_DEFAULT = 50;
+
+  public static final String OZONE_CLIENT_RETRY_INTERVAL =
+      "ozone.client.retry.interval";
+  public static final String OZONE_CLIENT_RETRY_INTERVAL_DEFAULT = "200ms";
+
   public static final String DFS_CONTAINER_RATIS_ENABLED_KEY
       = ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY;
   public static final boolean DFS_CONTAINER_RATIS_ENABLED_DEFAULT

http://git-wip-us.apache.org/repos/asf/hadoop/blob/873ef8ae/hadoop-hdds/common/src/main/resources/ozone-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 6f296c6..a9fd10b 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -244,6 +244,22 @@
     </description>
   </property>
   <property>
+    <name>ozone.client.max.retries</name>
+    <value>50</value>
+    <tag>OZONE, CLIENT</tag>
+    <description>Maximum number of retries by Ozone Client on encountering
+      exception while fetching committed block length.
+    </description>
+  </property>
+  <property>
+    <name>ozone.client.retry.interval</name>
+    <value>200ms</value>
+    <tag>OZONE, CLIENT</tag>
+    <description>Interval between retries by Ozone Client on encountering
+      exception while fetching committed block length.
+    </description>
+  </property>
+  <property>
     <name>ozone.client.protocol</name>
     <value>org.apache.hadoop.ozone.client.rpc.RpcClient</value>
     <tag>OZONE, CLIENT, MANAGEMENT</tag>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/873ef8ae/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java
index 0aaee31..5d57753 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java
@@ -17,14 +17,23 @@
  */
 package org.apache.hadoop.ozone.client;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.client.OzoneQuota;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
+import org.apache.hadoop.hdds.scm.container.common.helpers.BlockNotCommittedException;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.rest.response.BucketInfo;
 import org.apache.hadoop.ozone.client.rest.response.KeyInfo;
 import org.apache.hadoop.ozone.client.rest.response.VolumeInfo;
 import org.apache.hadoop.ozone.client.rest.response.VolumeOwner;
 
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
 /** A utility class for OzoneClient. */
 public final class OzoneClientUtils {
 
@@ -84,4 +93,23 @@ public final class OzoneClientUtils {
     keyInfo.setSize(key.getDataSize());
     return keyInfo;
   }
+
+  public static RetryPolicy createRetryPolicy(Configuration conf) {
+    int maxRetryCount =
+        conf.getInt(OzoneConfigKeys.OZONE_CLIENT_MAX_RETRIES, OzoneConfigKeys.
+            OZONE_CLIENT_MAX_RETRIES_DEFAULT);
+    long retryInterval = conf.getTimeDuration(OzoneConfigKeys.
+        OZONE_CLIENT_RETRY_INTERVAL, OzoneConfigKeys.
+        OZONE_CLIENT_RETRY_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS);
+    RetryPolicy basePolicy = RetryPolicies
+        .retryUpToMaximumCountWithFixedSleep(maxRetryCount, retryInterval,
+            TimeUnit.MILLISECONDS);
+    Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
+        new HashMap<Class<? extends Exception>, RetryPolicy>();
+    exceptionToPolicyMap.put(BlockNotCommittedException.class, basePolicy);
+    RetryPolicy retryPolicy = RetryPolicies
+        .retryByException(RetryPolicies.TRY_ONCE_THEN_FAIL,
+            exceptionToPolicyMap);
+    return retryPolicy;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/873ef8ae/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
index c632df6..21406b5 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
+import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
@@ -46,6 +47,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -63,7 +65,7 @@ import java.util.Optional;
  */
 public class ChunkGroupOutputStream extends OutputStream {
 
-  private static final Logger LOG =
+  public static final Logger LOG =
       LoggerFactory.getLogger(ChunkGroupOutputStream.class);
 
   // array list's get(index) is O(1)
@@ -80,6 +82,7 @@ public class ChunkGroupOutputStream extends OutputStream {
   private final String requestID;
   private boolean closed;
   private List<OmKeyLocationInfo> locationInfoList;
+  private final RetryPolicy retryPolicy;
   /**
    * A constructor for testing purpose only.
    */
@@ -95,6 +98,7 @@ public class ChunkGroupOutputStream extends OutputStream {
     requestID = null;
     closed = false;
     locationInfoList = null;
+    retryPolicy = null;
   }
 
   /**
@@ -124,7 +128,7 @@ public class ChunkGroupOutputStream extends OutputStream {
       StorageContainerLocationProtocolClientSideTranslatorPB scmClient,
       OzoneManagerProtocolClientSideTranslatorPB omClient,
       int chunkSize, String requestId, ReplicationFactor factor,
-      ReplicationType type) throws IOException {
+      ReplicationType type, RetryPolicy retryPolicy) throws IOException {
     this.streamEntries = new ArrayList<>();
     this.currentStreamIndex = 0;
     this.byteOffset = 0;
@@ -143,6 +147,7 @@ public class ChunkGroupOutputStream extends OutputStream {
     this.chunkSize = chunkSize;
     this.requestID = requestId;
     this.locationInfoList = new ArrayList<>();
+    this.retryPolicy = retryPolicy;
     LOG.debug("Expecting open key with one block, but got" +
         info.getKeyLocationVersions().size());
   }
@@ -305,6 +310,62 @@ public class ChunkGroupOutputStream extends OutputStream {
     }
   }
 
+  private long getCommittedBlockLength(ChunkOutputStreamEntry streamEntry)
+      throws IOException {
+    long blockLength;
+    ContainerProtos.GetCommittedBlockLengthResponseProto responseProto;
+    RetryPolicy.RetryAction action;
+    int numRetries = 0;
+
+    // TODO : At this point of time, we also need to allocate new blocks
+    // from a different container and may need to nullify
+    // all the remaining pre-allocated blocks in case they were
+    // pre-allocated on the same container which got closed now.This needs
+    // caching the closed container list on the client itself.
+    while (true) {
+      try {
+        responseProto = ContainerProtocolCalls
+            .getCommittedBlockLength(streamEntry.xceiverClient,
+                streamEntry.blockID, requestID);
+        blockLength = responseProto.getBlockLength();
+        return blockLength;
+      } catch (StorageContainerException sce) {
+        try {
+          action = retryPolicy.shouldRetry(sce, numRetries, 0, true);
+        } catch (Exception e) {
+          throw e instanceof IOException ? (IOException) e : new IOException(e);
+        }
+        if (action.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
+          if (action.reason != null) {
+            LOG.error(
+                "GetCommittedBlockLength request failed. " + action.reason,
+                sce);
+          }
+          throw sce;
+        }
+
+        // Throw the exception if the thread is interrupted
+        if (Thread.currentThread().isInterrupted()) {
+          LOG.warn("Interrupted while trying for connection");
+          throw sce;
+        }
+        Preconditions.checkArgument(
+            action.action == RetryPolicy.RetryAction.RetryDecision.RETRY);
+        try {
+          Thread.sleep(action.delayMillis);
+        } catch (InterruptedException e) {
+          throw (IOException) new InterruptedIOException(
+              "Interrupted: action=" + action + ", retry policy=" + retryPolicy)
+              .initCause(e);
+        }
+        numRetries++;
+        LOG.trace("Retrying GetCommittedBlockLength request. Already tried "
+            + numRetries + " time(s); retry policy is " + retryPolicy);
+        continue;
+      }
+    }
+  }
+
   /**
    * It performs following actions :
    * a. Updates the committed length at datanode for the current stream in
@@ -317,15 +378,6 @@ public class ChunkGroupOutputStream extends OutputStream {
    */
   private void handleCloseContainerException(ChunkOutputStreamEntry streamEntry,
       int streamIndex) throws IOException {
-    // TODO : If the block is still not committed and is in the
-    // pending openBlock Map, it will return BLOCK_NOT_COMMITTED
-    // exception. We should handle this by retrying the same operation
-    // n times and update the OzoneManager with the actual block length
-    // written. At this point of time, we also need to allocate new blocks
-    // from a different container and may need to nullify
-    // all the remaining pre-allocated blocks in case they were
-    // pre-allocated on the same container which got closed now.This needs
-    // caching the closed container list on the client itself.
     long committedLength = 0;
     ByteBuffer buffer = streamEntry.getBuffer();
     if (buffer == null) {
@@ -342,11 +394,7 @@ public class ChunkGroupOutputStream extends OutputStream {
     // for this block associated with the stream here.
     if (streamEntry.currentPosition >= chunkSize
         || streamEntry.currentPosition != buffer.position()) {
-      ContainerProtos.GetCommittedBlockLengthResponseProto responseProto =
-          ContainerProtocolCalls
-              .getCommittedBlockLength(streamEntry.xceiverClient,
-                  streamEntry.blockID, requestID);
-      committedLength = responseProto.getBlockLength();
+      committedLength = getCommittedBlockLength(streamEntry);
       // update the length of the current stream
       locationInfoList.get(streamIndex).setLength(committedLength);
     }
@@ -481,6 +529,7 @@ public class ChunkGroupOutputStream extends OutputStream {
     private String requestID;
     private ReplicationType type;
     private ReplicationFactor factor;
+    private RetryPolicy retryPolicy;
 
     public Builder setHandler(OpenKeySession handler) {
       this.openHandler = handler;
@@ -526,8 +575,14 @@ public class ChunkGroupOutputStream extends OutputStream {
 
     public ChunkGroupOutputStream build() throws IOException {
       return new ChunkGroupOutputStream(openHandler, xceiverManager, scmClient,
-          omClient, chunkSize, requestID, factor, type);
+          omClient, chunkSize, requestID, factor, type, retryPolicy);
+    }
+
+    public Builder setRetryPolicy(RetryPolicy rPolicy) {
+      this.retryPolicy = rPolicy;
+      return this;
     }
+
   }
 
   private static class ChunkOutputStreamEntry extends OutputStream {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/873ef8ae/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
index fc70514..387f41f 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
@@ -37,6 +38,7 @@ import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.ozone.client.VolumeArgs;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.client.io.ChunkGroupInputStream;
 import org.apache.hadoop.ozone.client.io.ChunkGroupOutputStream;
 import org.apache.hadoop.ozone.client.io.LengthInputStream;
@@ -97,6 +99,7 @@ public class RpcClient implements ClientProtocol {
   private final UserGroupInformation ugi;
   private final OzoneAcl.OzoneACLRights userRights;
   private final OzoneAcl.OzoneACLRights groupRights;
+  private final RetryPolicy retryPolicy;
 
    /**
     * Creates RpcClient instance with the given configuration.
@@ -137,6 +140,7 @@ public class RpcClient implements ClientProtocol {
                 Client.getRpcTimeout(conf)));
 
     this.xceiverClientManager = new XceiverClientManager(conf);
+    retryPolicy = OzoneClientUtils.createRetryPolicy(conf);
 
     int configuredChunkSize = conf.getInt(
         ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY,
@@ -469,6 +473,7 @@ public class RpcClient implements ClientProtocol {
             .setRequestID(requestId)
             .setType(HddsProtos.ReplicationType.valueOf(type.toString()))
             .setFactor(HddsProtos.ReplicationFactor.valueOf(factor.getValue()))
+            .setRetryPolicy(retryPolicy)
             .build();
     groupOutputStream.addPreallocateBlocks(
         openKey.getKeyInfo().getLatestVersionLocations(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/873ef8ae/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
index 50d7ec5..9f12633 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
@@ -22,6 +22,10 @@ import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.HddsDatanodeService;
+import org.apache.hadoop.hdds.scm.container.common.helpers.
+    StorageContainerException;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
@@ -45,6 +49,7 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.event.Level;
 
 import java.io.IOException;
 import java.security.MessageDigest;
@@ -52,6 +57,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.UUID;
+import java.util.Random;
 
 /**
  * Tests Close Container Exception handling by Ozone Client.
@@ -67,6 +73,7 @@ public class TestCloseContainerHandlingByClient {
   private static String volumeName;
   private static String bucketName;
   private static String keyString;
+  private static int maxRetries;
 
   /**
    * Create a MiniDFSCluster for testing.
@@ -78,6 +85,9 @@ public class TestCloseContainerHandlingByClient {
   @BeforeClass
   public static void init() throws Exception {
     conf = new OzoneConfiguration();
+    // generate a no between 1 to 10
+    maxRetries = new Random().nextInt(10);
+    conf.setInt(OzoneConfigKeys.OZONE_CLIENT_MAX_RETRIES, maxRetries);
     chunkSize = (int) OzoneConsts.MB;
     blockSize = 4 * chunkSize;
     conf.setInt(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY, chunkSize);
@@ -286,17 +296,8 @@ public class TestCloseContainerHandlingByClient {
 
     ChunkGroupOutputStream groupOutputStream =
         (ChunkGroupOutputStream) outputStream.getOutputStream();
-    long clientId = groupOutputStream.getOpenID();
-    OMMetadataManager metadataManager =
-        cluster.getOzoneManager().getMetadataManager();
-    byte[] openKey =
-        metadataManager.getOpenKeyBytes(
-            volumeName, bucketName, keyName, clientId);
-    byte[] openKeyData = metadataManager.getOpenKeyTable().get(openKey);
-    OmKeyInfo keyInfo = OmKeyInfo.getFromProtobuf(
-        OzoneManagerProtocolProtos.KeyInfo.parseFrom(openKeyData));
     List<OmKeyLocationInfo> locationInfoList =
-        keyInfo.getKeyLocationVersions().get(0).getBlocksLatestVersionOnly();
+        getLocationInfos(groupOutputStream, keyName);
     List<Long> containerIdList = new ArrayList<>();
     List<Pipeline> pipelineList = new ArrayList<>();
     for (OmKeyLocationInfo info : locationInfoList) {
@@ -318,7 +319,6 @@ public class TestCloseContainerHandlingByClient {
                 new CloseContainerCommand(containerID, type, pipeline.getId()));
       }
     }
-
     int index = 0;
     for (long containerID : containerIdList) {
       Pipeline pipeline = pipelineList.get(index);
@@ -333,7 +333,6 @@ public class TestCloseContainerHandlingByClient {
       }
       index++;
     }
-
   }
 
   private OzoneOutputStream createKey(String keyName, ReplicationType type,
@@ -345,6 +344,20 @@ public class TestCloseContainerHandlingByClient {
         .createKey(keyName, size, type, factor);
   }
 
+  private List<OmKeyLocationInfo> getLocationInfos(
+      ChunkGroupOutputStream groupOutputStream, String keyName)
+      throws IOException {
+    long clientId = groupOutputStream.getOpenID();
+    OMMetadataManager metadataManager =
+        cluster.getOzoneManager().getMetadataManager();
+    byte[] openKey = metadataManager
+        .getOpenKeyBytes(volumeName, bucketName, keyName, clientId);
+    byte[] openKeyData = metadataManager.getOpenKeyTable().get(openKey);
+    OmKeyInfo keyInfo = OmKeyInfo.getFromProtobuf(
+        OzoneManagerProtocolProtos.KeyInfo.parseFrom(openKeyData));
+    return keyInfo.getKeyLocationVersions().get(0).getBlocksLatestVersionOnly();
+  }
+
   private void validateData(String keyName, byte[] data) throws Exception {
     byte[] readData = new byte[data.length];
     OzoneInputStream is =
@@ -399,4 +412,58 @@ public class TestCloseContainerHandlingByClient {
     dataString.concat(dataString);
     validateData(keyName, dataString.getBytes());
   }
+
+  @Test
+  public void testRetriesOnBlockNotCommittedException() throws Exception {
+    String keyName = "blockcommitexceptiontest";
+    OzoneOutputStream key = createKey(keyName, ReplicationType.STAND_ALONE, 0);
+    ChunkGroupOutputStream groupOutputStream =
+        (ChunkGroupOutputStream) key.getOutputStream();
+    GenericTestUtils.setLogLevel(ChunkGroupOutputStream.LOG, Level.TRACE);
+    GenericTestUtils.LogCapturer logCapturer =
+        GenericTestUtils.LogCapturer.captureLogs(ChunkGroupOutputStream.LOG);
+
+    Assert.assertTrue(key.getOutputStream() instanceof ChunkGroupOutputStream);
+    String dataString = fixedLengthString(keyString, (3 * chunkSize));
+    key.write(dataString.getBytes());
+    List<OmKeyLocationInfo> locationInfos =
+        getLocationInfos(groupOutputStream, keyName);
+    long containerID = locationInfos.get(0).getContainerID();
+    List<DatanodeDetails> datanodes =
+        cluster.getStorageContainerManager().getScmContainerManager()
+            .getContainerWithPipeline(containerID).getPipeline().getMachines();
+    Assert.assertEquals(1, datanodes.size());
+    // move the container on the datanode to Closing state, this will ensure
+    // closing the key will hit BLOCK_NOT_COMMITTED_EXCEPTION while trying
+    // to fetch the committed length
+    for (HddsDatanodeService datanodeService : cluster.getHddsDatanodes()) {
+      if (datanodes.get(0).equals(datanodeService.getDatanodeDetails())) {
+        datanodeService.getDatanodeStateMachine().getContainer()
+            .getContainerSet().getContainer(containerID).getContainerData()
+            .setState(ContainerProtos.ContainerLifeCycleState.CLOSING);
+      }
+    }
+    dataString = fixedLengthString(keyString, (chunkSize * 1 / 2));
+    key.write(dataString.getBytes());
+    try {
+      key.close();
+      Assert.fail("Expected Exception not thrown");
+    } catch (IOException ioe) {
+      Assert.assertTrue(ioe instanceof StorageContainerException);
+      Assert.assertTrue(((StorageContainerException) ioe).getResult()
+          == ContainerProtos.Result.BLOCK_NOT_COMMITTED);
+    }
+    // It should retry only for max retries times
+    for (int i = 1; i <= maxRetries; i++) {
+      Assert.assertTrue(logCapturer.getOutput()
+          .contains("Retrying GetCommittedBlockLength request"));
+      Assert.assertTrue(logCapturer.getOutput().contains("Already tried " + i));
+    }
+    Assert.assertTrue(logCapturer.getOutput()
+        .contains("GetCommittedBlockLength request failed."));
+    Assert.assertTrue(logCapturer.getOutput().contains(
+        "retries get failed due to exceeded maximum allowed retries number"
+            + ": " + maxRetries));
+    logCapturer.stopCapturing();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/873ef8ae/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
index ec33990..0d62432 100644
--- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
+++ b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.ozone.web.storage;
 import com.google.common.base.Strings;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.client.io.LengthInputStream;
 import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
@@ -85,6 +87,7 @@ public final class DistributedStorageHandler implements StorageHandler {
   private final boolean useRatis;
   private final HddsProtos.ReplicationType type;
   private final HddsProtos.ReplicationFactor factor;
+  private final RetryPolicy retryPolicy;
 
   /**
    * Creates a new DistributedStorageHandler.
@@ -119,6 +122,7 @@ public final class DistributedStorageHandler implements StorageHandler {
         OMConfigKeys.OZONE_OM_USER_RIGHTS_DEFAULT);
     groupRights = conf.getEnum(OMConfigKeys.OZONE_OM_GROUP_RIGHTS,
         OMConfigKeys.OZONE_OM_GROUP_RIGHTS_DEFAULT);
+    retryPolicy = OzoneClientUtils.createRetryPolicy(conf);
     if(chunkSize > ScmConfigKeys.OZONE_SCM_CHUNK_MAX_SIZE) {
       LOG.warn("The chunk size ({}) is not allowed to be more than"
               + " the maximum size ({}),"
@@ -420,6 +424,7 @@ public final class DistributedStorageHandler implements StorageHandler {
             .setRequestID(args.getRequestID())
             .setType(xceiverClientManager.getType())
             .setFactor(xceiverClientManager.getFactor())
+            .setRetryPolicy(retryPolicy)
             .build();
     groupOutputStream.addPreallocateBlocks(
         openKey.getKeyInfo().getLatestVersionLocations(),


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[49/50] [abbrv] hadoop git commit: HDFS-13310. The DatanodeProtocol should have a DNA_BACKUP to backup blocks. Original patch contributed by Ewan Higgs. Followup work and fixed contributed by Virajith Jalaparthi.

Posted by eh...@apache.org.
HDFS-13310. The DatanodeProtocol should have a DNA_BACKUP to backup blocks. Original patch contributed by Ewan Higgs. Followup work and fixed contributed by Virajith Jalaparthi.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8cdd033a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8cdd033a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8cdd033a

Branch: refs/heads/HDFS-12090
Commit: 8cdd033a06693020c620c8057200e2da5b469ffc
Parents: 211034a
Author: Ewan Higgs <ew...@wdc.com>
Authored: Mon Jul 23 13:14:04 2018 +0200
Committer: Ewan Higgs <ew...@wdc.com>
Committed: Mon Sep 3 14:40:50 2018 +0200

----------------------------------------------------------------------
 .../BlockSyncTaskExecutionFeedback.java         |  67 ++++++
 .../protocol/SyncTaskExecutionOutcome.java      |  25 +++
 .../protocol/SyncTaskExecutionResult.java       |  46 ++++
 .../DatanodeProtocolClientSideTranslatorPB.java |   8 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |   6 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 208 ++++++++++++++++++-
 .../server/blockmanagement/DatanodeManager.java |   4 +-
 .../hdfs/server/datanode/BPServiceActor.java    |   9 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   8 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |   8 +-
 .../hdfs/server/protocol/BlockSyncTask.java     |  83 ++++++++
 .../protocol/BulkSyncTaskExecutionFeedback.java |  36 ++++
 .../hdfs/server/protocol/DatanodeProtocol.java  |  20 +-
 .../hdfs/server/protocol/SyncCommand.java       |  39 ++++
 .../src/main/proto/DatanodeProtocol.proto       |  88 +++++++-
 .../blockmanagement/TestDatanodeManager.java    |   2 +-
 .../TestNameNodePrunesMissingStorages.java      |   2 +-
 .../datanode/InternalDataNodeTestUtils.java     |   3 +-
 .../server/datanode/TestBPOfferService.java     |   5 +-
 .../hdfs/server/datanode/TestBlockRecovery.java |   4 +-
 .../server/datanode/TestDataNodeLifeline.java   |   9 +-
 .../TestDatanodeProtocolRetryPolicy.java        |   4 +-
 .../server/datanode/TestFsDatasetCache.java     |   4 +-
 .../hdfs/server/datanode/TestStorageReport.java |   4 +-
 .../server/namenode/NNThroughputBenchmark.java  |   8 +-
 .../hdfs/server/namenode/NameNodeAdapter.java   |   5 +-
 .../hdfs/server/namenode/TestDeadDatanode.java  |   4 +-
 27 files changed, 658 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockSyncTaskExecutionFeedback.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockSyncTaskExecutionFeedback.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockSyncTaskExecutionFeedback.java
new file mode 100644
index 0000000..2e5393e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockSyncTaskExecutionFeedback.java
@@ -0,0 +1,67 @@
+/*
+ * 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.hdfs.server.protocol;
+
+import java.util.UUID;
+
+/**
+ * Feedback for a BlockSyncTask.
+ */
+public class BlockSyncTaskExecutionFeedback {
+
+  private UUID syncTaskId;
+  private SyncTaskExecutionOutcome outcome;
+  private SyncTaskExecutionResult result;
+  private String syncMountId;
+
+  public BlockSyncTaskExecutionFeedback(UUID syncTaskId,
+      SyncTaskExecutionOutcome outcome, SyncTaskExecutionResult result,
+      String syncMountId) {
+    this.syncTaskId = syncTaskId;
+    this.outcome = outcome;
+    this.result = result;
+    this.syncMountId = syncMountId;
+  }
+
+  public static BlockSyncTaskExecutionFeedback finishedSuccessfully(
+      UUID syncTaskId, String syncMountId, SyncTaskExecutionResult result) {
+    return new BlockSyncTaskExecutionFeedback(syncTaskId,
+        SyncTaskExecutionOutcome.FINISHED_SUCCESSFULLY, result, syncMountId);
+  }
+
+  public static BlockSyncTaskExecutionFeedback failedWithException(
+      UUID syncTaskId, String syncMountId, Exception e) {
+    return new BlockSyncTaskExecutionFeedback(syncTaskId,
+        SyncTaskExecutionOutcome.EXCEPTION, null, syncMountId);
+  }
+
+  public UUID getSyncTaskId() {
+    return syncTaskId;
+  }
+
+  public SyncTaskExecutionOutcome getOutcome() {
+    return outcome;
+  }
+
+  public SyncTaskExecutionResult getResult() {
+    return result;
+  }
+
+  public String getSyncMountId() {
+    return syncMountId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncTaskExecutionOutcome.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncTaskExecutionOutcome.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncTaskExecutionOutcome.java
new file mode 100644
index 0000000..492575b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncTaskExecutionOutcome.java
@@ -0,0 +1,25 @@
+/*
+ * 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.hdfs.server.protocol;
+
+/**
+ * SyncTaskExecutionOutcome is whether the SyncTask was successful or not.
+ */
+public enum SyncTaskExecutionOutcome {
+  FINISHED_SUCCESSFULLY,
+  EXCEPTION
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncTaskExecutionResult.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncTaskExecutionResult.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncTaskExecutionResult.java
new file mode 100644
index 0000000..b623dc5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncTaskExecutionResult.java
@@ -0,0 +1,46 @@
+/*
+ * 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.hdfs.server.protocol;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Result of a SyncTask.
+ */
+public class SyncTaskExecutionResult {
+
+  /** result is the opaque byte stream result of a task. e.g. PartHandle */
+  private ByteBuffer result;
+  private Long numberOfBytes;
+
+  public SyncTaskExecutionResult(ByteBuffer result, Long numberOfBytes) {
+    this.result = result;
+    this.numberOfBytes = numberOfBytes;
+  }
+
+  public static SyncTaskExecutionResult emptyResult() {
+    return new SyncTaskExecutionResult(ByteBuffer.wrap(new byte[0]), 0L);
+  }
+
+  public ByteBuffer getResult() {
+    return result;
+  }
+
+  public Long getNumberOfBytes() {
+    return numberOfBytes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
index e4125dc..20b314c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageBlock
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReceivedDeletedBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionRequestProto;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -138,8 +139,8 @@ public class DatanodeProtocolClientSideTranslatorPB implements
       VolumeFailureSummary volumeFailureSummary,
       boolean requestFullBlockReportLease,
       @Nonnull SlowPeerReports slowPeers,
-      @Nonnull SlowDiskReports slowDisks)
-          throws IOException {
+      @Nonnull SlowDiskReports slowDisks,
+      BulkSyncTaskExecutionFeedback feedback) throws IOException {
     HeartbeatRequestProto.Builder builder = HeartbeatRequestProto.newBuilder()
         .setRegistration(PBHelper.convert(registration))
         .setXmitsInProgress(xmitsInProgress).setXceiverCount(xceiverCount)
@@ -162,6 +163,9 @@ public class DatanodeProtocolClientSideTranslatorPB implements
     if (slowDisks.haveSlowDisks()) {
       builder.addAllSlowDisks(PBHelper.convertSlowDiskInfo(slowDisks));
     }
+    if(feedback != null && !feedback.getFeedbacks().isEmpty()) {
+      builder.setBulkSyncTaskExecutionFeedback(PBHelper.convert(feedback));
+    }
 
     HeartbeatResponseProto resp;
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
index 5cba284..a51ce85 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
@@ -109,6 +109,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
   @Override
   public HeartbeatResponseProto sendHeartbeat(RpcController controller,
       HeartbeatRequestProto request) throws ServiceException {
+
     HeartbeatResponse response;
     try {
       final StorageReport[] report = PBHelperClient.convertStorageReports(
@@ -122,7 +123,10 @@ public class DatanodeProtocolServerSideTranslatorPB implements
           request.getXceiverCount(), request.getFailedVolumes(),
           volumeFailureSummary, request.getRequestFullBlockReportLease(),
           PBHelper.convertSlowPeerInfo(request.getSlowPeersList()),
-          PBHelper.convertSlowDiskInfo(request.getSlowDisksList()));
+          PBHelper.convertSlowDiskInfo(request.getSlowDisksList()),
+          PBHelper.convertBulkSyncTaskExecutionFeedback(
+              request.getBulkSyncTaskExecutionFeedback())
+      );
     } catch (IOException e) {
       throw new ServiceException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index ac01348..d520a40 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hdfs.protocolPB;
 
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -24,9 +27,10 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.UUID;
+import java.util.stream.Collectors;
 
 import com.google.protobuf.ByteString;
-
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.hdfs.DFSUtilClient;
@@ -43,38 +47,44 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommand
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockECReconstructionCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockSyncTaskProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BulkSyncTaskExecutionFeedbackProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.FinalizeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.KeyUpdateCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos
-    .SlowDiskReportProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.SlowDiskReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.SlowPeerReportProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.SyncCommandProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.SyncTaskExecutionFeedbackProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.SyncTaskExecutionOutcomeProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.SyncTaskExecutionResultProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.SyncTaskIdProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
 import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECReconstructionInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ProvidedStorageLocationProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfosProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ProvidedStorageLocationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlockWithLocationsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.BlocksWithLocationsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.CheckpointCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.CheckpointSignatureProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.ExportedBlockKeysProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NNHAStatusHeartbeatProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamenodeRegistrationProto.NamenodeRoleProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NamespaceInfoProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.NNHAStatusHeartbeatProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RecoveringBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RemoteEditLogManifestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.RemoteEditLogProto;
@@ -89,18 +99,23 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
+import org.apache.hadoop.hdfs.server.protocol.BlockSyncTask;
+import org.apache.hadoop.hdfs.server.protocol.BlockSyncTaskExecutionFeedback;
+import org.apache.hadoop.hdfs.server.protocol.SyncTaskExecutionOutcome;
+import org.apache.hadoop.hdfs.server.protocol.SyncTaskExecutionResult;
 import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
-import org.apache.hadoop.hdfs.server.protocol.BlockECReconstructionCommand.BlockECReconstructionInfo;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringStripedBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
@@ -119,7 +134,9 @@ import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import org.apache.hadoop.hdfs.server.protocol.SlowPeerReports;
+import org.apache.hadoop.hdfs.server.protocol.SyncCommand;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
+import org.apache.hadoop.ipc.ClientId;
 
 /**
  * Utilities for converting protobuf classes to and from implementation classes
@@ -469,11 +486,52 @@ public class PBHelper {
       return PBHelper.convert(proto.getBlkIdCmd());
     case BlockECReconstructionCommand:
       return PBHelper.convert(proto.getBlkECReconstructionCmd());
+    case SyncCommand:
+      return PBHelper.convert(proto.getSyncCommand());
     default:
       return null;
     }
   }
-  
+
+  private static SyncCommand convert(SyncCommandProto backupCommand) {
+    List<BlockSyncTaskProto> syncTasksProtoList =
+        backupCommand.getSyncTasksList();
+    List<BlockSyncTask> syncTasksList =
+        new ArrayList(syncTasksProtoList.size());
+    for (BlockSyncTaskProto syncTaskProto : syncTasksProtoList) {
+      syncTasksList.add(convertSyncTask(syncTaskProto));
+    }
+
+    return new SyncCommand(DatanodeProtocol.DNA_BACKUP, syncTasksList);
+  }
+
+  private static BlockSyncTask convertSyncTask(
+      BlockSyncTaskProto syncTaskProto) {
+    SyncTaskIdProto syncTaskIdProto = syncTaskProto.getSyncTaskId();
+    UUID syncTaskId = convert(syncTaskIdProto);
+    try {
+      return new BlockSyncTask(syncTaskId,
+          new URI(syncTaskProto.getUri()),
+          PBHelperClient.convertLocatedBlocks(
+              syncTaskProto.getLocatedBlocksList()),
+          syncTaskProto.getPartNumber(),
+          syncTaskProto.getUploadHandle().toByteArray(),
+          syncTaskProto.getOffset(),
+          syncTaskProto.getLength(),
+          syncTaskIdProto.getSyncMountId());
+    } catch (URISyntaxException e) {
+      throw new IllegalArgumentException();
+    }
+  }
+
+  public static UUID convert(SyncTaskIdProto syncTaskIdProto) {
+    byte[] clientId = syncTaskIdProto.getSyncTaskId().toByteArray();
+    long syncTaskIdMsb = ClientId.getMsb(clientId);
+    long syncTaskIdLsb = ClientId.getLsb(clientId);
+    return new UUID(syncTaskIdMsb, syncTaskIdLsb);
+  }
+
+
   public static BalancerBandwidthCommandProto convert(
       BalancerBandwidthCommand bbCmd) {
     return BalancerBandwidthCommandProto.newBuilder()
@@ -603,6 +661,10 @@ public class PBHelper {
           .setBlkECReconstructionCmd(
               convert((BlockECReconstructionCommand) datanodeCommand));
       break;
+    case DatanodeProtocol.DNA_BACKUP:
+      builder.setCmdType(DatanodeCommandProto.Type.SyncCommand)
+          .setSyncCommand(convert((SyncCommand) datanodeCommand));
+      break;
     case DatanodeProtocol.DNA_UNKNOWN: //Not expected
     default:
       builder.setCmdType(DatanodeCommandProto.Type.NullDatanodeCommand);
@@ -1124,4 +1186,130 @@ public class PBHelper {
 
     return new FileRegion(block, providedStorageLocation);
   }
+
+  private static SyncCommandProto convert(SyncCommand syncCommand) {
+    SyncCommandProto.Builder builder = SyncCommandProto.newBuilder();
+
+    List<BlockSyncTaskProto> syncTaskProtos = syncCommand.getSyncTasks()
+        .stream()
+        .map(syncTask -> convert(syncTask))
+        .collect(Collectors.toList());
+
+    builder.addAllSyncTasks(syncTaskProtos);
+
+    return builder.build();
+  }
+
+  private static BlockSyncTaskProto convert(BlockSyncTask blockSyncTask) {
+    BlockSyncTaskProto.Builder builder = BlockSyncTaskProto.newBuilder();
+    builder.addAllLocatedBlocks(
+        PBHelperClient.convertLocatedBlocks2(blockSyncTask.getLocatedBlocks()));
+    builder.setUploadHandle(
+        ByteString.copyFrom(blockSyncTask.getUploadHandle()));
+    builder.setPartNumber(blockSyncTask.getPartNumber());
+    builder.setUri(blockSyncTask.getRemoteURI().toString());
+    builder.setOffset(blockSyncTask.getOffset());
+    builder.setLength(blockSyncTask.getLength());
+
+    return builder.build();
+  }
+
+  public static SyncTaskIdProto convert(UUID syncTaskId, String syncMountId) {
+    SyncTaskIdProto.Builder builder = SyncTaskIdProto.newBuilder();
+    ByteBuffer syncTaskIdBytes = ByteBuffer.wrap(new byte[16]);
+    syncTaskIdBytes.putLong(syncTaskId.getMostSignificantBits());
+    syncTaskIdBytes.putLong(syncTaskId.getLeastSignificantBits());
+    builder.setSyncTaskId(ByteString.copyFrom(syncTaskIdBytes.array()));
+    builder.setSyncMountId(syncMountId);
+    return builder.build();
+  }
+
+
+  public static BulkSyncTaskExecutionFeedbackProto convert(
+      BulkSyncTaskExecutionFeedback bulkFeedback) {
+    return BulkSyncTaskExecutionFeedbackProto.newBuilder()
+        .addAllFeedbacks(bulkFeedback.getFeedbacks().stream()
+            .map(f -> convert(f)).collect(Collectors.toList()))
+        .build();
+  }
+
+  public static SyncTaskExecutionFeedbackProto convert(
+      BlockSyncTaskExecutionFeedback feedback) {
+    SyncTaskExecutionFeedbackProto.Builder builder =
+        SyncTaskExecutionFeedbackProto.newBuilder()
+            .setSyncTaskId(
+                convert(feedback.getSyncTaskId(), feedback.getSyncMountId()))
+            .setOutcome(convert(feedback.getOutcome()));
+    if (feedback.getResult() != null) {
+      builder.setResult(convert(feedback.getResult()));
+    }
+    return builder.build();
+  }
+
+  public static SyncTaskExecutionOutcomeProto convert(
+      SyncTaskExecutionOutcome outcome) {
+    switch (outcome) {
+    case FINISHED_SUCCESSFULLY:
+      return SyncTaskExecutionOutcomeProto.FINISHED_SUCCESSFULLY;
+    case EXCEPTION:
+      return SyncTaskExecutionOutcomeProto.EXCEPTION;
+    default:
+      throw new IllegalArgumentException(
+          "Unknown SyncTaskExecutionOutcome: " + outcome);
+    }
+  }
+
+  public static SyncTaskExecutionResultProto convert(
+      SyncTaskExecutionResult result) {
+    SyncTaskExecutionResultProto.Builder builder =
+        SyncTaskExecutionResultProto.newBuilder();
+    if (result.getResult() != null) {
+      builder.setResult(ByteString.copyFrom(result.getResult()));
+    }
+    if (result.getNumberOfBytes() != null) {
+      builder.setNumberOfBytes(result.getNumberOfBytes());
+    }
+    return builder.build();
+  }
+
+  public static BulkSyncTaskExecutionFeedback convertBulkSyncTaskExecutionFeedback(
+      BulkSyncTaskExecutionFeedbackProto bulkSyncTaskExecutionFeedback) {
+    return new BulkSyncTaskExecutionFeedback(
+        bulkSyncTaskExecutionFeedback.getFeedbacksList().stream()
+            .map(feedback -> convert(feedback)).collect(Collectors.toList()));
+  }
+
+  public static BlockSyncTaskExecutionFeedback convert(
+      SyncTaskExecutionFeedbackProto feedback) {
+    return new BlockSyncTaskExecutionFeedback(convert(feedback.getSyncTaskId()),
+        convert(feedback.getOutcome()),
+        feedback.hasResult() ? convert(feedback.getResult()) : null,
+        feedback.getSyncTaskId().getSyncMountId());
+  }
+
+  public static SyncTaskExecutionOutcome convert(
+      SyncTaskExecutionOutcomeProto outcome) {
+    switch (outcome) {
+    case FINISHED_SUCCESSFULLY:
+      return SyncTaskExecutionOutcome.FINISHED_SUCCESSFULLY;
+    case EXCEPTION:
+      return SyncTaskExecutionOutcome.EXCEPTION;
+    default:
+      throw new IllegalArgumentException(
+          "Unknown SyncTaskExecutionOutcomeProto: " + outcome);
+    }
+  }
+
+  public static SyncTaskExecutionResult convert(
+      SyncTaskExecutionResultProto result) {
+    byte[] bytes = null;
+    if (result.getResult() != null) {
+      bytes = result.getResult().toByteArray();
+    }
+
+    ByteBuffer byteBuffer =
+        (bytes == null) ? null : ByteBuffer.wrap(bytes).asReadOnlyBuffer();
+    return new SyncTaskExecutionResult(byteBuffer, result.getNumberOfBytes());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 1131506..45d3890 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -1636,7 +1636,9 @@ public class DatanodeManager {
       int maxTransfers, int failedVolumes,
       VolumeFailureSummary volumeFailureSummary,
       @Nonnull SlowPeerReports slowPeers,
-      @Nonnull SlowDiskReports slowDisks) throws IOException {
+      @Nonnull SlowDiskReports slowDisks,
+      BulkSyncTaskExecutionFeedback bulkSyncTaskExecutionFeedback)
+      throws IOException {
     final DatanodeDescriptor nodeinfo;
     try {
       nodeinfo = getDatanode(nodeReg);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
index 8f7a186..870cfda 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -512,6 +513,10 @@ class BPServiceActor implements Runnable {
             SlowDiskReports.create(dn.getDiskMetrics().getDiskOutliersStats()) :
             SlowDiskReports.EMPTY_REPORT;
 
+    // TODO - collect feedback from SyncTasks here.
+    BulkSyncTaskExecutionFeedback bulkSyncTaskExecutionFeedback =
+        new BulkSyncTaskExecutionFeedback(Collections.emptyList());
+
     HeartbeatResponse response = bpNamenode.sendHeartbeat(bpRegistration,
         reports,
         dn.getFSDataset().getCacheCapacity(),
@@ -522,13 +527,13 @@ class BPServiceActor implements Runnable {
         volumeFailureSummary,
         requestBlockReportLease,
         slowPeers,
-        slowDisks);
+        slowDisks,
+        bulkSyncTaskExecutionFeedback);
 
     if (outliersReportDue) {
       // If the report was due and successfully sent, schedule the next one.
       scheduler.scheduleNextOutlierReport();
     }
-
     return response;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 74c9f10..0aad07d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -103,6 +103,7 @@ import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.server.namenode.metrics.ReplicatedBlocksMBean;
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import static org.apache.hadoop.util.Time.now;
 import static org.apache.hadoop.util.Time.monotonicNow;
@@ -3925,8 +3926,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       VolumeFailureSummary volumeFailureSummary,
       boolean requestFullBlockReportLease,
       @Nonnull SlowPeerReports slowPeers,
-      @Nonnull SlowDiskReports slowDisks)
-          throws IOException {
+      @Nonnull SlowDiskReports slowDisks,
+      BulkSyncTaskExecutionFeedback bulkSyncTaskExecutionFeedback)
+      throws IOException {
     readLock();
     try {
       //get datanode commands
@@ -3935,7 +3937,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
           nodeReg, reports, getBlockPoolId(), cacheCapacity, cacheUsed,
           xceiverCount, maxTransfer, failedVolumes, volumeFailureSummary,
-          slowPeers, slowDisks);
+          slowPeers, slowDisks, bulkSyncTaskExecutionFeedback);
       long blockReportLeaseId = 0;
       if (requestFullBlockReportLease) {
         blockReportLeaseId =  blockManager.requestBlockReportLeaseId(nodeReg);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index ec5ce9d..fe71e19 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -158,6 +158,7 @@ import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfyManager;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -1516,14 +1517,15 @@ public class NameNodeRpcServer implements NamenodeProtocols {
       int failedVolumes, VolumeFailureSummary volumeFailureSummary,
       boolean requestFullBlockReportLease,
       @Nonnull SlowPeerReports slowPeers,
-      @Nonnull SlowDiskReports slowDisks)
-          throws IOException {
+      @Nonnull SlowDiskReports slowDisks,
+      BulkSyncTaskExecutionFeedback bulkSyncTaskExecutionFeedback)
+      throws IOException {
     checkNNStartup();
     verifyRequest(nodeReg);
     return namesystem.handleHeartbeat(nodeReg, report,
         dnCacheCapacity, dnCacheUsed, xceiverCount, xmitsInProgress,
         failedVolumes, volumeFailureSummary, requestFullBlockReportLease,
-        slowPeers, slowDisks);
+        slowPeers, slowDisks, bulkSyncTaskExecutionFeedback);
   }
 
   @Override // DatanodeProtocol

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockSyncTask.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockSyncTask.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockSyncTask.java
new file mode 100644
index 0000000..875a409
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockSyncTask.java
@@ -0,0 +1,83 @@
+/*
+ * 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.hdfs.server.protocol;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import java.net.URI;
+import java.util.List;
+import java.util.UUID;
+
+/**
+ * A BlockSyncTask is an operation that is sent to the datanodes to copy
+ * blocks to an external storage endpoint as a part of an orchestrated
+ * synchronization across multiple datanodes.
+ * BlockSyncTask is intended to be an immutable POJO.
+ */
+public class BlockSyncTask {
+  private final UUID syncTaskId;
+  private final URI remoteURI;
+  private final List<LocatedBlock> locatedBlocks;
+  private String syncMountId;
+  private final int partNumber;
+  private byte[] uploadHandle;
+  private final int offset;
+  private final long length;
+
+  public BlockSyncTask(UUID syncTaskId, URI remoteURI,
+      List<LocatedBlock> locatedBlocks, Integer partNumber, byte[] uploadHandle,
+      int offset, long length, String syncMountId) {
+    this.syncTaskId = syncTaskId;
+    this.remoteURI = remoteURI;
+    this.locatedBlocks = locatedBlocks;
+    this.syncMountId = syncMountId;
+    this.partNumber = partNumber;
+    this.uploadHandle = uploadHandle;
+    this.offset = offset;
+    this.length = length;
+  }
+
+  public int getPartNumber() {
+    return partNumber;
+  }
+
+  public byte[] getUploadHandle() {
+    return uploadHandle;
+  }
+
+  public int getOffset() {
+    return offset;
+  }
+
+  public long getLength() {
+    return length;
+  }
+
+  public UUID getSyncTaskId() {
+    return syncTaskId;
+  }
+
+  public URI getRemoteURI() {
+    return remoteURI;
+  }
+
+  public List<LocatedBlock> getLocatedBlocks() {
+    return locatedBlocks;
+  }
+
+  public String getSyncMountId() {
+    return syncMountId;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BulkSyncTaskExecutionFeedback.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BulkSyncTaskExecutionFeedback.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BulkSyncTaskExecutionFeedback.java
new file mode 100644
index 0000000..0d459e8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BulkSyncTaskExecutionFeedback.java
@@ -0,0 +1,36 @@
+/*
+ * 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.hdfs.server.protocol;
+
+import java.util.Collection;
+
+/**
+ * Feedback for a collection of {@link BlockSyncTask}s.
+ */
+public class BulkSyncTaskExecutionFeedback {
+
+  private Collection<BlockSyncTaskExecutionFeedback> feedbacks;
+
+  public BulkSyncTaskExecutionFeedback(
+      Collection<BlockSyncTaskExecutionFeedback> feedbacks) {
+    this.feedbacks = feedbacks;
+  }
+
+  public Collection<BlockSyncTaskExecutionFeedback> getFeedbacks() {
+    return feedbacks;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
index 311b68f..702053a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
@@ -81,6 +81,7 @@ public interface DatanodeProtocol {
   final static int DNA_ERASURE_CODING_RECONSTRUCTION = 11; // erasure coding reconstruction command
   int DNA_BLOCK_STORAGE_MOVEMENT = 12; // block storage movement command
   int DNA_DROP_SPS_WORK_COMMAND = 13; // drop sps work command
+  final static int DNA_BACKUP = 14; // back up data to PROVIDED stores.
 
   /** 
    * Register Datanode.
@@ -112,20 +113,19 @@ public interface DatanodeProtocol {
    * @param slowPeers Details of peer DataNodes that were detected as being
    *                  slow to respond to packet writes. Empty report if no
    *                  slow peers were detected by the DataNode.
+   * @param bulkSyncTaskExecutionFeedback Result of the execution of the
+   *                                      sync tasks.
    * @throws IOException on error
    */
   @Idempotent
   public HeartbeatResponse sendHeartbeat(DatanodeRegistration registration,
-                                       StorageReport[] reports,
-                                       long dnCacheCapacity,
-                                       long dnCacheUsed,
-                                       int xmitsInProgress,
-                                       int xceiverCount,
-                                       int failedVolumes,
-                                       VolumeFailureSummary volumeFailureSummary,
-                                       boolean requestFullBlockReportLease,
-                                       @Nonnull SlowPeerReports slowPeers,
-                                       @Nonnull SlowDiskReports slowDisks)
+      StorageReport[] reports, long cacheCapacity, long cacheUsed,
+      int xmitsInProgress, int xceiverCount, int failedVolumes,
+      VolumeFailureSummary volumeFailureSummary,
+      boolean requestFullBlockReportLease,
+      @Nonnull SlowPeerReports slowPeers,
+      @Nonnull SlowDiskReports slowDisks,
+      BulkSyncTaskExecutionFeedback bulkSyncTaskExecutionFeedback)
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncCommand.java
new file mode 100644
index 0000000..7e2c242
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/SyncCommand.java
@@ -0,0 +1,39 @@
+/**
+ * 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.hdfs.server.protocol;
+
+import java.util.List;
+
+/**
+ * A SyncCommand is an instruction to a DataNode to move the
+ * give file to specified target DataNodes provided storage.
+ */
+public class SyncCommand extends DatanodeCommand {
+
+  private final List<BlockSyncTask> syncTasks;
+
+  public SyncCommand(int action, List<BlockSyncTask> syncTasks) {
+    super(action);
+    this.syncTasks = syncTasks;
+  }
+
+  public List<BlockSyncTask> getSyncTasks() {
+    return syncTasks;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index 4a8f9f0..f4e3481 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -60,6 +60,7 @@ message DatanodeCommandProto {
     NullDatanodeCommand = 7;
     BlockIdCommand = 8;
     BlockECReconstructionCommand = 9;
+    SyncCommand = 10;
   }
 
   required Type cmdType = 1;    // Type of the command
@@ -74,6 +75,7 @@ message DatanodeCommandProto {
   optional RegisterCommandProto registerCmd = 7;
   optional BlockIdCommandProto blkIdCmd = 8;
   optional BlockECReconstructionCommandProto blkECReconstructionCmd = 9;
+  optional SyncCommandProto syncCommand = 10;
 }
 
 /**
@@ -154,6 +156,89 @@ message BlockECReconstructionCommandProto {
   repeated BlockECReconstructionInfoProto blockECReconstructioninfo = 1;
 }
 
+message SyncTaskIdProto {
+  required bytes syncTaskId = 1;
+  required string syncMountId = 2;
+}
+
+/**
+ * Instruct the datanode to perform a backup command
+ */
+message SyncCommandProto {
+  repeated BlockSyncTaskProto syncTasks = 1;
+}
+
+/**
+ * A block synchronization task as part of an orchestrated synchronization
+ * across potentially multiple datanodes (i.e. multipart put part).
+ */
+message BlockSyncTaskProto {
+  required SyncTaskIdProto syncTaskId = 1;
+
+  required bytes uploadHandle = 2;
+  required int32 partNumber = 3;
+  repeated LocatedBlockProto locatedBlocks = 4;
+  required string uri = 5;
+  required int32 offset = 6;
+  required int64 length = 7;
+}
+
+/**
+ * Block storage movement information
+ */
+message BlockMovingInfoProto {
+  required BlockProto block = 1;
+  required DatanodeInfoProto sourceDnInfo = 2;
+  required DatanodeInfoProto targetDnInfo = 3;
+  required StorageTypeProto sourceStorageType = 4;
+  required StorageTypeProto targetStorageType = 5;
+}
+
+/**
+ * Blocks for which storage movements has been attempted and finished
+ * with either success or failure.
+ */
+message BlocksStorageMoveAttemptFinishedProto {
+  repeated BlockProto blocks = 1;
+}
+
+/**
+ * A collection of feedbacks for a collection of sync tasks.
+ */
+message BulkSyncTaskExecutionFeedbackProto {
+  repeated SyncTaskExecutionFeedbackProto feedbacks = 1;
+}
+
+/**
+ * Feedback for a sync task that has been executed.
+ * syncTaskId - identifier for the task.
+ * outcome - success/error.
+ * operation - the type of operation.
+ * result - if the outcome is successful, the results of the sync task.
+ */
+message SyncTaskExecutionFeedbackProto {
+  required SyncTaskIdProto syncTaskId = 1;
+  required SyncTaskExecutionOutcomeProto outcome = 2;
+  optional SyncTaskExecutionResultProto result = 3;
+}
+
+/**
+ * Success of failure indication of a sync task.
+ */
+enum SyncTaskExecutionOutcomeProto {
+  FINISHED_SUCCESSFULLY = 0;
+  EXCEPTION = 1;
+}
+
+/**
+ * result - the opaque result data from the sync task.
+ * numberOfBytes - the number of bytes copied.
+ */
+message SyncTaskExecutionResultProto {
+  optional bytes result = 1;
+  optional int64 numberOfBytes = 2;
+}
+
 /**
  * registration - Information of the datanode registering with the namenode
  */
@@ -211,6 +296,7 @@ message HeartbeatRequestProto {
   optional bool requestFullBlockReportLease = 9 [ default = false ];
   repeated SlowPeerReportProto slowPeers = 10;
   repeated SlowDiskReportProto slowDisks = 11;
+  optional BulkSyncTaskExecutionFeedbackProto bulkSyncTaskExecutionFeedback = 12;
 }
 
 /**
@@ -276,7 +362,7 @@ message StorageBlockReportProto {
  */
 message BlockReportResponseProto {
   optional DatanodeCommandProto cmd = 1;
-} 
+}
 
 /**
  * registration - datanode registration information

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
index dd6f40a..8ba0396 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
@@ -589,7 +589,7 @@ public class TestDatanodeManager {
     Mockito.when(dm.getDatanode(dnReg)).thenReturn(nodeInfo);
     DatanodeCommand[] cmds = dm.handleHeartbeat(
         dnReg, new StorageReport[1], "bp-123", 0, 0, 10, maxTransfers, 0, null,
-        SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT);
+        SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT, null);
 
     long expectedNumCmds = Arrays.stream(
         new int[]{numReplicationTasks, numECTasks})

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
index 05b6d30..e81fb1f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
@@ -116,7 +116,7 @@ public class TestNameNodePrunesMissingStorages {
       cluster.stopDataNode(0);
       cluster.getNameNodeRpc().sendHeartbeat(dnReg, prunedReports, 0L, 0L, 0, 0,
           0, null, true, SlowPeerReports.EMPTY_REPORT,
-          SlowDiskReports.EMPTY_REPORT);
+          SlowDiskReports.EMPTY_REPORT, null);
 
       // Check that the missing storage was pruned.
       assertThat(dnDescriptor.getStorageInfos().length, is(expectedStoragesAfterTest));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
index b453991..2a6975f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/InternalDataNodeTestUtils.java
@@ -167,7 +167,8 @@ public class InternalDataNodeTestUtils {
             Mockito.anyInt(), Mockito.any(VolumeFailureSummary.class),
             Mockito.anyBoolean(),
             Mockito.any(SlowPeerReports.class),
-            Mockito.any(SlowDiskReports.class))).thenReturn(
+            Mockito.any(SlowDiskReports.class),
+            null)).thenReturn(
         new HeartbeatResponse(new DatanodeCommand[0], new NNHAStatusHeartbeat(
             HAServiceState.ACTIVE, 1), null, ThreadLocalRandom.current()
             .nextLong() | 1L));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
index 4863ca1..62b84d9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
+
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
@@ -157,7 +159,8 @@ public class TestBPOfferService {
           Mockito.any(VolumeFailureSummary.class),
           Mockito.anyBoolean(),
           Mockito.any(SlowPeerReports.class),
-          Mockito.any(SlowDiskReports.class));
+          Mockito.any(SlowDiskReports.class),
+          Mockito.any(BulkSyncTaskExecutionFeedback.class));
     mockHaStatuses[nnIdx] = new NNHAStatusHeartbeat(HAServiceState.STANDBY, 0);
     datanodeCommands[nnIdx] = new DatanodeCommand[0];
     return mock;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
index 07fd4ae..09fae14 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.datanode;
 import org.apache.hadoop.hdfs.AppendTestUtil;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
@@ -232,7 +233,8 @@ public class TestBlockRecovery {
             Mockito.any(VolumeFailureSummary.class),
             Mockito.anyBoolean(),
             Mockito.any(SlowPeerReports.class),
-            Mockito.any(SlowDiskReports.class)))
+            Mockito.any(SlowDiskReports.class),
+            Mockito.any(BulkSyncTaskExecutionFeedback.class)))
         .thenReturn(new HeartbeatResponse(
             new DatanodeCommand[0],
             new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 1),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java
index 28427bc..6374540 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeLifeline.java
@@ -24,13 +24,14 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHEC
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIFELINE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY;
 
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.any;
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.anyBoolean;
 import static org.mockito.Mockito.anyInt;
 import static org.mockito.Mockito.anyLong;
@@ -172,7 +173,8 @@ public class TestDataNodeLifeline {
             any(VolumeFailureSummary.class),
             anyBoolean(),
             any(SlowPeerReports.class),
-            any(SlowDiskReports.class));
+            any(SlowDiskReports.class),
+            any(BulkSyncTaskExecutionFeedback.class));
 
     // Intercept lifeline to trigger latch count-down on each call.
     doAnswer(new LatchCountingAnswer<Void>(lifelinesSent))
@@ -237,7 +239,8 @@ public class TestDataNodeLifeline {
             any(VolumeFailureSummary.class),
             anyBoolean(),
             any(SlowPeerReports.class),
-            any(SlowDiskReports.class));
+            any(SlowDiskReports.class),
+            any(BulkSyncTaskExecutionFeedback.class));
 
     // While waiting on the latch for the expected number of heartbeat messages,
     // poll DataNode tracking information.  We expect that the DataNode always

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
index bb1d9ef..2d7dea9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hdfs.server.datanode;
 
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -222,7 +223,8 @@ public class TestDatanodeProtocolRetryPolicy {
            Mockito.any(VolumeFailureSummary.class),
            Mockito.anyBoolean(),
            Mockito.any(SlowPeerReports.class),
-           Mockito.any(SlowDiskReports.class));
+           Mockito.any(SlowDiskReports.class),
+           Mockito.any(BulkSyncTaskExecutionFeedback.class));
 
     dn = new DataNode(conf, locations, null, null) {
       @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
index 2dbd5b9..2cf0135 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import net.jcip.annotations.NotThreadSafe;
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
 import static org.junit.Assert.assertEquals;
@@ -208,7 +209,8 @@ public class TestFsDatasetCache {
           (StorageReport[]) any(), anyLong(), anyLong(),
           anyInt(), anyInt(), anyInt(), (VolumeFailureSummary) any(),
           anyBoolean(), any(SlowPeerReports.class),
-          any(SlowDiskReports.class));
+          any(SlowDiskReports.class),
+          any(BulkSyncTaskExecutionFeedback.class));
     } finally {
       lock.writeLock().unlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
index 5f62ddb..0f0bc1d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
@@ -110,7 +111,8 @@ public class TestStorageReport {
         anyLong(), anyLong(), anyInt(), anyInt(), anyInt(),
         Mockito.any(VolumeFailureSummary.class), Mockito.anyBoolean(),
         Mockito.any(SlowPeerReports.class),
-        Mockito.any(SlowDiskReports.class));
+        Mockito.any(SlowDiskReports.class),
+        Mockito.any(BulkSyncTaskExecutionFeedback.class));
 
     StorageReport[] reports = captor.getValue();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
index 3a3c471..9940174 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
@@ -956,8 +956,8 @@ public class NNThroughputBenchmark implements Tool {
           DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED, 0L) };
       DatanodeCommand[] cmds = dataNodeProto.sendHeartbeat(dnRegistration, rep,
           0L, 0L, 0, 0, 0, null, true,
-          SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT)
-          .getCommands();
+          SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT,
+          null).getCommands();
       if(cmds != null) {
         for (DatanodeCommand cmd : cmds ) {
           if(LOG.isDebugEnabled()) {
@@ -1007,8 +1007,8 @@ public class NNThroughputBenchmark implements Tool {
           false, DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED, 0) };
       DatanodeCommand[] cmds = dataNodeProto.sendHeartbeat(dnRegistration,
           rep, 0L, 0L, 0, 0, 0, null, true,
-          SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT)
-          .getCommands();
+          SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT,
+          null).getCommands();
       if (cmds != null) {
         for (DatanodeCommand cmd : cmds) {
           if (cmd.getAction() == DatanodeProtocol.DNA_TRANSFER) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
index b85527a..8589f75 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
@@ -17,11 +17,13 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import static org.mockito.Mockito.spy;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.commons.lang3.reflect.FieldUtils;
@@ -130,7 +132,8 @@ public class NameNodeAdapter {
     return namesystem.handleHeartbeat(nodeReg,
         BlockManagerTestUtil.getStorageReportsForDatanode(dd),
         dd.getCacheCapacity(), dd.getCacheRemaining(), 0, 0, 0, null, true,
-        SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT);
+        SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT,
+        new BulkSyncTaskExecutionFeedback(new ArrayList<>()));
   }
 
   public static boolean setReplication(final FSNamesystem ns,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8cdd033a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
index df74107..9f9a897 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
@@ -139,8 +139,8 @@ public class TestDeadDatanode {
         false, 0, 0, 0, 0, 0) };
     DatanodeCommand[] cmd =
         dnp.sendHeartbeat(reg, rep, 0L, 0L, 0, 0, 0, null, true,
-            SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT)
-        .getCommands();
+            SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT,
+            null).getCommands();
     assertEquals(1, cmd.length);
     assertEquals(cmd[0].getAction(), RegisterCommand.REGISTER
         .getAction());


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[10/50] [abbrv] hadoop git commit: HDDS-332. Remove the ability to configure ozone.handler.type Contributed by Nandakumar and Anu Engineer.

Posted by eh...@apache.org.
HDDS-332. Remove the ability to configure ozone.handler.type
Contributed by Nandakumar and Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/df21e1b1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/df21e1b1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/df21e1b1

Branch: refs/heads/HDFS-12090
Commit: df21e1b1ddcc8439b5fa1bb79388403f87742e65
Parents: 2172399
Author: Anu Engineer <ae...@apache.org>
Authored: Tue Aug 28 09:56:02 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Tue Aug 28 09:56:02 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/ozone/OzoneConfigKeys.java    |    7 -
 .../org/apache/hadoop/ozone/OzoneConsts.java    |    1 -
 .../common/src/main/resources/ozone-default.xml |   21 -
 .../apache/hadoop/ozone/RatisTestHelper.java    |    8 +-
 .../ozone/client/rest/TestOzoneRestClient.java  |    7 +-
 .../rpc/TestCloseContainerHandlingByClient.java |    2 -
 .../ozone/client/rpc/TestOzoneRpcClient.java    |    9 +-
 .../ozone/container/ContainerTestHelper.java    |   10 -
 .../TestContainerDeletionChoosingPolicy.java    |    8 +-
 .../common/impl/TestContainerPersistence.java   |  116 +-
 .../commandhandler/TestBlockDeletion.java       |    8 +-
 .../TestCloseContainerByPipeline.java           |   35 +-
 .../container/ozoneimpl/TestOzoneContainer.java |    2 -
 .../ozoneimpl/TestOzoneContainerRatis.java      |    2 -
 .../container/ozoneimpl/TestRatisManager.java   |    2 -
 .../hadoop/ozone/freon/TestDataValidate.java    |    7 +-
 .../apache/hadoop/ozone/freon/TestFreon.java    |    3 +-
 .../ozone/om/TestContainerReportWithKeys.java   |   12 +-
 .../om/TestMultipleContainerReadWrite.java      |    5 +-
 .../hadoop/ozone/om/TestOmBlockVersioning.java  |    7 +-
 .../apache/hadoop/ozone/om/TestOmMetrics.java   |    7 +-
 .../apache/hadoop/ozone/om/TestOmSQLCli.java    |    6 +-
 .../hadoop/ozone/om/TestOzoneManager.java       |    5 +-
 .../hadoop/ozone/ozShell/TestOzoneShell.java    |   20 +-
 .../ozone/web/TestDistributedOzoneVolumes.java  |  188 ---
 .../hadoop/ozone/web/TestLocalOzoneVolumes.java |  187 ---
 .../hadoop/ozone/web/TestOzoneVolumes.java      |  183 +++
 .../hadoop/ozone/web/TestOzoneWebAccess.java    |   10 +-
 .../hadoop/ozone/web/client/TestBuckets.java    |    9 +-
 .../hadoop/ozone/web/client/TestKeysRatis.java  |    4 +-
 .../ozone/web/client/TestOzoneClient.java       |    3 -
 .../hadoop/ozone/web/client/TestVolume.java     |   11 +-
 .../ozone/web/client/TestVolumeRatis.java       |    3 -
 .../server/datanode/ObjectStoreHandler.java     |  182 ++-
 .../web/handlers/StorageHandlerBuilder.java     |   18 +-
 .../web/localstorage/LocalStorageHandler.java   |  385 ------
 .../web/localstorage/OzoneMetadataManager.java  | 1138 ------------------
 .../hadoop/fs/ozone/TestOzoneFSInputStream.java |    6 +-
 38 files changed, 363 insertions(+), 2274 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
index 92f0c41..6ad9085 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java
@@ -66,16 +66,9 @@ public final class OzoneConfigKeys {
       "dfs.container.ratis.ipc.random.port";
   public static final boolean DFS_CONTAINER_RATIS_IPC_RANDOM_PORT_DEFAULT =
       false;
-
-  public static final String OZONE_LOCALSTORAGE_ROOT =
-      "ozone.localstorage.root";
-  public static final String OZONE_LOCALSTORAGE_ROOT_DEFAULT = "/tmp/ozone";
   public static final String OZONE_ENABLED =
       "ozone.enabled";
   public static final boolean OZONE_ENABLED_DEFAULT = false;
-  public static final String OZONE_HANDLER_TYPE_KEY =
-      "ozone.handler.type";
-  public static final String OZONE_HANDLER_TYPE_DEFAULT = "distributed";
   public static final String OZONE_TRACE_ENABLED_KEY =
       "ozone.trace.enabled";
   public static final boolean OZONE_TRACE_ENABLED_DEFAULT = false;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index 320a3ed..ab6df92 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -114,7 +114,6 @@ public final class OzoneConsts {
    * Ozone handler types.
    */
   public static final String OZONE_HANDLER_DISTRIBUTED = "distributed";
-  public static final String OZONE_HANDLER_LOCAL = "local";
 
   public static final String DELETING_KEY_PREFIX = "#deleting#";
   public static final String DELETED_KEY_PREFIX = "#deleted#";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-hdds/common/src/main/resources/ozone-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index f2544d9..c3e3095 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -278,17 +278,6 @@
     </description>
   </property>
   <property>
-    <name>ozone.handler.type</name>
-    <value>distributed</value>
-    <tag>OZONE, REST</tag>
-    <description>
-      Tells ozone which storage handler to use. The possible values are:
-      distributed - The Ozone distributed storage handler, which speaks to
-      OM/SCM on the backend and provides REST services to clients.
-      local - Local Storage handler strictly for testing - To be removed.
-    </description>
-  </property>
-  <property>
     <name>ozone.key.deleting.limit.per.task</name>
     <value>1000</value>
     <tag>OM, PERFORMANCE</tag>
@@ -417,16 +406,6 @@
     </description>
   </property>
   <property>
-    <name>ozone.localstorage.root</name>
-    <value>${hadoop.tmp.dir}/ozone</value>
-    <tag>OZONE, DEBUG</tag>
-    <description>
-      This is used only for testing purposes. This value is used by the local
-      storage handler to simulate a REST backend. This is useful only when
-      debugging the REST front end independent of OM and SCM. To be removed.
-    </description>
-  </property>
-  <property>
     <name>ozone.metadata.dirs</name>
     <value/>
     <tag>OZONE, OM, SCM, CONTAINER, REQUIRED, STORAGE</tag>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
index 7a9a09a..79d1bef 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
@@ -50,10 +50,9 @@ public interface RatisTestHelper {
     private final MiniOzoneCluster cluster;
 
     /**
-     * Create a {@link MiniOzoneCluster} for testing by setting
-     *   OZONE_ENABLED = true,
-     *   RATIS_ENABLED = true, and
-     *   OZONE_HANDLER_TYPE_KEY = "distributed".
+     * Create a {@link MiniOzoneCluster} for testing by setting.
+     *   OZONE_ENABLED = true
+     *   RATIS_ENABLED = true
      */
     public RatisTestSuite(final Class<?> clazz)
         throws IOException, TimeoutException, InterruptedException {
@@ -88,7 +87,6 @@ public interface RatisTestHelper {
   static OzoneConfiguration newOzoneConfiguration(
       Class<?> clazz, RpcType rpc) {
     final OzoneConfiguration conf = new OzoneConfiguration();
-    ContainerTestHelper.setOzoneLocalStorageRoot(clazz, conf);
     initRatisConf(rpc, conf);
     return conf;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java
index 0dc0399..73bb32d 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java
@@ -22,8 +22,6 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.BucketArgs;
 import org.apache.hadoop.ozone.client.ObjectStore;
 import org.apache.hadoop.ozone.client.OzoneBucket;
@@ -65,16 +63,13 @@ public class TestOzoneRestClient {
   /**
    * Create a MiniDFSCluster for testing.
    * <p>
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "distributed"
+   * Ozone is made active by setting OZONE_ENABLED = true
    *
    * @throws IOException
    */
   @BeforeClass
   public static void init() throws Exception {
     OzoneConfiguration conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
-        OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
     cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
     cluster.waitForClusterToBeReady();
     InetSocketAddress omHttpAddress = cluster.getOzoneManager()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
index e5ecd81..43bd23e 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
@@ -81,8 +81,6 @@ public class TestCloseContainerHandlingByClient {
   @BeforeClass
   public static void init() throws Exception {
     conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
-        OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
     chunkSize = (int)OzoneConsts.MB;
     blockSize = 4 * chunkSize;
     conf.setInt(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY, chunkSize);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
index e31b528..45b3843 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
@@ -24,9 +24,7 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.BucketArgs;
 import org.apache.hadoop.ozone.client.ObjectStore;
 import org.apache.hadoop.ozone.client.OzoneBucket;
@@ -81,16 +79,13 @@ public class TestOzoneRpcClient {
   /**
    * Create a MiniOzoneCluster for testing.
    * <p>
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "distributed"
+   * Ozone is made active by setting OZONE_ENABLED = true
    *
    * @throws IOException
    */
   @BeforeClass
   public static void init() throws Exception {
     OzoneConfiguration conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
-        OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
     conf.setInt(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE, 1);
     cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(10).build();
     cluster.waitForClusterToBeReady();
@@ -439,7 +434,7 @@ public class TestOzoneRpcClient {
     String volumeName = UUID.randomUUID().toString();
     String bucketName = UUID.randomUUID().toString();
 
-    String value = RandomStringUtils.random(RandomUtils.nextInt(0,1024));
+    String value = RandomStringUtils.random(RandomUtils.nextInt(0, 1024));
     store.createVolume(volumeName);
     OzoneVolume volume = store.getVolume(volumeName);
     volume.createBucket(bucketName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
index dc166b5..0eb1ad4 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
@@ -73,15 +72,6 @@ public final class ContainerTestHelper {
   private ContainerTestHelper() {
   }
 
-  public static void setOzoneLocalStorageRoot(
-      Class<?> clazz, OzoneConfiguration conf) {
-    String path = GenericTestUtils.getTempPath(clazz.getSimpleName());
-    path += conf.getTrimmed(
-        OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
-        OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
-    conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
-  }
-
   // TODO: mock multi-node pipeline
   /**
    * Create a pipeline with single node replica.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDeletionChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDeletionChoosingPolicy.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDeletionChoosingPolicy.java
index 0d46ecf..6478fc6 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDeletionChoosingPolicy.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerDeletionChoosingPolicy.java
@@ -28,7 +28,6 @@ import java.util.Random;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
@@ -45,17 +44,14 @@ import org.junit.Test;
  */
 public class TestContainerDeletionChoosingPolicy {
   private static String path;
-  private static ContainerSet containerSet;
-  private static OzoneConfiguration conf;
+  private  ContainerSet containerSet;
+  private OzoneConfiguration conf;
 
   @Before
   public void init() throws Throwable {
     conf = new OzoneConfiguration();
     path = GenericTestUtils
         .getTempPath(TestContainerDeletionChoosingPolicy.class.getSimpleName());
-    path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
-        OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
-    conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
index 8e11a97..e49e283 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
@@ -21,21 +21,20 @@ import com.google.common.collect.Maps;
 import org.apache.commons.codec.binary.Hex;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
+import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.helpers.KeyData;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
-import org.apache.hadoop.ozone.container.common.volume
-    .RoundRobinVolumeChoosingPolicy;
+import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
@@ -45,8 +44,6 @@ import org.apache.hadoop.ozone.container.keyvalue.impl.KeyManagerImpl;
 import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
 import org.apache.hadoop.ozone.container.keyvalue.interfaces.KeyManager;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
-import org.apache.hadoop.ozone.container.common.helpers.KeyData;
 import org.apache.hadoop.utils.MetadataStore;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -68,113 +65,80 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.ArrayList;
 import java.util.UUID;
 
+import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Stage.COMBINED;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_ROOT_PREFIX;
 import static org.apache.hadoop.ozone.container.ContainerTestHelper.getChunk;
 import static org.apache.hadoop.ozone.container.ContainerTestHelper.getData;
-import static org.apache.hadoop.ozone.container.ContainerTestHelper
-    .setDataChecksum;
-import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .Stage.COMBINED;
+import static org.apache.hadoop.ozone.container.ContainerTestHelper.setDataChecksum;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 /**
- * Simple tests to verify that container persistence works as expected.
- * Some of these tests are specific to {@link KeyValueContainer}. If a new
- * {@link ContainerProtos.ContainerType} is added, the tests need to be
- * modified.
+ * Simple tests to verify that container persistence works as expected. Some of
+ * these tests are specific to {@link KeyValueContainer}. If a new {@link
+ * ContainerProtos.ContainerType} is added, the tests need to be modified.
  */
 public class TestContainerPersistence {
-  @Rule
-  public ExpectedException exception = ExpectedException.none();
-
-  /**
-   * Set the timeout for every test.
-   */
-  @Rule
-  public Timeout testTimeout = new Timeout(300000);
-
+  private static final String DATANODE_UUID = UUID.randomUUID().toString();
+  private static final String SCM_ID = UUID.randomUUID().toString();
   private static Logger log =
       LoggerFactory.getLogger(TestContainerPersistence.class);
   private static String hddsPath;
-  private static String path;
   private static OzoneConfiguration conf;
-  private static List<StorageLocation> pathLists = new LinkedList<>();
-  private Long  containerID = 8888L;;
-  private static final String datanodeUuid = UUID.randomUUID().toString();
-  private static final String scmId = UUID.randomUUID().toString();
-
   private static ContainerSet containerSet;
   private static VolumeSet volumeSet;
   private static VolumeChoosingPolicy volumeChoosingPolicy;
   private static KeyManager keyManager;
   private static ChunkManager chunkManager;
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+  /**
+   * Set the timeout for every test.
+   */
+  @Rule
+  public Timeout testTimeout = new Timeout(300000);
+  private Long containerID = 8888L;
 
   @BeforeClass
   public static void init() throws Throwable {
     conf = new OzoneConfiguration();
     hddsPath = GenericTestUtils
         .getTempPath(TestContainerPersistence.class.getSimpleName());
-    path = hddsPath + conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
-        OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
-    conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
     conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, hddsPath);
-
-    File containerDir = new File(path);
-    if (containerDir.exists()) {
-      FileUtils.deleteDirectory(new File(path));
-    }
-    Assert.assertTrue(containerDir.mkdirs());
     volumeChoosingPolicy = new RoundRobinVolumeChoosingPolicy();
   }
 
   @AfterClass
   public static void shutdown() throws IOException {
-    FileUtils.deleteDirectory(new File(path));
     FileUtils.deleteDirectory(new File(hddsPath));
   }
 
   @Before
   public void setupPaths() throws IOException {
-    if (!new File(path).exists() && !new File(path).mkdirs()) {
-      throw new IOException("Unable to create paths. " + path);
-    }
-    StorageLocation loc = StorageLocation.parse(
-        Paths.get(path).resolve(CONTAINER_ROOT_PREFIX).toString());
-
-    pathLists.clear();
     containerSet = new ContainerSet();
-    volumeSet = new VolumeSet(datanodeUuid, conf);
+    volumeSet = new VolumeSet(DATANODE_UUID, conf);
     keyManager = new KeyManagerImpl(conf);
     chunkManager = new ChunkManagerImpl();
 
-    if (!new File(loc.getNormalizedUri()).mkdirs()) {
-      throw new IOException("unable to create paths. " +
-          loc.getNormalizedUri());
-    }
-    pathLists.add(loc);
-
     for (String dir : conf.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
       StorageLocation location = StorageLocation.parse(dir);
       FileUtils.forceMkdir(new File(location.getNormalizedUri()));
     }
- }
+  }
 
   @After
   public void cleanupDir() throws IOException {
     // Clean up SCM metadata
-    log.info("Deleting {}", path);
-    FileUtils.deleteDirectory(new File(path));
     log.info("Deleting {}", hddsPath);
     FileUtils.deleteDirectory(new File(hddsPath));
 
@@ -196,9 +160,9 @@ public class TestContainerPersistence {
     data.addMetadata("VOLUME", "shire");
     data.addMetadata("owner)", "bilbo");
     KeyValueContainer container = new KeyValueContainer(data, conf);
-    container.create(volumeSet, volumeChoosingPolicy, scmId);
+    container.create(volumeSet, volumeChoosingPolicy, SCM_ID);
     containerSet.addContainer(container);
-    return  container;
+    return container;
   }
 
   @Test
@@ -209,7 +173,7 @@ public class TestContainerPersistence {
         .containsKey(testContainerID));
     KeyValueContainerData kvData =
         (KeyValueContainerData) containerSet.getContainer(testContainerID)
-        .getContainerData();
+            .getContainerData();
 
     Assert.assertNotNull(kvData);
     Assert.assertTrue(new File(kvData.getMetadataPath()).exists());
@@ -287,7 +251,7 @@ public class TestContainerPersistence {
   }
 
   @Test
-  public void testGetContainerReports() throws Exception{
+  public void testGetContainerReports() throws Exception {
     final int count = 10;
     List<Long> containerIDs = new ArrayList<>();
 
@@ -296,7 +260,7 @@ public class TestContainerPersistence {
       Container container = addContainer(containerSet, testContainerID);
 
       // Close a bunch of containers.
-      if (i%3 == 0) {
+      if (i % 3 == 0) {
         container.close();
       }
       containerIDs.add(testContainerID);
@@ -307,7 +271,8 @@ public class TestContainerPersistence {
     List<StorageContainerDatanodeProtocolProtos.ContainerInfo> reports =
         containerSet.getContainerReport().getReportsList();
     Assert.assertEquals(10, reports.size());
-    for(StorageContainerDatanodeProtocolProtos.ContainerInfo   report : reports) {
+    for (StorageContainerDatanodeProtocolProtos.ContainerInfo report :
+        reports) {
       long actualContainerID = report.getContainerID();
       Assert.assertTrue(containerIDs.remove(actualContainerID));
     }
@@ -315,8 +280,8 @@ public class TestContainerPersistence {
   }
 
   /**
-   * This test creates 50 containers and reads them back 5 containers at a
-   * time and verifies that we did get back all containers.
+   * This test creates 50 containers and reads them back 5 containers at a time
+   * and verifies that we did get back all containers.
    *
    * @throws IOException
    */
@@ -426,7 +391,7 @@ public class TestContainerPersistence {
         sha.update(FileUtils.readFileToByteArray(fname.toFile()));
         String val = Hex.encodeHexString(sha.digest());
         Assert.assertEquals(fileHashMap.get(fname.getFileName().toString())
-                .getChecksum(), val);
+            .getChecksum(), val);
         count++;
         sha.reset();
       }
@@ -454,8 +419,8 @@ public class TestContainerPersistence {
   @Test
   public void testPartialRead() throws Exception {
     final int datalen = 1024;
-    final int start = datalen/4;
-    final int length = datalen/2;
+    final int start = datalen / 4;
+    final int length = datalen / 2;
 
     long testContainerID = getTestContainerID();
     Container container = addContainer(containerSet, testContainerID);
@@ -544,7 +509,8 @@ public class TestContainerPersistence {
     }
 
     // Request to read the whole data in a single go.
-    ChunkInfo largeChunk = getChunk(blockID.getLocalID(), 0, 0, datalen * chunkCount);
+    ChunkInfo largeChunk = getChunk(blockID.getLocalID(), 0, 0,
+        datalen * chunkCount);
     byte[] newdata = chunkManager.readChunk(container, blockID, largeChunk);
     MessageDigest newSha = MessageDigest.getInstance(OzoneConsts.FILE_HASH);
     newSha.update(newdata);
@@ -701,8 +667,8 @@ public class TestContainerPersistence {
   }
 
   /**
-   * Tries to update an existing and non-existing container.
-   * Verifies container map and persistent data both updated.
+   * Tries to update an existing and non-existing container. Verifies container
+   * map and persistent data both updated.
    *
    * @throws IOException
    */
@@ -743,7 +709,7 @@ public class TestContainerPersistence {
         orgContainerFile.getAbsolutePath(),
         newContainerFile.getAbsolutePath());
 
-    ContainerData actualContainerData =  ContainerDataYaml.readContainerFile(
+    ContainerData actualContainerData = ContainerDataYaml.readContainerFile(
         newContainerFile);
     Assert.assertEquals("shire_new",
         actualContainerData.getMetadata().get("VOLUME"));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
index 08bc556..3c77687 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.hdds.scm.block.SCMBlockDeletingService;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.OzoneTestUtils;
 import org.apache.hadoop.ozone.client.ObjectStore;
@@ -88,16 +87,11 @@ public class TestBlockDeletion {
     File baseDir = new File(path);
     baseDir.mkdirs();
 
-    path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
-        OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
-
-    conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
-    conf.setQuietMode(false);
     conf.setTimeDuration(OZONE_BLOCK_DELETING_SERVICE_INTERVAL, 100,
         TimeUnit.MILLISECONDS);
     conf.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL, 200,
         TimeUnit.MILLISECONDS);
-
+    conf.setQuietMode(false);
     cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(1)
         .setHbInterval(200)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java
index f40e425..ed9c54d 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java
@@ -1,19 +1,18 @@
 /**
- * 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
+ * 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.
+ * 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.statemachine.commandhandler;
@@ -26,8 +25,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.ozone.HddsDatanodeService;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.ObjectStore;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientFactory;
@@ -54,20 +51,16 @@ public class TestCloseContainerByPipeline {
   private static OzoneClient client;
   private static ObjectStore objectStore;
 
-
   /**
    * Create a MiniDFSCluster for testing.
    * <p>
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "distributed"
+   * Ozone is made active by setting OZONE_ENABLED = true
    *
    * @throws IOException
    */
   @BeforeClass
   public static void init() throws Exception {
     conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
-        OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
     cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(3).build();
     cluster.waitForClusterToBeReady();
@@ -243,7 +236,8 @@ public class TestCloseContainerByPipeline {
           () -> isContainerClosed(cluster, containerID, datanodeDetails), 500,
           15 * 1000);
       //double check if it's really closed (waitFor also throws an exception)
-      Assert.assertTrue(isContainerClosed(cluster, containerID, datanodeDetails));
+      Assert.assertTrue(isContainerClosed(cluster,
+          containerID, datanodeDetails));
     }
     Assert.assertFalse(logCapturer.getOutput().contains(
         "submitting CloseContainer request over STAND_ALONE "
@@ -257,13 +251,14 @@ public class TestCloseContainerByPipeline {
   private Boolean isContainerClosed(MiniOzoneCluster cluster, long containerID,
       DatanodeDetails datanode) {
     ContainerData containerData;
-    for (HddsDatanodeService datanodeService : cluster.getHddsDatanodes())
+    for (HddsDatanodeService datanodeService : cluster.getHddsDatanodes()) {
       if (datanode.equals(datanodeService.getDatanodeDetails())) {
         containerData =
             datanodeService.getDatanodeStateMachine().getContainer()
                 .getContainerSet().getContainer(containerID).getContainerData();
         return containerData.isClosed();
       }
+    }
     return false;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
index 45fd4b7..f112d26 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
@@ -91,8 +91,6 @@ public class TestOzoneContainer {
 
   static OzoneConfiguration newOzoneConfiguration() {
     final OzoneConfiguration conf = new OzoneConfiguration();
-    ContainerTestHelper.setOzoneLocalStorageRoot(
-        TestOzoneContainer.class, conf);
     return conf;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerRatis.java
index c686b0b..cf469ec 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerRatis.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerRatis.java
@@ -54,8 +54,6 @@ public class TestOzoneContainerRatis {
 
   static OzoneConfiguration newOzoneConfiguration() {
     final OzoneConfiguration conf = new OzoneConfiguration();
-    ContainerTestHelper.setOzoneLocalStorageRoot(
-        TestOzoneContainerRatis.class, conf);
     return conf;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestRatisManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestRatisManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestRatisManager.java
index 12c2b7b..a0c8cfe 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestRatisManager.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestRatisManager.java
@@ -47,8 +47,6 @@ public class TestRatisManager {
 
   static OzoneConfiguration newOzoneConfiguration() {
     final OzoneConfiguration conf = new OzoneConfiguration();
-    ContainerTestHelper.setOzoneLocalStorageRoot(
-        TestRatisManager.class, conf);
     return conf;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDataValidate.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDataValidate.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDataValidate.java
index 8b8072c..0a3c391 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDataValidate.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDataValidate.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.ozone.freon;
 
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -45,16 +43,13 @@ public class TestDataValidate {
   /**
    * Create a MiniDFSCluster for testing.
    * <p>
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "distributed"
+   * Ozone is made active by setting OZONE_ENABLED = true
    *
    * @throws IOException
    */
   @BeforeClass
   public static void init() throws Exception {
     conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
-        OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
     cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(5).build();
     cluster.waitForClusterToBeReady();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreon.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreon.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreon.java
index 203cf4d..022d6b5 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreon.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestFreon.java
@@ -41,8 +41,7 @@ public class TestFreon {
   /**
    * Create a MiniDFSCluster for testing.
    * <p>
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "distributed"
+   * Ozone is made active by setting OZONE_ENABLED = true
    *
    * @throws IOException
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestContainerReportWithKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestContainerReportWithKeys.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestContainerReportWithKeys.java
index c66b3de..c24cfbf 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestContainerReportWithKeys.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestContainerReportWithKeys.java
@@ -24,8 +24,6 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.*;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.container.common.impl.ContainerData;
@@ -33,7 +31,6 @@ import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
-import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -60,16 +57,13 @@ public class TestContainerReportWithKeys {
   /**
    * Create a MiniDFSCluster for testing.
    * <p>
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "distributed"
+   * Ozone is made active by setting OZONE_ENABLED = true
    *
    * @throws IOException
    */
   @BeforeClass
   public static void init() throws Exception {
     conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
-        OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
     scm = cluster.getStorageContainerManager();
@@ -117,10 +111,6 @@ public class TestContainerReportWithKeys {
         cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
             .get(0).getBlocksLatestVersionOnly().get(0);
 
-    ContainerData cd = getContainerData(keyInfo.getContainerID());
-
-/*    LOG.info("DN Container Data:  keyCount: {} used: {} ",
-        cd.getKeyCount(), cd.getBytesUsed());*/
 
     ContainerInfo cinfo = scm.getContainerInfo(keyInfo.getContainerID());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestMultipleContainerReadWrite.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestMultipleContainerReadWrite.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestMultipleContainerReadWrite.java
index 1389cba..f1ff2cf 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestMultipleContainerReadWrite.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestMultipleContainerReadWrite.java
@@ -62,8 +62,7 @@ public class TestMultipleContainerReadWrite {
   /**
    * Create a MiniDFSCluster for testing.
    * <p>
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "distributed"
+   * Ozone is made active by setting OZONE_ENABLED = true
    *
    * @throws IOException
    */
@@ -73,8 +72,6 @@ public class TestMultipleContainerReadWrite {
     // set to as small as 100 bytes per block.
     conf.setLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB, 1);
     conf.setInt(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE, 5);
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
-        OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
     storageHandler = new ObjectStoreHandler(conf).getStorageHandler();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmBlockVersioning.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmBlockVersioning.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmBlockVersioning.java
index 0eb1677..9f97a2c 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmBlockVersioning.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmBlockVersioning.java
@@ -22,8 +22,6 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
@@ -67,16 +65,13 @@ public class TestOmBlockVersioning {
   /**
    * Create a MiniDFSCluster for testing.
    * <p>
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "distributed"
+   * Ozone is made active by setting OZONE_ENABLED = true
    *
    * @throws IOException
    */
   @BeforeClass
   public static void init() throws Exception {
     conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
-        OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
     storageHandler = new ObjectStoreHandler(conf).getStorageHandler();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java
index 8d0f4b21..8063981 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmMetrics.java
@@ -23,9 +23,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -52,8 +50,6 @@ public class TestOmMetrics {
   @Before
   public void setup() throws Exception {
     OzoneConfiguration conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
-        OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
     ozoneManager = cluster.getOzoneManager();
@@ -104,7 +100,8 @@ public class TestOmMetrics {
     Mockito.doThrow(exception).when(mockVm).setOwner(null, null);
     Mockito.doThrow(exception).when(mockVm).listVolumes(null, null, null, 0);
 
-    org.apache.hadoop.test.Whitebox.setInternalState(ozoneManager, "volumeManager", mockVm);
+    org.apache.hadoop.test.Whitebox.setInternalState(ozoneManager,
+        "volumeManager", mockVm);
     doVolumeOps();
 
     omMetrics = getMetrics("OMMetrics");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSQLCli.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSQLCli.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSQLCli.java
index 005a012..ab26c00 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSQLCli.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSQLCli.java
@@ -20,7 +20,6 @@ import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.scm.cli.SQLCLI;
 import org.apache.hadoop.ozone.web.handlers.BucketArgs;
 import org.apache.hadoop.ozone.web.handlers.KeyArgs;
@@ -97,16 +96,13 @@ public class TestOmSQLCli {
   /**
    * Create a MiniDFSCluster for testing.
    * <p>
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "distributed"
+   * Ozone is made active by setting OZONE_ENABLED = true
    *
    * @throws IOException
    */
   @Before
   public void setup() throws Exception {
     conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
-        OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
     storageHandler = new ObjectStoreHandler(conf).getStorageHandler();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
index 5109453..4908c4d 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
@@ -108,8 +108,7 @@ public class TestOzoneManager {
   /**
    * Create a MiniDFSCluster for testing.
    * <p>
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "distributed"
+   * Ozone is made active by setting OZONE_ENABLED = true
    *
    * @throws IOException
    */
@@ -119,8 +118,6 @@ public class TestOzoneManager {
     clusterId = UUID.randomUUID().toString();
     scmId = UUID.randomUUID().toString();
     omId = UUID.randomUUID().toString();
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
-        OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
     conf.setInt(OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS, 2);
     cluster =  MiniOzoneCluster.newBuilder(conf)
         .setClusterId(clusterId)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
index 573f097..528828b 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
@@ -48,7 +48,6 @@ import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneAcl.OzoneACLRights;
 import org.apache.hadoop.ozone.OzoneAcl.OzoneACLType;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.OzoneBucket;
@@ -134,11 +133,6 @@ public class TestOzoneShell {
     baseDir = new File(path);
     baseDir.mkdirs();
 
-    path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
-        OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
-
-    conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
-    conf.setQuietMode(false);
     shell = new Shell();
     shell.setConf(conf);
 
@@ -146,6 +140,7 @@ public class TestOzoneShell {
         .setNumDatanodes(3)
         .build();
     conf.setInt(OZONE_REPLICATION, ReplicationFactor.THREE.getValue());
+    conf.setQuietMode(false);
     client = new RpcClient(conf);
     cluster.waitForClusterToBeReady();
   }
@@ -208,12 +203,15 @@ public class TestOzoneShell {
     testCreateVolume(volumeName, "");
     volumeName = "volume" + RandomStringUtils.randomNumeric(5);
     testCreateVolume("/////" + volumeName, "");
-    testCreateVolume("/////", "Volume name is required to create a volume");
+    testCreateVolume("/////", "Volume name is required " +
+        "to create a volume");
     testCreateVolume("/////vol/123",
-        "Illegal argument: Bucket or Volume name has an unsupported character : /");
+        "Illegal argument: Bucket or Volume name has " +
+            "an unsupported character : /");
   }
 
-  private void testCreateVolume(String volumeName, String errorMsg) throws Exception {
+  private void testCreateVolume(String volumeName, String errorMsg)
+      throws Exception {
     err.reset();
     String userName = "bilbo";
     String[] args = new String[] {"-createVolume", url + "/" + volumeName,
@@ -397,7 +395,7 @@ public class TestOzoneShell {
 
     // test -prefix option
     out.reset();
-    args = new String[] { "-listVolume", url + "/", "-user", user1, "-length",
+    args = new String[] {"-listVolume", url + "/", "-user", user1, "-length",
         "100", "-prefix", "test-vol-" + protocol + "1" };
     assertEquals(0, ToolRunner.run(shell, args));
     commandOutput = out.toString();
@@ -414,7 +412,7 @@ public class TestOzoneShell {
 
     // test -start option
     out.reset();
-    args = new String[] { "-listVolume", url + "/", "-user", user2, "-length",
+    args = new String[] {"-listVolume", url + "/", "-user", user2, "-length",
         "100", "-start", "test-vol-" + protocol + "15" };
     assertEquals(0, ToolRunner.run(shell, args));
     commandOutput = out.toString();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestDistributedOzoneVolumes.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestDistributedOzoneVolumes.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestDistributedOzoneVolumes.java
deleted file mode 100644
index e592d56..0000000
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestDistributedOzoneVolumes.java
+++ /dev/null
@@ -1,188 +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.ozone.web;
-
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.TestOzoneHelper;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.junit.Rule;
-import org.junit.BeforeClass;
-import org.junit.AfterClass;
-import org.junit.Test;
-import org.junit.Assert;
-
-import org.junit.rules.Timeout;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-/**
- * Test ozone volume in the distributed storage handler scenario.
- */
-public class TestDistributedOzoneVolumes extends TestOzoneHelper {
-  private static final org.slf4j.Logger LOG =
-      LoggerFactory.getLogger(TestDistributedOzoneVolumes.class);
-  /**
-   * Set the timeout for every test.
-   */
-  @Rule
-  public Timeout testTimeout = new Timeout(300000);
-
-  private static MiniOzoneCluster cluster = null;
-  private static int port = 0;
-
-  /**
-   * Create a MiniDFSCluster for testing.
-   * <p>
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "distributed"
-   *
-   * @throws IOException
-   */
-  @BeforeClass
-  public static void init() throws Exception {
-    OzoneConfiguration conf = new OzoneConfiguration();
-    Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
-        OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
-    cluster = MiniOzoneCluster.newBuilder(conf).build();
-    cluster.waitForClusterToBeReady();
-    port = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails()
-        .getPort(DatanodeDetails.Port.Name.REST).getValue();
-  }
-
-  /**
-   * Shutdown MiniDFSCluster.
-   */
-  @AfterClass
-  public static void shutdown() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * Creates Volumes on Ozone Store.
-   *
-   * @throws IOException
-   */
-  @Test
-  public void testCreateVolumes() throws IOException {
-    super.testCreateVolumes(port);
-    Assert.assertEquals(0, cluster.getOzoneManager()
-        .getMetrics().getNumVolumeCreateFails());
-  }
-
-  /**
-   * Create Volumes with Quota.
-   *
-   * @throws IOException
-   */
-  @Test
-  public void testCreateVolumesWithQuota() throws IOException {
-    super.testCreateVolumesWithQuota(port);
-    Assert.assertEquals(0, cluster.getOzoneManager()
-        .getMetrics().getNumVolumeCreateFails());
-  }
-
-  /**
-   * Create Volumes with Invalid Quota.
-   *
-   * @throws IOException
-   */
-  @Test
-  public void testCreateVolumesWithInvalidQuota() throws IOException {
-    super.testCreateVolumesWithInvalidQuota(port);
-    Assert.assertEquals(0, cluster.getOzoneManager()
-        .getMetrics().getNumVolumeCreateFails());
-  }
-
-  /**
-   * To create a volume a user name must be specified using OZONE_USER header.
-   * This test verifies that we get an error in case we call without a OZONE
-   * user name.
-   *
-   * @throws IOException
-   */
-  @Test
-  public void testCreateVolumesWithInvalidUser() throws IOException {
-    super.testCreateVolumesWithInvalidUser(port);
-    Assert.assertEquals(0, cluster.getOzoneManager()
-        .getMetrics().getNumVolumeCreateFails());
-  }
-
-  /**
-   * Only Admins can create volumes in Ozone. This test uses simple userauth as
-   * backend and hdfs and root are admin users in the simple backend.
-   * <p>
-   * This test tries to create a volume as user bilbo.
-   *
-   * @throws IOException
-   */
-  @Test
-  public void testCreateVolumesWithOutAdminRights() throws IOException {
-    super.testCreateVolumesWithOutAdminRights(port);
-    Assert.assertEquals(0, cluster.getOzoneManager()
-        .getMetrics().getNumVolumeCreateFails());
-  }
-
-  /**
-   * Create a bunch of volumes in a loop.
-   *
-   * @throws IOException
-   */
-  @Test
-  public void testCreateVolumesInLoop() throws IOException {
-    super.testCreateVolumesInLoop(port);
-    Assert.assertEquals(0, cluster.getOzoneManager()
-        .getMetrics().getNumVolumeCreateFails());
-  }
-  /**
-   * Get volumes owned by the user.
-   *
-   * @throws IOException
-   */
-  public void testGetVolumesByUser() throws IOException {
-    testGetVolumesByUser(port);
-  }
-
-  /**
-   * Admins can read volumes belonging to other users.
-   *
-   * @throws IOException
-   */
-  public void testGetVolumesOfAnotherUser() throws IOException {
-    super.testGetVolumesOfAnotherUser(port);
-  }
-
-  /**
-   * if you try to read volumes belonging to another user,
-   * then server always ignores it.
-   *
-   * @throws IOException
-   */
-  public void testGetVolumesOfAnotherUserShouldFail() throws IOException {
-    super.testGetVolumesOfAnotherUserShouldFail(port);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestLocalOzoneVolumes.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestLocalOzoneVolumes.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestLocalOzoneVolumes.java
deleted file mode 100644
index 441f771..0000000
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestLocalOzoneVolumes.java
+++ /dev/null
@@ -1,187 +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.ozone.web;
-
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.ozone.TestOzoneHelper;
-import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-
-import java.io.IOException;
-
-/**
- * Test ozone volume in the local storage handler scenario.
- */
-public class TestLocalOzoneVolumes extends TestOzoneHelper {
-  /**
-   * Set the timeout for every test.
-   */
-  @Rule
-  public Timeout testTimeout = new Timeout(300000);
-
-  private static MiniOzoneCluster cluster = null;
-  private static int port = 0;
-
-  /**
-   * Create a MiniDFSCluster for testing.
-   * <p>
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "local" , which uses a local directory to
-   * emulate Ozone backend.
-   *
-   * @throws IOException
-   */
-  @BeforeClass
-  public static void init() throws Exception {
-    OzoneConfiguration conf = new OzoneConfiguration();
-
-    String path = GenericTestUtils
-        .getTempPath(TestLocalOzoneVolumes.class.getSimpleName());
-    path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
-        OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
-
-    conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
-    Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
-
-    cluster = MiniOzoneCluster.newBuilder(conf).build();
-    cluster.waitForClusterToBeReady();
-    port = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails().getPort(
-            DatanodeDetails.Port.Name.REST).getValue();
-  }
-
-  /**
-   * Shutdown MiniDFSCluster.
-   */
-  @AfterClass
-  public static void shutdown() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  /**
-   * Creates Volumes on Ozone Store.
-   *
-   * @throws IOException
-   */
-  @Test
-  public void testCreateVolumes() throws IOException {
-    super.testCreateVolumes(port);
-  }
-
-  /**
-   * Create Volumes with Quota.
-   *
-   * @throws IOException
-   */
-  @Test
-  public void testCreateVolumesWithQuota() throws IOException {
-    super.testCreateVolumesWithQuota(port);
-  }
-
-  /**
-   * Create Volumes with Invalid Quota.
-   *
-   * @throws IOException
-   */
-  @Test
-  public void testCreateVolumesWithInvalidQuota() throws IOException {
-    super.testCreateVolumesWithInvalidQuota(port);
-  }
-
-  /**
-   * To create a volume a user name must be specified using OZONE_USER header.
-   * This test verifies that we get an error in case we call without a OZONE
-   * user name.
-   *
-   * @throws IOException
-   */
-  @Test
-  public void testCreateVolumesWithInvalidUser() throws IOException {
-    super.testCreateVolumesWithInvalidUser(port);
-  }
-
-  /**
-   * Only Admins can create volumes in Ozone. This test uses simple userauth as
-   * backend and hdfs and root are admin users in the simple backend.
-   * <p>
-   * This test tries to create a volume as user bilbo.
-   *
-   * @throws IOException
-   */
-  @Test
-  public void testCreateVolumesWithOutAdminRights() throws IOException {
-    super.testCreateVolumesWithOutAdminRights(port);
-  }
-
-  /**
-   * Create a bunch of volumes in a loop.
-   *
-   * @throws IOException
-   */
-  //@Test
-  public void testCreateVolumesInLoop() throws IOException {
-    super.testCreateVolumesInLoop(port);
-  }
-  /**
-   * Get volumes owned by the user.
-   *
-   * @throws IOException
-   */
-  @Test
-  public void testGetVolumesByUser() throws IOException {
-    super.testGetVolumesByUser(port);
-  }
-
-  /**
-   * Admins can read volumes belonging to other users.
-   *
-   * @throws IOException
-   */
-  @Test
-  public void testGetVolumesOfAnotherUser() throws IOException {
-    super.testGetVolumesOfAnotherUser(port);
-  }
-
-  /**
-   * if you try to read volumes belonging to another user,
-   * then server always ignores it.
-   *
-   * @throws IOException
-   */
-  @Test @Ignore
-  public void testGetVolumesOfAnotherUserShouldFail() throws IOException {
-    super.testGetVolumesOfAnotherUserShouldFail(port);
-  }
-
-  @Test
-  public void testListKeyOnEmptyBucket() throws IOException {
-    super.testListKeyOnEmptyBucket(port);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneVolumes.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneVolumes.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneVolumes.java
new file mode 100644
index 0000000..290e834
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneVolumes.java
@@ -0,0 +1,183 @@
+/*
+ * 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.web;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.TestOzoneHelper;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.junit.Rule;
+import org.junit.BeforeClass;
+import org.junit.AfterClass;
+import org.junit.Test;
+import org.junit.Assert;
+
+import org.junit.rules.Timeout;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * Test ozone volume in the distributed storage handler scenario.
+ */
+public class TestOzoneVolumes extends TestOzoneHelper {
+  private static final org.slf4j.Logger LOG =
+      LoggerFactory.getLogger(TestOzoneVolumes.class);
+  /**
+   * Set the timeout for every test.
+   */
+  @Rule
+  public Timeout testTimeout = new Timeout(300000);
+
+  private static MiniOzoneCluster cluster = null;
+  private static int port = 0;
+
+  /**
+   * Create a MiniDFSCluster for testing.
+   * <p>
+   * Ozone is made active by setting OZONE_ENABLED = true
+   *
+   * @throws IOException
+   */
+  @BeforeClass
+  public static void init() throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
+    cluster = MiniOzoneCluster.newBuilder(conf).build();
+    cluster.waitForClusterToBeReady();
+    port = cluster.getHddsDatanodes().get(0)
+        .getDatanodeDetails()
+        .getPort(DatanodeDetails.Port.Name.REST).getValue();
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterClass
+  public static void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Creates Volumes on Ozone Store.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testCreateVolumes() throws IOException {
+    super.testCreateVolumes(port);
+    Assert.assertEquals(0, cluster.getOzoneManager()
+        .getMetrics().getNumVolumeCreateFails());
+  }
+
+  /**
+   * Create Volumes with Quota.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testCreateVolumesWithQuota() throws IOException {
+    super.testCreateVolumesWithQuota(port);
+    Assert.assertEquals(0, cluster.getOzoneManager()
+        .getMetrics().getNumVolumeCreateFails());
+  }
+
+  /**
+   * Create Volumes with Invalid Quota.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testCreateVolumesWithInvalidQuota() throws IOException {
+    super.testCreateVolumesWithInvalidQuota(port);
+    Assert.assertEquals(0, cluster.getOzoneManager()
+        .getMetrics().getNumVolumeCreateFails());
+  }
+
+  /**
+   * To create a volume a user name must be specified using OZONE_USER header.
+   * This test verifies that we get an error in case we call without a OZONE
+   * user name.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testCreateVolumesWithInvalidUser() throws IOException {
+    super.testCreateVolumesWithInvalidUser(port);
+    Assert.assertEquals(0, cluster.getOzoneManager()
+        .getMetrics().getNumVolumeCreateFails());
+  }
+
+  /**
+   * Only Admins can create volumes in Ozone. This test uses simple userauth as
+   * backend and hdfs and root are admin users in the simple backend.
+   * <p>
+   * This test tries to create a volume as user bilbo.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testCreateVolumesWithOutAdminRights() throws IOException {
+    super.testCreateVolumesWithOutAdminRights(port);
+    Assert.assertEquals(0, cluster.getOzoneManager()
+        .getMetrics().getNumVolumeCreateFails());
+  }
+
+  /**
+   * Create a bunch of volumes in a loop.
+   *
+   * @throws IOException
+   */
+  @Test
+  public void testCreateVolumesInLoop() throws IOException {
+    super.testCreateVolumesInLoop(port);
+    Assert.assertEquals(0, cluster.getOzoneManager()
+        .getMetrics().getNumVolumeCreateFails());
+  }
+  /**
+   * Get volumes owned by the user.
+   *
+   * @throws IOException
+   */
+  public void testGetVolumesByUser() throws IOException {
+    testGetVolumesByUser(port);
+  }
+
+  /**
+   * Admins can read volumes belonging to other users.
+   *
+   * @throws IOException
+   */
+  public void testGetVolumesOfAnotherUser() throws IOException {
+    super.testGetVolumesOfAnotherUser(port);
+  }
+
+  /**
+   * if you try to read volumes belonging to another user,
+   * then server always ignores it.
+   *
+   * @throws IOException
+   */
+  public void testGetVolumesOfAnotherUserShouldFail() throws IOException {
+    super.testGetVolumesOfAnotherUserShouldFail(port);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
index c014a60..9828b95 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.ozone.web;
 
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.rest.headers.Header;
@@ -62,20 +61,13 @@ public class TestOzoneWebAccess {
   /**
    * Create a MiniDFSCluster for testing.
    *
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "local" , which uses a local directory to
-   * emulate Ozone backend.
-   *
+   * Ozone is made active by setting OZONE_ENABLED = true
    * @throws IOException
    */
   @BeforeClass
   public static void init() throws Exception {
     OzoneConfiguration conf = new OzoneConfiguration();
 
-    String path = GenericTestUtils
-        .getTempPath(TestOzoneWebAccess.class.getSimpleName());
-    conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
-
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
     port = cluster.getHddsDatanodes().get(0)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
index 39aa03c..2eedb58 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.ozone.web.client;
 
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.client.BucketArgs;
 import org.apache.hadoop.ozone.client.VolumeArgs;
@@ -86,9 +85,7 @@ public class TestBuckets {
   /**
    * Create a MiniDFSCluster for testing.
    * <p>
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "local" , which uses a local directory to
-   * emulate Ozone backend.
+   * Ozone is made active by setting OZONE_ENABLED = true
    *
    * @throws IOException
    */
@@ -100,10 +97,6 @@ public class TestBuckets {
 
     String path = GenericTestUtils
         .getTempPath(TestBuckets.class.getSimpleName());
-    path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
-        OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
-
-    conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
     cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(3)
         .build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
index 6a92022..5eeeacf 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
@@ -19,9 +19,9 @@ package org.apache.hadoop.ozone.web.client;
 
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.RatisTestHelper;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -58,7 +58,7 @@ public class TestKeysRatis {
   @BeforeClass
   public static void init() throws Exception {
     suite = new RatisTestHelper.RatisTestSuite(TestBucketsRatis.class);
-    path = suite.getConf().get(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT);
+    path = GenericTestUtils.getTempPath(TestKeysRatis.class.getSimpleName());
     ozoneCluster = suite.getCluster();
     ozoneCluster.waitForClusterToBeReady();
     client = suite.newOzoneClient();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestOzoneClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestOzoneClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestOzoneClient.java
index 86de8df..f8f57d7 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestOzoneClient.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestOzoneClient.java
@@ -45,7 +45,6 @@ import io.netty.handler.logging.LoggingHandler;
 
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.rest.headers.Header;
@@ -92,8 +91,6 @@ public class TestOzoneClient {
   public static void init() throws Exception {
     Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.ALL);
     OzoneConfiguration conf = new OzoneConfiguration();
-    conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
-        OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
     cluster = MiniOzoneCluster.newBuilder(conf).build();
     cluster.waitForClusterToBeReady();
     int port = cluster.getHddsDatanodes().get(0)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
index c9a0c38..31f9214 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
@@ -23,7 +23,6 @@ import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.hadoop.hdds.client.OzoneQuota;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.client.VolumeArgs;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.client.rest.RestClient;
@@ -79,9 +78,7 @@ public class TestVolume {
   /**
    * Create a MiniDFSCluster for testing.
    * <p>
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "local" , which uses a local directory to
-   * emulate Ozone backend.
+   * Ozone is made active by setting OZONE_ENABLED = true
    *
    * @throws IOException
    */
@@ -91,11 +88,8 @@ public class TestVolume {
 
     String path = GenericTestUtils
         .getTempPath(TestVolume.class.getSimpleName());
-    path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
-        OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
     FileUtils.deleteDirectory(new File(path));
 
-    conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
     Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
 
     cluster = MiniOzoneCluster.newBuilder(conf).build();
@@ -221,7 +215,8 @@ public class TestVolume {
     client.createVolume(volumeName);
     client.setVolumeQuota(volumeName, OzoneQuota.parseQuota("1000MB"));
     OzoneVolume newVol = client.getVolumeDetails(volumeName);
-    assertEquals(newVol.getQuota(), OzoneQuota.parseQuota("1000MB").sizeInBytes());
+    assertEquals(newVol.getQuota(),
+        OzoneQuota.parseQuota("1000MB").sizeInBytes());
     // verify if the creation time is missing after setting quota operation
     assertTrue(newVol.getCreationTime() > 0);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
index d6783ad..1a05a3c 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
@@ -72,11 +72,8 @@ public class TestVolumeRatis {
 
     String path = GenericTestUtils
         .getTempPath(TestVolume.class.getSimpleName());
-    path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
-        OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
     FileUtils.deleteDirectory(new File(path));
 
-    conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
     Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
 
     cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(3).build();


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[21/50] [abbrv] hadoop git commit: HDFS-13854. RBF: The ProcessingAvgTime and ProxyAvgTime should display by JMX with ms unit. Contributed by yanghuafeng.

Posted by eh...@apache.org.
HDFS-13854. RBF: The ProcessingAvgTime and ProxyAvgTime should display by JMX with ms unit. Contributed by yanghuafeng.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/64ad0298
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/64ad0298
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/64ad0298

Branch: refs/heads/HDFS-12090
Commit: 64ad0298d441559951bc9589a40f8aab17c93a5f
Parents: 2651e2c
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Wed Aug 29 08:29:50 2018 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Wed Aug 29 08:29:50 2018 +0530

----------------------------------------------------------------------
 .../federation/metrics/FederationRPCMetrics.java     | 13 ++-----------
 .../metrics/FederationRPCPerformanceMonitor.java     | 15 +++++----------
 2 files changed, 7 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/64ad0298/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java
index 9ab4e5a..cce4b86 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java
@@ -86,15 +86,6 @@ public class FederationRPCMetrics implements FederationRPCMBean {
   }
 
   /**
-   * Convert nanoseconds to milliseconds.
-   * @param ns Time in nanoseconds.
-   * @return Time in milliseconds.
-   */
-  private static double toMs(double ns) {
-    return ns / 1000000;
-  }
-
-  /**
    * Reset the metrics system.
    */
   public static void reset() {
@@ -230,7 +221,7 @@ public class FederationRPCMetrics implements FederationRPCMBean {
 
   @Override
   public double getProxyAvg() {
-    return toMs(proxy.lastStat().mean());
+    return proxy.lastStat().mean();
   }
 
   @Override
@@ -250,7 +241,7 @@ public class FederationRPCMetrics implements FederationRPCMBean {
 
   @Override
   public double getProcessingAvg() {
-    return toMs(processing.lastStat().mean());
+    return processing.lastStat().mean();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/64ad0298/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java
index 2c2741e..15725d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java
@@ -35,6 +35,8 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
+import static org.apache.hadoop.util.Time.monotonicNow;
+
 /**
  * Customizable RPC performance monitor. Receives events from the RPC server
  * and aggregates them via JMX.
@@ -120,12 +122,12 @@ public class FederationRPCPerformanceMonitor implements RouterRpcMonitor {
 
   @Override
   public void startOp() {
-    START_TIME.set(this.getNow());
+    START_TIME.set(monotonicNow());
   }
 
   @Override
   public long proxyOp() {
-    PROXY_TIME.set(this.getNow());
+    PROXY_TIME.set(monotonicNow());
     long processingTime = getProcessingTime();
     if (processingTime >= 0) {
       metrics.addProcessingTime(processingTime);
@@ -188,13 +190,6 @@ public class FederationRPCPerformanceMonitor implements RouterRpcMonitor {
     metrics.incrRouterFailureLocked();
   }
 
-  /**
-   * Get current time.
-   * @return Current time in nanoseconds.
-   */
-  private long getNow() {
-    return System.nanoTime();
-  }
 
   /**
    * Get time between we receiving the operation and sending it to the Namenode.
@@ -214,7 +209,7 @@ public class FederationRPCPerformanceMonitor implements RouterRpcMonitor {
    */
   private long getProxyTime() {
     if (PROXY_TIME.get() != null && PROXY_TIME.get() > 0) {
-      return getNow() - PROXY_TIME.get();
+      return monotonicNow() - PROXY_TIME.get();
     }
     return -1;
   }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[15/50] [abbrv] hadoop git commit: HDFS-13837. Enable debug log for LeaseRenewer in TestDistributedFileSystem. Contributed by Shweta.

Posted by eh...@apache.org.
HDFS-13837. Enable debug log for LeaseRenewer in TestDistributedFileSystem. Contributed by Shweta.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/33f42efc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/33f42efc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/33f42efc

Branch: refs/heads/HDFS-12090
Commit: 33f42efc947445b7755da6aad34b5e26b96ad663
Parents: ac515d2
Author: Shweta <sh...@cloudera.com>
Authored: Tue Aug 28 13:51:04 2018 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Tue Aug 28 13:56:32 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java  | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/33f42efc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
index 46323dd..cae0fbf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
@@ -100,12 +100,12 @@ import org.apache.hadoop.test.Whitebox;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.concurrent.HadoopExecutors;
-import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.InOrder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.slf4j.event.Level;
 
 public class TestDistributedFileSystem {
   private static final Random RAN = new Random();
@@ -113,7 +113,8 @@ public class TestDistributedFileSystem {
       TestDistributedFileSystem.class);
 
   static {
-    GenericTestUtils.setLogLevel(DFSClient.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(DFSClient.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(LeaseRenewer.LOG, Level.DEBUG);
   }
 
   private boolean dualPortTesting = false;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[28/50] [abbrv] hadoop git commit: HDFS-13863. FsDatasetImpl should log DiskOutOfSpaceException. Contributed by Fei Hui.

Posted by eh...@apache.org.
HDFS-13863. FsDatasetImpl should log DiskOutOfSpaceException. Contributed by Fei Hui.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/582cb10e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/582cb10e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/582cb10e

Branch: refs/heads/HDFS-12090
Commit: 582cb10ec74ed5666946a3769002ceb80ba660cb
Parents: d53a10b
Author: Yiqun Lin <yq...@apache.org>
Authored: Thu Aug 30 11:21:13 2018 +0800
Committer: Yiqun Lin <yq...@apache.org>
Committed: Thu Aug 30 11:21:13 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/582cb10e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index d7f133e..27196c2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -1397,6 +1397,9 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           datanode.getMetrics().incrRamDiskBlocksWrite();
         } catch (DiskOutOfSpaceException de) {
           // Ignore the exception since we just fall back to persistent storage.
+          LOG.warn("Insufficient space for placing the block on a transient "
+              + "volume, fall back to persistent storage: "
+              + de.getMessage());
         } finally {
           if (ref == null) {
             cacheManager.release(b.getNumBytes());


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[47/50] [abbrv] hadoop git commit: HDFS-13774. EC: 'hdfs ec -getPolicy' is not retrieving policy details when the special REPLICATION policy set on the directory. Contributed by Ayush Saxena.

Posted by eh...@apache.org.
HDFS-13774. EC: 'hdfs ec -getPolicy' is not retrieving policy details when the special REPLICATION policy set on the directory. Contributed by Ayush Saxena.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3801436e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3801436e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3801436e

Branch: refs/heads/HDFS-12090
Commit: 3801436e49822c724c8f4e393e6e5abbd0d2573a
Parents: 780df90
Author: Vinayakumar B <vi...@apache.org>
Authored: Mon Sep 3 14:37:57 2018 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Mon Sep 3 14:37:57 2018 +0530

----------------------------------------------------------------------
 .../hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md       | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3801436e/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
index 2e8cbbd..67e6b75 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
@@ -65,11 +65,11 @@ Architecture
 
       2. _The size of a striping cell._ This determines the granularity of striped reads and writes, including buffer sizes and encoding work.
 
-    Policies are named *codec*-*num data blocks*-*num parity blocks*-*cell size*. Currently, six built-in policies are supported: `RS-3-2-1024k`, `RS-6-3-1024k`, `RS-10-4-1024k`, `RS-LEGACY-6-3-1024k`, `XOR-2-1-1024k` and `REPLICATION`.
+    Policies are named *codec*-*num data blocks*-*num parity blocks*-*cell size*. Currently, five built-in policies are supported: `RS-3-2-1024k`, `RS-6-3-1024k`, `RS-10-4-1024k`, `RS-LEGACY-6-3-1024k`, `XOR-2-1-1024k`.
 
-    `REPLICATION` is a special policy. It can only be set on directory, to force the directory to adopt 3x replication scheme, instead of inheriting its ancestor's erasure coding policy. This policy makes it possible to interleave 3x replication scheme directory with erasure coding directory.
+    The default `REPLICATION` scheme is also supported. It can only be set on directory, to force the directory to adopt 3x replication scheme, instead of inheriting its ancestor's erasure coding policy. This policy makes it possible to interleave 3x replication scheme directory with erasure coding directory.
 
-    `REPLICATION` policy is always enabled. For other built-in policies, they are disabled by default.
+    `REPLICATION` is always enabled. Out of all the EC policies, RS(6,3) is enabled by default.
 
     Similar to HDFS storage policies, erasure coding policies are set on a directory. When a file is created, it inherits the EC policy of its nearest ancestor directory.
 
@@ -184,7 +184,7 @@ Below are the details about each command.
       This parameter can be omitted if a 'dfs.namenode.ec.system.default.policy' configuration is set.
       The EC policy of the path will be set with the default value in configuration.
 
-      `-replicate` apply the special `REPLICATION` policy on the directory, force the directory to adopt 3x replication scheme.
+      `-replicate` apply the default `REPLICATION` scheme on the directory, force the directory to adopt 3x replication scheme.
 
       `-replicate` and `-policy <policyName>` are optional arguments. They cannot be specified at the same time.
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[08/50] [abbrv] hadoop git commit: HDDS-381. Fix TestKeys#testPutAndGetKeyWithDnRestart. Contributed by Mukul Kumar Singh.

Posted by eh...@apache.org.
HDDS-381. Fix TestKeys#testPutAndGetKeyWithDnRestart. Contributed by Mukul Kumar Singh.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2172399c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2172399c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2172399c

Branch: refs/heads/HDFS-12090
Commit: 2172399c55b481ea0da8cf2e2cb91ea6d8140b27
Parents: 75691ad
Author: Nanda kumar <na...@apache.org>
Authored: Tue Aug 28 22:19:52 2018 +0530
Committer: Nanda kumar <na...@apache.org>
Committed: Tue Aug 28 22:19:52 2018 +0530

----------------------------------------------------------------------
 .../common/transport/server/GrpcXceiverService.java    |  8 +++-----
 .../java/org/apache/hadoop/ozone/MiniOzoneCluster.java |  3 ++-
 .../org/apache/hadoop/ozone/MiniOzoneClusterImpl.java  | 13 +++++++++++--
 .../statemachine/commandhandler/TestBlockDeletion.java |  9 +++++++--
 .../org/apache/hadoop/ozone/web/client/TestKeys.java   | 11 ++++-------
 5 files changed, 27 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2172399c/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java
index df6220c..db4a86a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/GrpcXceiverService.java
@@ -56,10 +56,8 @@ public class GrpcXceiverService extends
           ContainerCommandResponseProto resp = dispatcher.dispatch(request);
           responseObserver.onNext(resp);
         } catch (Throwable e) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("{} got exception when processing"
+          LOG.error("{} got exception when processing"
                     + " ContainerCommandRequestProto {}: {}", request, e);
-          }
           responseObserver.onError(e);
         }
       }
@@ -67,13 +65,13 @@ public class GrpcXceiverService extends
       @Override
       public void onError(Throwable t) {
         // for now we just log a msg
-        LOG.info("{}: ContainerCommand send on error. Exception: {}", t);
+        LOG.error("{}: ContainerCommand send on error. Exception: {}", t);
       }
 
       @Override
       public void onCompleted() {
         if (isClosed.compareAndSet(false, true)) {
-          LOG.info("{}: ContainerCommand send completed");
+          LOG.debug("{}: ContainerCommand send completed");
           responseObserver.onCompleted();
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2172399c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
index b568672..ae6a91e 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
@@ -152,7 +152,8 @@ public interface MiniOzoneCluster {
    *
    * @param i index of HddsDatanode in the MiniOzoneCluster
    */
-  void restartHddsDatanode(int i);
+  void restartHddsDatanode(int i) throws InterruptedException,
+      TimeoutException;
 
   /**
    * Shutdown a particular HddsDatanode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2172399c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
index 9b7e399..e06e2f6 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
@@ -216,7 +216,8 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
   }
 
   @Override
-  public void restartHddsDatanode(int i) {
+  public void restartHddsDatanode(int i) throws InterruptedException,
+      TimeoutException {
     HddsDatanodeService datanodeService = hddsDatanodes.get(i);
     datanodeService.stop();
     datanodeService.join();
@@ -230,7 +231,15 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
         .getPort(DatanodeDetails.Port.Name.RATIS).getValue();
     conf.setInt(DFS_CONTAINER_RATIS_IPC_PORT, ratisPort);
     conf.setBoolean(DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, false);
-    datanodeService.start(null);
+    hddsDatanodes.remove(i);
+    // wait for node to be removed from SCM healthy node list.
+    waitForClusterToBeReady();
+    HddsDatanodeService service =
+        HddsDatanodeService.createHddsDatanodeService(conf);
+    hddsDatanodes.add(i, service);
+    service.start(null);
+    // wait for the node to be identified as a healthy node again.
+    waitForClusterToBeReady();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2172399c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
index 5794151..08bc556 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java
@@ -57,8 +57,13 @@ import org.junit.Test;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.*;
+import java.util.Set;
+import java.util.List;
+import java.util.HashSet;
+import java.util.ArrayList;
+import java.util.UUID;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
 import static org.apache.hadoop.hdds
     .HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL;
@@ -109,7 +114,7 @@ public class TestBlockDeletion {
   @Test(timeout = 60000)
   @Ignore("Until delete background service is fixed.")
   public void testBlockDeletion()
-      throws IOException, InterruptedException {
+      throws IOException, InterruptedException, TimeoutException {
     String volumeName = UUID.randomUUID().toString();
     String bucketName = UUID.randomUUID().toString();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2172399c/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
index a87f3f6..21887be 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
@@ -77,7 +77,6 @@ import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.InputStream;
 import java.io.IOException;
-import java.net.URISyntaxException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.Arrays;
@@ -90,6 +89,8 @@ import java.util.stream.Collectors;
 
 import static org.apache.hadoop.hdds
     .HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys
+    .OZONE_SCM_STALENODE_INTERVAL;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -127,6 +128,7 @@ public class TestKeys {
     conf.setTimeDuration(OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL,
         1000, TimeUnit.MILLISECONDS);
     conf.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL, 1, TimeUnit.SECONDS);
+    conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 30, TimeUnit.SECONDS);
 
     path = GenericTestUtils.getTempPath(TestKeys.class.getSimpleName());
     Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
@@ -322,7 +324,7 @@ public class TestKeys {
   }
 
   private static void restartDatanode(MiniOzoneCluster cluster, int datanodeIdx)
-      throws OzoneException, URISyntaxException {
+      throws Exception {
     cluster.restartHddsDatanode(datanodeIdx);
   }
 
@@ -344,11 +346,6 @@ public class TestKeys {
 
     // restart the datanode
     restartDatanode(cluster, 0);
-    // TODO: Try removing sleep and adding a join for the MiniOzoneCluster start
-    // The ozoneContainer is not started and its metrics are not initialized
-    // which leads to NullPointerException in Dispatcher.
-    Thread.sleep(1000);
-    ozoneCluster.waitForClusterToBeReady();
     // verify getKey after the datanode restart
     String newFileName = helper.dir + "/"
         + OzoneUtils.getRequestID().toLowerCase();


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[04/50] [abbrv] hadoop git commit: HADOOP-15699. Fix some of testContainerManager failures in Windows. Contributed by Botong Huang.

Posted by eh...@apache.org.
HADOOP-15699. Fix some of testContainerManager failures in Windows. Contributed by Botong Huang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/602d1384
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/602d1384
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/602d1384

Branch: refs/heads/HDFS-12090
Commit: 602d13844a8d4c7b08ce185da01fde098ff8b9a6
Parents: 05b2bbe
Author: Giovanni Matteo Fumarola <gi...@apache.org>
Authored: Mon Aug 27 12:25:46 2018 -0700
Committer: Giovanni Matteo Fumarola <gi...@apache.org>
Committed: Mon Aug 27 12:25:46 2018 -0700

----------------------------------------------------------------------
 .../containermanager/TestContainerManager.java    | 18 ++++++------------
 1 file changed, 6 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/602d1384/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index ee5259f..d28340b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -320,9 +320,8 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
   @Test (timeout = 10000L)
   public void testAuxPathHandler() throws Exception {
-    File testDir = GenericTestUtils.getTestDir(GenericTestUtils.getTestDir(
-        TestContainerManager.class.getSimpleName() + "LocDir").
-        getAbsolutePath());
+    File testDir = GenericTestUtils
+        .getTestDir(TestContainerManager.class.getSimpleName() + "LocDir");
     testDir.mkdirs();
     File testFile = new File(testDir, "test");
     testFile.createNewFile();
@@ -1977,15 +1976,11 @@ public class TestContainerManager extends BaseContainerManagerTest {
     Signal signal = ContainerLaunch.translateCommandToSignal(command);
     containerManager.start();
 
-    File scriptFile = new File(tmpDir, "scriptFile.sh");
+    File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
     PrintWriter fileWriter = new PrintWriter(scriptFile);
     File processStartFile =
         new File(tmpDir, "start_file.txt").getAbsoluteFile();
-    fileWriter.write("\numask 0"); // So that start file is readable by the test
-    fileWriter.write("\necho Hello World! > " + processStartFile);
-    fileWriter.write("\necho $$ >> " + processStartFile);
-    fileWriter.write("\nexec sleep 1000s");
-    fileWriter.close();
+    writeScriptFile(fileWriter, "Hello world!", processStartFile, null, false);
 
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
@@ -2008,9 +2003,8 @@ public class TestContainerManager extends BaseContainerManagerTest {
         new HashMap<String, LocalResource>();
     localResources.put(destinationFile, rsrc_alpha);
     containerLaunchContext.setLocalResources(localResources);
-    List<String> commands = new ArrayList<>();
-    commands.add("/bin/bash");
-    commands.add(scriptFile.getAbsolutePath());
+    List<String> commands =
+        Arrays.asList(Shell.getRunScriptCommand(scriptFile));
     containerLaunchContext.setCommands(commands);
     StartContainerRequest scRequest =
         StartContainerRequest.newInstance(


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[50/50] [abbrv] hadoop git commit: HDFS-13421. [PROVIDED Phase 2] Implement DNA_BACKUP command in Datanode. Contributed by Ewan Higgs.

Posted by eh...@apache.org.
HDFS-13421. [PROVIDED Phase 2] Implement DNA_BACKUP command in Datanode. Contributed by Ewan Higgs.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/06477abc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/06477abc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/06477abc

Branch: refs/heads/HDFS-12090
Commit: 06477abcd93eb988b4afd0a2dff549e67e0dbd85
Parents: 8cdd033
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Wed Aug 1 12:13:31 2018 -0700
Committer: Ewan Higgs <ew...@wdc.com>
Committed: Mon Sep 3 14:42:33 2018 +0200

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/BlockInputStream.java    |  52 ++++++++
 .../hdfs/server/datanode/BPOfferService.java    |   6 +
 .../hadoop/hdfs/server/datanode/DataNode.java   |  18 +++
 .../SyncServiceSatisfierDatanodeWorker.java     |  97 +++++++++++++++
 .../SyncTaskExecutionFeedbackCollector.java     |  54 ++++++++
 .../executor/BlockSyncOperationExecutor.java    | 122 +++++++++++++++++++
 .../executor/BlockSyncReaderFactory.java        |  92 ++++++++++++++
 .../executor/BlockSyncTaskRunner.java           |  69 +++++++++++
 .../hadoop/hdfs/TestBlockInputStream.java       |  84 +++++++++++++
 .../TestBlockSyncOperationExecutor.java         |  94 ++++++++++++++
 10 files changed, 688 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/06477abc/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockInputStream.java
new file mode 100644
index 0000000..152f83e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockInputStream.java
@@ -0,0 +1,52 @@
+/*
+ * 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.hdfs;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * Facade around BlockReader that indeed implements the InputStream interface.
+ */
+public class BlockInputStream extends InputStream {
+  private final BlockReader blockReader;
+
+  public BlockInputStream(BlockReader blockReader) {
+    this.blockReader = blockReader;
+  }
+
+  @Override
+  public int read() throws IOException {
+    byte[] b = new byte[1];
+    int c = blockReader.read(b, 0, b.length);
+    if (c > 0) {
+      return b[0];
+    } else {
+      return -1;
+    }
+  }
+
+  @Override
+  public int read(byte b[], int off, int len) throws IOException {
+    return blockReader.read(b, off, len);
+  }
+
+  @Override
+  public long skip(long n) throws IOException {
+    return blockReader.skip(n);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06477abc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
index a25f6a9..b8eef5e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java
@@ -795,6 +795,12 @@ class BPOfferService {
           ((BlockECReconstructionCommand) cmd).getECTasks();
       dn.getErasureCodingWorker().processErasureCodingTasks(ecTasks);
       break;
+    case DatanodeProtocol.DNA_BACKUP:
+      LOG.info("DatanodeCommand action: DNA_BACKUP");
+      Collection<BlockSyncTask> backupTasks =
+          ((SyncCommand) cmd).getSyncTasks();
+      dn.getSyncServiceSatisfierDatanodeWorker().processSyncTasks(backupTasks);
+      break;
     default:
       LOG.warn("Unknown DatanodeCommand action: " + cmd.getAction());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06477abc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index c980395..44a548c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -387,6 +387,7 @@ public class DataNode extends ReconfigurableBase
   private String dnUserName = null;
   private BlockRecoveryWorker blockRecoveryWorker;
   private ErasureCodingWorker ecWorker;
+  private SyncServiceSatisfierDatanodeWorker syncServiceSatisfierDatanodeWorker;
   private final Tracer tracer;
   private final TracerConfigurationManager tracerConfigurationManager;
   private static final int NUM_CORES = Runtime.getRuntime()
@@ -1435,6 +1436,9 @@ public class DataNode extends ReconfigurableBase
 
     ecWorker = new ErasureCodingWorker(getConf(), this);
     blockRecoveryWorker = new BlockRecoveryWorker(this);
+    syncServiceSatisfierDatanodeWorker =
+        new SyncServiceSatisfierDatanodeWorker(getConf(), this);
+    syncServiceSatisfierDatanodeWorker.start();
 
     blockPoolManager = new BlockPoolManager(this);
     blockPoolManager.refreshNamenodes(getConf());
@@ -1987,6 +1991,11 @@ public class DataNode extends ReconfigurableBase
       }
     }
 
+    // stop syncServiceSatisfierDatanodeWorker
+    if (syncServiceSatisfierDatanodeWorker != null) {
+      syncServiceSatisfierDatanodeWorker.stop();
+    }
+
     List<BPOfferService> bposArray = (this.blockPoolManager == null)
         ? new ArrayList<BPOfferService>()
         : this.blockPoolManager.getAllNamenodeThreads();
@@ -2139,6 +2148,11 @@ public class DataNode extends ReconfigurableBase
       notifyAll();
     }
     tracer.close();
+
+    // Waiting to finish backup SPS worker thread.
+    if (syncServiceSatisfierDatanodeWorker != null) {
+      syncServiceSatisfierDatanodeWorker.waitToFinishWorkerThread();
+    }
   }
 
   /**
@@ -3631,4 +3645,8 @@ public class DataNode extends ReconfigurableBase
     }
     return this.diskBalancer;
   }
+
+  public SyncServiceSatisfierDatanodeWorker getSyncServiceSatisfierDatanodeWorker() {
+    return syncServiceSatisfierDatanodeWorker;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06477abc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SyncServiceSatisfierDatanodeWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SyncServiceSatisfierDatanodeWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SyncServiceSatisfierDatanodeWorker.java
new file mode 100644
index 0000000..7216e8f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SyncServiceSatisfierDatanodeWorker.java
@@ -0,0 +1,97 @@
+/*
+ * 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.hdfs.server.datanode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.datanode.syncservice.SyncTaskExecutionFeedbackCollector;
+import org.apache.hadoop.hdfs.server.datanode.syncservice.executor.BlockSyncOperationExecutor;
+import org.apache.hadoop.hdfs.server.datanode.syncservice.executor.BlockSyncReaderFactory;
+import org.apache.hadoop.hdfs.server.datanode.syncservice.executor.BlockSyncTaskRunner;
+import org.apache.hadoop.hdfs.server.protocol.BlockSyncTask;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+
+public class SyncServiceSatisfierDatanodeWorker {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(SyncServiceSatisfierDatanodeWorker.class);
+
+  private ExecutorService executorService;
+  private BlockSyncOperationExecutor syncOperationExecutor;
+  private SyncTaskExecutionFeedbackCollector syncTaskExecutionFeedbackCollector;
+
+  public SyncServiceSatisfierDatanodeWorker(Configuration conf, DataNode dataNode) throws IOException {
+    this.executorService = HadoopExecutors.newFixedThreadPool(4);
+    this.syncOperationExecutor =
+        BlockSyncOperationExecutor.createOnDataNode(conf,
+          (locatedBlock, config) -> {
+              try {
+                return BlockSyncReaderFactory.createBlockReader(dataNode, locatedBlock, config);
+              } catch (IOException e) {
+                throw new RuntimeException(e);
+              }
+            }
+          );
+    this.syncTaskExecutionFeedbackCollector = new SyncTaskExecutionFeedbackCollector();
+  }
+
+
+  public void start() {
+    this.executorService = HadoopExecutors.newFixedThreadPool(4);
+  }
+
+  public void stop() {
+    this.executorService.shutdown();
+  }
+
+  public void waitToFinishWorkerThread() {
+    try {
+      this.executorService.awaitTermination(3, TimeUnit.MINUTES);
+    } catch (InterruptedException e) {
+      LOG.warn("SyncServiceSatisfierDatanodeWorker interrupted during waiting for finalization.");
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  public void processSyncTasks(Collection<BlockSyncTask> blockSyncTasks) {
+
+    LOG.debug("Received SyncTasks: {}", blockSyncTasks);
+    for (BlockSyncTask blockSyncTask : blockSyncTasks) {
+      try {
+        executorService.submit(new BlockSyncTaskRunner(blockSyncTask,
+            syncOperationExecutor,
+            syncTaskExecutionFeedback -> syncTaskExecutionFeedbackCollector
+                .addFeedback(syncTaskExecutionFeedback)));
+      } catch (RejectedExecutionException e) {
+        LOG.warn("BlockSyncTask {} for {} was rejected: {}",
+            blockSyncTask.getSyncTaskId(), blockSyncTask.getRemoteURI(),
+            e.getCause());
+      }
+    }
+  }
+
+  public SyncTaskExecutionFeedbackCollector getSyncTaskExecutionFeedbackCollector() {
+    return syncTaskExecutionFeedbackCollector;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06477abc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/syncservice/SyncTaskExecutionFeedbackCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/syncservice/SyncTaskExecutionFeedbackCollector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/syncservice/SyncTaskExecutionFeedbackCollector.java
new file mode 100644
index 0000000..41cd441
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/syncservice/SyncTaskExecutionFeedbackCollector.java
@@ -0,0 +1,54 @@
+/*
+ * 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.hdfs.server.datanode.syncservice;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hdfs.server.protocol.BlockSyncTaskExecutionFeedback;
+import org.apache.hadoop.hdfs.server.protocol.BulkSyncTaskExecutionFeedback;
+
+import java.util.List;
+
+/**
+ * DatanodeSyncTaskExecutionFeedbackCollector collects feedback for the
+ * sync service tracker to determine what has happened and report statistics.
+ */
+public class SyncTaskExecutionFeedbackCollector {
+
+  private List<BlockSyncTaskExecutionFeedback> collectedFeedback;
+
+  public SyncTaskExecutionFeedbackCollector() {
+    this.collectedFeedback = Lists.newArrayList();
+  }
+
+  public void addFeedback(BlockSyncTaskExecutionFeedback feedback) {
+    synchronized (this) {
+      collectedFeedback.add(feedback);
+    }
+  }
+
+  public BulkSyncTaskExecutionFeedback packageFeedbackForHeartbeat() {
+
+    List<BlockSyncTaskExecutionFeedback> feedbackForHeartbeat;
+
+    synchronized (this) {
+      feedbackForHeartbeat = collectedFeedback;
+      collectedFeedback = Lists.newArrayList();
+    }
+    return new BulkSyncTaskExecutionFeedback(feedbackForHeartbeat);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06477abc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/syncservice/executor/BlockSyncOperationExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/syncservice/executor/BlockSyncOperationExecutor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/syncservice/executor/BlockSyncOperationExecutor.java
new file mode 100644
index 0000000..7fde230
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/syncservice/executor/BlockSyncOperationExecutor.java
@@ -0,0 +1,122 @@
+/*
+ * 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.hdfs.server.datanode.syncservice.executor;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BBUploadHandle;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.MultipartUploader;
+import org.apache.hadoop.fs.MultipartUploaderFactory;
+import org.apache.hadoop.fs.PartHandle;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.BlockInputStream;
+import org.apache.hadoop.hdfs.BlockReader;
+import org.apache.hadoop.hdfs.DFSInputStream;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.protocol.BlockSyncTask;
+import org.apache.hadoop.hdfs.server.protocol.SyncTaskExecutionResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.SequenceInputStream;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.Enumeration;
+import java.util.List;
+import java.util.Vector;
+
+/**
+ * BlockSyncOperationExecutor writes the blocks to the sync service remote
+ * endpoint.
+ */
+public class BlockSyncOperationExecutor  {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockSyncOperationExecutor.class);
+
+  private Configuration conf;
+  private BiFunction<LocatedBlock, Configuration, BlockReader>
+      createBlockReader;
+  private Function<FileSystem, MultipartUploader> multipartUploaderSupplier;
+
+  @VisibleForTesting
+  BlockSyncOperationExecutor(Configuration conf,
+      BiFunction<LocatedBlock, Configuration, BlockReader> createBlockReader,
+      Function<FileSystem, MultipartUploader> multipartUploaderSupplier) {
+    this.conf = conf;
+    this.createBlockReader = createBlockReader;
+    this.multipartUploaderSupplier = multipartUploaderSupplier;
+  }
+
+  public static BlockSyncOperationExecutor createOnDataNode(Configuration conf,
+      BiFunction<LocatedBlock, Configuration, BlockReader> createBlockReader) {
+    return new BlockSyncOperationExecutor(conf,
+        createBlockReader,
+        fs -> {
+          try {
+            return MultipartUploaderFactory.get(fs, conf);
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        });
+  }
+
+  public SyncTaskExecutionResult execute(BlockSyncTask blockSyncTask)
+      throws Exception {
+    LOG.info("Executing MetadataSyncTask {} (on {})",
+        blockSyncTask.getSyncTaskId(), blockSyncTask.getRemoteURI());
+
+      return doMultiPartPart(
+          blockSyncTask.getRemoteURI(),
+          blockSyncTask.getLocatedBlocks(),
+          blockSyncTask.getPartNumber(),
+          blockSyncTask.getUploadHandle(),
+          blockSyncTask.getOffset(),
+          blockSyncTask.getLength());
+  }
+
+  private SyncTaskExecutionResult doMultiPartPart(URI uri,
+      List<LocatedBlock> locatedBlocks, int partNumber, byte[] uploadHandle,
+      int offset, long length) throws IOException {
+    FileSystem fs = FileSystem.get(uri, conf);
+    Path filePath = new Path(uri);
+    Vector<InputStream> inputStreams = new Vector<>(locatedBlocks.size());
+    for (int i = 0; i < locatedBlocks.size(); ++i) {
+      LocatedBlock locatedBlock = locatedBlocks.get(i);
+      BlockReader reader = createBlockReader.apply(locatedBlock, conf);
+      if (i == 0) {
+        reader.skip(offset);
+      }
+      BlockInputStream inputStream = new BlockInputStream(reader);
+      inputStreams.add(inputStream);
+    }
+    Enumeration<InputStream> streamEnumeration = inputStreams.elements();
+    SequenceInputStream inputStream =
+        new SequenceInputStream(streamEnumeration);
+    MultipartUploader mpu = multipartUploaderSupplier.apply(fs);
+    PartHandle partHandle = mpu.putPart(filePath, inputStream,
+        partNumber, BBUploadHandle.from(ByteBuffer.wrap(uploadHandle)), length);
+    return new SyncTaskExecutionResult(partHandle.bytes(), length);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06477abc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/syncservice/executor/BlockSyncReaderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/syncservice/executor/BlockSyncReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/syncservice/executor/BlockSyncReaderFactory.java
new file mode 100644
index 0000000..cc5eb5c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/syncservice/executor/BlockSyncReaderFactory.java
@@ -0,0 +1,92 @@
+/*
+ * 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.hdfs.server.datanode.syncservice.executor;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FsTracer;
+import org.apache.hadoop.hdfs.BlockReader;
+import org.apache.hadoop.hdfs.ClientContext;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.client.impl.BlockReaderFactory;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.Token;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.EnumSet;
+
+/**
+ * BlockSyncReaderFactory constructs a block reader in the Datanode for the
+ * Sync Command to read blocks that will be written to the synchronization
+ * remote endpoint.
+ */
+public class BlockSyncReaderFactory {
+
+  public static BlockReader createBlockReader(DataNode dataNode,
+      LocatedBlock locatedBlock, Configuration conf) throws IOException {
+    ClientContext clientContext = ClientContext.getFromConf(conf);
+    Token<BlockTokenIdentifier> accessToken = dataNode.getBlockAccessToken(
+        locatedBlock.getBlock(),
+        EnumSet.of(BlockTokenIdentifier.AccessMode.READ),
+        locatedBlock.getStorageTypes(), locatedBlock.getStorageIDs());
+
+    DatanodeInfo datanodeInfo = locatedBlock.getLocations()[0];
+
+    Socket socked = NetUtils.getDefaultSocketFactory(conf).createSocket();
+    InetSocketAddress resolvedAddress =
+        datanodeInfo.getResolvedAddress();
+    socked.connect(resolvedAddress);
+
+    return new BlockReaderFactory(new DfsClientConf(conf))
+        .setConfiguration(conf)
+        .setBlock(locatedBlock.getBlock())
+        .setBlockToken(accessToken)
+        .setStartOffset(0)
+        .setLength(locatedBlock.getBlock().getNumBytes())
+        .setInetSocketAddress(datanodeInfo.getResolvedAddress())
+        .setVerifyChecksum(true)
+        .setDatanodeInfo(datanodeInfo)
+        .setClientName("BlockSyncOperationExecutor")
+        .setCachingStrategy(CachingStrategy.newDefaultStrategy())
+        .setRemotePeerFactory((addr, blockToken, datanodeId) -> {
+          Peer peer = null;
+          Socket sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
+          try {
+            sock.connect(addr, HdfsConstants.READ_TIMEOUT);
+            sock.setSoTimeout(HdfsConstants.READ_TIMEOUT);
+            peer = DFSUtilClient.peerFromSocket(sock);
+          } finally {
+            if (peer == null) {
+              IOUtils.closeQuietly(sock);
+            }
+          }
+          return peer;
+        })
+        .setClientCacheContext(clientContext)
+        .build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06477abc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/syncservice/executor/BlockSyncTaskRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/syncservice/executor/BlockSyncTaskRunner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/syncservice/executor/BlockSyncTaskRunner.java
new file mode 100644
index 0000000..660e39e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/syncservice/executor/BlockSyncTaskRunner.java
@@ -0,0 +1,69 @@
+/*
+ * 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.hdfs.server.datanode.syncservice.executor;
+
+import org.apache.hadoop.hdfs.server.protocol.BlockSyncTask;
+import org.apache.hadoop.hdfs.server.protocol.BlockSyncTaskExecutionFeedback;
+import org.apache.hadoop.hdfs.server.protocol.SyncTaskExecutionResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * BlockSyncTaskRunner glues together the sync task and the feedback reporting.
+ */
+import java.util.function.Consumer;
+
+public class BlockSyncTaskRunner implements Runnable {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockSyncTaskRunner.class);
+
+  private BlockSyncTask blockSyncTask;
+  private BlockSyncOperationExecutor syncOperationExecutor;
+
+  private Consumer<BlockSyncTaskExecutionFeedback> publishOutcomeCallback;
+
+  public BlockSyncTaskRunner(BlockSyncTask blockSyncTask,
+      BlockSyncOperationExecutor syncOperationExecutor,
+      Consumer<BlockSyncTaskExecutionFeedback> publishOutcomeCallback) {
+    this.blockSyncTask = blockSyncTask;
+    this.syncOperationExecutor = syncOperationExecutor;
+    this.publishOutcomeCallback = publishOutcomeCallback;
+  }
+
+  @Override
+  public void run() {
+    LOG.info("Executing BlockyncTask {} (on {})",
+        blockSyncTask.getSyncTaskId(), blockSyncTask.getRemoteURI());
+    try {
+      SyncTaskExecutionResult result =
+          syncOperationExecutor.execute(blockSyncTask);
+      publishOutcomeCallback.accept(BlockSyncTaskExecutionFeedback
+          .finishedSuccessfully(blockSyncTask.getSyncTaskId(),
+              blockSyncTask.getSyncMountId(),
+              result));
+    } catch (Exception e) {
+      LOG.error(
+          String.format("Exception executing MetadataSyncTask %s (on %s)",
+              blockSyncTask.getSyncTaskId(), blockSyncTask.getRemoteURI()), e);
+      publishOutcomeCallback.accept(BlockSyncTaskExecutionFeedback
+          .failedWithException(blockSyncTask.getSyncTaskId(),
+              blockSyncTask.getSyncMountId(), e));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06477abc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockInputStream.java
new file mode 100644
index 0000000..43d4881
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockInputStream.java
@@ -0,0 +1,84 @@
+/*
+ * 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.hdfs;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Test the BlockInputStream facade.
+ */
+@RunWith(MockitoJUnitRunner.class)
+public class TestBlockInputStream {
+  @Mock
+  private BlockReader blockReaderMock;
+
+  @Test
+  public void testBlockInputStreamReadChar() {
+    BlockInputStream is = new BlockInputStream(blockReaderMock);
+
+    try {
+      when(blockReaderMock.read(any(), eq(0), eq(1)))
+          .thenReturn(32);
+      // Making the mock perform the side effect of writing to buf is nasty.
+      is.read();
+      verify(blockReaderMock, times(1)).read(any(), eq(0), eq(1));
+    } catch (IOException e) {
+      fail("Could not even mock out read function.");
+    }
+  }
+
+  @Test
+  public void testBlockInputStreamReadBuf() {
+    BlockInputStream is = new BlockInputStream(blockReaderMock);
+
+    try {
+      byte[] buf = new byte[1024];
+      when(blockReaderMock.read(buf, 0, buf.length)).thenReturn(1024);
+      is.read(buf, 0, buf.length);
+      verify(blockReaderMock, times(1)).read(buf, 0, buf.length);
+    } catch (IOException e) {
+      fail("Could not even mock out read function.");
+    }
+  }
+
+  @Test
+  public void testBlockInputStreamSkip() {
+    BlockInputStream is = new BlockInputStream(blockReaderMock);
+
+    try {
+      when(blockReaderMock.skip(10)).thenReturn(10L);
+      long ret = is.skip(10);
+      assertEquals(10, ret);
+      verify(blockReaderMock, times(1)).skip(10L);
+    } catch (IOException e) {
+      fail("Could not even mock out skip function.");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06477abc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/syncservice/executor/TestBlockSyncOperationExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/syncservice/executor/TestBlockSyncOperationExecutor.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/syncservice/executor/TestBlockSyncOperationExecutor.java
new file mode 100644
index 0000000..e16d086
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/syncservice/executor/TestBlockSyncOperationExecutor.java
@@ -0,0 +1,94 @@
+/*
+ * 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.hdfs.server.datanode.syncservice.executor;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BBPartHandle;
+import org.apache.hadoop.fs.MultipartUploader;
+import org.apache.hadoop.fs.PartHandle;
+import org.apache.hadoop.hdfs.BlockReader;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.protocol.BlockSyncTask;
+import org.apache.hadoop.hdfs.server.protocol.SyncTaskExecutionResult;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestBlockSyncOperationExecutor {
+
+  @Mock
+  private BlockReader blockReaderMock;
+
+  @Mock
+  private MultipartUploader multipartUploaderMock;
+
+  @Test
+  public void executeMultipartPutFileSyncTask() throws Exception {
+    long blockLength = 42L;
+    Configuration conf = new Configuration();
+    BlockSyncOperationExecutor blockSyncOperationExecutor =
+        new BlockSyncOperationExecutor(conf,
+            (locatedBlock, config) -> blockReaderMock,
+            fs -> multipartUploaderMock);
+    String uploadHandleStr = "uploadHandle";
+    byte[] uploadHandle = uploadHandleStr.getBytes();
+    ByteBuffer byteBuffer = ByteBuffer.wrap(uploadHandle);
+    PartHandle partHandle = BBPartHandle.from(byteBuffer);
+    when(multipartUploaderMock.putPart(any(), any(), anyInt(), any(),
+        anyLong())).thenReturn(partHandle);
+    UUID syncTaskId = UUID.randomUUID();
+    URI remoteUri = new URI("remoteUri");
+    String syncMountId = "syncMountId";
+    Block block = new Block(42L, blockLength, 44L);
+    ExtendedBlock extendedBlock1 = new ExtendedBlock("poolId", block);
+    LocatedBlock locatedBlock = new LocatedBlock(extendedBlock1, null);
+    List<LocatedBlock> locatedBlocks = Lists.newArrayList(locatedBlock);
+    Integer partNumber = 85;
+    final int offset = 0;
+    final long length = locatedBlock.getBlockSize();
+
+
+    BlockSyncTask blockSyncTask = new BlockSyncTask(syncTaskId, remoteUri,
+        locatedBlocks, partNumber, uploadHandle, offset, length, syncMountId);
+
+    SyncTaskExecutionResult result =
+        blockSyncOperationExecutor.execute(blockSyncTask);
+
+    assertThat(result).isNotNull();
+    Long actualLength = result.getNumberOfBytes();
+    assertThat(actualLength).isEqualTo(blockLength);
+    assertThat(result.getResult()).isEqualTo(partHandle.bytes());
+  }
+}
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[17/50] [abbrv] hadoop git commit: HDDS-382. Remove RatisTestHelper#RatisTestSuite constructor argument and fix checkstyle in ContainerTestHelper, GenericTestUtils Contributed by Nandakumar.

Posted by eh...@apache.org.
HDDS-382. Remove RatisTestHelper#RatisTestSuite constructor argument and fix checkstyle in ContainerTestHelper, GenericTestUtils
Contributed by Nandakumar.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c5629d54
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c5629d54
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c5629d54

Branch: refs/heads/HDFS-12090
Commit: c5629d546d64091a14560df488a7f797a150337e
Parents: 33f42ef
Author: Anu Engineer <ae...@apache.org>
Authored: Tue Aug 28 14:06:19 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Tue Aug 28 14:06:19 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdds/scm/XceiverClient.java   |  6 +--
 .../hadoop/hdds/scm/XceiverClientGrpc.java      |  6 +--
 .../hadoop/hdds/scm/XceiverClientManager.java   |  2 +-
 .../hdds/scm/storage/ChunkInputStream.java      |  7 +--
 .../hdds/scm/storage/ChunkOutputStream.java     |  4 +-
 .../org/apache/hadoop/hdds/client/BlockID.java  |  5 +-
 .../hadoop/hdds/scm/XceiverClientSpi.java       |  2 -
 .../common/helpers/AllocatedBlock.java          |  4 +-
 .../container/common/helpers/ContainerInfo.java | 12 ++---
 .../common/helpers/ContainerWithPipeline.java   |  7 +--
 .../scm/container/common/helpers/Pipeline.java  | 11 ++---
 .../StorageContainerLocationProtocol.java       |  6 ++-
 ...rLocationProtocolClientSideTranslatorPB.java | 21 ++++----
 .../scm/storage/ContainerProtocolCalls.java     |  6 +--
 .../org/apache/hadoop/ozone/OzoneConsts.java    |  5 --
 .../ozone/container/common/helpers/KeyData.java |  8 ++--
 .../apache/hadoop/utils/HddsVersionInfo.java    |  6 ++-
 .../apache/hadoop/utils/TestMetadataStore.java  |  1 -
 .../hadoop/ozone/HddsDatanodeService.java       |  3 +-
 .../common/helpers/ContainerUtils.java          | 22 ++++-----
 .../container/common/impl/ContainerSet.java     |  2 +-
 .../common/impl/OpenContainerBlockMap.java      | 19 ++++----
 .../server/ratis/XceiverServerRatis.java        |  6 +--
 .../keyvalue/interfaces/KeyManager.java         |  4 +-
 .../ozone/protocol/commands/CommandStatus.java  | 16 +++----
 .../ozone/container/common/ScmTestMock.java     |  6 ++-
 .../common/interfaces/TestHandler.java          |  7 ---
 .../endpoint/TestHeartbeatEndpointTask.java     |  2 -
 .../TestRoundRobinVolumeChoosingPolicy.java     |  5 +-
 .../container/ozoneimpl/TestOzoneContainer.java |  3 +-
 .../hadoop/hdds/server/events/EventWatcher.java |  6 ++-
 .../hdds/server/events/TestEventQueue.java      |  3 --
 .../hadoop/hdds/scm/block/BlockManagerImpl.java | 18 +++----
 .../hdds/scm/block/DeletedBlockLogImpl.java     |  3 +-
 .../hdds/scm/block/SCMBlockDeletingService.java |  4 +-
 .../container/CloseContainerEventHandler.java   |  4 +-
 .../hdds/scm/container/ContainerMapping.java    |  4 +-
 .../scm/container/ContainerStateManager.java    |  7 +--
 .../replication/ReplicationManager.java         |  2 +-
 .../scm/container/states/ContainerStateMap.java |  2 +-
 .../hdds/scm/node/states/Node2ContainerMap.java |  4 +-
 .../scm/node/states/NodeNotFoundException.java  |  2 -
 .../hdds/scm/node/states/ReportResult.java      |  3 +-
 .../hdds/scm/pipelines/Node2PipelineMap.java    | 50 +++++++++-----------
 .../hdds/scm/pipelines/PipelineManager.java     |  6 +--
 .../hdds/scm/pipelines/PipelineSelector.java    |  7 +--
 .../scm/server/SCMClientProtocolServer.java     |  3 +-
 .../org/apache/hadoop/hdds/scm/TestUtils.java   |  8 ++--
 .../hadoop/hdds/scm/block/TestBlockManager.java |  1 -
 .../hdds/scm/block/TestDeletedBlockLog.java     |  7 +--
 .../command/TestCommandStatusReportHandler.java | 22 ++++-----
 .../TestCloseContainerEventHandler.java         |  1 -
 .../scm/container/TestContainerMapping.java     |  7 +--
 .../container/TestContainerReportHandler.java   |  2 +-
 .../TestSCMContainerPlacementCapacity.java      |  8 ++--
 .../TestSCMContainerPlacementRandom.java        |  4 +-
 .../replication/TestReplicationManager.java     | 11 ++---
 .../replication/TestReplicationQueue.java       |  4 +-
 .../hdds/scm/node/TestContainerPlacement.java   |  5 +-
 .../hadoop/hdds/scm/node/TestNodeManager.java   |  3 +-
 .../hdds/scm/node/TestNodeReportHandler.java    |  3 +-
 .../ozone/container/common/TestEndPoint.java    |  9 ++--
 .../placement/TestContainerPlacement.java       |  6 ++-
 .../apache/hadoop/ozone/client/ObjectStore.java |  7 ++-
 .../hdds/scm/pipeline/TestPipelineClose.java    |  4 --
 .../apache/hadoop/ozone/RatisTestHelper.java    |  8 ++--
 .../TestStorageContainerManagerHelper.java      |  2 -
 .../rpc/TestCloseContainerHandlingByClient.java |  3 +-
 .../ozone/container/ContainerTestHelper.java    |  2 -
 .../common/impl/TestContainerPersistence.java   |  1 -
 .../ozoneimpl/TestOzoneContainerRatis.java      |  3 +-
 .../container/ozoneimpl/TestRatisManager.java   |  4 +-
 .../hadoop/ozone/scm/TestAllocateContainer.java |  2 -
 .../hadoop/ozone/web/TestOzoneWebAccess.java    |  1 -
 .../hadoop/ozone/web/client/TestBuckets.java    |  5 +-
 .../ozone/web/client/TestBucketsRatis.java      |  2 +-
 .../hadoop/ozone/web/client/TestKeysRatis.java  |  2 +-
 .../apache/hadoop/ozone/om/KeyManagerImpl.java  | 13 -----
 .../apache/hadoop/fs/ozone/OzoneFileSystem.java |  2 -
 .../org/apache/hadoop/ozone/scm/cli/SQLCLI.java |  4 --
 80 files changed, 226 insertions(+), 282 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClient.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClient.java
index 097af17..5022618 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClient.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClient.java
@@ -38,7 +38,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
-import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Semaphore;
@@ -188,13 +187,12 @@ public class XceiverClient extends XceiverClientSpi {
   /**
    * Create a pipeline.
    *
-   * @param pipeline -  pipeline to be created.
+   * @param ignored -  pipeline to be created.
    */
   @Override
-  public void createPipeline(Pipeline pipeline)
+  public void createPipeline(Pipeline ignored)
       throws IOException {
     // For stand alone pipeline, there is no notion called setup pipeline.
-    return;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
index 35bc932..e2416c2 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
@@ -38,7 +38,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
-import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Semaphore;
@@ -218,13 +217,12 @@ public class XceiverClientGrpc extends XceiverClientSpi {
   /**
    * Create a pipeline.
    *
-   * @param pipeline -  pipeline to be created.
+   * @param ignored -  pipeline to be created.
    */
   @Override
-  public void createPipeline(Pipeline pipeline)
+  public void createPipeline(Pipeline ignored)
       throws IOException {
     // For stand alone pipeline, there is no notion called setup pipeline.
-    return;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java
index 125e5d5..9762406 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java
@@ -154,7 +154,7 @@ public class XceiverClientManager implements Closeable {
               break;
             case CHAINED:
             default:
-              throw new IOException ("not implemented" + pipeline.getType());
+              throw new IOException("not implemented" + pipeline.getType());
             }
             client.connect();
             return client;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java
index 020c684..a969b68 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java
@@ -65,7 +65,8 @@ public class ChunkInputStream extends InputStream implements Seekable {
    * @param chunks list of chunks to read
    * @param traceID container protocol call traceID
    */
-  public ChunkInputStream(BlockID blockID, XceiverClientManager xceiverClientManager,
+  public ChunkInputStream(
+      BlockID blockID, XceiverClientManager xceiverClientManager,
       XceiverClientSpi xceiverClient, List<ChunkInfo> chunks, String traceID) {
     this.blockID = blockID;
     this.traceID = traceID;
@@ -211,8 +212,8 @@ public class ChunkInputStream extends InputStream implements Seekable {
     if (pos < 0 || (chunks.size() == 0 && pos > 0)
         || pos >= chunkOffset[chunks.size() - 1] + chunks.get(chunks.size() - 1)
         .getLen()) {
-      throw new EOFException(
-          "EOF encountered pos: " + pos + " container key: " + blockID.getLocalID());
+      throw new EOFException("EOF encountered pos: " + pos + " container key: "
+          + blockID.getLocalID());
     }
     if (chunkIndex == -1) {
       chunkIndex = Arrays.binarySearch(chunkOffset, pos);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java
index 7309434..f2df3fa 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java
@@ -76,8 +76,8 @@ public class ChunkOutputStream extends OutputStream {
    * @param chunkSize chunk size
    */
   public ChunkOutputStream(BlockID blockID, String key,
-       XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient,
-       String traceID, int chunkSize) {
+       XceiverClientManager xceiverClientManager,
+       XceiverClientSpi xceiverClient, String traceID, int chunkSize) {
     this.blockID = blockID;
     this.key = key;
     this.traceID = traceID;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java
index 74e90e9..8149740 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/BlockID.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import java.util.Objects;
 
 /**
- * BlockID of ozone (containerID  localID)
+ * BlockID of ozone (containerID  localID).
  */
 public class BlockID {
   private long containerID;
@@ -65,7 +65,8 @@ public class BlockID {
         setContainerID(containerID).setLocalID(localID).build();
   }
 
-  public static BlockID getFromProtobuf(ContainerProtos.DatanodeBlockID blockID) {
+  public static BlockID getFromProtobuf(
+      ContainerProtos.DatanodeBlockID blockID) {
     return new BlockID(blockID.getContainerID(),
         blockID.getLocalID());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java
index b29e73d..b3b0da2 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdds.scm;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .ContainerCommandRequestProto;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
@@ -29,7 +28,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicInteger;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/AllocatedBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/AllocatedBlock.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/AllocatedBlock.java
index 9b89469..63781a8 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/AllocatedBlock.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/AllocatedBlock.java
@@ -43,8 +43,8 @@ public final class AllocatedBlock {
       return this;
     }
 
-    public Builder setBlockID(BlockID blockID) {
-      this.blockID = blockID;
+    public Builder setBlockID(BlockID blockId) {
+      this.blockID = blockId;
       return this;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java
index 311c118..465f4b9 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerInfo.java
@@ -396,13 +396,13 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
     private ReplicationType replicationType;
 
     public Builder setReplicationType(
-        ReplicationType replicationType) {
-      this.replicationType = replicationType;
+        ReplicationType repType) {
+      this.replicationType = repType;
       return this;
     }
 
-    public Builder setPipelineID(PipelineID pipelineID) {
-      this.pipelineID = pipelineID;
+    public Builder setPipelineID(PipelineID pipelineId) {
+      this.pipelineID = pipelineId;
       return this;
     }
 
@@ -447,8 +447,8 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
       return this;
     }
 
-    public Builder setDeleteTransactionId(long deleteTransactionId) {
-      this.deleteTransactionId = deleteTransactionId;
+    public Builder setDeleteTransactionId(long deleteTransactionID) {
+      this.deleteTransactionId = deleteTransactionID;
       return this;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerWithPipeline.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerWithPipeline.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerWithPipeline.java
index e71d429..64f42b3 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerWithPipeline.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/ContainerWithPipeline.java
@@ -26,8 +26,8 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 /**
  * Class wraps ozone container info.
  */
-public class ContainerWithPipeline
-    implements Comparator<ContainerWithPipeline>, Comparable<ContainerWithPipeline> {
+public class ContainerWithPipeline implements Comparator<ContainerWithPipeline>,
+    Comparable<ContainerWithPipeline> {
 
   private final ContainerInfo containerInfo;
   private final Pipeline pipeline;
@@ -45,7 +45,8 @@ public class ContainerWithPipeline
     return pipeline;
   }
 
-  public static ContainerWithPipeline fromProtobuf(HddsProtos.ContainerWithPipeline allocatedContainer) {
+  public static ContainerWithPipeline fromProtobuf(
+      HddsProtos.ContainerWithPipeline allocatedContainer) {
     return new ContainerWithPipeline(
         ContainerInfo.fromProtobuf(allocatedContainer.getContainerInfo()),
         Pipeline.getFromProtoBuf(allocatedContainer.getPipeline()));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/Pipeline.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/Pipeline.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/Pipeline.java
index 9270468..6757262 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/Pipeline.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/container/common/helpers/Pipeline.java
@@ -169,8 +169,8 @@ public class Pipeline {
    */
   public List<String> getDatanodeHosts() {
     List<String> dataHosts = new ArrayList<>();
-    for (DatanodeDetails id :getDatanodes().values()) {
-      dataHosts.add(id.getHostName());
+    for (DatanodeDetails datanode : getDatanodes().values()) {
+      dataHosts.add(datanode.getHostName());
     }
     return dataHosts;
   }
@@ -219,7 +219,7 @@ public class Pipeline {
    * Update the State of the pipeline.
    */
   public void setLifeCycleState(HddsProtos.LifeCycleState nextState) {
-     lifeCycleState = nextState;
+    lifeCycleState = nextState;
   }
 
   /**
@@ -244,9 +244,8 @@ public class Pipeline {
   public String toString() {
     final StringBuilder b = new StringBuilder(getClass().getSimpleName())
         .append("[");
-    getDatanodes().keySet().stream()
-        .forEach(id -> b.
-            append(id.endsWith(getLeaderID()) ? "*" + id : id));
+    getDatanodes().keySet().forEach(
+        node -> b.append(node.endsWith(getLeaderID()) ? "*" + id : id));
     b.append(" id:").append(id);
     if (getType() != null) {
       b.append(" type:").append(getType().toString());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
index 581fbd0..c55062b 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
@@ -38,7 +38,8 @@ public interface StorageContainerLocationProtocol {
    * set of datanodes that should be used creating this container.
    *
    */
-  ContainerWithPipeline allocateContainer(HddsProtos.ReplicationType replicationType,
+  ContainerWithPipeline allocateContainer(
+      HddsProtos.ReplicationType replicationType,
       HddsProtos.ReplicationFactor factor, String owner)
       throws IOException;
 
@@ -61,7 +62,8 @@ public interface StorageContainerLocationProtocol {
    * @return ContainerWithPipeline - the container info with the pipeline.
    * @throws IOException
    */
-  ContainerWithPipeline getContainerWithPipeline(long containerID) throws IOException;
+  ContainerWithPipeline getContainerWithPipeline(long containerID)
+      throws IOException;
 
   /**
    * Ask SCM a list of containers with a range of container names

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
index ac12ea2..0441469 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
@@ -97,8 +97,9 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB
    * @throws IOException
    */
   @Override
-  public ContainerWithPipeline allocateContainer(HddsProtos.ReplicationType type,
-      HddsProtos.ReplicationFactor factor, String owner) throws IOException {
+  public ContainerWithPipeline allocateContainer(
+      HddsProtos.ReplicationType type, HddsProtos.ReplicationFactor factor,
+      String owner) throws IOException {
 
     ContainerRequestProto request = ContainerRequestProto.newBuilder()
         .setReplicationFactor(factor)
@@ -116,7 +117,8 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB
       throw new IOException(response.hasErrorMessage() ?
           response.getErrorMessage() : "Allocate container failed.");
     }
-    return ContainerWithPipeline.fromProtobuf(response.getContainerWithPipeline());
+    return ContainerWithPipeline.fromProtobuf(
+        response.getContainerWithPipeline());
   }
 
   public ContainerInfo getContainer(long containerID) throws IOException {
@@ -138,17 +140,18 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB
   /**
    * {@inheritDoc}
    */
-  public ContainerWithPipeline getContainerWithPipeline(long containerID) throws IOException {
+  public ContainerWithPipeline getContainerWithPipeline(long containerID)
+      throws IOException {
     Preconditions.checkState(containerID >= 0,
         "Container ID cannot be negative");
-    GetContainerWithPipelineRequestProto request = GetContainerWithPipelineRequestProto
-        .newBuilder()
-        .setContainerID(containerID)
-        .build();
+    GetContainerWithPipelineRequestProto request =
+        GetContainerWithPipelineRequestProto.newBuilder()
+            .setContainerID(containerID).build();
     try {
       GetContainerWithPipelineResponseProto response =
           rpcProxy.getContainerWithPipeline(NULL_RPC_CONTROLLER, request);
-      return ContainerWithPipeline.fromProtobuf(response.getContainerWithPipeline());
+      return ContainerWithPipeline.fromProtobuf(
+          response.getContainerWithPipeline());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
index abad9e3..1f2fafb 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java
@@ -113,8 +113,8 @@ public final class ContainerProtocolCalls  {
    * @throws IOException if there is an I/O error while performing the call
    */
   public static ContainerProtos.GetCommittedBlockLengthResponseProto
-  getCommittedBlockLength(
-      XceiverClientSpi xceiverClient, BlockID blockID, String traceID)
+      getCommittedBlockLength(
+          XceiverClientSpi xceiverClient, BlockID blockID, String traceID)
       throws IOException {
     ContainerProtos.GetCommittedBlockLengthRequestProto.Builder
         getBlockLengthRequestBuilder =
@@ -375,7 +375,7 @@ public final class ContainerProtocolCalls  {
   }
 
   /**
-   * Reads the data given the blockID
+   * Reads the data given the blockID.
    *
    * @param client
    * @param blockID - ID of the block

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index ab6df92..15366fb 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -110,11 +110,6 @@ public final class OzoneConsts {
     }
   }
 
-  /**
-   * Ozone handler types.
-   */
-  public static final String OZONE_HANDLER_DISTRIBUTED = "distributed";
-
   public static final String DELETING_KEY_PREFIX = "#deleting#";
   public static final String DELETED_KEY_PREFIX = "#deleted#";
   public static final String DELETE_TRANSACTION_KEY_PREFIX = "#delTX#";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java
index 84a6f71..ee27021 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyData.java
@@ -37,7 +37,8 @@ public class KeyData {
 
   /**
    * Represent a list of chunks.
-   * In order to reduce memory usage, chunkList is declared as an {@link Object}.
+   * In order to reduce memory usage, chunkList is declared as an
+   * {@link Object}.
    * When #elements == 0, chunkList is null.
    * When #elements == 1, chunkList refers to the only element.
    * When #elements > 1, chunkList refers to the list.
@@ -157,7 +158,7 @@ public class KeyData {
   }
 
   /**
-   * Adds chinkInfo to the list
+   * Adds chinkInfo to the list.
    */
   public void addChunk(ContainerProtos.ChunkInfo chunkInfo) {
     if (chunkList == null) {
@@ -237,7 +238,8 @@ public class KeyData {
     } else {
       final int n = chunks.size();
       chunkList = n == 0? null: n == 1? chunks.get(0): chunks;
-      size = chunks.parallelStream().mapToLong(ContainerProtos.ChunkInfo::getLen).sum();
+      size = chunks.parallelStream().mapToLong(
+          ContainerProtos.ChunkInfo::getLen).sum();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/HddsVersionInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/HddsVersionInfo.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/HddsVersionInfo.java
index 59b9de6..e7f697a 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/HddsVersionInfo.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/HddsVersionInfo.java
@@ -36,7 +36,8 @@ import java.util.Properties;
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class HddsVersionInfo {
-  private static final Logger LOG = LoggerFactory.getLogger(HddsVersionInfo.class);
+  private static final Logger LOG = LoggerFactory.getLogger(
+      HddsVersionInfo.class);
 
   private Properties info;
 
@@ -95,7 +96,8 @@ public class HddsVersionInfo {
     return info.getProperty("protocVersion", "Unknown");
   }
 
-  private static HddsVersionInfo HDDS_VERSION_INFO = new HddsVersionInfo("hdds");
+  private static final HddsVersionInfo HDDS_VERSION_INFO =
+      new HddsVersionInfo("hdds");
   /**
    * Get the HDDS version.
    * @return the Hdds version string, eg. "0.6.3-dev"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
index 1bce022..30fc7f3 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/TestMetadataStore.java
@@ -55,7 +55,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
index f359e72..348196c 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
@@ -232,7 +232,8 @@ public class HddsDatanodeService implements ServicePlugin {
 
   public static void main(String[] args) {
     try {
-      if (DFSUtil.parseHelpArgument(args, "Starts HDDS Datanode", System.out, false)) {
+      if (DFSUtil.parseHelpArgument(
+          args, "Starts HDDS Datanode", System.out, false)) {
         System.exit(0);
       }
       Configuration conf = new OzoneConfiguration();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
index 469c969..d96849e 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerUtils.java
@@ -43,7 +43,6 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
-import java.nio.file.Path;
 import java.nio.file.Paths;
 import org.yaml.snakeyaml.Yaml;
 
@@ -54,8 +53,6 @@ import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
     .Result.NO_SUCH_ALGORITHM;
 import static org.apache.hadoop.ozone.container.common.impl.ContainerData
     .CHARSET_ENCODING;
-import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_EXTENSION;
-
 
 /**
  * A set of helper functions to create proper responses.
@@ -75,14 +72,13 @@ public final class ContainerUtils {
    * @return ContainerCommand Response Builder.
    */
   public static ContainerCommandResponseProto.Builder
-  getContainerCommandResponse(
-      ContainerCommandRequestProto request, Result result, String message) {
-    return
-        ContainerCommandResponseProto.newBuilder()
-            .setCmdType(request.getCmdType())
-            .setTraceID(request.getTraceID())
-            .setResult(result)
-            .setMessage(message);
+      getContainerCommandResponse(
+          ContainerCommandRequestProto request, Result result, String message) {
+    return ContainerCommandResponseProto.newBuilder()
+        .setCmdType(request.getCmdType())
+        .setTraceID(request.getTraceID())
+        .setResult(result)
+        .setMessage(message);
   }
 
   /**
@@ -287,7 +283,7 @@ public final class ContainerUtils {
   }
 
   /**
-   * Get the .container file from the containerBaseDir
+   * Get the .container file from the containerBaseDir.
    * @param containerBaseDir container base directory. The name of this
    *                         directory is same as the containerID
    * @return the .container file
@@ -301,7 +297,7 @@ public final class ContainerUtils {
   }
 
   /**
-   * ContainerID can be decoded from the container base directory name
+   * ContainerID can be decoded from the container base directory name.
    */
   public static long getContainerID(File containerBaseDir) {
     return Long.parseLong(containerBaseDir.getName());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
index f92ab52..7f4f147 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java
@@ -132,7 +132,7 @@ public class ContainerSet {
   }
 
   /**
-   * Return a copy of the containerMap
+   * Return a copy of the containerMap.
    * @return containerMap
    */
   public Map<Long, Container> getContainerMap() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
index ab7789b..1ef3d0d 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/OpenContainerBlockMap.java
@@ -45,10 +45,12 @@ public class OpenContainerBlockMap {
   /**
    * Map: localId -> KeyData.
    *
-   * In order to support {@link #getAll()}, the update operations are synchronized.
+   * In order to support {@link #getAll()}, the update operations are
+   * synchronized.
    */
   static class KeyDataMap {
-    private final ConcurrentMap<Long, KeyData> blocks = new ConcurrentHashMap<>();
+    private final ConcurrentMap<Long, KeyData> blocks =
+        new ConcurrentHashMap<>();
 
     KeyData get(long localId) {
       return blocks.get(localId);
@@ -59,7 +61,8 @@ public class OpenContainerBlockMap {
       return blocks.size();
     }
 
-    synchronized KeyData computeIfAbsent(long localId, Function<Long, KeyData> f) {
+    synchronized KeyData computeIfAbsent(
+        long localId, Function<Long, KeyData> f) {
       return blocks.computeIfAbsent(localId, f);
     }
 
@@ -76,7 +79,8 @@ public class OpenContainerBlockMap {
    *
    * For now, we will track all open blocks of a container in the blockMap.
    */
-  private final ConcurrentMap<Long, KeyDataMap> containers = new ConcurrentHashMap<>();
+  private final ConcurrentMap<Long, KeyDataMap> containers =
+      new ConcurrentHashMap<>();
 
   /**
    * Removes the Container matching with specified containerId.
@@ -109,7 +113,7 @@ public class OpenContainerBlockMap {
   }
 
   /**
-   * returns the list of open to the openContainerBlockMap
+   * Returns the list of open to the openContainerBlockMap.
    * @param containerId container id
    * @return List of open Keys(blocks)
    */
@@ -130,15 +134,14 @@ public class OpenContainerBlockMap {
   }
 
   /**
-   * Returns true if the block exists in the map, false otherwise
+   * Returns true if the block exists in the map, false otherwise.
    *
    * @param blockID
    * @return True, if it exists, false otherwise
    */
   public boolean checkIfBlockExists(BlockID blockID) {
     KeyDataMap keyDataMap = containers.get(blockID.getContainerID());
-    return keyDataMap == null ? false :
-        keyDataMap.get(blockID.getLocalID()) != null;
+    return keyDataMap != null && keyDataMap.get(blockID.getLocalID()) != null;
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
index f8c7af2..8256722 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
@@ -71,10 +71,10 @@ import java.util.concurrent.atomic.AtomicLong;
  */
 public final class XceiverServerRatis implements XceiverServerSpi {
   static final Logger LOG = LoggerFactory.getLogger(XceiverServerRatis.class);
-  private static final AtomicLong callIdCounter = new AtomicLong();
+  private static final AtomicLong CALL_ID_COUNTER = new AtomicLong();
 
   private static long nextCallId() {
-    return callIdCounter.getAndIncrement() & Long.MAX_VALUE;
+    return CALL_ID_COUNTER.getAndIncrement() & Long.MAX_VALUE;
   }
 
   private final int port;
@@ -307,6 +307,6 @@ public final class XceiverServerRatis implements XceiverServerSpi {
       RaftClientRequest.Type type) {
     return new RaftClientRequest(clientId, server.getId(),
         PipelineID.getFromProtobuf(pipelineID).getRaftGroupID(),
-        nextCallId(),0, Message.valueOf(request.toByteString()), type);
+        nextCallId(), 0, Message.valueOf(request.toByteString()), type);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
index 37871be..84f771a 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/KeyManager.java
@@ -67,8 +67,8 @@ public interface KeyManager {
    * @param count    - Number of keys to return.
    * @return List of Keys that match the criteria.
    */
-  List<KeyData> listKey(Container container, long startLocalID, int count) throws
-      IOException;
+  List<KeyData> listKey(Container container, long startLocalID, int count)
+      throws IOException;
 
   /**
    * Returns the last committed block length for the block.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CommandStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CommandStatus.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CommandStatus.java
index bf99700..32cf7c2 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CommandStatus.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/CommandStatus.java
@@ -109,23 +109,23 @@ public class CommandStatus {
       return new CommandStatusBuilder();
     }
 
-    public CommandStatusBuilder setType(Type type) {
-      this.type = type;
+    public CommandStatusBuilder setType(Type commandType) {
+      this.type = commandType;
       return this;
     }
 
-    public CommandStatusBuilder setCmdId(Long cmdId) {
-      this.cmdId = cmdId;
+    public CommandStatusBuilder setCmdId(Long commandId) {
+      this.cmdId = commandId;
       return this;
     }
 
-    public CommandStatusBuilder setStatus(Status status) {
-      this.status = status;
+    public CommandStatusBuilder setStatus(Status commandStatus) {
+      this.status = commandStatus;
       return this;
     }
 
-    public CommandStatusBuilder setMsg(String msg) {
-      this.msg = msg;
+    public CommandStatusBuilder setMsg(String message) {
+      this.msg = message;
       return this;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
index 8827d1d..751775f 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ScmTestMock.java
@@ -193,11 +193,13 @@ public class ScmTestMock implements StorageContainerDatanodeProtocol {
     rpcCount.incrementAndGet();
     heartbeatCount.incrementAndGet();
     if(heartbeat.hasCommandStatusReport()){
-      cmdStatusList.addAll(heartbeat.getCommandStatusReport().getCmdStatusList());
+      cmdStatusList.addAll(heartbeat.getCommandStatusReport()
+          .getCmdStatusList());
       commandStatusReport.incrementAndGet();
     }
     sleepIfNeeded();
-    return SCMHeartbeatResponseProto.newBuilder().addAllCommands(scmCommandRequests)
+    return SCMHeartbeatResponseProto.newBuilder().addAllCommands(
+        scmCommandRequests)
         .setDatanodeUUID(heartbeat.getDatanodeDetails().getUuid())
         .build();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java
index c9733f8..b658295 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/interfaces/TestHandler.java
@@ -19,17 +19,12 @@
 package org.apache.hadoop.ozone.container.common.interfaces;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
 import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
 import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
-import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
-import static org.junit.Assert.fail;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -37,8 +32,6 @@ import org.junit.rules.TestRule;
 import org.junit.rules.Timeout;
 import org.mockito.Mockito;
 
-import java.util.UUID;
-
 /**
  * Tests Handler interface.
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/states/endpoint/TestHeartbeatEndpointTask.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/states/endpoint/TestHeartbeatEndpointTask.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/states/endpoint/TestHeartbeatEndpointTask.java
index 13de11f..69a6a33 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/states/endpoint/TestHeartbeatEndpointTask.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/states/endpoint/TestHeartbeatEndpointTask.java
@@ -22,8 +22,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.ContainerInfo;
-import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerAction;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.CommandStatusReportsProto;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestRoundRobinVolumeChoosingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestRoundRobinVolumeChoosingPolicy.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestRoundRobinVolumeChoosingPolicy.java
index a45a639..80594d35 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestRoundRobinVolumeChoosingPolicy.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestRoundRobinVolumeChoosingPolicy.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.ozone.container.common.volume;
 
 import org.apache.hadoop.fs.GetSpaceUsed;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
@@ -43,8 +42,8 @@ public class TestRoundRobinVolumeChoosingPolicy {
   private List<HddsVolume> volumes;
 
   private final String baseDir = MiniDFSCluster.getBaseDirectory();
-	private final String volume1 = baseDir + "disk1";
-	private final String volume2 = baseDir + "disk2";
+  private final String volume1 = baseDir + "disk1";
+  private final String volume2 = baseDir + "disk2";
   private static final String DUMMY_IP_ADDR = "0.0.0.0";
 
   @Before

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
index 173a8b2..fea126b 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainer.java
@@ -62,7 +62,8 @@ public class TestOzoneContainer {
     conf = new OzoneConfiguration();
     conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, folder.getRoot()
         .getAbsolutePath());
-    conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, folder.newFolder().getAbsolutePath());
+    conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS,
+        folder.newFolder().getAbsolutePath());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventWatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventWatcher.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventWatcher.java
index 38386d4..e3fee63 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventWatcher.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/events/EventWatcher.java
@@ -180,9 +180,11 @@ public abstract class EventWatcher<TIMEOUT_PAYLOAD extends
 
   }
 
-  protected abstract void onTimeout(EventPublisher publisher, TIMEOUT_PAYLOAD payload);
+  protected abstract void onTimeout(
+      EventPublisher publisher, TIMEOUT_PAYLOAD payload);
 
-  protected abstract void onFinished(EventPublisher publisher, TIMEOUT_PAYLOAD payload);
+  protected abstract void onFinished(
+      EventPublisher publisher, TIMEOUT_PAYLOAD payload);
 
   public List<TIMEOUT_PAYLOAD> getTimeoutEvents(
       Predicate<? super TIMEOUT_PAYLOAD> predicate) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/events/TestEventQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/events/TestEventQueue.java b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/events/TestEventQueue.java
index 2bdf705..0c1200f 100644
--- a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/events/TestEventQueue.java
+++ b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/server/events/TestEventQueue.java
@@ -22,9 +22,6 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.util.Set;
-import java.util.stream.Collectors;
-
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
index ca2a6a0..82d9a28 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
@@ -161,11 +161,11 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
     lock.lock();
     try {
       for (int i = 0; i < count; i++) {
-        ContainerWithPipeline containerWithPipeline = null;
+        ContainerWithPipeline containerWithPipeline;
         try {
           // TODO: Fix this later when Ratis is made the Default.
-          containerWithPipeline = containerManager.allocateContainer(type, factor,
-              owner);
+          containerWithPipeline = containerManager.allocateContainer(
+              type, factor, owner);
 
           if (containerWithPipeline == null) {
             LOG.warn("Unable to allocate container.");
@@ -293,12 +293,12 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
 
   private String getChannelName(ReplicationType type) {
     switch (type) {
-      case RATIS:
-        return "RA" + UUID.randomUUID().toString().substring(3);
-      case STAND_ALONE:
-        return "SA" + UUID.randomUUID().toString().substring(3);
-      default:
-        return "RA" + UUID.randomUUID().toString().substring(3);
+    case RATIS:
+      return "RA" + UUID.randomUUID().toString().substring(3);
+    case STAND_ALONE:
+      return "SA" + UUID.randomUUID().toString().substring(3);
+    default:
+      return "RA" + UUID.randomUUID().toString().substring(3);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java
index df97c27..49af65c 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/DeletedBlockLogImpl.java
@@ -232,7 +232,8 @@ public class DeletedBlockLogImpl implements DeletedBlockLog {
     lock.lock();
     try {
       Set<UUID> dnsWithCommittedTxn;
-      for (DeleteBlockTransactionResult transactionResult : transactionResults) {
+      for (DeleteBlockTransactionResult transactionResult :
+          transactionResults) {
         if (isTransactionFailed(transactionResult)) {
           continue;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/SCMBlockDeletingService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/SCMBlockDeletingService.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/SCMBlockDeletingService.java
index 699fd37..de3fe26 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/SCMBlockDeletingService.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/SCMBlockDeletingService.java
@@ -109,8 +109,8 @@ public class SCMBlockDeletingService extends BackgroundService {
 
   public void handlePendingDeletes(PendingDeleteStatusList deletionStatusList) {
     DatanodeDetails dnDetails = deletionStatusList.getDatanodeDetails();
-    for (PendingDeleteStatusList.PendingDeleteStatus deletionStatus : deletionStatusList
-        .getPendingDeleteStatuses()) {
+    for (PendingDeleteStatusList.PendingDeleteStatus deletionStatus :
+        deletionStatusList.getPendingDeleteStatuses()) {
       LOG.info(
           "Block deletion txnID mismatch in datanode {} for containerID {}."
               + " Datanode delete txnID: {}, SCM txnID: {}",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java
index c723dfa..863907e 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/CloseContainerEventHandler.java
@@ -62,8 +62,8 @@ public class CloseContainerEventHandler implements EventHandler<ContainerID> {
           containerManager.getContainerWithPipeline(containerID.getId());
       info = containerWithPipeline.getContainerInfo();
       if (info == null) {
-        LOG.error("Failed to update the container state. Container with id : {} "
-            + "does not exist", containerID.getId());
+        LOG.error("Failed to update the container state. Container with id : {}"
+            + " does not exist", containerID.getId());
         return;
       }
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
index b000bfd..d506b0c 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerMapping.java
@@ -600,8 +600,8 @@ public class ContainerMapping implements Mapping {
         .setReplicationType(knownState.getReplicationType())
         .setReplicationFactor(knownState.getReplicationFactor());
 
-    // TODO: If current state doesn't have this DN in list of DataNodes with replica
-    // then add it in list of replicas.
+    // TODO: If current state doesn't have this DN in list of DataNodes with
+    // replica then add it in list of replicas.
 
     // If used size is greater than allocated size, we will be updating
     // allocated size with used size. This update is done as a fallback

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
index 5eb8195..7afed42 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
@@ -288,9 +288,10 @@ public class ContainerStateManager implements Closeable {
    * @return ContainerWithPipeline
    * @throws IOException  on Failure.
    */
-  public ContainerWithPipeline allocateContainer(PipelineSelector selector, HddsProtos
-      .ReplicationType type, HddsProtos.ReplicationFactor replicationFactor,
-      String owner) throws IOException {
+  public ContainerWithPipeline allocateContainer(PipelineSelector selector,
+      HddsProtos.ReplicationType type,
+      HddsProtos.ReplicationFactor replicationFactor, String owner)
+      throws IOException {
 
     Pipeline pipeline = selector.getReplicationPipeline(type,
         replicationFactor);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java
index 5f78722..4a980f7 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationManager.java
@@ -182,7 +182,7 @@ public class ReplicationManager implements Runnable {
   }
 
   /**
-   * Event for the ReplicationCommandWatcher to repeate the embedded request
+   * Event for the ReplicationCommandWatcher to repeate the embedded request.
    * in case fof timeout.
    */
   public static class ReplicationRequestToRepeat

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java
index 6c6ce65..4d34cb7 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/states/ContainerStateMap.java
@@ -381,7 +381,7 @@ public class ContainerStateMap {
   }
 
   /**
-   * Returns Open containers in the SCM by the Pipeline
+   * Returns Open containers in the SCM by the Pipeline.
    *
    * @param pipelineID - Pipeline id.
    * @return NavigableSet<ContainerID>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMap.java
index 8ed6d59..d4d475e 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMap.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/Node2ContainerMap.java
@@ -84,8 +84,8 @@ public class Node2ContainerMap {
    * @throws SCMException - if we don't know about this datanode, for new DN
    *                      use insertNewDatanode.
    */
-  public void setContainersForDatanode(UUID datanodeID, Set<ContainerID> containers)
-      throws SCMException {
+  public void setContainersForDatanode(UUID datanodeID,
+      Set<ContainerID> containers) throws SCMException {
     Preconditions.checkNotNull(datanodeID);
     Preconditions.checkNotNull(containers);
     if (dn2ContainerMap

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeNotFoundException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeNotFoundException.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeNotFoundException.java
index 52a352e..c44a08c 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeNotFoundException.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeNotFoundException.java
@@ -17,8 +17,6 @@
 
 package org.apache.hadoop.hdds.scm.node.states;
 
-import java.io.IOException;
-
 /**
  * This exception represents that the node that is being accessed does not
  * exist in NodeStateMap.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/ReportResult.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/ReportResult.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/ReportResult.java
index 2697629..9bb6cf1 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/ReportResult.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/ReportResult.java
@@ -94,7 +94,8 @@ public class ReportResult {
       if (nullSafeMissingContainers == null) {
         nullSafeMissingContainers = Collections.emptySet();
       }
-      return new ReportResult(status, nullSafeMissingContainers, nullSafeNewContainers);
+      return new ReportResult(status, nullSafeMissingContainers,
+          nullSafeNewContainers);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/Node2PipelineMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/Node2PipelineMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/Node2PipelineMap.java
index b860082..4a7fa81 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/Node2PipelineMap.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/Node2PipelineMap.java
@@ -23,39 +23,31 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
-import java.util.Map;
-import java.util.HashSet;
-import java.util.Collections;
-
 import java.util.concurrent.ConcurrentHashMap;
 
-import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
-    .DUPLICATE_DATANODE;
-
+import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes.DUPLICATE_DATANODE;
 
 /**
- * This data structure maintains the list of pipelines which the given datanode
- * is a part of.
- * This information will be added whenever a new pipeline allocation happens.
+ * This data structure maintains the list of pipelines which the given datanode is a part of. This
+ * information will be added whenever a new pipeline allocation happens.
  *
- * TODO: this information needs to be regenerated from pipeline reports on
- * SCM restart
+ * <p>TODO: this information needs to be regenerated from pipeline reports on SCM restart
  */
 public class Node2PipelineMap {
   private final Map<UUID, Set<Pipeline>> dn2PipelineMap;
 
-  /**
-   * Constructs a Node2PipelineMap Object.
-   */
+  /** Constructs a Node2PipelineMap Object. */
   public Node2PipelineMap() {
     dn2PipelineMap = new ConcurrentHashMap<>();
   }
 
   /**
-   * Returns true if this a datanode that is already tracked by
-   * Node2PipelineMap.
+   * Returns true if this a datanode that is already tracked by Node2PipelineMap.
    *
    * @param datanodeID - UUID of the Datanode.
    * @return True if this is tracked, false if this map does not know about it.
@@ -71,18 +63,17 @@ public class Node2PipelineMap {
    * @param datanodeID -- Datanode UUID
    * @param pipelines - set of pipelines.
    */
-  private void insertNewDatanode(UUID datanodeID, Set<Pipeline> pipelines)
-      throws SCMException {
+  private void insertNewDatanode(UUID datanodeID, Set<Pipeline> pipelines) throws SCMException {
     Preconditions.checkNotNull(pipelines);
     Preconditions.checkNotNull(datanodeID);
-    if(dn2PipelineMap.putIfAbsent(datanodeID, pipelines) != null) {
-      throw new SCMException("Node already exists in the map",
-          DUPLICATE_DATANODE);
+    if (dn2PipelineMap.putIfAbsent(datanodeID, pipelines) != null) {
+      throw new SCMException("Node already exists in the map", DUPLICATE_DATANODE);
     }
   }
 
   /**
    * Removes datanode Entry from the map.
+   *
    * @param datanodeID - Datanode ID.
    */
   public synchronized void removeDatanode(UUID datanodeID) {
@@ -98,20 +89,19 @@ public class Node2PipelineMap {
    */
   public Set<Pipeline> getPipelines(UUID datanode) {
     Preconditions.checkNotNull(datanode);
-    return dn2PipelineMap.computeIfPresent(datanode, (k, v) ->
-        Collections.unmodifiableSet(v));
+    return dn2PipelineMap.computeIfPresent(datanode, (k, v) -> Collections.unmodifiableSet(v));
   }
 
   /**
    * Adds a pipeline entry to a given dataNode in the map.
+   *
    * @param pipeline Pipeline to be added
    */
   public synchronized void addPipeline(Pipeline pipeline) {
     for (DatanodeDetails details : pipeline.getDatanodes().values()) {
       UUID dnId = details.getUuid();
       dn2PipelineMap
-          .computeIfAbsent(dnId,
-              k -> Collections.synchronizedSet(new HashSet<>()))
+          .computeIfAbsent(dnId, k -> Collections.synchronizedSet(new HashSet<>()))
           .add(pipeline);
     }
   }
@@ -119,8 +109,12 @@ public class Node2PipelineMap {
   public synchronized void removePipeline(Pipeline pipeline) {
     for (DatanodeDetails details : pipeline.getDatanodes().values()) {
       UUID dnId = details.getUuid();
-      dn2PipelineMap.computeIfPresent(dnId,
-          (k, v) -> {v.remove(pipeline); return v;});
+      dn2PipelineMap.computeIfPresent(
+          dnId,
+          (k, v) -> {
+            v.remove(pipeline);
+            return v;
+          });
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java
index 7d91ee4..5b1a7f7 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineManager.java
@@ -111,7 +111,7 @@ public abstract class PipelineManager {
       ReplicationFactor replicationFactor);
 
   /**
-   * Initialize the pipeline
+   * Initialize the pipeline.
    * TODO: move the initialization to Ozone Client later
    */
   public abstract void initializePipeline(Pipeline pipeline) throws IOException;
@@ -176,7 +176,7 @@ public abstract class PipelineManager {
   }
 
   /**
-   * Remove the pipeline from active allocation
+   * Remove the pipeline from active allocation.
    * @param pipeline pipeline to be finalized
    */
   public synchronized void finalizePipeline(Pipeline pipeline) {
@@ -193,7 +193,7 @@ public abstract class PipelineManager {
   }
 
   /**
-   * list members in the pipeline .
+   * list members in the pipeline.
    * @return the datanode
    */
   public abstract List<DatanodeDetails> getMembers(PipelineID pipelineID)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
index 5343bce..b02beb3 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipelines/PipelineSelector.java
@@ -126,7 +126,7 @@ public class PipelineSelector {
   }
 
   /**
-   * Event and State Transition Mapping:
+   * Event and State Transition Mapping.
    *
    * State: ALLOCATED ---------------> CREATING
    * Event:                CREATE
@@ -293,7 +293,7 @@ public class PipelineSelector {
       pipeline = manager.getPipeline(replicationFactor, replicationType);
     } else {
       // if a new pipeline is created, initialize its state machine
-      updatePipelineState(pipeline,HddsProtos.LifeCycleEvent.CREATE);
+      updatePipelineState(pipeline, HddsProtos.LifeCycleEvent.CREATE);
 
       //TODO: move the initialization of pipeline to Ozone Client
       manager.initializePipeline(pipeline);
@@ -334,7 +334,8 @@ public class PipelineSelector {
   /**
    * Close a given pipeline.
    */
-  public void closePipelineIfNoOpenContainers(Pipeline pipeline) throws IOException {
+  public void closePipelineIfNoOpenContainers(Pipeline pipeline)
+      throws IOException {
     if (pipeline.getLifeCycleState() != LifeCycleState.CLOSING) {
       return;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
index aefcf1b..77e495d 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
@@ -165,7 +165,8 @@ public class SCMClientProtocolServer implements
   }
 
   @Override
-  public ContainerWithPipeline getContainerWithPipeline(long containerID) throws IOException {
+  public ContainerWithPipeline getContainerWithPipeline(long containerID)
+      throws IOException {
     String remoteUser = getRpcRemoteUsername();
     getScm().checkAdminAccess(remoteUser);
     return scm.getScmContainerManager()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
index c466570..d617680 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/TestUtils.java
@@ -74,7 +74,7 @@ public final class TestUtils {
         + "." + random.nextInt(256)
         + "." + random.nextInt(256)
         + "." + random.nextInt(256);
-   return createDatanodeDetails(uuid.toString(), "localhost", ipAddress);
+    return createDatanodeDetails(uuid.toString(), "localhost", ipAddress);
   }
 
   /**
@@ -259,12 +259,12 @@ public final class TestUtils {
     StorageTypeProto storageTypeProto =
         type == null ? StorageTypeProto.DISK : type;
     srb.setStorageType(storageTypeProto);
-   return srb.build();
+    return srb.build();
   }
 
 
   /**
-   * Generates random container reports
+   * Generates random container reports.
    *
    * @return ContainerReportsProto
    */
@@ -281,7 +281,7 @@ public final class TestUtils {
    */
   public static ContainerReportsProto getRandomContainerReports(
       int numberOfContainers) {
-   List<ContainerInfo> containerInfos = new ArrayList<>();
+    List<ContainerInfo> containerInfos = new ArrayList<>();
     for (int i = 0; i < numberOfContainers; i++) {
       containerInfos.add(getRandomContainerInfo(i));
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
index 2beb4e7..68c5813 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
@@ -39,7 +39,6 @@ import org.junit.rules.ExpectedException;
 import java.io.File;
 import java.io.IOException;
 import java.nio.file.Paths;
-import java.util.Collections;
 
 import static org.apache.hadoop.ozone.OzoneConsts.GB;
 import static org.apache.hadoop.ozone.OzoneConsts.MB;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java
index 1d06ea4..9f0e336 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestDeletedBlockLog.java
@@ -102,8 +102,8 @@ public class TestDeletedBlockLog {
     ContainerInfo containerInfo =
         new ContainerInfo.Builder().setContainerID(1).build();
     Pipeline pipeline =
-        new Pipeline(null, LifeCycleState.CLOSED, ReplicationType.RATIS,
-            ReplicationFactor.THREE, null);
+        new Pipeline(null, LifeCycleState.CLOSED,
+            ReplicationType.RATIS, ReplicationFactor.THREE, null);
     pipeline.addMember(dnList.get(0));
     pipeline.addMember(dnList.get(1));
     pipeline.addMember(dnList.get(2));
@@ -379,7 +379,8 @@ public class TestDeletedBlockLog {
     Assert.assertTrue(transactions.isFull());
   }
 
-  private void mockContainerInfo(long containerID, DatanodeDetails dd) throws IOException {
+  private void mockContainerInfo(long containerID, DatanodeDetails dd)
+      throws IOException {
     Pipeline pipeline =
         new Pipeline("fake", LifeCycleState.OPEN,
             ReplicationType.STAND_ALONE, ReplicationFactor.ONE,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/command/TestCommandStatusReportHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/command/TestCommandStatusReportHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/command/TestCommandStatusReportHandler.java
index eca5b87..65a2e29 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/command/TestCommandStatusReportHandler.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/command/TestCommandStatusReportHandler.java
@@ -48,12 +48,11 @@ import static org.junit.Assert.assertFalse;
 
 public class TestCommandStatusReportHandler implements EventPublisher {
 
-  private static Logger LOG = LoggerFactory
+  private static final Logger LOG = LoggerFactory
       .getLogger(TestCommandStatusReportHandler.class);
   private CommandStatusReportHandler cmdStatusReportHandler;
   private String storagePath = GenericTestUtils.getRandomizedTempPath()
       .concat("/" + UUID.randomUUID().toString());
-  ;
 
   @Before
   public void setup() {
@@ -69,10 +68,9 @@ public class TestCommandStatusReportHandler implements EventPublisher {
         .emptyList());
     cmdStatusReportHandler.onMessage(report, this);
     assertFalse(logCapturer.getOutput().contains("DeleteBlockCommandStatus"));
-    assertFalse(logCapturer.getOutput().contains
-        ("CloseContainerCommandStatus"));
-    assertFalse(logCapturer.getOutput().contains
-        ("ReplicateCommandStatus"));
+    assertFalse(logCapturer.getOutput().contains(
+        "CloseContainerCommandStatus"));
+    assertFalse(logCapturer.getOutput().contains("ReplicateCommandStatus"));
 
 
     report = this.getStatusReport(this.getCommandStatusList());
@@ -93,13 +91,13 @@ public class TestCommandStatusReportHandler implements EventPublisher {
 
   }
 
-  private CommandStatusReportFromDatanode getStatusReport(List<CommandStatus>
-      reports) {
-    CommandStatusReportsProto report = TestUtils.createCommandStatusReport
-        (reports);
+  private CommandStatusReportFromDatanode getStatusReport(
+      List<CommandStatus> reports) {
+    CommandStatusReportsProto report = TestUtils.createCommandStatusReport(
+        reports);
     DatanodeDetails dn = TestUtils.randomDatanodeDetails();
-    return new SCMDatanodeHeartbeatDispatcher.CommandStatusReportFromDatanode
-        (dn, report);
+    return new SCMDatanodeHeartbeatDispatcher.CommandStatusReportFromDatanode(
+        dn, report);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
index 4790c82..38050c9 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestCloseContainerEventHandler.java
@@ -37,7 +37,6 @@ import org.junit.Test;
 import java.io.File;
 import java.io.IOException;
 
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent.CREATE;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent.CREATED;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
index b0b39f1..c5686f5 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerMapping.java
@@ -178,8 +178,8 @@ public class TestContainerMapping {
     mapping
         .updateContainerState(contInfo.getContainerID(), LifeCycleEvent.CLOSE);
     ContainerInfo finalContInfo = contInfo;
-    LambdaTestUtils.intercept(SCMException.class,"No entry exist for "
-        + "containerId:" , () -> mapping.getContainerWithPipeline(
+    LambdaTestUtils.intercept(SCMException.class, "No entry exist for "
+        + "containerId:", () -> mapping.getContainerWithPipeline(
         finalContInfo.getContainerID()));
 
     mapping.getStateManager().getContainerStateMap()
@@ -376,7 +376,8 @@ public class TestContainerMapping {
   @Test
   public void testFlushAllContainers() throws IOException {
     ContainerInfo info = createContainer();
-    List<ContainerInfo> containers = mapping.getStateManager().getAllContainers();
+    List<ContainerInfo> containers = mapping.getStateManager()
+        .getAllContainers();
     Assert.assertTrue(containers.size() > 0);
     mapping.flushContainerInfo();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
index 443b4b2..66f0966 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
@@ -86,7 +86,7 @@ public class TestContainerReportHandler implements EventPublisher {
                     .setContainerID((Long) invocation.getArguments()[0])
                     .setState(LifeCycleState.CLOSED)
                     .build()
-        );
+      );
 
     ContainerStateManager containerStateManager =
         new ContainerStateManager(conf, mapping);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementCapacity.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementCapacity.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementCapacity.java
index fea1e4b..764daff 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementCapacity.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementCapacity.java
@@ -52,13 +52,13 @@ public class TestSCMContainerPlacementCapacity {
         .thenReturn(new ArrayList<>(datanodes));
 
     when(mockNodeManager.getNodeStat(anyObject()))
-        .thenReturn(new SCMNodeMetric(100l, 0L, 100L));
+        .thenReturn(new SCMNodeMetric(100L, 0L, 100L));
     when(mockNodeManager.getNodeStat(datanodes.get(2)))
-        .thenReturn(new SCMNodeMetric(100l, 90L, 10L));
+        .thenReturn(new SCMNodeMetric(100L, 90L, 10L));
     when(mockNodeManager.getNodeStat(datanodes.get(3)))
-        .thenReturn(new SCMNodeMetric(100l, 80L, 20L));
+        .thenReturn(new SCMNodeMetric(100L, 80L, 20L));
     when(mockNodeManager.getNodeStat(datanodes.get(4)))
-        .thenReturn(new SCMNodeMetric(100l, 70L, 30L));
+        .thenReturn(new SCMNodeMetric(100L, 70L, 30L));
 
     SCMContainerPlacementCapacity scmContainerPlacementRandom =
         new SCMContainerPlacementCapacity(mockNodeManager, conf);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[43/50] [abbrv] hadoop git commit: HDDS-357. Use DBStore and TableStore for OzoneManager non-background service. Contributed by Nandakumar.

Posted by eh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
index 21d2411..151fddf 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
@@ -19,77 +19,178 @@ package org.apache.hadoop.ozone.om;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
-import org.apache.commons.lang3.tuple.ImmutablePair;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hdds.client.BlockID;
-import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
-import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
-import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
-import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
-import org.apache.hadoop.ozone.common.BlockGroup;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.common.BlockGroup;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketInfo;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeInfo;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
-
 import org.apache.hadoop.util.Time;
-import org.apache.hadoop.utils.BatchOperation;
-import org.apache.hadoop.utils.MetadataKeyFilters;
-import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
-import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter;
-import org.apache.hadoop.utils.MetadataStore;
-import org.apache.hadoop.utils.MetadataStoreBuilder;
+import org.apache.hadoop.utils.db.DBStore;
+import org.apache.hadoop.utils.db.DBStoreBuilder;
+import org.apache.hadoop.utils.db.Table;
+import org.apache.hadoop.utils.db.TableIterator;
+import org.eclipse.jetty.util.StringUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
-import java.util.ArrayList;
 import java.util.Map;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.stream.Collectors;
 
+import static org.apache.hadoop.hdds.server.ServerUtils.getOzoneMetaDirPath;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS_DEFAULT;
-import static org.apache.hadoop.ozone.OzoneConsts.DELETING_KEY_PREFIX;
 import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
-import static org.apache.hadoop.ozone.OzoneConsts.OPEN_KEY_ID_DELIMINATOR;
-import static org.apache.hadoop.ozone.OzoneConsts.OPEN_KEY_PREFIX;
-import static org.apache.hadoop.ozone.om.OMConfigKeys
-    .OZONE_OM_DB_CACHE_SIZE_DEFAULT;
-import static org.apache.hadoop.ozone.om.OMConfigKeys
-    .OZONE_OM_DB_CACHE_SIZE_MB;
-import static org.apache.hadoop.hdds.server.ServerUtils.getOzoneMetaDirPath;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
 
 /**
  * Ozone metadata manager interface.
  */
 public class OmMetadataManagerImpl implements OMMetadataManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OmMetadataManagerImpl.class);
+
+  /**
+   * OM RocksDB Structure .
+   * <p>
+   * OM DB stores metadata as KV pairs in different column families.
+   * <p>
+   * OM DB Schema:
+   * |-------------------------------------------------------------------|
+   * |  Column Family     |        VALUE                                 |
+   * |-------------------------------------------------------------------|
+   * | userTable          |     user->VolumeList                         |
+   * |-------------------------------------------------------------------|
+   * | volumeTable        |     /volume->VolumeInfo                      |
+   * |-------------------------------------------------------------------|
+   * | bucketTable        |     /volume/bucket-> BucketInfo              |
+   * |-------------------------------------------------------------------|
+   * | keyTable           | /volumeName/bucketName/keyName->KeyInfo      |
+   * |-------------------------------------------------------------------|
+   * | deletedTable       | /volumeName/bucketName/keyName->KeyInfo      |
+   * |-------------------------------------------------------------------|
+   * | openKey            | /volumeName/bucketName/keyName/id->KeyInfo   |
+   * |-------------------------------------------------------------------|
+   */
+
+  private static final String USER_TABLE = "userTable";
+  private static final String VOLUME_TABLE = "volumeTable";
+  private static final String BUCKET_TABLE = "bucketTable";
+  private static final String KEY_TABLE = "keyTable";
+  private static final String DELETED_TABLE = "deletedTable";
+  private static final String OPEN_KEY_TABLE = "openKeyTable";
 
-  private final MetadataStore store;
+  private final DBStore store;
+
+  // TODO: Make this lock move into Table instead of *ONE* lock for the whole
+  // DB.
   private final ReadWriteLock lock;
   private final long openKeyExpireThresholdMS;
 
+  private final Table userTable;
+  private final Table volumeTable;
+  private final Table bucketTable;
+  private final Table keyTable;
+  private final Table deletedTable;
+  private final Table openKeyTable;
+
   public OmMetadataManagerImpl(OzoneConfiguration conf) throws IOException {
     File metaDir = getOzoneMetaDirPath(conf);
-    final int cacheSize = conf.getInt(OZONE_OM_DB_CACHE_SIZE_MB,
-        OZONE_OM_DB_CACHE_SIZE_DEFAULT);
-    File omDBFile = new File(metaDir.getPath(), OM_DB_NAME);
-    this.store = MetadataStoreBuilder.newBuilder()
-        .setConf(conf)
-        .setDbFile(omDBFile)
-        .setCacheSize(cacheSize * OzoneConsts.MB)
-        .build();
     this.lock = new ReentrantReadWriteLock();
     this.openKeyExpireThresholdMS = 1000 * conf.getInt(
         OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS,
         OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS_DEFAULT);
+
+    this.store = DBStoreBuilder.newBuilder(conf)
+        .setName(OM_DB_NAME)
+        .setPath(Paths.get(metaDir.getPath()))
+        .addTable(USER_TABLE)
+        .addTable(VOLUME_TABLE)
+        .addTable(BUCKET_TABLE)
+        .addTable(KEY_TABLE)
+        .addTable(DELETED_TABLE)
+        .addTable(OPEN_KEY_TABLE)
+        .build();
+
+    userTable = this.store.getTable(USER_TABLE);
+    checkTableStatus(userTable, USER_TABLE);
+
+    volumeTable = this.store.getTable(VOLUME_TABLE);
+    checkTableStatus(volumeTable, VOLUME_TABLE);
+
+    bucketTable = this.store.getTable(BUCKET_TABLE);
+    checkTableStatus(bucketTable, BUCKET_TABLE);
+
+    keyTable = this.store.getTable(KEY_TABLE);
+    checkTableStatus(keyTable, KEY_TABLE);
+
+    deletedTable = this.store.getTable(DELETED_TABLE);
+    checkTableStatus(deletedTable, DELETED_TABLE);
+
+    openKeyTable = this.store.getTable(OPEN_KEY_TABLE);
+    checkTableStatus(openKeyTable, OPEN_KEY_TABLE);
+
+  }
+
+  @Override
+  public Table getUserTable() {
+    return userTable;
+  }
+
+  @Override
+  public Table getVolumeTable() {
+    return volumeTable;
+  }
+
+  @Override
+  public Table getBucketTable() {
+    return bucketTable;
+  }
+
+  @Override
+  public Table getKeyTable() {
+    return keyTable;
+  }
+
+  @Override
+  public Table getDeletedTable() {
+    return deletedTable;
+  }
+
+  @Override
+  public Table getOpenKeyTable() {
+    return openKeyTable;
+  }
+
+  private void checkTableStatus(Table table, String name) throws IOException {
+    String logMessage = "Unable to get a reference to %s table. Cannot " +
+        "continue.";
+    String errMsg = "Inconsistent DB state, Table - %s. Please check the logs" +
+        "for more info.";
+    if (table == null) {
+      LOG.error(String.format(logMessage, name));
+      throw new IOException(String.format(errMsg, name));
+    }
   }
 
   /**
@@ -104,7 +205,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
    * Stop metadata manager.
    */
   @Override
-  public void stop() throws IOException {
+  public void stop() throws Exception {
     if (store != null) {
       store.close();
     }
@@ -112,86 +213,75 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
 
   /**
    * Get metadata store.
+   *
    * @return store - metadata store.
    */
   @VisibleForTesting
   @Override
-  public MetadataStore getStore() {
+  public DBStore getStore() {
     return store;
   }
 
   /**
    * Given a volume return the corresponding DB key.
+   *
    * @param volume - Volume name
    */
+  @Override
   public byte[] getVolumeKey(String volume) {
-    String dbVolumeName = OzoneConsts.OM_VOLUME_PREFIX + volume;
-    return DFSUtil.string2Bytes(dbVolumeName);
+    return DFSUtil.string2Bytes(OzoneConsts.OM_KEY_PREFIX + volume);
   }
 
   /**
    * Given a user return the corresponding DB key.
+   *
    * @param user - User name
    */
+  @Override
   public byte[] getUserKey(String user) {
-    String dbUserName = OzoneConsts.OM_USER_PREFIX + user;
-    return DFSUtil.string2Bytes(dbUserName);
+    return DFSUtil.string2Bytes(user);
   }
 
   /**
    * Given a volume and bucket, return the corresponding DB key.
+   *
    * @param volume - User name
    * @param bucket - Bucket name
    */
+  @Override
   public byte[] getBucketKey(String volume, String bucket) {
-    String bucketKeyString = OzoneConsts.OM_VOLUME_PREFIX + volume
-        + OzoneConsts.OM_BUCKET_PREFIX + bucket;
-    return DFSUtil.string2Bytes(bucketKeyString);
-  }
+    StringBuilder builder =
+        new StringBuilder().append(OM_KEY_PREFIX).append(volume);
 
-  /**
-   * @param volume
-   * @param bucket
-   * @return
-   */
-  private String getBucketWithDBPrefix(String volume, String bucket) {
-    StringBuffer sb = new StringBuffer();
-    sb.append(OzoneConsts.OM_VOLUME_PREFIX)
-        .append(volume)
-        .append(OzoneConsts.OM_BUCKET_PREFIX);
-    if (!Strings.isNullOrEmpty(bucket)) {
-      sb.append(bucket);
+    if (StringUtils.isNotBlank(bucket)) {
+      builder.append(OM_KEY_PREFIX).append(bucket);
     }
-    return sb.toString();
-  }
-
-  @Override
-  public String getKeyWithDBPrefix(String volume, String bucket, String key) {
-    String keyVB = OzoneConsts.OM_KEY_PREFIX + volume
-        + OzoneConsts.OM_KEY_PREFIX + bucket
-        + OzoneConsts.OM_KEY_PREFIX;
-    return Strings.isNullOrEmpty(key) ? keyVB : keyVB + key;
-  }
-
-  @Override
-  public byte[] getDBKeyBytes(String volume, String bucket, String key) {
-    return DFSUtil.string2Bytes(getKeyWithDBPrefix(volume, bucket, key));
+    return DFSUtil.string2Bytes(builder.toString());
   }
 
   @Override
-  public byte[] getDeletedKeyName(byte[] keyName) {
-    return DFSUtil.string2Bytes(
-        DELETING_KEY_PREFIX + DFSUtil.bytes2String(keyName));
+  public byte[] getOzoneKeyBytes(String volume, String bucket, String key) {
+    StringBuilder builder = new StringBuilder()
+        .append(OM_KEY_PREFIX).append(volume);
+    // TODO : Throw if the Bucket is null?
+    builder.append(OM_KEY_PREFIX).append(bucket);
+    if (StringUtil.isNotBlank(key)) {
+      builder.append(OM_KEY_PREFIX).append(key);
+    }
+    return DFSUtil.string2Bytes(builder.toString());
   }
 
   @Override
-  public byte[] getOpenKeyNameBytes(String keyName, int id) {
-    return DFSUtil.string2Bytes(OPEN_KEY_PREFIX + id +
-        OPEN_KEY_ID_DELIMINATOR + keyName);
+  public byte[] getOpenKeyBytes(String volume, String bucket,
+                                    String key, long id) {
+    String openKey = OM_KEY_PREFIX + volume + OM_KEY_PREFIX + bucket +
+        OM_KEY_PREFIX + key + OM_KEY_PREFIX + id;
+    return DFSUtil.string2Bytes(openKey);
   }
 
   /**
    * Returns the read lock used on Metadata DB.
+   *
    * @return readLock
    */
   @Override
@@ -201,6 +291,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
 
   /**
    * Returns the write lock used on Metadata DB.
+   *
    * @return writeLock
    */
   @Override
@@ -209,71 +300,79 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
   }
 
   /**
-   * Returns the value associated with this key.
-   * @param key - key
-   * @return value
+   * Returns true if the firstArray startsWith the bytes of secondArray.
+   *
+   * @param firstArray - Byte array
+   * @param secondArray - Byte array
+   * @return true if the first array bytes match the bytes in the second array.
    */
-  @Override
-  public byte[] get(byte[] key) throws IOException {
-    return store.get(key);
-  }
+  private boolean startsWith(byte[] firstArray, byte[] secondArray) {
 
-  /**
-   * Puts a Key into Metadata DB.
-   * @param key   - key
-   * @param value - value
-   */
-  @Override
-  public void put(byte[] key, byte[] value) throws IOException {
-    store.put(key, value);
-  }
+    if (firstArray == null) {
+      // if both are null, then the arrays match, else if first is null and
+      // second is not, then this function returns false.
+      return secondArray == null;
+    }
 
-  /**
-   * Deletes a Key from Metadata DB.
-   * @param key   - key
-   */
-  public void delete(byte[] key) throws IOException {
-    store.delete(key);
-  }
 
-  @Override
-  public void writeBatch(BatchOperation batch) throws IOException {
-    this.store.writeBatch(batch);
+    if (secondArray != null) {
+      // If the second array is longer then first array cannot be starting with
+      // the bytes of second array.
+      if (secondArray.length > firstArray.length) {
+        return false;
+      }
+
+      for (int ndx = 0; ndx < secondArray.length; ndx++) {
+        if (firstArray[ndx] != secondArray[ndx]) {
+          return false;
+        }
+      }
+      return true; //match, return true.
+    }
+    return false; // if first is not null and second is null, we define that
+    // array does not start with same chars.
   }
 
   /**
    * Given a volume, check if it is empty, i.e there are no buckets inside it.
+   * We iterate in the bucket table and see if there is any key that starts with
+   * the volume prefix. We actually look for /volume/, since if we don't have
+   * the trailing slash it is possible that we might match some other volume.
+   * <p>
+   * For example, vol1 and vol122 might match, to avoid that we look for /vol1/
+   *
    * @param volume - Volume name
    * @return true if the volume is empty
    */
+  @Override
   public boolean isVolumeEmpty(String volume) throws IOException {
-    String dbVolumeRootName = OzoneConsts.OM_VOLUME_PREFIX + volume
-        + OzoneConsts.OM_BUCKET_PREFIX;
-    byte[] dbVolumeRootKey = DFSUtil.string2Bytes(dbVolumeRootName);
-    ImmutablePair<byte[], byte[]> volumeRoot =
-        store.peekAround(0, dbVolumeRootKey);
-    if (volumeRoot != null) {
-      return !DFSUtil.bytes2String(volumeRoot.getKey())
-          .startsWith(dbVolumeRootName);
+    byte[] volumePrefix = getVolumeKey(volume + OM_KEY_PREFIX);
+    try (TableIterator<Table.KeyValue> bucketIter = bucketTable.iterator()) {
+      Table.KeyValue kv = bucketIter.seek(volumePrefix);
+      if (kv != null && startsWith(kv.getKey(), volumePrefix)) {
+        return false; // we found at least one bucket with this volume prefix.
+      }
     }
     return true;
   }
 
   /**
-   * Given a volume/bucket, check if it is empty,
-   * i.e there are no keys inside it.
+   * Given a volume/bucket, check if it is empty, i.e there are no keys inside
+   * it. Prefix is /volume/bucket/, and we lookup the keyTable.
+   *
    * @param volume - Volume name
    * @param bucket - Bucket name
    * @return true if the bucket is empty
    */
+  @Override
   public boolean isBucketEmpty(String volume, String bucket)
       throws IOException {
-    String keyRootName = getKeyWithDBPrefix(volume, bucket, null);
-    byte[] keyRoot = DFSUtil.string2Bytes(keyRootName);
-    ImmutablePair<byte[], byte[]> firstKey = store.peekAround(0, keyRoot);
-    if (firstKey != null) {
-      return !DFSUtil.bytes2String(firstKey.getKey())
-          .startsWith(keyRootName);
+    byte[] keyPrefix = getBucketKey(volume, bucket + OM_KEY_PREFIX);
+    try (TableIterator<Table.KeyValue> keyIter = keyTable.iterator()) {
+      Table.KeyValue kv = keyIter.seek(keyPrefix);
+      if (kv != null && startsWith(kv.getKey(), keyPrefix)) {
+        return false; // we found at least one key with this vol/bucket prefix.
+      }
     }
     return true;
   }
@@ -283,8 +382,8 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
    */
   @Override
   public List<OmBucketInfo> listBuckets(final String volumeName,
-                                        final String startBucket, final String bucketPrefix,
-                                        final int maxNumOfBuckets) throws IOException {
+      final String startBucket, final String bucketPrefix,
+      final int maxNumOfBuckets) throws IOException {
     List<OmBucketInfo> result = new ArrayList<>();
     if (Strings.isNullOrEmpty(volumeName)) {
       throw new OMException("Volume name is required.",
@@ -292,49 +391,61 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
     }
 
     byte[] volumeNameBytes = getVolumeKey(volumeName);
-    if (store.get(volumeNameBytes) == null) {
+    if (volumeTable.get(volumeNameBytes) == null) {
       throw new OMException("Volume " + volumeName + " not found.",
           ResultCodes.FAILED_VOLUME_NOT_FOUND);
     }
 
 
-    // A bucket starts with /#volume/#bucket_prefix
-    MetadataKeyFilter filter = (preKey, currentKey, nextKey) -> {
-      if (currentKey != null) {
-        String bucketNamePrefix =
-                getBucketWithDBPrefix(volumeName, bucketPrefix);
-        String bucket = DFSUtil.bytes2String(currentKey);
-        return bucket.startsWith(bucketNamePrefix);
-      }
-      return false;
-    };
-
-    List<Map.Entry<byte[], byte[]>> rangeResult;
-    if (!Strings.isNullOrEmpty(startBucket)) {
-      // Since we are excluding start key from the result,
-      // the maxNumOfBuckets is incremented.
-      rangeResult = store.getSequentialRangeKVs(
-          getBucketKey(volumeName, startBucket),
-          maxNumOfBuckets + 1, filter);
-      if (!rangeResult.isEmpty()) {
-        //Remove start key from result.
-        rangeResult.remove(0);
-      }
+    byte[] startKey;
+    boolean skipStartKey = false;
+    if (StringUtil.isNotBlank(startBucket)) {
+      // if the user has specified a start key, we need to seek to that key
+      // and avoid that key in the response set.
+      startKey = getBucketKey(volumeName, startBucket);
+      skipStartKey = true;
     } else {
-      rangeResult = store.getSequentialRangeKVs(null, maxNumOfBuckets, filter);
+      // If the user has specified a prefix key, we need to get to the first
+      // of the keys with the prefix match. We can leverage RocksDB to do that.
+      // However, if the user has specified only a prefix, we cannot skip
+      // the first prefix key we see, the boolean skipStartKey allows us to
+      // skip the startkey or not depending on what patterns are specified.
+      startKey = getBucketKey(volumeName, bucketPrefix);
     }
 
-    for (Map.Entry<byte[], byte[]> entry : rangeResult) {
-      OmBucketInfo info = OmBucketInfo.getFromProtobuf(
-          BucketInfo.parseFrom(entry.getValue()));
-      result.add(info);
+    byte[] seekPrefix;
+    if (StringUtil.isNotBlank(bucketPrefix)) {
+      seekPrefix = getBucketKey(volumeName, bucketPrefix);
+    } else {
+      seekPrefix = getVolumeKey(volumeName + OM_KEY_PREFIX);
+    }
+    int currentCount = 0;
+    try (TableIterator<Table.KeyValue> bucketIter = bucketTable.iterator()) {
+      Table.KeyValue kv = bucketIter.seek(startKey);
+      while (currentCount < maxNumOfBuckets && bucketIter.hasNext()) {
+        kv = bucketIter.next();
+        // Skip the Start Bucket if needed.
+        if (kv != null && skipStartKey &&
+            Arrays.equals(kv.getKey(), startKey)) {
+          continue;
+        }
+        if (kv != null && startsWith(kv.getKey(), seekPrefix)) {
+          result.add(OmBucketInfo.getFromProtobuf(
+              BucketInfo.parseFrom(kv.getValue())));
+          currentCount++;
+        } else {
+          // The SeekPrefix does not match any more, we can break out of the
+          // loop.
+          break;
+        }
+      }
     }
     return result;
   }
 
   @Override
   public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
-                                  String startKey, String keyPrefix, int maxKeys) throws IOException {
+      String startKey, String keyPrefix, int maxKeys) throws IOException {
     List<OmKeyInfo> result = new ArrayList<>();
     if (Strings.isNullOrEmpty(volumeName)) {
       throw new OMException("Volume name is required.",
@@ -347,47 +458,61 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
     }
 
     byte[] bucketNameBytes = getBucketKey(volumeName, bucketName);
-    if (store.get(bucketNameBytes) == null) {
+    if (getBucketTable().get(bucketNameBytes) == null) {
       throw new OMException("Bucket " + bucketName + " not found.",
           ResultCodes.FAILED_BUCKET_NOT_FOUND);
     }
 
-    MetadataKeyFilter filter = new KeyPrefixFilter()
-        .addFilter(getKeyWithDBPrefix(volumeName, bucketName, keyPrefix));
-
-    List<Map.Entry<byte[], byte[]>> rangeResult;
-    if (!Strings.isNullOrEmpty(startKey)) {
-      //Since we are excluding start key from the result,
-      // the maxNumOfBuckets is incremented.
-      rangeResult = store.getSequentialRangeKVs(
-          getDBKeyBytes(volumeName, bucketName, startKey),
-          maxKeys + 1, filter);
-      if (!rangeResult.isEmpty()) {
-        //Remove start key from result.
-        rangeResult.remove(0);
-      }
+    byte[] seekKey;
+    boolean skipStartKey = false;
+    if (StringUtil.isNotBlank(startKey)) {
+      // Seek to the specified key.
+      seekKey = getOzoneKeyBytes(volumeName, bucketName, startKey);
+      skipStartKey = true;
     } else {
-      rangeResult = store.getSequentialRangeKVs(null, maxKeys, filter);
+      // This allows us to seek directly to the first key with the right prefix.
+      seekKey = getOzoneKeyBytes(volumeName, bucketName, keyPrefix);
     }
 
-    for (Map.Entry<byte[], byte[]> entry : rangeResult) {
-      OmKeyInfo info = OmKeyInfo.getFromProtobuf(
-          KeyInfo.parseFrom(entry.getValue()));
-      result.add(info);
+    byte[] seekPrefix;
+    if (StringUtil.isNotBlank(keyPrefix)) {
+      seekPrefix = getOzoneKeyBytes(volumeName, bucketName, keyPrefix);
+    } else {
+      seekPrefix = getBucketKey(volumeName, bucketName + OM_KEY_PREFIX);
+    }
+    int currentCount = 0;
+    try (TableIterator<Table.KeyValue> keyIter = getKeyTable().iterator()) {
+      Table.KeyValue kv = keyIter.seek(seekKey);
+      while (currentCount < maxKeys && keyIter.hasNext()) {
+        kv = keyIter.next();
+        // Skip the Start key if needed.
+        if (kv != null && skipStartKey && Arrays.equals(kv.getKey(), seekKey)) {
+          continue;
+        }
+        if (kv != null && startsWith(kv.getKey(), seekPrefix)) {
+          result.add(OmKeyInfo.getFromProtobuf(
+              KeyInfo.parseFrom(kv.getValue())));
+          currentCount++;
+        } else {
+          // The SeekPrefix does not match any more, we can break out of the
+          // loop.
+          break;
+        }
+      }
     }
     return result;
   }
 
   @Override
   public List<OmVolumeArgs> listVolumes(String userName,
-                                        String prefix, String startKey, int maxKeys) throws IOException {
+      String prefix, String startKey, int maxKeys) throws IOException {
     List<OmVolumeArgs> result = Lists.newArrayList();
     VolumeList volumes;
-    if (Strings.isNullOrEmpty(userName)) {
-      volumes = getAllVolumes();
-    } else {
-      volumes = getVolumesByUser(userName);
+    if (StringUtil.isBlank(userName)) {
+      throw new OMException("User name is required to list Volumes.",
+          ResultCodes.FAILED_USER_NOT_FOUND);
     }
+    volumes = getVolumesByUser(userName);
 
     if (volumes == null || volumes.getVolumeNamesCount() == 0) {
       return result;
@@ -406,7 +531,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
         continue;
       }
       if (startKeyFound && result.size() < maxKeys) {
-        byte[] volumeInfo = store.get(this.getVolumeKey(volumeName));
+        byte[] volumeInfo = getVolumeTable().get(this.getVolumeKey(volumeName));
         if (volumeInfo == null) {
           // Could not get volume info by given volume name,
           // since the volume name is loaded from db,
@@ -433,7 +558,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
       throws OMException {
     VolumeList volumes = null;
     try {
-      byte[] volumesInBytes = store.get(userNameKey);
+      byte[] volumesInBytes = getUserTable().get(userNameKey);
       if (volumesInBytes == null) {
         // No volume found for this user, return an empty list
         return VolumeList.newBuilder().build();
@@ -447,32 +572,12 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
     return volumes;
   }
 
-  private VolumeList getAllVolumes() throws IOException {
-    // Scan all users in database
-    KeyPrefixFilter filter =
-        new KeyPrefixFilter().addFilter(OzoneConsts.OM_USER_PREFIX);
-    // We are not expecting a huge number of users per cluster,
-    // it should be fine to scan all users in db and return us a
-    // list of volume names in string per user.
-    List<Map.Entry<byte[], byte[]>> rangeKVs = store
-        .getSequentialRangeKVs(null, Integer.MAX_VALUE, filter);
-
-    VolumeList.Builder builder = VolumeList.newBuilder();
-    for (Map.Entry<byte[], byte[]> entry : rangeKVs) {
-      VolumeList volumes = this.getVolumesByUser(entry.getKey());
-      builder.addAllVolumeNames(volumes.getVolumeNamesList());
-    }
-
-    return builder.build();
-  }
-
   @Override
   public List<BlockGroup> getPendingDeletionKeys(final int count)
       throws IOException {
     List<BlockGroup> keyBlocksList = Lists.newArrayList();
-    List<Map.Entry<byte[], byte[]>> rangeResult =
-        store.getRangeKVs(null, count,
-            MetadataKeyFilters.getDeletingKeyFilter());
+    // TODO: Fix this later, Not part of this patch.
+    List<Map.Entry<byte[], byte[]>> rangeResult = Collections.emptyList();
     for (Map.Entry<byte[], byte[]> entry : rangeResult) {
       OmKeyInfo info =
           OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(entry.getValue()));
@@ -482,7 +587,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
         return Collections.emptyList();
       }
       List<BlockID> item = latest.getLocationList().stream()
-          .map(b->new BlockID(b.getContainerID(), b.getLocalID()))
+          .map(b -> new BlockID(b.getContainerID(), b.getLocalID()))
           .collect(Collectors.toList());
       BlockGroup keyBlocks = BlockGroup.newBuilder()
           .setKeyName(DFSUtil.bytes2String(entry.getKey()))
@@ -497,11 +602,9 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
   public List<BlockGroup> getExpiredOpenKeys() throws IOException {
     List<BlockGroup> keyBlocksList = Lists.newArrayList();
     long now = Time.now();
-    final MetadataKeyFilter openKeyFilter =
-        new KeyPrefixFilter().addFilter(OPEN_KEY_PREFIX);
-    List<Map.Entry<byte[], byte[]>> rangeResult =
-        store.getSequentialRangeKVs(null, Integer.MAX_VALUE,
-            openKeyFilter);
+    // TODO: Fix the getExpiredOpenKeys, Not part of this patch.
+    List<Map.Entry<byte[], byte[]>> rangeResult = Collections.emptyList();
+
     for (Map.Entry<byte[], byte[]> entry : rangeResult) {
       OmKeyInfo info =
           OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(entry.getValue()));
@@ -513,7 +616,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
       // Get block keys as a list.
       List<BlockID> item = info.getLatestVersionLocations()
           .getBlocksLatestVersionOnly().stream()
-          .map(b->new BlockID(b.getContainerID(), b.getLocalID()))
+          .map(b -> new BlockID(b.getContainerID(), b.getLocalID()))
           .collect(Collectors.toList());
       BlockGroup keyBlocks = BlockGroup.newBuilder()
           .setKeyName(DFSUtil.bytes2String(entry.getKey()))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
index 71fa921..c06508d 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
@@ -21,14 +21,27 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.protobuf.BlockingService;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.ScmInfo;
+import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
+import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
+import org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolPB;
+import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
+import org.apache.hadoop.hdds.server.ServiceRuntimeInfoImpl;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.hdds.server.ServiceRuntimeInfoImpl;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.util.MBeans;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.ozone.common.Storage.StorageState;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
 import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
@@ -39,36 +52,12 @@ import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
 import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
 import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB;
-import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
-import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
-import org.apache.hadoop.metrics2.util.MBeans;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
-    .ServicePort;
-import org.apache.hadoop.ozone.protocol.proto
-    .OzoneManagerProtocolProtos.OzoneAclInfo;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneAclInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServicePort;
 import org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB;
-import org.apache.hadoop.hdds.scm.ScmInfo;
-import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
-import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
-import org.apache.hadoop.hdds.scm.protocolPB
-    .ScmBlockLocationProtocolClientSideTranslatorPB;
-import org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolPB;
-import org.apache.hadoop.hdds.scm.protocolPB
-    .StorageContainerLocationProtocolClientSideTranslatorPB;
-import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.StringUtils;
-
-import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForBlockClients;
-import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
-import static org.apache.hadoop.hdds.HddsUtils.isHddsEnabled;
-import static org.apache.hadoop.ozone.OmUtils.getOmAddress;
-import static org.apache.hadoop.hdds.server.ServerUtils
-    .updateRPCListenAddress;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -81,18 +70,17 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForBlockClients;
+import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
+import static org.apache.hadoop.hdds.HddsUtils.isHddsEnabled;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY;
+import static org.apache.hadoop.hdds.server.ServerUtils.updateRPCListenAddress;
+import static org.apache.hadoop.ozone.OmUtils.getOmAddress;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED;
-import static org.apache.hadoop.ozone.om.OMConfigKeys
-    .OZONE_OM_ADDRESS_KEY;
-import static org.apache.hadoop.ozone.om.OMConfigKeys
-    .OZONE_OM_HANDLER_COUNT_DEFAULT;
-import static org.apache.hadoop.ozone.om.OMConfigKeys
-    .OZONE_OM_HANDLER_COUNT_KEY;
-import static org.apache.hadoop.ozone.protocol.proto
-    .OzoneManagerProtocolProtos.OzoneManagerService
-    .newReflectiveBlockingService;
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos
-    .NodeState.HEALTHY;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HANDLER_COUNT_DEFAULT;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HANDLER_COUNT_KEY;
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneManagerService.newReflectiveBlockingService;
 import static org.apache.hadoop.util.ExitUtil.terminate;
 
 /**
@@ -108,33 +96,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
       "Usage: \n ozone om [genericOptions] " + "[ "
           + StartupOption.CREATEOBJECTSTORE.getName() + " ]\n " + "ozone om [ "
           + StartupOption.HELP.getName() + " ]\n";
-
-  /** Startup options. */
-  public enum StartupOption {
-    CREATEOBJECTSTORE("-createObjectStore"),
-    HELP("-help"),
-    REGULAR("-regular");
-
-    private final String name;
-
-    StartupOption(String arg) {
-      this.name = arg;
-    }
-
-    public String getName() {
-      return name;
-    }
-
-    public static StartupOption parse(String value) {
-      for (StartupOption option : StartupOption.values()) {
-        if (option.name.equalsIgnoreCase(value)) {
-          return option;
-        }
-      }
-      return null;
-    }
-  }
-
   private final OzoneConfiguration configuration;
   private final RPC.Server omRpcServer;
   private final InetSocketAddress omRpcAddress;
@@ -238,20 +199,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     return scmContainerClient;
   }
 
-  @VisibleForTesting
-  public KeyManager getKeyManager() {
-    return keyManager;
-  }
-
-  @VisibleForTesting
-  public ScmInfo getScmInfo() throws IOException {
-    return scmBlockClient.getScmInfo();
-  }
-
-  @VisibleForTesting
-  public OMStorage getOmStorage() {
-    return omStorage;
-  }
   /**
    * Starts an RPC server, if configured.
    *
@@ -260,7 +207,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    * @param protocol RPC protocol provided by RPC server
    * @param instance RPC protocol implementation instance
    * @param handlerCount RPC server handler count
-   *
    * @return RPC server
    * @throws IOException if there is an I/O error while creating RPC server
    */
@@ -282,18 +228,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   }
 
   /**
-   * Get metadata manager.
-   * @return metadata manager.
-   */
-  public OMMetadataManager getMetadataManager() {
-    return metadataManager;
-  }
-
-  public OMMetrics getMetrics() {
-    return metrics;
-  }
-
-  /**
    * Main entry point for starting OzoneManager.
    *
    * @param argv arguments
@@ -329,6 +263,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
 
   /**
    * Constructs OM instance based on command line arguments.
+   *
    * @param argv Command line arguments
    * @param conf OzoneConfiguration
    * @return OM instance
@@ -336,7 +271,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    */
 
   public static OzoneManager createOm(String[] argv,
-                                      OzoneConfiguration conf) throws IOException {
+      OzoneConfiguration conf) throws IOException {
     if (!isHddsEnabled(conf)) {
       System.err.println("OM cannot be started in secure mode or when " +
           OZONE_ENABLED + " is set to false");
@@ -363,9 +298,11 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
 
   /**
    * Initializes the OM instance.
+   *
    * @param conf OzoneConfiguration
    * @return true if OM initialization succeeds, false otherwise
-   * @throws IOException in case ozone metadata directory path is not accessible
+   * @throws IOException in case ozone metadata directory path is not
+   *                     accessible
    */
 
   private static boolean omInit(OzoneConfiguration conf) throws IOException {
@@ -406,14 +343,17 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
 
   /**
    * Parses the command line options for OM initialization.
+   *
    * @param args command line arguments
    * @return StartupOption if options are valid, null otherwise
    */
   private static StartupOption parseArguments(String[] args) {
     if (args == null || args.length == 0) {
       return StartupOption.REGULAR;
-    } else if (args.length == 1) {
-      return StartupOption.parse(args[0]);
+    } else {
+      if (args.length == 1) {
+        return StartupOption.parse(args[0]);
+      }
     }
     return null;
   }
@@ -432,6 +372,34 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
         String.format("%s not started", description);
   }
 
+  @VisibleForTesting
+  public KeyManager getKeyManager() {
+    return keyManager;
+  }
+
+  @VisibleForTesting
+  public ScmInfo getScmInfo() throws IOException {
+    return scmBlockClient.getScmInfo();
+  }
+
+  @VisibleForTesting
+  public OMStorage getOmStorage() {
+    return omStorage;
+  }
+
+  /**
+   * Get metadata manager.
+   *
+   * @return metadata manager.
+   */
+  public OMMetadataManager getMetadataManager() {
+    return metadataManager;
+  }
+
+  public OMMetrics getMetrics() {
+    return metrics;
+  }
+
   /**
    * Start service.
    */
@@ -533,8 +501,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    *
    * @param volume - volume
    * @param userAcl - user acls which needs to be checked for access
-   * @return true if the user has required access for the volume,
-   *         false otherwise
+   * @return true if the user has required access for the volume, false
+   * otherwise
    * @throws IOException
    */
   @Override
@@ -597,7 +565,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    */
   @Override
   public List<OmVolumeArgs> listVolumeByUser(String userName, String prefix,
-                                             String prevKey, int maxKeys) throws IOException {
+      String prevKey, int maxKeys) throws IOException {
     try {
       metrics.incNumVolumeLists();
       return volumeManager.listVolumes(userName, prefix, prevKey, maxKeys);
@@ -651,7 +619,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    */
   @Override
   public List<OmBucketInfo> listBuckets(String volumeName,
-                                        String startKey, String prefix, int maxNumOfBuckets)
+      String startKey, String prefix, int maxNumOfBuckets)
       throws IOException {
     try {
       metrics.incNumBucketLists();
@@ -702,7 +670,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   }
 
   @Override
-  public void commitKey(OmKeyArgs args, int clientID)
+  public void commitKey(OmKeyArgs args, long clientID)
       throws IOException {
     try {
       metrics.incNumKeyCommits();
@@ -714,7 +682,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   }
 
   @Override
-  public OmKeyLocationInfo allocateBlock(OmKeyArgs args, int clientID)
+  public OmKeyLocationInfo allocateBlock(OmKeyArgs args, long clientID)
       throws IOException {
     try {
       metrics.incNumBlockAllocateCalls();
@@ -773,7 +741,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
 
   @Override
   public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
-                                  String startKey, String keyPrefix, int maxKeys) throws IOException {
+      String startKey, String keyPrefix, int maxKeys) throws IOException {
     try {
       metrics.incNumKeyLists();
       return keyManager.listKeys(volumeName, bucketName,
@@ -786,6 +754,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
 
   /**
    * Sets bucket property from args.
+   *
    * @param args - BucketArgs.
    * @throws IOException
    */
@@ -801,9 +770,9 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     }
   }
 
-
   /**
    * Deletes an existing empty bucket from volume.
+   *
    * @param volume - Name of the volume.
    * @param bucket - Name of the bucket.
    * @throws IOException
@@ -853,8 +822,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
         .setNodeType(HddsProtos.NodeType.OM)
         .setHostname(omRpcAddress.getHostName())
         .addServicePort(ServicePort.newBuilder()
-                .setType(ServicePort.Type.RPC)
-                .setValue(omRpcAddress.getPort())
+            .setType(ServicePort.Type.RPC)
+            .setValue(omRpcAddress.getPort())
             .build());
     if (httpServer.getHttpAddress() != null) {
       omServiceInfoBuilder.addServicePort(ServicePort.newBuilder()
@@ -908,4 +877,32 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     // metrics.incNumGetServiceListFails()
     return services;
   }
+
+  /**
+   * Startup options.
+   */
+  public enum StartupOption {
+    CREATEOBJECTSTORE("-createObjectStore"),
+    HELP("-help"),
+    REGULAR("-regular");
+
+    private final String name;
+
+    StartupOption(String arg) {
+      this.name = arg;
+    }
+
+    public static StartupOption parse(String value) {
+      for (StartupOption option : StartupOption.values()) {
+        if (option.name.equalsIgnoreCase(value)) {
+          return option;
+        }
+      }
+      return null;
+    }
+
+    public String getName() {
+      return name;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManagerImpl.java
index e50145d..419b0aa 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManagerImpl.java
@@ -28,7 +28,9 @@ import org.apache.hadoop.ozone.protocol.proto
     .OzoneManagerProtocolProtos.VolumeInfo;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.util.Time;
-import org.apache.hadoop.utils.BatchOperation;
+import org.apache.hadoop.utils.RocksDBStore;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteBatch;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -67,10 +69,10 @@ public class VolumeManagerImpl implements VolumeManager {
 
   // Helpers to add and delete volume from user list
   private void addVolumeToOwnerList(String volume, String owner,
-      BatchOperation batchOperation) throws IOException {
+      WriteBatch batchOperation) throws RocksDBException, IOException {
     // Get the volume list
     byte[] dbUserKey = metadataManager.getUserKey(owner);
-    byte[] volumeList  = metadataManager.get(dbUserKey);
+    byte[] volumeList  = metadataManager.getUserTable().get(dbUserKey);
     List<String> prevVolList = new LinkedList<>();
     if (volumeList != null) {
       VolumeList vlist = VolumeList.parseFrom(volumeList);
@@ -87,15 +89,15 @@ public class VolumeManagerImpl implements VolumeManager {
     prevVolList.add(volume);
     VolumeList newVolList = VolumeList.newBuilder()
         .addAllVolumeNames(prevVolList).build();
-    batchOperation.put(dbUserKey, newVolList.toByteArray());
+    batchOperation.put(metadataManager.getUserTable().getHandle(),
+        dbUserKey, newVolList.toByteArray());
   }
 
   private void delVolumeFromOwnerList(String volume, String owner,
-                                      BatchOperation batchOperation)
-      throws IOException {
+      WriteBatch batch) throws RocksDBException, IOException {
     // Get the volume list
     byte[] dbUserKey = metadataManager.getUserKey(owner);
-    byte[] volumeList  = metadataManager.get(dbUserKey);
+    byte[] volumeList  = metadataManager.getUserTable().get(dbUserKey);
     List<String> prevVolList = new LinkedList<>();
     if (volumeList != null) {
       VolumeList vlist = VolumeList.parseFrom(volumeList);
@@ -108,11 +110,12 @@ public class VolumeManagerImpl implements VolumeManager {
     // Remove the volume from the list
     prevVolList.remove(volume);
     if (prevVolList.size() == 0) {
-      batchOperation.delete(dbUserKey);
+      batch.delete(dbUserKey);
     } else {
       VolumeList newVolList = VolumeList.newBuilder()
           .addAllVolumeNames(prevVolList).build();
-      batchOperation.put(dbUserKey, newVolList.toByteArray());
+      batch.put(metadataManager.getUserTable().getHandle(),
+          dbUserKey, newVolList.toByteArray());
     }
   }
 
@@ -126,7 +129,7 @@ public class VolumeManagerImpl implements VolumeManager {
     metadataManager.writeLock().lock();
     try {
       byte[] dbVolumeKey = metadataManager.getVolumeKey(args.getVolume());
-      byte[] volumeInfo = metadataManager.get(dbVolumeKey);
+      byte[] volumeInfo = metadataManager.getVolumeTable().get(dbVolumeKey);
 
       // Check of the volume already exists
       if (volumeInfo != null) {
@@ -134,37 +137,45 @@ public class VolumeManagerImpl implements VolumeManager {
         throw new OMException(ResultCodes.FAILED_VOLUME_ALREADY_EXISTS);
       }
 
-      BatchOperation batch = new BatchOperation();
-      // Write the vol info
-      List<HddsProtos.KeyValue> metadataList = new LinkedList<>();
-      for (Map.Entry<String, String> entry : args.getKeyValueMap().entrySet()) {
-        metadataList.add(HddsProtos.KeyValue.newBuilder()
-            .setKey(entry.getKey()).setValue(entry.getValue()).build());
+      try(WriteBatch batch = new WriteBatch()) {
+        // Write the vol info
+        List<HddsProtos.KeyValue> metadataList = new LinkedList<>();
+        for (Map.Entry<String, String> entry :
+            args.getKeyValueMap().entrySet()) {
+          metadataList.add(HddsProtos.KeyValue.newBuilder()
+              .setKey(entry.getKey()).setValue(entry.getValue()).build());
+        }
+        List<OzoneAclInfo> aclList = args.getAclMap().ozoneAclGetProtobuf();
+
+        VolumeInfo newVolumeInfo = VolumeInfo.newBuilder()
+            .setAdminName(args.getAdminName())
+            .setOwnerName(args.getOwnerName())
+            .setVolume(args.getVolume())
+            .setQuotaInBytes(args.getQuotaInBytes())
+            .addAllMetadata(metadataList)
+            .addAllVolumeAcls(aclList)
+            .setCreationTime(Time.now())
+            .build();
+        batch.put(metadataManager.getVolumeTable().getHandle(),
+            dbVolumeKey, newVolumeInfo.toByteArray());
+
+        // Add volume to user list
+        addVolumeToOwnerList(args.getVolume(), args.getOwnerName(), batch);
+        metadataManager.getStore().write(batch);
       }
-      List<OzoneAclInfo> aclList = args.getAclMap().ozoneAclGetProtobuf();
-
-      VolumeInfo newVolumeInfo = VolumeInfo.newBuilder()
-          .setAdminName(args.getAdminName())
-          .setOwnerName(args.getOwnerName())
-          .setVolume(args.getVolume())
-          .setQuotaInBytes(args.getQuotaInBytes())
-          .addAllMetadata(metadataList)
-          .addAllVolumeAcls(aclList)
-          .setCreationTime(Time.now())
-          .build();
-      batch.put(dbVolumeKey, newVolumeInfo.toByteArray());
-
-      // Add volume to user list
-      addVolumeToOwnerList(args.getVolume(), args.getOwnerName(), batch);
-      metadataManager.writeBatch(batch);
       LOG.debug("created volume:{} user:{}", args.getVolume(),
           args.getOwnerName());
-    } catch (IOException ex) {
+    } catch (RocksDBException | IOException ex) {
       if (!(ex instanceof OMException)) {
         LOG.error("Volume creation failed for user:{} volume:{}",
             args.getOwnerName(), args.getVolume(), ex);
       }
-      throw ex;
+      if(ex instanceof RocksDBException) {
+        throw RocksDBStore.toIOException("Volume creation failed.",
+            (RocksDBException) ex);
+      } else {
+        throw (IOException) ex;
+      }
     } finally {
       metadataManager.writeLock().unlock();
     }
@@ -184,7 +195,7 @@ public class VolumeManagerImpl implements VolumeManager {
     metadataManager.writeLock().lock();
     try {
       byte[] dbVolumeKey = metadataManager.getVolumeKey(volume);
-      byte[] volInfo = metadataManager.get(dbVolumeKey);
+      byte[] volInfo = metadataManager.getVolumeTable().get(dbVolumeKey);
       if (volInfo == null) {
         LOG.debug("Changing volume ownership failed for user:{} volume:{}",
             owner, volume);
@@ -195,28 +206,34 @@ public class VolumeManagerImpl implements VolumeManager {
       OmVolumeArgs volumeArgs = OmVolumeArgs.getFromProtobuf(volumeInfo);
       Preconditions.checkState(volume.equals(volumeInfo.getVolume()));
 
-      BatchOperation batch = new BatchOperation();
-      delVolumeFromOwnerList(volume, volumeArgs.getOwnerName(), batch);
-      addVolumeToOwnerList(volume, owner, batch);
-
-      OmVolumeArgs newVolumeArgs =
-          OmVolumeArgs.newBuilder().setVolume(volumeArgs.getVolume())
-              .setAdminName(volumeArgs.getAdminName())
-              .setOwnerName(owner)
-              .setQuotaInBytes(volumeArgs.getQuotaInBytes())
-              .setCreationTime(volumeArgs.getCreationTime())
-              .build();
-
-      VolumeInfo newVolumeInfo = newVolumeArgs.getProtobuf();
-      batch.put(dbVolumeKey, newVolumeInfo.toByteArray());
-
-      metadataManager.writeBatch(batch);
-    } catch (IOException ex) {
+      try(WriteBatch batch = new WriteBatch()) {
+        delVolumeFromOwnerList(volume, volumeArgs.getOwnerName(), batch);
+        addVolumeToOwnerList(volume, owner, batch);
+
+        OmVolumeArgs newVolumeArgs =
+            OmVolumeArgs.newBuilder().setVolume(volumeArgs.getVolume())
+                .setAdminName(volumeArgs.getAdminName())
+                .setOwnerName(owner)
+                .setQuotaInBytes(volumeArgs.getQuotaInBytes())
+                .setCreationTime(volumeArgs.getCreationTime())
+                .build();
+
+        VolumeInfo newVolumeInfo = newVolumeArgs.getProtobuf();
+        batch.put(metadataManager.getVolumeTable().getHandle(),
+            dbVolumeKey, newVolumeInfo.toByteArray());
+        metadataManager.getStore().write(batch);
+      }
+    } catch (RocksDBException | IOException ex) {
       if (!(ex instanceof OMException)) {
         LOG.error("Changing volume ownership failed for user:{} volume:{}",
             owner, volume, ex);
       }
-      throw ex;
+      if(ex instanceof RocksDBException) {
+        throw RocksDBStore.toIOException("Volume creation failed.",
+            (RocksDBException) ex);
+      } else {
+        throw (IOException) ex;
+      }
     } finally {
       metadataManager.writeLock().unlock();
     }
@@ -234,7 +251,7 @@ public class VolumeManagerImpl implements VolumeManager {
     metadataManager.writeLock().lock();
     try {
       byte[] dbVolumeKey = metadataManager.getVolumeKey(volume);
-      byte[] volInfo = metadataManager.get(dbVolumeKey);
+      byte[] volInfo = metadataManager.getVolumeTable().get(dbVolumeKey);
       if (volInfo == null) {
         LOG.debug("volume:{} does not exist", volume);
         throw new OMException(ResultCodes.FAILED_VOLUME_NOT_FOUND);
@@ -253,7 +270,8 @@ public class VolumeManagerImpl implements VolumeManager {
               .setCreationTime(volumeArgs.getCreationTime()).build();
 
       VolumeInfo newVolumeInfo = newVolumeArgs.getProtobuf();
-      metadataManager.put(dbVolumeKey, newVolumeInfo.toByteArray());
+      metadataManager.getVolumeTable().put(dbVolumeKey,
+          newVolumeInfo.toByteArray());
     } catch (IOException ex) {
       if (!(ex instanceof OMException)) {
         LOG.error("Changing volume quota failed for volume:{} quota:{}", volume,
@@ -276,7 +294,7 @@ public class VolumeManagerImpl implements VolumeManager {
     metadataManager.readLock().lock();
     try {
       byte[] dbVolumeKey = metadataManager.getVolumeKey(volume);
-      byte[] volInfo = metadataManager.get(dbVolumeKey);
+      byte[] volInfo = metadataManager.getVolumeTable().get(dbVolumeKey);
       if (volInfo == null) {
         LOG.debug("volume:{} does not exist", volume);
         throw new OMException(ResultCodes.FAILED_VOLUME_NOT_FOUND);
@@ -307,9 +325,9 @@ public class VolumeManagerImpl implements VolumeManager {
     Preconditions.checkNotNull(volume);
     metadataManager.writeLock().lock();
     try {
-      BatchOperation batch = new BatchOperation();
+
       byte[] dbVolumeKey = metadataManager.getVolumeKey(volume);
-      byte[] volInfo = metadataManager.get(dbVolumeKey);
+      byte[] volInfo = metadataManager.getVolumeTable().get(dbVolumeKey);
       if (volInfo == null) {
         LOG.debug("volume:{} does not exist", volume);
         throw new OMException(ResultCodes.FAILED_VOLUME_NOT_FOUND);
@@ -324,14 +342,22 @@ public class VolumeManagerImpl implements VolumeManager {
       Preconditions.checkState(volume.equals(volumeInfo.getVolume()));
       // delete the volume from the owner list
       // as well as delete the volume entry
-      delVolumeFromOwnerList(volume, volumeInfo.getOwnerName(), batch);
-      batch.delete(dbVolumeKey);
-      metadataManager.writeBatch(batch);
-    } catch (IOException ex) {
+      try(WriteBatch batch = new WriteBatch()) {
+        delVolumeFromOwnerList(volume, volumeInfo.getOwnerName(), batch);
+        batch.delete(metadataManager.getVolumeTable().getHandle(),
+            dbVolumeKey);
+        metadataManager.getStore().write(batch);
+      }
+    } catch (RocksDBException| IOException ex) {
       if (!(ex instanceof OMException)) {
         LOG.error("Delete volume failed for volume:{}", volume, ex);
       }
-      throw ex;
+      if(ex instanceof RocksDBException) {
+        throw RocksDBStore.toIOException("Volume creation failed.",
+            (RocksDBException) ex);
+      } else {
+        throw (IOException) ex;
+      }
     } finally {
       metadataManager.writeLock().unlock();
     }
@@ -352,7 +378,7 @@ public class VolumeManagerImpl implements VolumeManager {
     metadataManager.readLock().lock();
     try {
       byte[] dbVolumeKey = metadataManager.getVolumeKey(volume);
-      byte[] volInfo = metadataManager.get(dbVolumeKey);
+      byte[] volInfo = metadataManager.getVolumeTable().get(dbVolumeKey);
       if (volInfo == null) {
         LOG.debug("volume:{} does not exist", volume);
         throw  new OMException(ResultCodes.FAILED_VOLUME_NOT_FOUND);
@@ -378,7 +404,7 @@ public class VolumeManagerImpl implements VolumeManager {
    */
   @Override
   public List<OmVolumeArgs> listVolumes(String userName,
-                                        String prefix, String startKey, int maxKeys) throws IOException {
+      String prefix, String startKey, int maxKeys) throws IOException {
     metadataManager.readLock().lock();
     try {
       return metadataManager.listVolumes(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java
index 45ec2d0..06d782b 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java
@@ -526,8 +526,7 @@ public class OzoneManagerProtocolServerSideTranslatorPB implements
           .setFactor(factor)
           .setDataSize(keyArgs.getDataSize())
           .build();
-      int id = request.getClientID();
-      impl.commitKey(omKeyArgs, id);
+      impl.commitKey(omKeyArgs, request.getClientID());
       resp.setStatus(Status.OK);
     } catch (IOException e) {
       resp.setStatus(exceptionToResponseStatus(e));
@@ -547,8 +546,8 @@ public class OzoneManagerProtocolServerSideTranslatorPB implements
           .setBucketName(keyArgs.getBucketName())
           .setKeyName(keyArgs.getKeyName())
           .build();
-      int id = request.getClientID();
-      OmKeyLocationInfo newLocation = impl.allocateBlock(omKeyArgs, id);
+      OmKeyLocationInfo newLocation = impl.allocateBlock(omKeyArgs,
+          request.getClientID());
       resp.setKeyLocation(newLocation.getProtobuf());
       resp.setStatus(Status.OK);
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestBucketManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestBucketManagerImpl.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestBucketManagerImpl.java
index 1ecac7f..9684a1f 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestBucketManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestBucketManagerImpl.java
@@ -17,33 +17,26 @@
 package org.apache.hadoop.ozone.om;
 
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
-import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
-import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
-import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
 import org.junit.runner.RunWith;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
 import org.mockito.runners.MockitoJUnitRunner;
-import org.mockito.stubbing.Answer;
 
+import java.io.File;
 import java.io.IOException;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
 import java.util.LinkedList;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import static org.mockito.Mockito.any;
+import java.util.List;
 
 /**
  * Tests BucketManagerImpl, mocks OMMetadataManager for testing.
@@ -53,86 +46,35 @@ public class TestBucketManagerImpl {
   @Rule
   public ExpectedException thrown = ExpectedException.none();
 
-  private OMMetadataManager getMetadataManagerMock(String... volumesToCreate)
-      throws IOException {
-    OMMetadataManager metadataManager = Mockito.mock(OMMetadataManager.class);
-    Map<String, byte[]> metadataDB = new HashMap<>();
-    ReadWriteLock lock = new ReentrantReadWriteLock();
-
-    Mockito.when(metadataManager.writeLock()).thenReturn(lock.writeLock());
-    Mockito.when(metadataManager.readLock()).thenReturn(lock.readLock());
-    Mockito.when(metadataManager.getVolumeKey(any(String.class))).thenAnswer(
-        (InvocationOnMock invocation) ->
-            DFSUtil.string2Bytes(
-                OzoneConsts.OM_VOLUME_PREFIX + invocation.getArguments()[0]));
-    Mockito.when(metadataManager
-        .getBucketKey(any(String.class), any(String.class))).thenAnswer(
-            (InvocationOnMock invocation) ->
-                DFSUtil.string2Bytes(
-                    OzoneConsts.OM_VOLUME_PREFIX
-                        + invocation.getArguments()[0]
-                        + OzoneConsts.OM_BUCKET_PREFIX
-                        + invocation.getArguments()[1]));
-
-    Mockito.doAnswer(
-        new Answer<Boolean>() {
-          @Override
-          public Boolean answer(InvocationOnMock invocation)
-              throws Throwable {
-            String keyRootName =  OzoneConsts.OM_KEY_PREFIX
-                + invocation.getArguments()[0]
-                + OzoneConsts.OM_KEY_PREFIX
-                + invocation.getArguments()[1]
-                + OzoneConsts.OM_KEY_PREFIX;
-            Iterator<String> keyIterator = metadataDB.keySet().iterator();
-            while(keyIterator.hasNext()) {
-              if(keyIterator.next().startsWith(keyRootName)) {
-                return false;
-              }
-            }
-            return true;
-          }
-        }).when(metadataManager).isBucketEmpty(any(String.class),
-        any(String.class));
-
-    Mockito.doAnswer(
-        new Answer<Void>() {
-          @Override
-          public Void answer(InvocationOnMock invocation) throws Throwable {
-            metadataDB.put(DFSUtil.bytes2String(
-                (byte[])invocation.getArguments()[0]),
-                (byte[])invocation.getArguments()[1]);
-            return null;
-          }
-        }).when(metadataManager).put(any(byte[].class), any(byte[].class));
-
-    Mockito.when(metadataManager.get(any(byte[].class))).thenAnswer(
-        (InvocationOnMock invocation) ->
-            metadataDB.get(DFSUtil.bytes2String(
-                (byte[])invocation.getArguments()[0]))
-    );
-    Mockito.doAnswer(
-        new Answer<Void>() {
-          @Override
-          public Void answer(InvocationOnMock invocation) throws Throwable {
-            metadataDB.remove(DFSUtil.bytes2String(
-                (byte[])invocation.getArguments()[0]));
-            return null;
-          }
-        }).when(metadataManager).delete(any(byte[].class));
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
 
-    for(String volumeName : volumesToCreate) {
-      byte[] dummyVolumeInfo = DFSUtil.string2Bytes(volumeName);
-      metadataDB.put(OzoneConsts.OM_VOLUME_PREFIX + volumeName,
-                     dummyVolumeInfo);
+  private OzoneConfiguration createNewTestPath() throws IOException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    File newFolder = folder.newFolder();
+    if (!newFolder.exists()) {
+      Assert.assertTrue(newFolder.mkdirs());
     }
-    return metadataManager;
+    ServerUtils.setOzoneMetaDirPath(conf, newFolder.toString());
+    return conf;
+  }
+
+  private OmMetadataManagerImpl createSampleVol() throws IOException {
+    OzoneConfiguration conf = createNewTestPath();
+    OmMetadataManagerImpl metaMgr = new OmMetadataManagerImpl(conf);
+    byte[] volumeKey = metaMgr.getVolumeKey("sampleVol");
+    // This is a simple hack for testing, we just test if the volume via a
+    // null check, do not parse the value part. So just write some dummy value.
+    metaMgr.getVolumeTable().put(volumeKey, volumeKey);
+    return metaMgr;
   }
 
   @Test
-  public void testCreateBucketWithoutVolume() throws IOException {
+  public void testCreateBucketWithoutVolume() throws Exception {
     thrown.expectMessage("Volume doesn't exist");
-    OMMetadataManager metaMgr = getMetadataManagerMock();
+    OzoneConfiguration conf = createNewTestPath();
+    OmMetadataManagerImpl metaMgr =
+        new OmMetadataManagerImpl(conf);
     try {
       BucketManager bucketManager = new BucketManagerImpl(metaMgr);
       OmBucketInfo bucketInfo = OmBucketInfo.newBuilder()
@@ -140,29 +82,35 @@ public class TestBucketManagerImpl {
           .setBucketName("bucketOne")
           .build();
       bucketManager.createBucket(bucketInfo);
-    } catch(OMException omEx) {
+    } catch (OMException omEx) {
       Assert.assertEquals(ResultCodes.FAILED_VOLUME_NOT_FOUND,
           omEx.getResult());
       throw omEx;
+    } finally {
+      metaMgr.getStore().close();
     }
   }
 
   @Test
-  public void testCreateBucket() throws IOException {
-    OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol");
+  public void testCreateBucket() throws Exception {
+    OmMetadataManagerImpl metaMgr = createSampleVol();
+
     BucketManager bucketManager = new BucketManagerImpl(metaMgr);
     OmBucketInfo bucketInfo = OmBucketInfo.newBuilder()
         .setVolumeName("sampleVol")
         .setBucketName("bucketOne")
         .build();
     bucketManager.createBucket(bucketInfo);
-    Assert.assertNotNull(bucketManager.getBucketInfo("sampleVol", "bucketOne"));
+    Assert.assertNotNull(bucketManager.getBucketInfo("sampleVol",
+        "bucketOne"));
+    metaMgr.getStore().close();
   }
 
   @Test
-  public void testCreateAlreadyExistingBucket() throws IOException {
+  public void testCreateAlreadyExistingBucket() throws Exception {
     thrown.expectMessage("Bucket already exist");
-    OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol");
+    OmMetadataManagerImpl metaMgr = createSampleVol();
+
     try {
       BucketManager bucketManager = new BucketManagerImpl(metaMgr);
       OmBucketInfo bucketInfo = OmBucketInfo.newBuilder()
@@ -171,30 +119,37 @@ public class TestBucketManagerImpl {
           .build();
       bucketManager.createBucket(bucketInfo);
       bucketManager.createBucket(bucketInfo);
-    } catch(OMException omEx) {
+    } catch (OMException omEx) {
       Assert.assertEquals(ResultCodes.FAILED_BUCKET_ALREADY_EXISTS,
           omEx.getResult());
       throw omEx;
+    } finally {
+      metaMgr.getStore().close();
     }
   }
 
   @Test
-  public void testGetBucketInfoForInvalidBucket() throws IOException {
+  public void testGetBucketInfoForInvalidBucket() throws Exception {
     thrown.expectMessage("Bucket not found");
+    OmMetadataManagerImpl metaMgr = createSampleVol();
     try {
-      OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol");
+
+
       BucketManager bucketManager = new BucketManagerImpl(metaMgr);
       bucketManager.getBucketInfo("sampleVol", "bucketOne");
-    } catch(OMException omEx) {
+    } catch (OMException omEx) {
       Assert.assertEquals(ResultCodes.FAILED_BUCKET_NOT_FOUND,
           omEx.getResult());
       throw omEx;
+    } finally {
+      metaMgr.getStore().close();
     }
   }
 
   @Test
-  public void testGetBucketInfo() throws IOException {
-    OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol");
+  public void testGetBucketInfo() throws Exception {
+    OmMetadataManagerImpl metaMgr = createSampleVol();
+
     BucketManager bucketManager = new BucketManagerImpl(metaMgr);
     OmBucketInfo bucketInfo = OmBucketInfo.newBuilder()
         .setVolumeName("sampleVol")
@@ -210,11 +165,13 @@ public class TestBucketManagerImpl {
     Assert.assertEquals(StorageType.DISK,
         result.getStorageType());
     Assert.assertEquals(false, result.getIsVersionEnabled());
+    metaMgr.getStore().close();
   }
 
   @Test
-  public void testSetBucketPropertyAddACL() throws IOException {
-    OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol");
+  public void testSetBucketPropertyAddACL() throws Exception {
+    OmMetadataManagerImpl metaMgr = createSampleVol();
+
     List<OzoneAcl> acls = new LinkedList<>();
     OzoneAcl ozoneAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER,
         "root", OzoneAcl.OzoneACLRights.READ);
@@ -247,11 +204,13 @@ public class TestBucketManagerImpl {
         "sampleVol", "bucketOne");
     Assert.assertEquals(2, updatedResult.getAcls().size());
     Assert.assertTrue(updatedResult.getAcls().contains(newAcl));
+    metaMgr.getStore().close();
   }
 
   @Test
-  public void testSetBucketPropertyRemoveACL() throws IOException {
-    OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol");
+  public void testSetBucketPropertyRemoveACL() throws Exception {
+    OmMetadataManagerImpl metaMgr = createSampleVol();
+
     List<OzoneAcl> acls = new LinkedList<>();
     OzoneAcl aclOne = new OzoneAcl(OzoneAcl.OzoneACLType.USER,
         "root", OzoneAcl.OzoneACLRights.READ);
@@ -283,11 +242,13 @@ public class TestBucketManagerImpl {
         "sampleVol", "bucketOne");
     Assert.assertEquals(1, updatedResult.getAcls().size());
     Assert.assertFalse(updatedResult.getAcls().contains(aclTwo));
+    metaMgr.getStore().close();
   }
 
   @Test
-  public void testSetBucketPropertyChangeStorageType() throws IOException {
-    OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol");
+  public void testSetBucketPropertyChangeStorageType() throws Exception {
+    OmMetadataManagerImpl metaMgr = createSampleVol();
+
     BucketManager bucketManager = new BucketManagerImpl(metaMgr);
     OmBucketInfo bucketInfo = OmBucketInfo.newBuilder()
         .setVolumeName("sampleVol")
@@ -309,11 +270,13 @@ public class TestBucketManagerImpl {
         "sampleVol", "bucketOne");
     Assert.assertEquals(StorageType.SSD,
         updatedResult.getStorageType());
+    metaMgr.getStore().close();
   }
 
   @Test
-  public void testSetBucketPropertyChangeVersioning() throws IOException {
-    OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol");
+  public void testSetBucketPropertyChangeVersioning() throws Exception {
+    OmMetadataManagerImpl metaMgr = createSampleVol();
+
     BucketManager bucketManager = new BucketManagerImpl(metaMgr);
     OmBucketInfo bucketInfo = OmBucketInfo.newBuilder()
         .setVolumeName("sampleVol")
@@ -333,21 +296,22 @@ public class TestBucketManagerImpl {
     OmBucketInfo updatedResult = bucketManager.getBucketInfo(
         "sampleVol", "bucketOne");
     Assert.assertTrue(updatedResult.getIsVersionEnabled());
+    metaMgr.getStore().close();
   }
 
   @Test
-  public void testDeleteBucket() throws IOException {
+  public void testDeleteBucket() throws Exception {
     thrown.expectMessage("Bucket not found");
-    OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol");
+    OmMetadataManagerImpl metaMgr = createSampleVol();
     BucketManager bucketManager = new BucketManagerImpl(metaMgr);
-    for(int i = 0; i < 5; i++) {
+    for (int i = 0; i < 5; i++) {
       OmBucketInfo bucketInfo = OmBucketInfo.newBuilder()
           .setVolumeName("sampleVol")
           .setBucketName("bucket_" + i)
           .build();
       bucketManager.createBucket(bucketInfo);
     }
-    for(int i = 0; i < 5; i++) {
+    for (int i = 0; i < 5; i++) {
       Assert.assertEquals("bucket_" + i,
           bucketManager.getBucketInfo(
               "sampleVol", "bucket_" + i).getBucketName());
@@ -356,22 +320,23 @@ public class TestBucketManagerImpl {
       bucketManager.deleteBucket("sampleVol", "bucket_1");
       Assert.assertNotNull(bucketManager.getBucketInfo(
           "sampleVol", "bucket_2"));
-    } catch(IOException ex) {
+    } catch (IOException ex) {
       Assert.fail(ex.getMessage());
     }
     try {
       bucketManager.getBucketInfo("sampleVol", "bucket_1");
-    } catch(OMException omEx) {
+    } catch (OMException omEx) {
       Assert.assertEquals(ResultCodes.FAILED_BUCKET_NOT_FOUND,
           omEx.getResult());
       throw omEx;
     }
+    metaMgr.getStore().close();
   }
 
   @Test
-  public void testDeleteNonEmptyBucket() throws IOException {
+  public void testDeleteNonEmptyBucket() throws Exception {
     thrown.expectMessage("Bucket is not empty");
-    OMMetadataManager metaMgr = getMetadataManagerMock("sampleVol");
+    OmMetadataManagerImpl metaMgr = createSampleVol();
     BucketManager bucketManager = new BucketManagerImpl(metaMgr);
     OmBucketInfo bucketInfo = OmBucketInfo.newBuilder()
         .setVolumeName("sampleVol")
@@ -379,16 +344,19 @@ public class TestBucketManagerImpl {
         .build();
     bucketManager.createBucket(bucketInfo);
     //Create keys in bucket
-    metaMgr.put(DFSUtil.string2Bytes("/sampleVol/bucketOne/key_one"),
+    metaMgr.getKeyTable().put(DFSUtil.string2Bytes("/sampleVol/bucketOne" +
+            "/key_one"),
         DFSUtil.string2Bytes("value_one"));
-    metaMgr.put(DFSUtil.string2Bytes("/sampleVol/bucketOne/key_two"),
+    metaMgr.getKeyTable().put(DFSUtil.string2Bytes("/sampleVol/bucketOne" +
+            "/key_two"),
         DFSUtil.string2Bytes("value_two"));
     try {
       bucketManager.deleteBucket("sampleVol", "bucketOne");
-    } catch(OMException omEx) {
+    } catch (OMException omEx) {
       Assert.assertEquals(ResultCodes.FAILED_BUCKET_NOT_EMPTY,
           omEx.getResult());
       throw omEx;
     }
+    metaMgr.getStore().close();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java
index 51018a1..080840a 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java
@@ -57,9 +57,8 @@ import java.sql.Statement;
 
 import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_SUFFIX;
 import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
 import static org.apache.hadoop.ozone.OzoneConsts.OM_USER_PREFIX;
-import static org.apache.hadoop.ozone.OzoneConsts.OM_BUCKET_PREFIX;
-import static org.apache.hadoop.ozone.OzoneConsts.OM_VOLUME_PREFIX;
 import static org.apache.hadoop.ozone.OzoneConsts.OPEN_CONTAINERS_DB;
 
 /**
@@ -412,12 +411,15 @@ public class SQLCLI  extends Configured implements Tool {
     }
   }
 
+  // TODO: This has to be fixed.
+  // we don't have prefix anymore. now each key is written into different
+  // table. The logic has to be changed.
   private KeyType getKeyType(String key) {
     if (key.startsWith(OM_USER_PREFIX)) {
       return KeyType.USER;
-    } else if (key.startsWith(OM_VOLUME_PREFIX)) {
-      return key.replaceFirst(OM_VOLUME_PREFIX, "")
-          .contains(OM_BUCKET_PREFIX) ? KeyType.BUCKET : KeyType.VOLUME;
+    } else if (key.startsWith(OM_KEY_PREFIX)) {
+      return key.replaceFirst(OM_KEY_PREFIX, "")
+          .contains(OM_KEY_PREFIX) ? KeyType.BUCKET : KeyType.VOLUME;
     }else {
       return KeyType.KEY;
     }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[33/50] [abbrv] hadoop git commit: HADOOP-15706. Typo in compatibility doc: SHOUD -> SHOULD (Contributed by Laszlo Kollar via Daniel Templeton)

Posted by eh...@apache.org.
HADOOP-15706. Typo in compatibility doc: SHOUD -> SHOULD
(Contributed by Laszlo Kollar via Daniel Templeton)

Change-Id: I6e2459d0700df7f3bad4eac8297a11690191c3ba


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f2c2a68e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f2c2a68e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f2c2a68e

Branch: refs/heads/HDFS-12090
Commit: f2c2a68ec208f640e778fc41f95f0284fcc44729
Parents: 5a0babf
Author: Daniel Templeton <te...@apache.org>
Authored: Thu Aug 30 09:12:36 2018 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Thu Aug 30 09:12:36 2018 -0700

----------------------------------------------------------------------
 .../hadoop-common/src/site/markdown/Compatibility.md               | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f2c2a68e/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md b/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
index 6b17c62..03d162a 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
@@ -187,7 +187,7 @@ existing documentation and tests and/or adding new documentation or tests.
 
 #### Java Binary compatibility for end-user applications i.e. Apache Hadoop ABI
 
-Apache Hadoop revisions SHOUD retain binary compatability such that end-user
+Apache Hadoop revisions SHOULD retain binary compatability such that end-user
 applications continue to work without any modifications. Minor Apache Hadoop
 revisions within the same major revision MUST retain compatibility such that
 existing MapReduce applications (e.g. end-user applications and projects such


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[11/50] [abbrv] hadoop git commit: HDDS-359. RocksDB Profiles support. Contributed by Anu Engineer.

Posted by eh...@apache.org.
HDDS-359. RocksDB Profiles support. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c61824a1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c61824a1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c61824a1

Branch: refs/heads/HDFS-12090
Commit: c61824a18940ef37dc7201717a3115a78bf942d4
Parents: df21e1b
Author: Márton Elek <el...@apache.org>
Authored: Tue Aug 28 19:22:30 2018 +0200
Committer: Márton Elek <el...@apache.org>
Committed: Tue Aug 28 19:33:13 2018 +0200

----------------------------------------------------------------------
 .../org/apache/hadoop/hdds/HddsConfigKeys.java  |   6 +
 .../hadoop/utils/db/DBConfigFromFile.java       | 134 +++++++++++++
 .../org/apache/hadoop/utils/db/DBProfile.java   | 120 +++++++++++
 .../apache/hadoop/utils/db/DBStoreBuilder.java  | 201 +++++++++++++++++++
 .../org/apache/hadoop/utils/db/RDBStore.java    |  32 +--
 .../org/apache/hadoop/utils/db/TableConfig.java |  93 +++++++++
 .../common/src/main/resources/ozone-default.xml |  10 +
 .../hadoop/utils/db/TestDBConfigFromFile.java   | 116 +++++++++++
 .../hadoop/utils/db/TestDBStoreBuilder.java     | 174 ++++++++++++++++
 .../apache/hadoop/utils/db/TestRDBStore.java    |  17 +-
 .../hadoop/utils/db/TestRDBTableStore.java      |  11 +-
 .../common/src/test/resources/test.db.ini       | 145 +++++++++++++
 hadoop-hdds/pom.xml                             |   1 +
 13 files changed, 1040 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
index d25af80..8272ed7 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdds;
 
+import org.apache.hadoop.utils.db.DBProfile;
+
 /**
  * This class contains constants for configuration keys and default values
  * used in hdds.
@@ -58,4 +60,8 @@ public final class HddsConfigKeys {
   public static final String HDDS_DATANODE_VOLUME_CHOOSING_POLICY =
       "hdds.datanode.volume.choosing.policy";
 
+  // DB Profiles used by ROCKDB instances.
+  public static final String HDDS_DB_PROFILE = "hdds.db.profile";
+  public static final DBProfile HDDS_DEFAULT_DB_PROFILE = DBProfile.SSD;
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBConfigFromFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBConfigFromFile.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBConfigFromFile.java
new file mode 100644
index 0000000..753a460
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBConfigFromFile.java
@@ -0,0 +1,134 @@
+/*
+ * 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.utils.db;
+
+import com.google.common.base.Preconditions;
+import org.eclipse.jetty.util.StringUtil;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Env;
+import org.rocksdb.OptionsUtil;
+import org.rocksdb.RocksDBException;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+
+/**
+ * A Class that controls the standard config options of RocksDB.
+ * <p>
+ * Important : Some of the functions in this file are magic functions designed
+ * for the use of OZONE developers only. Due to that this information is
+ * documented in this files only and is *not* intended for end user consumption.
+ * Please do not use this information to tune your production environments.
+ * Please remember the SpiderMan principal; with great power comes great
+ * responsibility.
+ */
+public final class DBConfigFromFile {
+  public static final String CONFIG_DIR = "HADOOP_CONF_DIR";
+
+  private DBConfigFromFile() {
+  }
+
+  public static File getConfigLocation() throws IOException {
+    String path = System.getenv(CONFIG_DIR);
+
+    // Make testing easy.
+    // If there is No Env. defined, let us try to read the JVM property
+    if (StringUtil.isBlank(path)) {
+      path = System.getProperty(CONFIG_DIR);
+    }
+
+    if (StringUtil.isBlank(path)) {
+      throw new IOException("Unable to find the configuration directory. "
+          + "Please make sure that HADOOP_CONF_DIR is setup correctly ");
+    }
+    return new File(path);
+
+  }
+
+  /**
+   * This class establishes a magic pattern where we look for DBFile.ini as the
+   * options for RocksDB.
+   *
+   * @param dbFileName - The DBFile Name. For example, OzoneManager.db
+   * @return Name of the DB File options
+   */
+  public static String getOptionsFileNameFromDB(String dbFileName) {
+    Preconditions.checkNotNull(dbFileName);
+    return dbFileName + ".ini";
+  }
+
+  /**
+   * One of the Magic functions designed for the use of Ozone Developers *ONLY*.
+   * This function takes the name of DB file and looks up the a .ini file that
+   * follows the ROCKSDB config format and uses that file for DBOptions and
+   * Column family Options. The Format for this file is specified by RockDB.
+   * <p>
+   * Here is a sample config from RocksDB sample Repo.
+   * <p>
+   * https://github.com/facebook/rocksdb/blob/master/examples
+   * /rocksdb_option_file_example.ini
+   * <p>
+   * We look for a specific pattern, say OzoneManager.db will have its configs
+   * specified in OzoneManager.db.ini. This option is used only by the
+   * performance testing group to allow tuning of all parameters freely.
+   * <p>
+   * For the end users we offer a set of Predefined options that is easy to use
+   * and the user does not need to become an expert in RockDB config.
+   * <p>
+   * This code assumes the .ini file is placed in the same directory as normal
+   * config files. That is in $HADOOP_DIR/etc/hadoop. For example, if we want to
+   * control OzoneManager.db configs from a file, we need to create a file
+   * called OzoneManager.db.ini and place that file in $HADOOP_DIR/etc/hadoop.
+   *
+   * @param dbFileName - The DB File Name, for example, OzoneManager.db.
+   * @param cfDescs - ColumnFamily Handles.
+   * @return DBOptions, Options to be used for opening/creating the DB.
+   * @throws IOException
+   */
+  public static DBOptions readFromFile(String dbFileName,
+      List<ColumnFamilyDescriptor> cfDescs) throws IOException {
+    Preconditions.checkNotNull(dbFileName);
+    Preconditions.checkNotNull(cfDescs);
+    Preconditions.checkArgument(cfDescs.size() > 0);
+
+    //TODO: Add Documentation on how to support RocksDB Mem Env.
+    Env env = Env.getDefault();
+    DBOptions options = null;
+    Path optionsFile = Paths.get(getConfigLocation().toString(),
+        getOptionsFileNameFromDB(dbFileName));
+
+    if (optionsFile.toFile().exists()) {
+      options = new DBOptions();
+      try {
+        OptionsUtil.loadOptionsFromFile(optionsFile.toString(),
+            env, options, cfDescs, true);
+
+      } catch (RocksDBException rdEx) {
+        RDBTable.toIOException("Unable to find/open Options file.", rdEx);
+      }
+    }
+    return options;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBProfile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBProfile.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBProfile.java
new file mode 100644
index 0000000..4d3d6bc
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBProfile.java
@@ -0,0 +1,120 @@
+/*
+ * 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.utils.db;
+
+import org.apache.hadoop.conf.StorageUnit;
+import org.rocksdb.BlockBasedTableConfig;
+import org.rocksdb.BloomFilter;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.CompactionStyle;
+import org.rocksdb.DBOptions;
+
+import java.math.BigDecimal;
+
+/**
+ * User visible configs based RocksDB tuning page. Documentation for Options.
+ * <p>
+ * https://github.com/facebook/rocksdb/blob/master/include/rocksdb/options.h
+ * <p>
+ * Most tuning parameters are based on this URL.
+ * <p>
+ * https://github.com/facebook/rocksdb/wiki/Setup-Options-and-Basic-Tuning
+ */
+public enum DBProfile {
+  //TODO : Add more profiles like TEST etc.
+  SSD {
+    @Override
+    public String toString() {
+      return "DBProfile.SSD";
+    }
+
+    @Override
+    public ColumnFamilyOptions getColumnFamilyOptions() {
+
+      // Set BlockCacheSize to 256 MB. This should not be an issue for HADOOP.
+      final long blockCacheSize = toLong(StorageUnit.MB.toBytes(256.00));
+
+      // Set the Default block size to 16KB
+      final long blockSize = toLong(StorageUnit.KB.toBytes(16));
+
+      // Write Buffer Size -- set to 128 MB
+      final long writeBufferSize = toLong(StorageUnit.MB.toBytes(128));
+
+      return new ColumnFamilyOptions()
+          .setLevelCompactionDynamicLevelBytes(true)
+          .setWriteBufferSize(writeBufferSize)
+          .setTableFormatConfig(
+              new BlockBasedTableConfig()
+                  .setBlockCacheSize(blockCacheSize)
+                  .setBlockSize(blockSize)
+                  .setCacheIndexAndFilterBlocks(true)
+                  .setPinL0FilterAndIndexBlocksInCache(true)
+                  .setFilter(new BloomFilter()));
+    }
+
+    @Override
+    public DBOptions getDBOptions() {
+      final int maxBackgroundCompactions = 4;
+      final int maxBackgroundFlushes = 2;
+      final long bytesPerSync = toLong(StorageUnit.MB.toBytes(1.00));
+      final boolean createIfMissing = true;
+      final boolean createMissingColumnFamilies = true;
+      return new DBOptions()
+          .setIncreaseParallelism(Runtime.getRuntime().availableProcessors())
+          .setMaxBackgroundCompactions(maxBackgroundCompactions)
+          .setMaxBackgroundFlushes(maxBackgroundFlushes)
+          .setBytesPerSync(bytesPerSync)
+          .setCreateIfMissing(createIfMissing)
+          .setCreateMissingColumnFamilies(createMissingColumnFamilies);
+    }
+
+
+  },
+  DISK {
+    @Override
+    public String toString() {
+      return "DBProfile.DISK";
+    }
+
+    @Override
+    public DBOptions getDBOptions() {
+      final long readAheadSize = toLong(StorageUnit.MB.toBytes(4.00));
+      return SSD.getDBOptions().setCompactionReadaheadSize(readAheadSize);
+    }
+
+    @Override
+    public ColumnFamilyOptions getColumnFamilyOptions() {
+      ColumnFamilyOptions columnFamilyOptions = SSD.getColumnFamilyOptions();
+      columnFamilyOptions.setCompactionStyle(CompactionStyle.LEVEL);
+      return columnFamilyOptions;
+    }
+
+
+  };
+
+  private static long toLong(double value) {
+    BigDecimal temp = new BigDecimal(value);
+    return temp.longValue();
+  }
+
+  public abstract DBOptions getDBOptions();
+
+  public abstract ColumnFamilyOptions getColumnFamilyOptions();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStoreBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStoreBuilder.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStoreBuilder.java
new file mode 100644
index 0000000..5ba9b9c
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStoreBuilder.java
@@ -0,0 +1,201 @@
+/*
+ * 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.utils.db;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.eclipse.jetty.util.StringUtil;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.RocksDB;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DB_PROFILE;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DEFAULT_DB_PROFILE;
+
+/**
+ * DBStore Builder.
+ */
+public final class DBStoreBuilder {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DBStoreBuilder.class);
+  private Set<TableConfig> tables;
+  private DBProfile dbProfile;
+  private DBOptions rocksDBOption;
+  private String dbname;
+  private Path dbPath;
+  private List<String> tableNames;
+  private Configuration configuration;
+
+  private DBStoreBuilder(Configuration configuration) {
+    tables = new HashSet<>();
+    tableNames = new LinkedList<>();
+    this.configuration = configuration;
+  }
+
+  public static DBStoreBuilder newBuilder(Configuration configuration) {
+    return new DBStoreBuilder(configuration);
+  }
+
+  public DBStoreBuilder setProfile(DBProfile profile) {
+    dbProfile = profile;
+    return this;
+  }
+
+  public DBStoreBuilder setName(String name) {
+    dbname = name;
+    return this;
+  }
+
+  public DBStoreBuilder addTable(String tableName) {
+    tableNames.add(tableName);
+    return this;
+  }
+
+  public DBStoreBuilder addTable(String tableName, ColumnFamilyOptions option)
+      throws IOException {
+    TableConfig tableConfig = new TableConfig(tableName, option);
+    if (!tables.add(tableConfig)) {
+      String message = "Unable to add the table: " + tableName +
+          ".  Please check if this table name is already in use.";
+      LOG.error(message);
+      throw new IOException(message);
+    }
+    LOG.info("using custom profile for table: {}", tableName);
+    return this;
+  }
+
+  public DBStoreBuilder setDBOption(DBOptions option) {
+    rocksDBOption = option;
+    return this;
+  }
+
+  public DBStoreBuilder setPath(Path path) {
+    Preconditions.checkNotNull(path);
+    dbPath = path;
+    return this;
+  }
+
+  /**
+   * Builds a DBStore instance and returns that.
+   *
+   * @return DBStore
+   */
+  public DBStore build() throws IOException {
+    if(StringUtil.isBlank(dbname) || (dbPath == null)) {
+      LOG.error("Required Parameter missing.");
+      throw new IOException("Required parameter is missing. Please make sure "
+          + "sure Path and DB name is provided.");
+    }
+    processDBProfile();
+    processTables();
+    DBOptions options = getDbProfile();
+    File dbFile = getDBFile();
+    if (!dbFile.getParentFile().exists()) {
+      throw new IOException("The DB destination directory should exist.");
+    }
+    return new RDBStore(dbFile, options, tables);
+  }
+
+  /**
+   * if the DBProfile is not set, we will default to using default from the
+   * config file.
+   */
+  private void processDBProfile() {
+    if (dbProfile == null) {
+      dbProfile = this.configuration.getEnum(HDDS_DB_PROFILE,
+          HDDS_DEFAULT_DB_PROFILE);
+    }
+  }
+
+  private void processTables() throws IOException {
+    if (tableNames.size() > 0) {
+      for (String name : tableNames) {
+        addTable(name, dbProfile.getColumnFamilyOptions());
+        LOG.info("Using default column profile:{} for Table:{}",
+            dbProfile.toString(), name);
+      }
+    }
+    addTable(DFSUtil.bytes2String(RocksDB.DEFAULT_COLUMN_FAMILY),
+        dbProfile.getColumnFamilyOptions());
+    LOG.info("Using default column profile:{} for Table:{}",
+        dbProfile.toString(),
+        DFSUtil.bytes2String(RocksDB.DEFAULT_COLUMN_FAMILY));
+  }
+
+  private DBOptions getDbProfile() {
+    if (rocksDBOption != null) {
+      return rocksDBOption;
+    }
+    DBOptions option = null;
+    if (StringUtil.isNotBlank(dbname)) {
+      List<ColumnFamilyDescriptor> columnFamilyDescriptors = new LinkedList<>();
+
+      for (TableConfig tc : tables) {
+        columnFamilyDescriptors.add(tc.getDescriptor());
+      }
+
+      if (columnFamilyDescriptors.size() > 0) {
+        try {
+          option = DBConfigFromFile.readFromFile(dbname,
+              columnFamilyDescriptors);
+          if(option != null) {
+            LOG.info("Using Configs from {}.ini file", dbname);
+          }
+        } catch (IOException ex) {
+          LOG.info("Unable to read ROCKDB config", ex);
+        }
+      }
+    }
+
+    if (option == null) {
+      LOG.info("Using default options. {}", dbProfile.toString());
+      return dbProfile.getDBOptions();
+    }
+    return option;
+  }
+
+  private File getDBFile() throws IOException {
+    if (dbPath == null) {
+      LOG.error("DB path is required.");
+      throw new IOException("A Path to for DB file is needed.");
+    }
+
+    if (StringUtil.isBlank(dbname)) {
+      LOG.error("DBName is a required.");
+      throw new IOException("A valid DB name is required.");
+    }
+    return Paths.get(dbPath.toString(), dbname).toFile();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
index c719d31..85508d5 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.utils.RocksDBStoreMBean;
 import org.apache.ratis.shaded.com.google.common.annotations.VisibleForTesting;
 import org.rocksdb.ColumnFamilyDescriptor;
 import org.rocksdb.ColumnFamilyHandle;
-import org.rocksdb.ColumnFamilyOptions;
+
 import org.rocksdb.DBOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
@@ -38,12 +38,12 @@ import org.slf4j.LoggerFactory;
 import javax.management.ObjectName;
 import java.io.File;
 import java.io.IOException;
-import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Hashtable;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 /**
  * RocksDB Store that supports creating Tables in DB.
@@ -58,7 +58,7 @@ public class RDBStore implements DBStore {
   private final Hashtable<String, ColumnFamilyHandle> handleTable;
   private ObjectName statMBeanName;
 
-  public RDBStore(File dbFile, DBOptions options, List<String> families)
+  public RDBStore(File dbFile, DBOptions options, Set<TableConfig> families)
       throws IOException {
     Preconditions.checkNotNull(dbFile, "DB file location cannot be null");
     Preconditions.checkNotNull(families);
@@ -69,10 +69,8 @@ public class RDBStore implements DBStore {
         new ArrayList<>();
     final List<ColumnFamilyHandle> columnFamilyHandles = new ArrayList<>();
 
-    for (String family : families) {
-      columnFamilyDescriptors.add(
-          new ColumnFamilyDescriptor(family.getBytes(StandardCharsets.UTF_8),
-              new ColumnFamilyOptions()));
+    for (TableConfig family : families) {
+      columnFamilyDescriptors.add(family.getDescriptor());
     }
 
     dbOptions = options;
@@ -141,19 +139,23 @@ public class RDBStore implements DBStore {
     for (final ColumnFamilyHandle handle : handleTable.values()) {
       handle.close();
     }
-    if (dbOptions != null) {
-      dbOptions.close();
-    }
-    if (writeOptions != null) {
-      writeOptions.close();
-    }
+
     if (statMBeanName != null) {
       MBeans.unregister(statMBeanName);
       statMBeanName = null;
     }
+
     if (db != null) {
       db.close();
     }
+
+    if (dbOptions != null) {
+      dbOptions.close();
+    }
+
+    if (writeOptions != null) {
+      writeOptions.close();
+    }
   }
 
   @Override
@@ -221,7 +223,7 @@ public class RDBStore implements DBStore {
   @Override
   public long getEstimatedKeyCount() throws IOException {
     try {
-      return Long.parseLong(db.getProperty("rocksdb.estimate-num-keys"));
+      return db.getLongProperty("rocksdb.estimate-num-keys");
     } catch (RocksDBException e) {
       throw toIOException("Unable to get the estimated count.", e);
     }
@@ -244,7 +246,7 @@ public class RDBStore implements DBStore {
   @Override
   public ArrayList<Table> listTables() throws IOException {
     ArrayList<Table> returnList = new ArrayList<>();
-    for (ColumnFamilyHandle handle: handleTable.values())  {
+    for (ColumnFamilyHandle handle : handleTable.values()) {
       returnList.add(new RDBTable(db, handle, writeOptions));
     }
     return returnList;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TableConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TableConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TableConfig.java
new file mode 100644
index 0000000..897028a
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/TableConfig.java
@@ -0,0 +1,93 @@
+/*
+ * 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.utils.db;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyOptions;
+
+/**
+ * Class that maintains Table Configuration.
+ */
+public class TableConfig {
+  private final String name;
+  private final ColumnFamilyOptions columnFamilyOptions;
+
+
+  /**
+   * Constructs a Table Config.
+   * @param name - Name of the Table.
+   * @param columnFamilyOptions - Column Family options.
+   */
+  public TableConfig(String name, ColumnFamilyOptions columnFamilyOptions) {
+    this.name = name;
+    this.columnFamilyOptions = columnFamilyOptions;
+  }
+
+  /**
+   * Returns the Name for this Table.
+   * @return - Name String
+   */
+  public String getName() {
+    return name;
+  }
+
+  /**
+   * Returns a ColumnFamilyDescriptor for this table.
+   * @return ColumnFamilyDescriptor
+   */
+  public ColumnFamilyDescriptor getDescriptor() {
+    return  new ColumnFamilyDescriptor(DFSUtil.string2Bytes(name),
+        columnFamilyOptions);
+  }
+
+  /**
+   * Returns Column family options for this Table.
+   * @return  ColumnFamilyOptions used for the Table.
+   */
+  public ColumnFamilyOptions getColumnFamilyOptions() {
+    return columnFamilyOptions;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    TableConfig that = (TableConfig) o;
+    return new EqualsBuilder()
+        .append(getName(), that.getName())
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37)
+        .append(getName())
+        .toHashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/main/resources/ozone-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index c3e3095..6d2ee09 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -1098,4 +1098,14 @@
       datanode unless the datanode confirms the completion.
     </description>
   </property>
+
+  <property>
+    <name>ozone.db.profile</name>
+    <value>DBProfile.SSD</value>
+    <tag>OZONE, OM, PERFORMANCE, REQUIRED</tag>
+    <description>This property allows user to pick a configuration
+    that tunes the RocksDB settings for the hardware it is running
+    on. Right now, we have SSD and DISK as profile options.</description>
+  </property>
+
 </configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBConfigFromFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBConfigFromFile.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBConfigFromFile.java
new file mode 100644
index 0000000..b20ca70
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBConfigFromFile.java
@@ -0,0 +1,116 @@
+/*
+ * 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.utils.db;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.RocksDB;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.hadoop.utils.db.DBConfigFromFile.getOptionsFileNameFromDB;
+
+/**
+ * DBConf tests.
+ */
+public class TestDBConfigFromFile {
+  private final static String DB_FILE = "test.db";
+  private final static String INI_FILE = getOptionsFileNameFromDB(DB_FILE);
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty(DBConfigFromFile.CONFIG_DIR,
+        folder.newFolder().toString());
+    ClassLoader classLoader = getClass().getClassLoader();
+    File testData = new File(classLoader.getResource(INI_FILE).getFile());
+    File dest = Paths.get(
+        System.getProperty(DBConfigFromFile.CONFIG_DIR), INI_FILE).toFile();
+    FileUtils.copyFile(testData, dest);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+  }
+
+  @Test
+  public void readFromFile() throws IOException {
+    final List<String> families =
+        Arrays.asList(DFSUtil.bytes2String(RocksDB.DEFAULT_COLUMN_FAMILY),
+            "First", "Second", "Third",
+            "Fourth", "Fifth",
+            "Sixth");
+    final List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        new ArrayList<>();
+    for (String family : families) {
+      columnFamilyDescriptors.add(
+          new ColumnFamilyDescriptor(family.getBytes(StandardCharsets.UTF_8),
+              new ColumnFamilyOptions()));
+    }
+
+    final DBOptions options = DBConfigFromFile.readFromFile(DB_FILE,
+        columnFamilyDescriptors);
+
+    // Some Random Values Defined in the test.db.ini, we verify that we are
+    // able to get values that are defined in the test.db.ini.
+    Assert.assertNotNull(options);
+    Assert.assertEquals(551615L, options.maxManifestFileSize());
+    Assert.assertEquals(1000L, options.keepLogFileNum());
+    Assert.assertEquals(1048576, options.writableFileMaxBufferSize());
+  }
+
+  @Test
+  public void readFromFileInvalidConfig() throws IOException {
+    final List<String> families =
+        Arrays.asList(DFSUtil.bytes2String(RocksDB.DEFAULT_COLUMN_FAMILY),
+            "First", "Second", "Third",
+            "Fourth", "Fifth",
+            "Sixth");
+    final List<ColumnFamilyDescriptor> columnFamilyDescriptors =
+        new ArrayList<>();
+    for (String family : families) {
+      columnFamilyDescriptors.add(
+          new ColumnFamilyDescriptor(family.getBytes(StandardCharsets.UTF_8),
+              new ColumnFamilyOptions()));
+    }
+
+    final DBOptions options = DBConfigFromFile.readFromFile("badfile.db.ini",
+        columnFamilyDescriptors);
+
+    // This has to return a Null, since we have config defined for badfile.db
+    Assert.assertNull(options);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBStoreBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBStoreBuilder.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBStoreBuilder.java
new file mode 100644
index 0000000..3e1f364
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBStoreBuilder.java
@@ -0,0 +1,174 @@
+/*
+ * 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.utils.db;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+/**
+ * Tests RDBStore creation.
+ */
+public class TestDBStoreBuilder {
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty(DBConfigFromFile.CONFIG_DIR,
+        folder.newFolder().toString());
+  }
+
+  @Test
+  public void builderWithoutAnyParams() throws IOException {
+    Configuration conf = new Configuration();
+    thrown.expect(IOException.class);
+    DBStoreBuilder.newBuilder(conf).build();
+  }
+
+  @Test
+  public void builderWithOneParamV1() throws IOException {
+    Configuration conf = new Configuration();
+    thrown.expect(IOException.class);
+    DBStoreBuilder.newBuilder(conf)
+        .setName("Test.db")
+        .build();
+  }
+
+  @Test
+  public void builderWithOneParamV2() throws IOException {
+    Configuration conf = new Configuration();
+    File newFolder = folder.newFolder();
+    if(!newFolder.exists()) {
+      Assert.assertTrue(newFolder.mkdirs());
+    }
+    thrown.expect(IOException.class);
+    DBStoreBuilder.newBuilder(conf)
+        .setPath(newFolder.toPath())
+        .build();
+  }
+
+  @Test
+  public void builderWithOpenClose() throws Exception {
+    Configuration conf = new Configuration();
+    File newFolder = folder.newFolder();
+    if(!newFolder.exists()) {
+      Assert.assertTrue(newFolder.mkdirs());
+    }
+    DBStore dbStore = DBStoreBuilder.newBuilder(conf)
+        .setName("Test.db")
+        .setPath(newFolder.toPath())
+        .build();
+    // Nothing to do just open and Close.
+    dbStore.close();
+  }
+
+  @Test
+  public void builderWithDoubleTableName() throws Exception {
+    Configuration conf = new Configuration();
+    File newFolder = folder.newFolder();
+    if(!newFolder.exists()) {
+      Assert.assertTrue(newFolder.mkdirs());
+    }
+    thrown.expect(IOException.class);
+    DBStoreBuilder.newBuilder(conf)
+        .setName("Test.db")
+        .setPath(newFolder.toPath())
+        .addTable("FIRST")
+        .addTable("FIRST")
+        .build();
+    // Nothing to do , This will throw so we do not have to close.
+
+  }
+
+  @Test
+  public void builderWithDataWrites() throws Exception {
+    Configuration conf = new Configuration();
+    File newFolder = folder.newFolder();
+    if(!newFolder.exists()) {
+      Assert.assertTrue(newFolder.mkdirs());
+    }
+    try (DBStore dbStore = DBStoreBuilder.newBuilder(conf)
+        .setName("Test.db")
+        .setPath(newFolder.toPath())
+        .addTable("First")
+        .addTable("Second")
+        .build()) {
+      try (Table firstTable = dbStore.getTable("First")) {
+        byte[] key =
+            RandomStringUtils.random(9).getBytes(StandardCharsets.UTF_8);
+        byte[] value =
+            RandomStringUtils.random(9).getBytes(StandardCharsets.UTF_8);
+        firstTable.put(key, value);
+        byte[] temp = firstTable.get(key);
+        Arrays.equals(value, temp);
+      }
+
+      try (Table secondTable = dbStore.getTable("Second")) {
+        Assert.assertTrue(secondTable.isEmpty());
+      }
+    }
+  }
+
+  @Test
+  public void builderWithDiskProfileWrites() throws Exception {
+    Configuration conf = new Configuration();
+    File newFolder = folder.newFolder();
+    if(!newFolder.exists()) {
+      Assert.assertTrue(newFolder.mkdirs());
+    }
+    try (DBStore dbStore = DBStoreBuilder.newBuilder(conf)
+        .setName("Test.db")
+        .setPath(newFolder.toPath())
+        .addTable("First")
+        .addTable("Second")
+        .setProfile(DBProfile.DISK)
+        .build()) {
+      try (Table firstTable = dbStore.getTable("First")) {
+        byte[] key =
+            RandomStringUtils.random(9).getBytes(StandardCharsets.UTF_8);
+        byte[] value =
+            RandomStringUtils.random(9).getBytes(StandardCharsets.UTF_8);
+        firstTable.put(key, value);
+        byte[] temp = firstTable.get(key);
+        Arrays.equals(value, temp);
+      }
+
+      try (Table secondTable = dbStore.getTable("Second")) {
+        Assert.assertTrue(secondTable.isEmpty());
+      }
+    }
+  }
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java
index 94a650b..0b673d3 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBStore.java
@@ -28,6 +28,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 import org.junit.rules.TemporaryFolder;
+import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.Statistics;
@@ -39,8 +40,10 @@ import java.lang.management.ManagementFactory;
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 /**
  * RDBStore Tests.
@@ -57,6 +60,7 @@ public class TestRDBStore {
   public ExpectedException thrown = ExpectedException.none();
   private RDBStore rdbStore = null;
   private DBOptions options = null;
+  private Set<TableConfig> configSet;
 
   @Before
   public void setUp() throws Exception {
@@ -67,7 +71,12 @@ public class TestRDBStore {
     Statistics statistics = new Statistics();
     statistics.setStatsLevel(StatsLevel.ALL);
     options = options.setStatistics(statistics);
-    rdbStore = new RDBStore(folder.newFolder(), options, families);
+    configSet = new HashSet<>();
+    for(String name : families) {
+      TableConfig newConfig = new TableConfig(name, new ColumnFamilyOptions());
+      configSet.add(newConfig);
+    }
+    rdbStore = new RDBStore(folder.newFolder(), options, configSet);
   }
 
   @After
@@ -80,7 +89,7 @@ public class TestRDBStore {
   @Test
   public void compactDB() throws Exception {
     try (RDBStore newStore =
-             new RDBStore(folder.newFolder(), options, families)) {
+             new RDBStore(folder.newFolder(), options, configSet)) {
       Assert.assertNotNull("DB Store cannot be null", newStore);
       try (Table firstTable = newStore.getTable(families.get(1))) {
         Assert.assertNotNull("Table cannot be null", firstTable);
@@ -101,7 +110,7 @@ public class TestRDBStore {
   @Test
   public void close() throws Exception {
     RDBStore newStore =
-        new RDBStore(folder.newFolder(), options, families);
+        new RDBStore(folder.newFolder(), options, configSet);
     Assert.assertNotNull("DBStore cannot be null", newStore);
     // This test does not assert anything if there is any error this test
     // will throw and fail.
@@ -157,7 +166,7 @@ public class TestRDBStore {
   @Test
   public void getEstimatedKeyCount() throws Exception {
     try (RDBStore newStore =
-             new RDBStore(folder.newFolder(), options, families)) {
+             new RDBStore(folder.newFolder(), options, configSet)) {
       Assert.assertNotNull("DB Store cannot be null", newStore);
       // Write 100 keys to the first table.
       try (Table firstTable = newStore.getTable(families.get(1))) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java
index 677a1f9..cd25548 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestRDBTableStore.java
@@ -27,6 +27,7 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.DBOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.Statistics;
@@ -35,8 +36,10 @@ import org.rocksdb.WriteBatch;
 
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Set;
 
 /**
  * Tests for RocksDBTable Store.
@@ -62,7 +65,13 @@ public class TestRDBTableStore {
     Statistics statistics = new Statistics();
     statistics.setStatsLevel(StatsLevel.ALL);
     options = options.setStatistics(statistics);
-    rdbStore = new RDBStore(folder.newFolder(), options, families);
+
+    Set<TableConfig> configSet = new HashSet<>();
+    for(String name : families) {
+      TableConfig newConfig = new TableConfig(name, new ColumnFamilyOptions());
+      configSet.add(newConfig);
+    }
+    rdbStore = new RDBStore(folder.newFolder(), options, configSet);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/common/src/test/resources/test.db.ini
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/resources/test.db.ini b/hadoop-hdds/common/src/test/resources/test.db.ini
new file mode 100644
index 0000000..6666cd2
--- /dev/null
+++ b/hadoop-hdds/common/src/test/resources/test.db.ini
@@ -0,0 +1,145 @@
+# This is a RocksDB option file.
+#
+# A typical RocksDB options file has four sections, which are
+# Version section, DBOptions section, at least one CFOptions
+# section, and one TableOptions section for each column family.
+# The RocksDB options file in general follows the basic INI
+# file format with the following extensions / modifications:
+#
+#  * Escaped characters
+#    We escaped the following characters:
+#     - \n -- line feed - new line
+#     - \r -- carriage return
+#     - \\ -- backslash \
+#     - \: -- colon symbol :
+#     - \# -- hash tag #
+#  * Comments
+#    We support # style comments.  Comments can appear at the ending
+#    part of a line.
+#  * Statements
+#    A statement is of the form option_name = value.
+#    Each statement contains a '=', where extra white-spaces
+#    are supported. However, we don't support multi-lined statement.
+#    Furthermore, each line can only contain at most one statement.
+#  * Sections
+#    Sections are of the form [SecitonTitle "SectionArgument"],
+#    where section argument is optional.
+#  * List
+#    We use colon-separated string to represent a list.
+#    For instance, n1:n2:n3:n4 is a list containing four values.
+#
+# Below is an example of a RocksDB options file:
+
+
+#----------------------IMPORTANT------------------------------------#
+### FAKE VALUES FOR TESTING ONLY ### DO NOT USE THESE FOR PRODUCTION.
+#----------------------IMPORTANT------------------------------------#
+[DBOptions]
+  stats_dump_period_sec=600
+  max_manifest_file_size=551615
+  bytes_per_sync=8388608
+  delayed_write_rate=2097152
+  WAL_ttl_seconds=0
+  WAL_size_limit_MB=0
+  max_subcompactions=1
+  wal_dir=
+  wal_bytes_per_sync=0
+  db_write_buffer_size=0
+  keep_log_file_num=1000
+  table_cache_numshardbits=4
+  max_file_opening_threads=1
+  writable_file_max_buffer_size=1048576
+  random_access_max_buffer_size=1048576
+  use_fsync=false
+  max_total_wal_size=0
+  max_open_files=-1
+  skip_stats_update_on_db_open=false
+  max_background_compactions=16
+  manifest_preallocation_size=4194304
+  max_background_flushes=7
+  is_fd_close_on_exec=true
+  max_log_file_size=0
+  advise_random_on_open=true
+  create_missing_column_families=false
+  paranoid_checks=true
+  delete_obsolete_files_period_micros=21600000000
+  log_file_time_to_roll=0
+  compaction_readahead_size=0
+  create_if_missing=false
+  use_adaptive_mutex=false
+  enable_thread_tracking=false
+  allow_fallocate=true
+  error_if_exists=false
+  recycle_log_file_num=0
+  skip_log_error_on_recovery=false
+  db_log_dir=
+  new_table_reader_for_compaction_inputs=true
+  allow_mmap_reads=false
+  allow_mmap_writes=false
+  use_direct_reads=false
+  use_direct_writes=false
+
+
+[CFOptions "default"]
+  compaction_style=kCompactionStyleLevel
+  compaction_filter=nullptr
+  num_levels=6
+  table_factory=BlockBasedTable
+  comparator=leveldb.BytewiseComparator
+  max_sequential_skip_in_iterations=8
+  soft_rate_limit=0.000000
+  max_bytes_for_level_base=1073741824
+  memtable_prefix_bloom_probes=6
+  memtable_prefix_bloom_bits=0
+  memtable_prefix_bloom_huge_page_tlb_size=0
+  max_successive_merges=0
+  arena_block_size=16777216
+  min_write_buffer_number_to_merge=1
+  target_file_size_multiplier=1
+  source_compaction_factor=1
+  max_bytes_for_level_multiplier=8
+  max_bytes_for_level_multiplier_additional=2:3:5
+  compaction_filter_factory=nullptr
+  max_write_buffer_number=8
+  level0_stop_writes_trigger=20
+  compression=kSnappyCompression
+  level0_file_num_compaction_trigger=4
+  purge_redundant_kvs_while_flush=true
+  max_write_buffer_number_to_maintain=0
+  memtable_factory=SkipListFactory
+  max_grandparent_overlap_factor=8
+  expanded_compaction_factor=25
+  hard_pending_compaction_bytes_limit=137438953472
+  inplace_update_num_locks=10000
+  level_compaction_dynamic_level_bytes=true
+  level0_slowdown_writes_trigger=12
+  filter_deletes=false
+  verify_checksums_in_compaction=true
+  min_partial_merge_operands=2
+  paranoid_file_checks=false
+  target_file_size_base=134217728
+  optimize_filters_for_hits=false
+  merge_operator=PutOperator
+  compression_per_level=kNoCompression:kNoCompression:kNoCompression:kSnappyCompression:kSnappyCompression:kSnappyCompression
+  compaction_measure_io_stats=false
+  prefix_extractor=nullptr
+  bloom_locality=0
+  write_buffer_size=134217728
+  disable_auto_compactions=false
+  inplace_update_support=false
+
+[TableOptions/BlockBasedTable "default"]
+  format_version=2
+  whole_key_filtering=true
+  no_block_cache=false
+  checksum=kCRC32c
+  filter_policy=rocksdb.BuiltinBloomFilter
+  block_size_deviation=10
+  block_size=8192
+  block_restart_interval=16
+  cache_index_and_filter_blocks=false
+  pin_l0_filter_and_index_blocks_in_cache=false
+  pin_top_level_index_and_filter=false
+  index_type=kBinarySearch
+  hash_index_allow_collision=true
+  flush_block_policy_factory=FlushBlockBySizePolicyFactory
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c61824a1/hadoop-hdds/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/pom.xml b/hadoop-hdds/pom.xml
index f655c2f..428fd11 100644
--- a/hadoop-hdds/pom.xml
+++ b/hadoop-hdds/pom.xml
@@ -105,6 +105,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
             <exclude>src/test/resources/incorrect.container</exclude>
             <exclude>src/test/resources/additionalfields.container</exclude>
             <exclude>src/test/resources/incorrect.checksum.container</exclude>
+            <exclude>src/test/resources/test.db.ini</exclude>
           </excludes>
         </configuration>
       </plugin>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[30/50] [abbrv] hadoop git commit: HADOOP-15667. FileSystemMultipartUploader should verify that UploadHandle has non-0 length. Contributed by Ewan Higgs

Posted by eh...@apache.org.
HADOOP-15667. FileSystemMultipartUploader should verify that UploadHandle has non-0 length.
Contributed by Ewan Higgs


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2e6c1109
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2e6c1109
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2e6c1109

Branch: refs/heads/HDFS-12090
Commit: 2e6c1109dcdeedb59a3345047e9201271c9a0b27
Parents: 781437c
Author: Steve Loughran <st...@apache.org>
Authored: Thu Aug 30 14:33:16 2018 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Aug 30 14:33:16 2018 +0100

----------------------------------------------------------------------
 .../hadoop/fs/FileSystemMultipartUploader.java  |  6 ++-
 .../org/apache/hadoop/fs/MultipartUploader.java | 11 +++++
 .../AbstractContractMultipartUploaderTest.java  | 43 ++++++++++++++++++++
 .../hadoop/fs/s3a/S3AMultipartUploader.java     | 10 ++---
 4 files changed, 61 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e6c1109/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemMultipartUploader.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemMultipartUploader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemMultipartUploader.java
index a700a9f..f13b50b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemMultipartUploader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemMultipartUploader.java
@@ -68,6 +68,7 @@ public class FileSystemMultipartUploader extends MultipartUploader {
       throws IOException {
 
     byte[] uploadIdByteArray = uploadId.toByteArray();
+    checkUploadId(uploadIdByteArray);
     Path collectorPath = new Path(new String(uploadIdByteArray, 0,
         uploadIdByteArray.length, Charsets.UTF_8));
     Path partPath =
@@ -101,6 +102,8 @@ public class FileSystemMultipartUploader extends MultipartUploader {
       List<Pair<Integer, PartHandle>> handles, UploadHandle multipartUploadId)
       throws IOException {
 
+    checkUploadId(multipartUploadId.toByteArray());
+
     if (handles.isEmpty()) {
       throw new IOException("Empty upload");
     }
@@ -133,8 +136,7 @@ public class FileSystemMultipartUploader extends MultipartUploader {
   @Override
   public void abort(Path filePath, UploadHandle uploadId) throws IOException {
     byte[] uploadIdByteArray = uploadId.toByteArray();
-    Preconditions.checkArgument(uploadIdByteArray.length != 0,
-        "UploadId is empty");
+    checkUploadId(uploadIdByteArray);
     Path collectorPath = new Path(new String(uploadIdByteArray, 0,
         uploadIdByteArray.length, Charsets.UTF_8));
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e6c1109/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java
index 47fd9f2..76f58d3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.util.List;
 
+import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -91,4 +92,14 @@ public abstract class MultipartUploader {
   public abstract void abort(Path filePath, UploadHandle multipartUploadId)
       throws IOException;
 
+  /**
+   * Utility method to validate uploadIDs
+   * @param uploadId
+   * @throws IllegalArgumentException
+   */
+  protected void checkUploadId(byte[] uploadId)
+      throws IllegalArgumentException {
+    Preconditions.checkArgument(uploadId.length > 0,
+        "Empty UploadId is not valid");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e6c1109/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java
index c0e1600..85a6861 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java
@@ -297,4 +297,47 @@ public abstract class AbstractContractMultipartUploaderTest extends
         () -> mpu.complete(dest, new ArrayList<>(), handle));
     mpu.abort(dest, handle);
   }
+
+  /**
+   * When we pass empty uploadID, putPart throws IllegalArgumentException.
+   * @throws Exception
+   */
+  @Test
+  public void testPutPartEmptyUploadID() throws Exception {
+    describe("Expect IllegalArgumentException when putPart uploadID is empty");
+    FileSystem fs = getFileSystem();
+    Path dest = path("testCompleteEmptyUpload");
+    MultipartUploader mpu = MultipartUploaderFactory.get(fs, null);
+    mpu.initialize(dest);
+    UploadHandle emptyHandle =
+        BBUploadHandle.from(ByteBuffer.wrap(new byte[0]));
+    byte[] payload = generatePayload(1);
+    InputStream is = new ByteArrayInputStream(payload);
+    intercept(IllegalArgumentException.class,
+        () -> mpu.putPart(dest, is, 1, emptyHandle, payload.length));
+  }
+
+  /**
+   * When we pass empty uploadID, complete throws IllegalArgumentException.
+   * @throws Exception
+   */
+  @Test
+  public void testCompleteEmptyUploadID() throws Exception {
+    describe("Expect IllegalArgumentException when complete uploadID is empty");
+    FileSystem fs = getFileSystem();
+    Path dest = path("testCompleteEmptyUpload");
+    MultipartUploader mpu = MultipartUploaderFactory.get(fs, null);
+    UploadHandle realHandle = mpu.initialize(dest);
+    UploadHandle emptyHandle =
+        BBUploadHandle.from(ByteBuffer.wrap(new byte[0]));
+    List<Pair<Integer, PartHandle>> partHandles = new ArrayList<>();
+    byte[] payload = generatePayload(1);
+    InputStream is = new ByteArrayInputStream(payload);
+    PartHandle partHandle = mpu.putPart(dest, is, 1, realHandle,
+        payload.length);
+    partHandles.add(Pair.of(1, partHandle));
+
+    intercept(IllegalArgumentException.class,
+        () -> mpu.complete(dest, partHandles, emptyHandle));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e6c1109/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AMultipartUploader.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AMultipartUploader.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AMultipartUploader.java
index 6a1df54..4a6cb8c 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AMultipartUploader.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AMultipartUploader.java
@@ -84,9 +84,10 @@ public class S3AMultipartUploader extends MultipartUploader {
   public PartHandle putPart(Path filePath, InputStream inputStream,
       int partNumber, UploadHandle uploadId, long lengthInBytes)
       throws IOException {
-    final WriteOperationHelper writeHelper = s3a.getWriteOperationHelper();
-    String key = s3a.pathToKey(filePath);
     byte[] uploadIdBytes = uploadId.toByteArray();
+    checkUploadId(uploadIdBytes);
+    String key = s3a.pathToKey(filePath);
+    final WriteOperationHelper writeHelper = s3a.getWriteOperationHelper();
     String uploadIdString = new String(uploadIdBytes, 0, uploadIdBytes.length,
         Charsets.UTF_8);
     UploadPartRequest request = writeHelper.newUploadPartRequest(key,
@@ -155,11 +156,6 @@ public class S3AMultipartUploader extends MultipartUploader {
     }
   }
 
-  private void checkUploadId(byte[] uploadId) throws IllegalArgumentException {
-    Preconditions.checkArgument(uploadId.length > 0,
-        "Empty UploadId is not valid");
-  }
-
   /**
    * Build the payload for marshalling.
    * @param eTag upload etag


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[18/50] [abbrv] hadoop git commit: HDFS-13731. ReencryptionUpdater fails with ConcurrentModificationException during processCheckpoints. Contributed by Zsolt Venczel.

Posted by eh...@apache.org.
HDFS-13731. ReencryptionUpdater fails with ConcurrentModificationException during processCheckpoints. Contributed by Zsolt Venczel.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3e18b957
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3e18b957
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3e18b957

Branch: refs/heads/HDFS-12090
Commit: 3e18b957ebdf20925224ab9c28e6c2f4b6bbdb24
Parents: c5629d5
Author: Zsolt Venczel <zv...@cloudera.com>
Authored: Tue Aug 28 15:11:58 2018 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Tue Aug 28 15:13:43 2018 -0700

----------------------------------------------------------------------
 .../server/namenode/ReencryptionHandler.java    |  6 +--
 .../server/namenode/ReencryptionUpdater.java    | 52 ++++++++++----------
 2 files changed, 30 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e18b957/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java
index c8c8d68..a8acccd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java
@@ -714,10 +714,10 @@ public class ReencryptionHandler implements Runnable {
           zst = new ZoneSubmissionTracker();
           submissions.put(zoneId, zst);
         }
+        Future future = batchService.submit(new EDEKReencryptCallable(zoneId,
+            currentBatch, reencryptionHandler));
+        zst.addTask(future);
       }
-      Future future = batchService.submit(new EDEKReencryptCallable(zoneId,
-          currentBatch, reencryptionHandler));
-      zst.addTask(future);
       LOG.info("Submitted batch (start:{}, size:{}) of zone {} to re-encrypt.",
           currentBatch.getFirstFilePath(), currentBatch.size(), zoneId);
       currentBatch = new ReencryptionBatch(reencryptBatchSize);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3e18b957/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionUpdater.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionUpdater.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionUpdater.java
index a5923a7..15cfa92 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionUpdater.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionUpdater.java
@@ -383,32 +383,34 @@ public final class ReencryptionUpdater implements Runnable {
     final LinkedList<Future> tasks = tracker.getTasks();
     final List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
     ListIterator<Future> iter = tasks.listIterator();
-    while (iter.hasNext()) {
-      Future<ReencryptionTask> curr = iter.next();
-      if (curr.isCancelled()) {
-        break;
-      }
-      if (!curr.isDone() || !curr.get().processed) {
-        // still has earlier tasks not completed, skip here.
-        break;
-      }
-      ReencryptionTask task = curr.get();
-      LOG.debug("Updating re-encryption checkpoint with completed task."
-          + " last: {} size:{}.", task.lastFile, task.batch.size());
-      assert zoneId == task.zoneId;
-      try {
-        final XAttr xattr = FSDirEncryptionZoneOp
-            .updateReencryptionProgress(dir, zoneNode, status, task.lastFile,
-                task.numFilesUpdated, task.numFailures);
-        xAttrs.clear();
-        xAttrs.add(xattr);
-      } catch (IOException ie) {
-        LOG.warn("Failed to update re-encrypted progress to xattr for zone {}",
-            zonePath, ie);
-        ++task.numFailures;
+    synchronized (handler) {
+      while (iter.hasNext()) {
+        Future<ReencryptionTask> curr = iter.next();
+        if (curr.isCancelled()) {
+          break;
+        }
+        if (!curr.isDone() || !curr.get().processed) {
+          // still has earlier tasks not completed, skip here.
+          break;
+        }
+        ReencryptionTask task = curr.get();
+        LOG.debug("Updating re-encryption checkpoint with completed task."
+            + " last: {} size:{}.", task.lastFile, task.batch.size());
+        assert zoneId == task.zoneId;
+        try {
+          final XAttr xattr = FSDirEncryptionZoneOp
+              .updateReencryptionProgress(dir, zoneNode, status, task.lastFile,
+                  task.numFilesUpdated, task.numFailures);
+          xAttrs.clear();
+          xAttrs.add(xattr);
+        } catch (IOException ie) {
+          LOG.warn("Failed to update re-encrypted progress to xattr" +
+                  " for zone {}", zonePath, ie);
+          ++task.numFailures;
+        }
+        ++tracker.numCheckpointed;
+        iter.remove();
       }
-      ++tracker.numCheckpointed;
-      iter.remove();
     }
     if (tracker.isCompleted()) {
       LOG.debug("Removed re-encryption tracker for zone {} because it completed"


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[22/50] [abbrv] hadoop git commit: YARN-8723. Fix a typo in CS init error message when resource calculator is not correctly set. Contributed by Abhishek Modi.

Posted by eh...@apache.org.
YARN-8723. Fix a typo in CS init error message when resource calculator is not correctly set. Contributed by Abhishek Modi.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3fa46394
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3fa46394
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3fa46394

Branch: refs/heads/HDFS-12090
Commit: 3fa46394214181ed1cc7f06b886282bbdf67a10f
Parents: 64ad029
Author: Weiwei Yang <ww...@apache.org>
Authored: Wed Aug 29 10:46:13 2018 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Wed Aug 29 11:13:44 2018 +0800

----------------------------------------------------------------------
 .../resourcemanager/scheduler/capacity/CapacityScheduler.java      | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3fa46394/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
index dec1301..81dcf86 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
@@ -348,7 +348,7 @@ public class CapacityScheduler extends
         throw new YarnRuntimeException("RM uses DefaultResourceCalculator which"
             + " used only memory as resource-type but invalid resource-types"
             + " specified " + ResourceUtils.getResourceTypes() + ". Use"
-            + " DomainantResourceCalculator instead to make effective use of"
+            + " DominantResourceCalculator instead to make effective use of"
             + " these resource-types");
       }
       this.usePortForNodeName = this.conf.getUsePortForNodeName();


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[34/50] [abbrv] hadoop git commit: HDFS-13027. Handle possible NPEs due to deleted blocks in race condition. Contributed by Vinayakumar B.

Posted by eh...@apache.org.
HDFS-13027. Handle possible NPEs due to deleted blocks in race condition. Contributed by Vinayakumar B.

(cherry picked from commit 65977e5d8124be2bc208af25beed934933f170b3)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c36d69a7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c36d69a7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c36d69a7

Branch: refs/heads/HDFS-12090
Commit: c36d69a7b30927eaea16335e06cfcc247accde35
Parents: f2c2a68
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed Aug 29 22:40:13 2018 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Thu Aug 30 22:15:51 2018 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java    | 2 +-
 .../apache/hadoop/hdfs/server/blockmanagement/BlockManager.java | 4 ++++
 .../org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java    | 2 +-
 .../org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java    | 5 ++++-
 4 files changed, 10 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c36d69a7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
index 43f4f47..d160f61 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@ -52,7 +52,7 @@ public abstract class BlockInfo extends Block
   /**
    * Block collection ID.
    */
-  private long bcId;
+  private volatile long bcId;
 
   /** For implementing {@link LightWeightGSet.LinkedElement} interface. */
   private LightWeightGSet.LinkedElement nextLinkedElement;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c36d69a7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 17f6f6e..675221a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -4171,6 +4171,10 @@ public class BlockManager implements BlockStatsMXBean {
     int numExtraRedundancy = 0;
     while(it.hasNext()) {
       final BlockInfo block = it.next();
+      if (block.isDeleted()) {
+        //Orphan block, will be handled eventually, skip
+        continue;
+      }
       int expectedReplication = this.getExpectedRedundancyNum(block);
       NumberReplicas num = countNodes(block);
       if (shouldProcessExtraRedundancy(num, expectedReplication)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c36d69a7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 6ba0e0b..74c9f10 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -4128,7 +4128,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         while (it.hasNext()) {
           Block b = it.next();
           BlockInfo blockInfo = blockManager.getStoredBlock(b);
-          if (blockInfo == null) {
+          if (blockInfo == null || blockInfo.isDeleted()) {
             LOG.info("Cannot find block info for block " + b);
           } else {
             BlockCollection bc = getBlockCollection(blockInfo);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c36d69a7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 5e7bab5..0201ca1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -264,12 +264,13 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       return;
     }
 
+    namenode.getNamesystem().readLock();
     try {
       //get blockInfo
       Block block = new Block(Block.getBlockId(blockId));
       //find which file this block belongs to
       BlockInfo blockInfo = blockManager.getStoredBlock(block);
-      if(blockInfo == null) {
+      if (blockInfo == null || blockInfo.isDeleted()) {
         out.println("Block "+ blockId +" " + NONEXISTENT_STATUS);
         LOG.warn("Block "+ blockId + " " + NONEXISTENT_STATUS);
         return;
@@ -329,6 +330,8 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       out.println(e.getMessage());
       out.print("\n\n" + errMsg);
       LOG.warn("Error in looking up block", e);
+    } finally {
+      namenode.getNamesystem().readUnlock("fsck");
     }
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[03/50] [abbrv] hadoop git commit: YARN-8675. Remove default hostname for docker containers when net=host. Contributed by Suma Shivaprasad

Posted by eh...@apache.org.
YARN-8675. Remove default hostname for docker containers when net=host. Contributed by Suma Shivaprasad


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/05b2bbeb
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/05b2bbeb
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/05b2bbeb

Branch: refs/heads/HDFS-12090
Commit: 05b2bbeb357d4fa03e71f2bfd5d8eeb0ea6c3f60
Parents: c9b6395
Author: Billie Rinaldi <bi...@apache.org>
Authored: Mon Aug 27 11:34:33 2018 -0700
Committer: Billie Rinaldi <bi...@apache.org>
Committed: Mon Aug 27 11:34:33 2018 -0700

----------------------------------------------------------------------
 .../runtime/DockerLinuxContainerRuntime.java    | 49 ++++++++++++--------
 1 file changed, 29 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/05b2bbeb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
index 1872830..00771ff 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
@@ -134,8 +134,8 @@ import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.r
  *   <li>
  *     {@code YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_HOSTNAME} sets the
  *     hostname to be used by the Docker container. If not specified, a
- *     hostname will be derived from the container ID.  This variable is
- *     ignored if the network is 'host' and Registry DNS is not enabled.
+ *     hostname will be derived from the container ID and set as default
+ *     hostname for networks other than 'host'.
  *   </li>
  *   <li>
  *     {@code YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER}
@@ -549,22 +549,34 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
     }
   }
 
-  /** Set a DNS friendly hostname. */
-  private void setHostname(DockerRunCommand runCommand, String
-      containerIdStr, String name)
+  /** Set a DNS friendly hostname.
+   *  Only add hostname if network is not host or if hostname is
+   *  specified via YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_HOSTNAME
+   *  in host network mode
+   */
+  private void setHostname(DockerRunCommand runCommand,
+      String containerIdStr, String network, String name)
       throws ContainerExecutionException {
-    if (name == null || name.isEmpty()) {
-      name = RegistryPathUtils.encodeYarnID(containerIdStr);
 
-      String domain = conf.get(RegistryConstants.KEY_DNS_DOMAIN);
-      if (domain != null) {
-        name += ("." + domain);
+    if (network.equalsIgnoreCase("host")) {
+      if (name != null && !name.isEmpty()) {
+        LOG.info("setting hostname in container to: " + name);
+        runCommand.setHostname(name);
       }
-      validateHostname(name);
-    }
+    } else {
+      //get default hostname
+      if (name == null || name.isEmpty()) {
+        name = RegistryPathUtils.encodeYarnID(containerIdStr);
 
-    LOG.info("setting hostname in container to: " + name);
-    runCommand.setHostname(name);
+        String domain = conf.get(RegistryConstants.KEY_DNS_DOMAIN);
+        if (domain != null) {
+          name += ("." + domain);
+        }
+        validateHostname(name);
+      }
+      LOG.info("setting hostname in container to: " + name);
+      runCommand.setHostname(name);
+    }
   }
 
   /**
@@ -823,12 +835,9 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
     DockerRunCommand runCommand = new DockerRunCommand(containerIdStr,
         dockerRunAsUser, imageName)
         .setNetworkType(network);
-    // Only add hostname if network is not host or if Registry DNS is enabled.
-    if (!network.equalsIgnoreCase("host") ||
-        conf.getBoolean(RegistryConstants.KEY_DNS_ENABLED,
-            RegistryConstants.DEFAULT_DNS_ENABLED)) {
-      setHostname(runCommand, containerIdStr, hostname);
-    }
+
+    setHostname(runCommand, containerIdStr, network, hostname);
+
     runCommand.setCapabilities(capabilities);
 
     runCommand.addAllReadWriteMountLocations(containerLogDirs);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[12/50] [abbrv] hadoop git commit: YARN-8488. Added SUCCEEDED/FAILED states to YARN service. Contributed by Suma Shivaprasad

Posted by eh...@apache.org.
YARN-8488.  Added SUCCEEDED/FAILED states to YARN service.
            Contributed by Suma Shivaprasad


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fd089caf
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fd089caf
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fd089caf

Branch: refs/heads/HDFS-12090
Commit: fd089caf69cf608a91564c9c3d20cbf84e7fd60c
Parents: c61824a
Author: Eric Yang <ey...@apache.org>
Authored: Tue Aug 28 13:55:28 2018 -0400
Committer: Eric Yang <ey...@apache.org>
Committed: Tue Aug 28 13:55:28 2018 -0400

----------------------------------------------------------------------
 .../hadoop/yarn/service/ServiceScheduler.java   | 100 ++++++++++---
 .../service/api/records/ComponentState.java     |   2 +-
 .../service/api/records/ContainerState.java     |   3 +-
 .../yarn/service/api/records/ServiceState.java  |   2 +-
 .../component/instance/ComponentInstance.java   | 144 ++++++++++++++-----
 .../timelineservice/ServiceTimelineEvent.java   |   5 +-
 .../ServiceTimelinePublisher.java               |  33 ++++-
 .../yarn/service/MockRunningServiceContext.java |  18 ++-
 .../hadoop/yarn/service/ServiceTestUtils.java   |   9 +-
 .../yarn/service/component/TestComponent.java   |  55 ++++++-
 .../component/TestComponentRestartPolicy.java   |   1 -
 .../instance/TestComponentInstance.java         |  35 ++---
 .../TestServiceTimelinePublisher.java           |   4 +-
 13 files changed, 322 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd089caf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
index 384659f..b49ef2a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
@@ -59,6 +59,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.service.api.ServiceApiConstants;
+import org.apache.hadoop.yarn.service.api.records.ContainerState;
 import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.api.records.ServiceState;
 import org.apache.hadoop.yarn.service.api.records.ConfigFile;
@@ -80,6 +81,8 @@ import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
 import org.apache.hadoop.yarn.service.utils.ServiceRegistryUtils;
 import org.apache.hadoop.yarn.service.utils.ServiceUtils;
 import org.apache.hadoop.yarn.util.BoundedAppender;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.hadoop.yarn.util.resource.ResourceUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -102,7 +105,8 @@ import java.util.concurrent.TimeUnit;
 
 import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY;
 import static org.apache.hadoop.registry.client.api.RegistryConstants.*;
-import static org.apache.hadoop.yarn.api.records.ContainerExitStatus.KILLED_AFTER_APP_COMPLETION;
+import static org.apache.hadoop.yarn.api.records.ContainerExitStatus
+    .KILLED_AFTER_APP_COMPLETION;
 import static org.apache.hadoop.yarn.service.api.ServiceApiConstants.*;
 import static org.apache.hadoop.yarn.service.component.ComponentEventType.*;
 import static org.apache.hadoop.yarn.service.exceptions.LauncherExitCodes
@@ -137,6 +141,8 @@ public class ServiceScheduler extends CompositeService {
 
   private ServiceTimelinePublisher serviceTimelinePublisher;
 
+  private boolean timelineServiceEnabled;
+
   // Global diagnostics that will be reported to RM on eRxit.
   // The unit the number of characters. This will be limited to 64 * 1024
   // characters.
@@ -169,6 +175,8 @@ public class ServiceScheduler extends CompositeService {
   private volatile FinalApplicationStatus finalApplicationStatus =
       FinalApplicationStatus.ENDED;
 
+  private Clock systemClock;
+
   // For unit test override since we don't want to terminate UT process.
   private ServiceUtils.ProcessTerminationHandler
       terminationHandler = new ServiceUtils.ProcessTerminationHandler();
@@ -176,6 +184,8 @@ public class ServiceScheduler extends CompositeService {
   public ServiceScheduler(ServiceContext context) {
     super(context.getService().getName());
     this.context = context;
+    this.app = context.getService();
+    this.systemClock = SystemClock.getInstance();
   }
 
   public void buildInstance(ServiceContext context, Configuration configuration)
@@ -254,8 +264,14 @@ public class ServiceScheduler extends CompositeService {
         YarnServiceConf.DEFAULT_CONTAINER_RECOVERY_TIMEOUT_MS,
         app.getConfiguration(), getConfig());
 
+    if (YarnConfiguration
+        .timelineServiceV2Enabled(getConfig())) {
+      timelineServiceEnabled = true;
+    }
+
     serviceManager = createServiceManager();
     context.setServiceManager(serviceManager);
+
   }
 
   protected YarnRegistryViewForProviders createYarnRegistryOperations(
@@ -311,21 +327,38 @@ public class ServiceScheduler extends CompositeService {
     // only stop the entire service when a graceful stop has been initiated
     // (e.g. via client RPC, not through the AM receiving a SIGTERM)
     if (gracefulStop) {
+
       if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
-        // mark component-instances/containers as STOPPED
-        for (ContainerId containerId : getLiveInstances().keySet()) {
-          serviceTimelinePublisher.componentInstanceFinished(containerId,
-              KILLED_AFTER_APP_COMPLETION, diagnostics.toString());
+
+        // mark other component-instances/containers as STOPPED
+        final Map<ContainerId, ComponentInstance> liveInst =
+            getLiveInstances();
+        for (Map.Entry<ContainerId, ComponentInstance> instance : liveInst
+            .entrySet()) {
+          if (!ComponentInstance.isFinalState(
+              instance.getValue().getContainerSpec().getState())) {
+            LOG.info("{} Component instance state changed from {} to {}",
+                instance.getValue().getCompInstanceName(),
+                instance.getValue().getContainerSpec().getState(),
+                ContainerState.STOPPED);
+            serviceTimelinePublisher.componentInstanceFinished(
+                instance.getKey(), KILLED_AFTER_APP_COMPLETION,
+                ContainerState.STOPPED, getDiagnostics().toString());
+          }
         }
+
+        LOG.info("Service state changed to {}", finalApplicationStatus);
         // mark attempt as unregistered
-        serviceTimelinePublisher
-            .serviceAttemptUnregistered(context, diagnostics.toString());
+        serviceTimelinePublisher.serviceAttemptUnregistered(context,
+            finalApplicationStatus, diagnostics.toString());
       }
+
       // unregister AM
-      amRMClient.unregisterApplicationMaster(FinalApplicationStatus.ENDED,
+      amRMClient.unregisterApplicationMaster(finalApplicationStatus,
           diagnostics.toString(), "");
-      LOG.info("Service {} unregistered with RM, with attemptId = {} " +
-          ", diagnostics = {} ", app.getName(), context.attemptId, diagnostics);
+      LOG.info("Service {} unregistered with RM, with attemptId = {} "
+              + ", diagnostics = {} ", app.getName(), context.attemptId,
+          diagnostics);
     }
     super.serviceStop();
   }
@@ -911,7 +944,7 @@ public class ServiceScheduler extends CompositeService {
 *   (which #failed-instances + #suceeded-instances = #total-n-containers)
 * The service will be terminated.
 */
-  public synchronized void terminateServiceIfAllComponentsFinished() {
+  public void terminateServiceIfAllComponentsFinished() {
     boolean shouldTerminate = true;
 
     // Succeeded comps and failed comps, for logging purposes.
@@ -920,7 +953,30 @@ public class ServiceScheduler extends CompositeService {
 
     for (Component comp : getAllComponents().values()) {
       ComponentRestartPolicy restartPolicy = comp.getRestartPolicyHandler();
-      if (!restartPolicy.shouldTerminate(comp)) {
+
+      if (restartPolicy.shouldTerminate(comp)) {
+        if (restartPolicy.hasCompletedSuccessfully(comp)) {
+          comp.getComponentSpec().setState(org.apache.hadoop
+              .yarn.service.api.records.ComponentState.SUCCEEDED);
+          LOG.info("{} Component state changed from {} to {}",
+              comp.getName(), comp.getComponentSpec().getState(),
+              org.apache.hadoop
+                  .yarn.service.api.records.ComponentState.SUCCEEDED);
+        } else {
+          comp.getComponentSpec().setState(org.apache.hadoop
+              .yarn.service.api.records.ComponentState.FAILED);
+          LOG.info("{} Component state changed from {} to {}",
+              comp.getName(), comp.getComponentSpec().getState(),
+              org.apache.hadoop
+                  .yarn.service.api.records.ComponentState.FAILED);
+        }
+
+        if (isTimelineServiceEnabled()) {
+          // record in ATS
+          serviceTimelinePublisher.componentFinished(comp.getComponentSpec(),
+              comp.getComponentSpec().getState(), systemClock.getTime());
+        }
+      } else {
         shouldTerminate = false;
         break;
       }
@@ -929,7 +985,7 @@ public class ServiceScheduler extends CompositeService {
 
       if (nFailed > 0) {
         failedComponents.add(comp.getName());
-      } else{
+      } else {
         succeededComponents.add(comp.getName());
       }
     }
@@ -944,16 +1000,28 @@ public class ServiceScheduler extends CompositeService {
       LOG.info("Failed components: [" + org.apache.commons.lang3.StringUtils
           .join(failedComponents, ",") + "]");
 
+      int exitStatus = EXIT_SUCCESS;
       if (failedComponents.isEmpty()) {
         setGracefulStop(FinalApplicationStatus.SUCCEEDED);
-        getTerminationHandler().terminate(EXIT_SUCCESS);
-      } else{
+        app.setState(ServiceState.SUCCEEDED);
+      } else {
         setGracefulStop(FinalApplicationStatus.FAILED);
-        getTerminationHandler().terminate(EXIT_FALSE);
+        app.setState(ServiceState.FAILED);
+        exitStatus = EXIT_FALSE;
       }
+
+      getTerminationHandler().terminate(exitStatus);
     }
   }
 
+  public Clock getSystemClock() {
+    return systemClock;
+  }
+
+  public boolean isTimelineServiceEnabled() {
+    return timelineServiceEnabled;
+  }
+
   public ServiceUtils.ProcessTerminationHandler getTerminationHandler() {
     return terminationHandler;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd089caf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ComponentState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ComponentState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ComponentState.java
index 3e7ed11..472f374 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ComponentState.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ComponentState.java
@@ -26,5 +26,5 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceStability.Unstable
 @ApiModel(description = "The current state of a component.")
 public enum ComponentState {
-  FLEXING, STABLE, NEEDS_UPGRADE, UPGRADING;
+  FLEXING, STABLE, NEEDS_UPGRADE, UPGRADING, SUCCEEDED, FAILED;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd089caf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ContainerState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ContainerState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ContainerState.java
index 6e39073..cac527a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ContainerState.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ContainerState.java
@@ -26,5 +26,6 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
 public enum ContainerState {
-  RUNNING_BUT_UNREADY, READY, STOPPED, NEEDS_UPGRADE, UPGRADING;
+  RUNNING_BUT_UNREADY, READY, STOPPED, NEEDS_UPGRADE, UPGRADING, SUCCEEDED,
+  FAILED;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd089caf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java
index 0b3c037..3876173 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java
@@ -30,5 +30,5 @@ import org.apache.hadoop.classification.InterfaceStability;
 @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
 public enum ServiceState {
   ACCEPTED, STARTED, STABLE, STOPPED, FAILED, FLEX, UPGRADING,
-  UPGRADING_AUTO_FINALIZE, EXPRESS_UPGRADING;
+  UPGRADING_AUTO_FINALIZE, EXPRESS_UPGRADING, SUCCEEDED;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd089caf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
index ed5e68e..afd8c67 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.client.api.NMClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -41,7 +42,9 @@ import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.service.ServiceScheduler;
 import org.apache.hadoop.yarn.service.api.records.Artifact;
+import org.apache.hadoop.yarn.service.api.records.ComponentState;
 import org.apache.hadoop.yarn.service.api.records.ContainerState;
+import org.apache.hadoop.yarn.service.api.records.ServiceState;
 import org.apache.hadoop.yarn.service.component.Component;
 import org.apache.hadoop.yarn.service.component.ComponentEvent;
 import org.apache.hadoop.yarn.service.component.ComponentEventType;
@@ -68,6 +71,9 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
 import static org.apache.hadoop.registry.client.types.yarn.YarnRegistryAttributes.*;
+
+import static org.apache.hadoop.yarn.api.records.ContainerExitStatus
+    .KILLED_AFTER_APP_COMPLETION;
 import static org.apache.hadoop.yarn.api.records.ContainerExitStatus.KILLED_BY_APPMASTER;
 import static org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEventType.*;
 import static org.apache.hadoop.yarn.service.component.instance.ComponentInstanceState.*;
@@ -242,15 +248,22 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
   }
 
   @VisibleForTesting
-  static void handleComponentInstanceRelaunch(
-      ComponentInstance compInstance, ComponentInstanceEvent event,
-      boolean failureBeforeLaunch) {
+  static void handleComponentInstanceRelaunch(ComponentInstance compInstance,
+      ComponentInstanceEvent event, boolean failureBeforeLaunch,
+      String containerDiag) {
     Component comp = compInstance.getComponent();
 
     // Do we need to relaunch the service?
-    boolean hasContainerFailed = hasContainerFailed(event.getStatus());
+    boolean hasContainerFailed = failureBeforeLaunch || hasContainerFailed(
+        event.getStatus());
 
     ComponentRestartPolicy restartPolicy = comp.getRestartPolicyHandler();
+    ContainerState containerState =
+        hasContainerFailed ? ContainerState.FAILED : ContainerState.SUCCEEDED;
+
+    if (compInstance.getContainerSpec() != null) {
+      compInstance.getContainerSpec().setState(containerState);
+    }
 
     if (restartPolicy.shouldRelaunchInstance(compInstance, event.getStatus())) {
       // re-ask the failed container.
@@ -259,25 +272,47 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
 
       StringBuilder builder = new StringBuilder();
       builder.append(compInstance.getCompInstanceId()).append(": ");
-      builder.append(event.getContainerId()).append(" completed. Reinsert back to pending list and requested ");
+      builder.append(event.getContainerId()).append(
+          " completed. Reinsert back to pending list and requested ");
       builder.append("a new container.").append(System.lineSeparator());
-      builder.append(" exitStatus=").append(failureBeforeLaunch ? null : event.getStatus().getExitStatus());
+      builder.append(" exitStatus=").append(
+          failureBeforeLaunch ? null : event.getStatus().getExitStatus());
       builder.append(", diagnostics=");
-      builder.append(failureBeforeLaunch ? FAILED_BEFORE_LAUNCH_DIAG : event.getStatus().getDiagnostics());
+      builder.append(failureBeforeLaunch ?
+          FAILED_BEFORE_LAUNCH_DIAG :
+          event.getStatus().getDiagnostics());
 
-      if (event.getStatus().getExitStatus() != 0) {
+      if (event.getStatus() != null && event.getStatus().getExitStatus() != 0) {
         LOG.error(builder.toString());
-      } else {
+      } else{
         LOG.info(builder.toString());
       }
-    } else {
+
+      if (compInstance.timelineServiceEnabled) {
+        // record in ATS
+        LOG.info("Publishing component instance status {} {} ",
+            event.getContainerId(), containerState);
+        compInstance.serviceTimelinePublisher.componentInstanceFinished(
+            event.getContainerId(), event.getStatus().getExitStatus(),
+            containerState, containerDiag);
+      }
+
+    } else{
       // When no relaunch, update component's #succeeded/#failed
       // instances.
       if (hasContainerFailed) {
         comp.markAsFailed(compInstance);
-      } else {
+      } else{
         comp.markAsSucceeded(compInstance);
       }
+
+      if (compInstance.timelineServiceEnabled) {
+        // record in ATS
+        compInstance.serviceTimelinePublisher.componentInstanceFinished(
+            event.getContainerId(), event.getStatus().getExitStatus(),
+            containerState, containerDiag);
+      }
+
       LOG.info(compInstance.getCompInstanceId() + (!hasContainerFailed ?
           " succeeded" :
           " failed") + " without retry, exitStatus=" + event.getStatus());
@@ -287,8 +322,8 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
 
   public static boolean hasContainerFailed(ContainerStatus containerStatus) {
     //Mark conainer as failed if we cant get its exit status i.e null?
-    return containerStatus == null || containerStatus.getExitStatus() !=
-        ContainerExitStatus.SUCCESS;
+    return containerStatus == null || containerStatus
+        .getExitStatus() != ContainerExitStatus.SUCCESS;
   }
 
   private static class ContainerStoppedTransition extends  BaseTransition {
@@ -307,9 +342,10 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
         ComponentInstanceEvent event) {
 
       Component comp = compInstance.component;
-      String containerDiag =
-          compInstance.getCompInstanceId() + ": " + (failedBeforeLaunching ?
-              FAILED_BEFORE_LAUNCH_DIAG : event.getStatus().getDiagnostics());
+      String containerDiag = compInstance.getCompInstanceId() + ": " + (
+          failedBeforeLaunching ?
+              FAILED_BEFORE_LAUNCH_DIAG :
+              event.getStatus().getDiagnostics());
       compInstance.diagnostics.append(containerDiag + System.lineSeparator());
       compInstance.cancelContainerStatusRetriever();
       if (compInstance.getState().equals(ComponentInstanceState.UPGRADING)) {
@@ -329,36 +365,69 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
       // Check if it exceeds the failure threshold, but only if health threshold
       // monitor is not enabled
       if (!comp.isHealthThresholdMonitorEnabled()
-          && comp.currentContainerFailure
-              .get() > comp.maxContainerFailurePerComp) {
+          && comp.currentContainerFailure.get()
+          > comp.maxContainerFailurePerComp) {
         String exitDiag = MessageFormat.format(
-            "[COMPONENT {0}]: Failed {1} times, exceeded the limit - {2}. Shutting down now... "
-                + System.lineSeparator(),
-            comp.getName(), comp.currentContainerFailure.get(), comp.maxContainerFailurePerComp);
+            "[COMPONENT {0}]: Failed {1} times, exceeded the limit - {2}. "
+                + "Shutting down now... "
+                + System.lineSeparator(), comp.getName(),
+            comp.currentContainerFailure.get(),
+            comp.maxContainerFailurePerComp);
         compInstance.diagnostics.append(exitDiag);
         // append to global diagnostics that will be reported to RM.
         scheduler.getDiagnostics().append(containerDiag);
         scheduler.getDiagnostics().append(exitDiag);
         LOG.warn(exitDiag);
+
+        compInstance.getContainerSpec().setState(ContainerState.FAILED);
+        comp.getComponentSpec().setState(ComponentState.FAILED);
+        comp.getScheduler().getApp().setState(ServiceState.FAILED);
+
+        if (compInstance.timelineServiceEnabled) {
+          // record in ATS
+          compInstance.scheduler.getServiceTimelinePublisher()
+              .componentInstanceFinished(compInstance.getContainer().getId(),
+                  failedBeforeLaunching ?
+                      -1 :
+                      event.getStatus().getExitStatus(), ContainerState.FAILED,
+                  containerDiag);
+
+          // mark other component-instances/containers as STOPPED
+          for (ContainerId containerId : scheduler.getLiveInstances()
+              .keySet()) {
+            if (!compInstance.container.getId().equals(containerId)
+                && !isFinalState(compInstance.getContainerSpec().getState())) {
+              compInstance.getContainerSpec().setState(ContainerState.STOPPED);
+              compInstance.scheduler.getServiceTimelinePublisher()
+                  .componentInstanceFinished(containerId,
+                      KILLED_AFTER_APP_COMPLETION, ContainerState.STOPPED,
+                      scheduler.getDiagnostics().toString());
+            }
+          }
+
+          compInstance.scheduler.getServiceTimelinePublisher()
+              .componentFinished(comp.getComponentSpec(), ComponentState.FAILED,
+                  scheduler.getSystemClock().getTime());
+
+          compInstance.scheduler.getServiceTimelinePublisher()
+              .serviceAttemptUnregistered(comp.getContext(),
+                  FinalApplicationStatus.FAILED,
+                  scheduler.getDiagnostics().toString());
+        }
+
         shouldFailService = true;
       }
 
       if (!failedBeforeLaunching) {
         // clean up registry
-        // If the container failed before launching, no need to cleanup registry,
+        // If the container failed before launching, no need to cleanup
+        // registry,
         // because it was not registered before.
-        // hdfs dir content will be overwritten when a new container gets started,
+        // hdfs dir content will be overwritten when a new container gets
+        // started,
         // so no need remove.
-        compInstance.scheduler.executorService
-            .submit(() -> compInstance.cleanupRegistry(event.getContainerId()));
-
-        if (compInstance.timelineServiceEnabled) {
-          // record in ATS
-          compInstance.serviceTimelinePublisher
-              .componentInstanceFinished(event.getContainerId(),
-                  event.getStatus().getExitStatus(), containerDiag);
-        }
-        compInstance.containerSpec.setState(ContainerState.STOPPED);
+        compInstance.scheduler.executorService.submit(
+            () -> compInstance.cleanupRegistry(event.getContainerId()));
       }
 
       // remove the failed ContainerId -> CompInstance mapping
@@ -367,7 +436,7 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
       // According to component restart policy, handle container restart
       // or finish the service (if all components finished)
       handleComponentInstanceRelaunch(compInstance, event,
-          failedBeforeLaunching);
+          failedBeforeLaunching, containerDiag);
 
       if (shouldFailService) {
         scheduler.getTerminationHandler().terminate(-1);
@@ -375,6 +444,11 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
     }
   }
 
+  public static boolean isFinalState(ContainerState state) {
+    return ContainerState.FAILED.equals(state) || ContainerState.STOPPED
+        .equals(state) || ContainerState.SUCCEEDED.equals(state);
+  }
+
   private static class ContainerUpgradeTransition extends BaseTransition {
 
     @Override
@@ -586,7 +660,7 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
 
     if (timelineServiceEnabled) {
       serviceTimelinePublisher.componentInstanceFinished(containerId,
-          KILLED_BY_APPMASTER, diagnostics.toString());
+          KILLED_BY_APPMASTER, ContainerState.STOPPED, diagnostics.toString());
     }
     cancelContainerStatusRetriever();
     scheduler.executorService.submit(() ->

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd089caf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelineEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelineEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelineEvent.java
index 6c3428a..832dad7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelineEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelineEvent.java
@@ -32,5 +32,8 @@ public enum ServiceTimelineEvent {
 
   COMPONENT_INSTANCE_IP_HOST_UPDATE,
 
-  COMPONENT_INSTANCE_BECOME_READY
+  COMPONENT_INSTANCE_BECOME_READY,
+
+  COMPONENT_FINISHED
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd089caf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelinePublisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelinePublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelinePublisher.java
index 6c73ebb..79f37c0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelinePublisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelinePublisher.java
@@ -28,6 +28,8 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.service.ServiceContext;
 import org.apache.hadoop.yarn.service.api.records.*;
+import org.apache.hadoop.yarn.service.api.records.Component;
+import org.apache.hadoop.yarn.service.api.records.ComponentState;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.slf4j.Logger;
@@ -42,7 +44,6 @@ import java.util.Map.Entry;
 import java.util.Set;
 
 import static org.apache.hadoop.yarn.service.api.records.ContainerState.READY;
-import static org.apache.hadoop.yarn.service.api.records.ContainerState.STOPPED;
 import static org.apache.hadoop.yarn.service.timelineservice.ServiceTimelineMetricsConstants.DIAGNOSTICS_INFO;
 
 /**
@@ -130,12 +131,11 @@ public class ServiceTimelinePublisher extends CompositeService {
   }
 
   public void serviceAttemptUnregistered(ServiceContext context,
-      String diagnostics) {
+      FinalApplicationStatus status, String diagnostics) {
     TimelineEntity entity = createServiceAttemptEntity(
         context.attemptId.getApplicationId().toString());
     Map<String, Object> entityInfos = new HashMap<String, Object>();
-    entityInfos.put(ServiceTimelineMetricsConstants.STATE,
-        FinalApplicationStatus.ENDED);
+    entityInfos.put(ServiceTimelineMetricsConstants.STATE, status);
     entityInfos.put(DIAGNOSTICS_INFO, diagnostics);
     entity.addInfo(entityInfos);
 
@@ -180,7 +180,7 @@ public class ServiceTimelinePublisher extends CompositeService {
   }
 
   public void componentInstanceFinished(ContainerId containerId,
-      int exitCode, String diagnostics) {
+      int exitCode, ContainerState state, String diagnostics) {
     TimelineEntity entity = createComponentInstanceEntity(
         containerId.toString());
 
@@ -189,7 +189,7 @@ public class ServiceTimelinePublisher extends CompositeService {
     entityInfos.put(ServiceTimelineMetricsConstants.EXIT_STATUS_CODE,
         exitCode);
     entityInfos.put(DIAGNOSTICS_INFO, diagnostics);
-    entityInfos.put(ServiceTimelineMetricsConstants.STATE, STOPPED);
+    entityInfos.put(ServiceTimelineMetricsConstants.STATE, state);
     entity.addInfo(entityInfos);
 
     // add an event
@@ -375,4 +375,25 @@ public class ServiceTimelinePublisher extends CompositeService {
       log.error("Error when publishing entity " + entity, e);
     }
   }
+
+  public void componentFinished(
+      Component comp,
+      ComponentState state, long finishTime) {
+    createComponentEntity(comp.getName());
+    TimelineEntity entity = createComponentEntity(comp.getName());
+
+    // create info keys
+    Map<String, Object> entityInfos = new HashMap<String, Object>();
+    entityInfos.put(ServiceTimelineMetricsConstants.STATE, state);
+    entity.addInfo(entityInfos);
+
+    // add an event
+    TimelineEvent startEvent = new TimelineEvent();
+    startEvent
+        .setId(ServiceTimelineEvent.COMPONENT_FINISHED.toString());
+    startEvent.setTimestamp(finishTime);
+    entity.addEvent(startEvent);
+
+    putEntity(entity);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd089caf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockRunningServiceContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockRunningServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockRunningServiceContext.java
index 89888c5..321b2cd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockRunningServiceContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockRunningServiceContext.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEvent;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEventType;
 import org.apache.hadoop.yarn.service.containerlaunch.ContainerLaunchService;
 import org.apache.hadoop.yarn.service.registry.YarnRegistryViewForProviders;
+import org.apache.hadoop.yarn.service.utils.ServiceUtils;
 import org.mockito.stubbing.Answer;
 
 import java.io.IOException;
@@ -92,7 +93,18 @@ public class MockRunningServiceContext extends ServiceContext {
       public ContainerLaunchService getContainerLaunchService() {
         return mockLaunchService;
       }
+
+      @Override public ServiceUtils.ProcessTerminationHandler
+      getTerminationHandler() {
+        return new
+        ServiceUtils.ProcessTerminationHandler() {
+          public void terminate(int exitCode) {
+          }
+        };
+      }
     };
+
+
     this.scheduler.init(fsWatcher.getConf());
 
     ServiceTestUtils.createServiceManager(this);
@@ -116,8 +128,10 @@ public class MockRunningServiceContext extends ServiceContext {
       Component component = new org.apache.hadoop.yarn.service.component.
           Component(componentSpec, 1L, context);
       componentState.put(component.getName(), component);
-      component.handle(new ComponentEvent(component.getName(),
-          ComponentEventType.FLEX));
+      component.handle(
+          new ComponentEvent(component.getName(), ComponentEventType.FLEX)
+              .setDesired(
+                  component.getComponentSpec().getNumberOfContainers()));
 
       for (int i = 0; i < componentSpec.getNumberOfContainers(); i++) {
         counter++;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd089caf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/ServiceTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/ServiceTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/ServiceTestUtils.java
index 170c20b..6b49ab0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/ServiceTestUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/ServiceTestUtils.java
@@ -63,7 +63,6 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.net.URL;
 import java.nio.file.Paths;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeoutException;
@@ -119,14 +118,10 @@ public class ServiceTestUtils {
             Component.RestartPolicyEnum.NEVER, null));
     exampleApp.addComponent(
         createComponent("terminating-comp2", 2, "sleep 1000",
-            Component.RestartPolicyEnum.ON_FAILURE, new ArrayList<String>() {{
-                add("terminating-comp1");
-            }}));
+            Component.RestartPolicyEnum.ON_FAILURE, null));
     exampleApp.addComponent(
         createComponent("terminating-comp3", 2, "sleep 1000",
-            Component.RestartPolicyEnum.ON_FAILURE, new ArrayList<String>() {{
-                add("terminating-comp2");
-            }}));
+            Component.RestartPolicyEnum.ON_FAILURE, null));
 
     return exampleApp;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd089caf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponent.java
index 2e17c7f..e1a4d9d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponent.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.yarn.service.ServiceTestUtils;
 import org.apache.hadoop.yarn.service.TestServiceManager;
 import org.apache.hadoop.yarn.service.api.records.ComponentState;
 import org.apache.hadoop.yarn.service.api.records.Service;
+import org.apache.hadoop.yarn.service.api.records.ServiceState;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEvent;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEventType;
@@ -147,7 +148,8 @@ public class TestComponent {
   }
 
   @Test
-  public void testComponentStateUpdatesWithTerminatingComponents() throws
+  public void testComponentStateReachesStableStateWithTerminatingComponents()
+      throws
       Exception {
     final String serviceName =
         "testComponentStateUpdatesWithTerminatingComponents";
@@ -198,6 +200,57 @@ public class TestComponent {
     }
   }
 
+  @Test
+  public void testComponentStateUpdatesWithTerminatingComponents()
+      throws
+      Exception {
+    final String serviceName =
+        "testComponentStateUpdatesWithTerminatingComponents";
+
+    Service testService = ServiceTestUtils.createTerminatingJobExample(
+        serviceName);
+    TestServiceManager.createDef(serviceName, testService);
+
+    ServiceContext context = new MockRunningServiceContext(rule, testService);
+
+    for (Component comp : context.scheduler.getAllComponents().values()) {
+      Iterator<ComponentInstance> instanceIter = comp.
+          getAllComponentInstances().iterator();
+
+      while (instanceIter.hasNext()) {
+
+        ComponentInstance componentInstance = instanceIter.next();
+        Container instanceContainer = componentInstance.getContainer();
+
+        //stop 1 container
+        ContainerStatus containerStatus = ContainerStatus.newInstance(
+            instanceContainer.getId(),
+            org.apache.hadoop.yarn.api.records.ContainerState.COMPLETE,
+            "successful", 0);
+        comp.handle(new ComponentEvent(comp.getName(),
+            ComponentEventType.CONTAINER_COMPLETED).setStatus(containerStatus)
+            .setContainerId(instanceContainer.getId()));
+        componentInstance.handle(
+            new ComponentInstanceEvent(componentInstance.getContainer().getId(),
+                ComponentInstanceEventType.STOP).setStatus(containerStatus));
+      }
+
+      ComponentState componentState =
+          comp.getComponentSpec().getState();
+      Assert.assertEquals(
+          ComponentState.SUCCEEDED,
+          componentState);
+    }
+
+    ServiceState serviceState =
+        testService.getState();
+    Assert.assertEquals(
+        ServiceState.SUCCEEDED,
+        serviceState);
+  }
+
+
+
   private static org.apache.hadoop.yarn.service.api.records.Component
       createSpecWithEnv(String serviceName, String compName, String key,
       String val) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd089caf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponentRestartPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponentRestartPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponentRestartPolicy.java
index 60f5c91..03158cf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponentRestartPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponentRestartPolicy.java
@@ -110,7 +110,6 @@ public class TestComponentRestartPolicy {
 
     assertEquals(true, restartPolicy.isReadyForDownStream(component));
 
-
     when(component.getNumSucceededInstances()).thenReturn(new Long(2));
     when(component.getNumFailedInstances()).thenReturn(new Long(1));
     when(component.getNumDesiredInstances()).thenReturn(3);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd089caf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/instance/TestComponentInstance.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/instance/TestComponentInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/instance/TestComponentInstance.java
index f428904..e039981 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/instance/TestComponentInstance.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/instance/TestComponentInstance.java
@@ -204,6 +204,8 @@ public class TestComponentInstance {
     when(componentInstance.getComponent()).thenReturn(component);
     when(componentInstance.getCompInstanceName()).thenReturn(
         "compInstance" + instanceId);
+    Container container = mock(Container.class);
+    when(componentInstance.getContainerSpec()).thenReturn(container);
 
     ServiceUtils.ProcessTerminationHandler terminationHandler = mock(
         ServiceUtils.ProcessTerminationHandler.class);
@@ -227,12 +229,15 @@ public class TestComponentInstance {
     Mockito.doNothing().when(serviceScheduler).setGracefulStop(
         any(FinalApplicationStatus.class));
 
+    final String containerDiag = "Container succeeded";
+
     ComponentInstanceEvent componentInstanceEvent = mock(
         ComponentInstanceEvent.class);
     ContainerId containerId = ContainerId.newContainerId(ApplicationAttemptId
         .newInstance(ApplicationId.newInstance(1234L, 1), 1), 1);
     ContainerStatus containerStatus = ContainerStatus.newInstance(containerId,
-        org.apache.hadoop.yarn.api.records.ContainerState.COMPLETE, "hello", 0);
+        org.apache.hadoop.yarn.api.records.ContainerState.COMPLETE,
+        containerDiag, 0);
 
     when(componentInstanceEvent.getStatus()).thenReturn(containerStatus);
 
@@ -245,7 +250,7 @@ public class TestComponentInstance {
         comp.getAllComponentInstances().iterator().next();
 
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent, false);
+        componentInstanceEvent, false, containerDiag);
 
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, never()).markAsFailed(any(ComponentInstance.class));
@@ -262,7 +267,7 @@ public class TestComponentInstance {
     componentInstance = comp.getAllComponentInstances().iterator().next();
     containerStatus.setExitStatus(1);
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent, false);
+        componentInstanceEvent, false, containerDiag);
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, never()).markAsFailed(any(ComponentInstance.class));
     verify(comp, times(1)).reInsertPendingInstance(
@@ -286,7 +291,7 @@ public class TestComponentInstance {
     when(comp.getNumSucceededInstances()).thenReturn(new Long(1));
 
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent, false);
+        componentInstanceEvent, false, containerDiag);
     verify(comp, times(1)).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, never()).markAsFailed(any(ComponentInstance.class));
     verify(comp, times(0)).reInsertPendingInstance(
@@ -304,8 +309,7 @@ public class TestComponentInstance {
 
     when(comp.getNumFailedInstances()).thenReturn(new Long(1));
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent, false);
-
+        componentInstanceEvent, false, containerDiag);
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, times(1)).markAsFailed(any(ComponentInstance.class));
     verify(comp, times(0)).reInsertPendingInstance(
@@ -323,7 +327,7 @@ public class TestComponentInstance {
     componentInstance = comp.getAllComponentInstances().iterator().next();
     containerStatus.setExitStatus(1);
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent, false);
+        componentInstanceEvent, false, containerDiag);
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, never()).markAsFailed(any(ComponentInstance.class));
     verify(comp, times(1)).reInsertPendingInstance(
@@ -340,7 +344,7 @@ public class TestComponentInstance {
     componentInstance = comp.getAllComponentInstances().iterator().next();
     containerStatus.setExitStatus(1);
     ComponentInstance.handleComponentInstanceRelaunch(componentInstance,
-        componentInstanceEvent, false);
+        componentInstanceEvent, false, containerDiag);
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, times(1)).markAsFailed(any(ComponentInstance.class));
     verify(comp, times(0)).reInsertPendingInstance(
@@ -363,8 +367,7 @@ public class TestComponentInstance {
     containerStatus.setExitStatus(1);
     ComponentInstance commponentInstance = iter.next();
     ComponentInstance.handleComponentInstanceRelaunch(commponentInstance,
-        componentInstanceEvent, false);
-
+        componentInstanceEvent, false, containerDiag);
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
     verify(comp, never()).markAsFailed(any(ComponentInstance.class));
     verify(comp, times(1)).reInsertPendingInstance(
@@ -404,7 +407,7 @@ public class TestComponentInstance {
       when(component2Instance.getComponent().getNumFailedInstances())
           .thenReturn(new Long(failed2Instances.size()));
       ComponentInstance.handleComponentInstanceRelaunch(component2Instance,
-          componentInstanceEvent, false);
+          componentInstanceEvent, false, containerDiag);
     }
 
     Map<String, ComponentInstance> failed1Instances = new HashMap<>();
@@ -418,7 +421,7 @@ public class TestComponentInstance {
       when(component1Instance.getComponent().getNumFailedInstances())
           .thenReturn(new Long(failed1Instances.size()));
       ComponentInstance.handleComponentInstanceRelaunch(component1Instance,
-          componentInstanceEvent, false);
+          componentInstanceEvent, false, containerDiag);
     }
 
     verify(comp, never()).markAsSucceeded(any(ComponentInstance.class));
@@ -458,7 +461,7 @@ public class TestComponentInstance {
       when(component2Instance.getComponent().getNumSucceededInstances())
           .thenReturn(new Long(succeeded2Instances.size()));
       ComponentInstance.handleComponentInstanceRelaunch(component2Instance,
-          componentInstanceEvent, false);
+          componentInstanceEvent, false, containerDiag);
     }
 
     Map<String, ComponentInstance> succeeded1Instances = new HashMap<>();
@@ -471,7 +474,7 @@ public class TestComponentInstance {
       when(component1Instance.getComponent().getNumSucceededInstances())
           .thenReturn(new Long(succeeded1Instances.size()));
       ComponentInstance.handleComponentInstanceRelaunch(component1Instance,
-          componentInstanceEvent, false);
+          componentInstanceEvent, false, containerDiag);
     }
 
     verify(comp, times(2)).markAsSucceeded(any(ComponentInstance.class));
@@ -500,7 +503,7 @@ public class TestComponentInstance {
 
     for (ComponentInstance component2Instance : component2Instances) {
       ComponentInstance.handleComponentInstanceRelaunch(component2Instance,
-          componentInstanceEvent, false);
+          componentInstanceEvent, false, containerDiag);
     }
 
     succeeded1Instances = new HashMap<>();
@@ -511,7 +514,7 @@ public class TestComponentInstance {
       when(component1Instance.getComponent().getSucceededInstances())
           .thenReturn(succeeded1Instances.values());
       ComponentInstance.handleComponentInstanceRelaunch(component1Instance,
-          componentInstanceEvent, false);
+          componentInstanceEvent, false, containerDiag);
     }
 
     verify(comp, times(2)).markAsSucceeded(any(ComponentInstance.class));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd089caf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/timelineservice/TestServiceTimelinePublisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/timelineservice/TestServiceTimelinePublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/timelineservice/TestServiceTimelinePublisher.java
index cff7229..a77e6c8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/timelineservice/TestServiceTimelinePublisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/timelineservice/TestServiceTimelinePublisher.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.service.timelineservice;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity.Identifier;
 import org.apache.hadoop.yarn.client.api.TimelineV2Client;
@@ -122,7 +123,8 @@ public class TestServiceTimelinePublisher {
     context.attemptId = ApplicationAttemptId
         .newInstance(ApplicationId.fromString(service.getId()), 1);
     String exitDiags = "service killed";
-    serviceTimelinePublisher.serviceAttemptUnregistered(context, exitDiags);
+    serviceTimelinePublisher.serviceAttemptUnregistered(context,
+        FinalApplicationStatus.ENDED, exitDiags);
     lastPublishedEntities =
         ((DummyTimelineClient) timelineClient).getLastPublishedEntities();
     for (TimelineEntity timelineEntity : lastPublishedEntities) {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[14/50] [abbrv] hadoop git commit: HDDS-376. Create custom message structure for use in AuditLogging Contributed by Dinesh Chitlangia.

Posted by eh...@apache.org.
HDDS-376. Create custom message structure for use in AuditLogging
Contributed by Dinesh Chitlangia.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ac515d22
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ac515d22
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ac515d22

Branch: refs/heads/HDFS-12090
Commit: ac515d22d84478acbed92ef4024d9a3d3f329c8a
Parents: cb9d371
Author: Anu Engineer <ae...@apache.org>
Authored: Tue Aug 28 12:59:08 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Tue Aug 28 12:59:08 2018 -0700

----------------------------------------------------------------------
 .../apache/hadoop/ozone/audit/AuditLogger.java  |  66 ++++------
 .../apache/hadoop/ozone/audit/AuditMessage.java |  64 ++++++++++
 .../apache/hadoop/ozone/audit/package-info.java |  19 ++-
 .../ozone/audit/TestOzoneAuditLogger.java       | 124 ++++++++++++-------
 4 files changed, 177 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac515d22/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLogger.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLogger.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLogger.java
index 46ffaab..ee20c66 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLogger.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLogger.java
@@ -21,10 +21,8 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Marker;
-import org.apache.logging.log4j.message.StructuredDataMessage;
 import org.apache.logging.log4j.spi.ExtendedLogger;
 
-import java.util.Map;
 
 /**
  * Class to define Audit Logger for Ozone.
@@ -32,16 +30,13 @@ import java.util.Map;
 public class AuditLogger {
 
   private ExtendedLogger logger;
-
-  private static final String SUCCESS = AuditEventStatus.SUCCESS.getStatus();
-  private static final String FAILURE = AuditEventStatus.FAILURE.getStatus();
   private static final String FQCN = AuditLogger.class.getName();
   private static final Marker WRITE_MARKER = AuditMarker.WRITE.getMarker();
   private static final Marker READ_MARKER = AuditMarker.READ.getMarker();
 
   /**
    * Parametrized Constructor to initialize logger.
-   * @param type
+   * @param type Audit Logger Type
    */
   public AuditLogger(AuditLoggerType type){
     initializeLogger(type);
@@ -60,68 +55,53 @@ public class AuditLogger {
     return logger;
   }
 
-  public void logWriteSuccess(AuditAction type, Map<String, String> data) {
-    logWriteSuccess(type, data, Level.INFO);
+  public void logWriteSuccess(AuditMessage msg) {
+    logWriteSuccess(Level.INFO, msg);
   }
 
-  public void logWriteSuccess(AuditAction type, Map<String, String> data, Level
-      level) {
-    StructuredDataMessage msg = new StructuredDataMessage("", SUCCESS,
-        type.getAction(), data);
+  public void logWriteSuccess(Level level, AuditMessage msg) {
     this.logger.logIfEnabled(FQCN, level, WRITE_MARKER, msg, null);
   }
 
-
-  public void logWriteFailure(AuditAction type, Map<String, String> data) {
-    logWriteFailure(type, data, Level.INFO, null);
+  public void logWriteFailure(AuditMessage msg) {
+    logWriteFailure(Level.ERROR, msg);
   }
 
-  public void logWriteFailure(AuditAction type, Map<String, String> data, Level
-      level) {
-    logWriteFailure(type, data, level, null);
+  public void logWriteFailure(Level level, AuditMessage msg) {
+    logWriteFailure(level, msg, null);
   }
 
-  public void logWriteFailure(AuditAction type, Map<String, String> data,
-      Throwable exception) {
-    logWriteFailure(type, data, Level.INFO, exception);
+  public void logWriteFailure(AuditMessage msg, Throwable exception) {
+    logWriteFailure(Level.ERROR, msg, exception);
   }
 
-  public void logWriteFailure(AuditAction type, Map<String, String> data, Level
-      level, Throwable exception) {
-    StructuredDataMessage msg = new StructuredDataMessage("", FAILURE,
-        type.getAction(), data);
+  public void logWriteFailure(Level level, AuditMessage msg,
+      Throwable exception) {
     this.logger.logIfEnabled(FQCN, level, WRITE_MARKER, msg, exception);
   }
 
-  public void logReadSuccess(AuditAction type, Map<String, String> data) {
-    logReadSuccess(type, data, Level.INFO);
+  public void logReadSuccess(AuditMessage msg) {
+    logReadSuccess(Level.INFO, msg);
   }
 
-  public void logReadSuccess(AuditAction type, Map<String, String> data, Level
-      level) {
-    StructuredDataMessage msg = new StructuredDataMessage("", SUCCESS,
-        type.getAction(), data);
+  public void logReadSuccess(Level level, AuditMessage msg) {
     this.logger.logIfEnabled(FQCN, level, READ_MARKER, msg, null);
   }
 
-  public void logReadFailure(AuditAction type, Map<String, String> data) {
-    logReadFailure(type, data, Level.INFO, null);
+  public void logReadFailure(AuditMessage msg) {
+    logReadFailure(Level.ERROR, msg);
   }
 
-  public void logReadFailure(AuditAction type, Map<String, String> data, Level
-      level) {
-    logReadFailure(type, data, level, null);
+  public void logReadFailure(Level level, AuditMessage msg) {
+    logReadFailure(level, msg, null);
   }
 
-  public void logReadFailure(AuditAction type, Map<String, String> data,
-      Throwable exception) {
-    logReadFailure(type, data, Level.INFO, exception);
+  public void logReadFailure(AuditMessage msg, Throwable exception) {
+    logReadFailure(Level.ERROR, msg, exception);
   }
 
-  public void logReadFailure(AuditAction type, Map<String, String> data, Level
-      level, Throwable exception) {
-    StructuredDataMessage msg = new StructuredDataMessage("", FAILURE,
-        type.getAction(), data);
+  public void logReadFailure(Level level, AuditMessage msg,
+      Throwable exception) {
     this.logger.logIfEnabled(FQCN, level, READ_MARKER, msg, exception);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac515d22/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditMessage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditMessage.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditMessage.java
new file mode 100644
index 0000000..858695a
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditMessage.java
@@ -0,0 +1,64 @@
+/**
+ * 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.audit;
+
+import org.apache.logging.log4j.message.Message;
+
+import java.util.Map;
+
+/**
+ * Defines audit message structure.
+ */
+public class AuditMessage implements Message {
+
+  private String message;
+
+  public AuditMessage(String user, String ip, String op,
+      Map<String, String> params, String ret){
+
+    this.message = String.format("user=%s ip=%s op=%s %s ret=%s",
+                                  user, ip, op, params, ret);
+  }
+
+  @Override
+  public String getFormattedMessage() {
+    return message;
+  }
+
+  @Override
+  public String getFormat() {
+    return null;
+  }
+
+  @Override
+  public Object[] getParameters() {
+    return new Object[0];
+  }
+
+  @Override
+  public Throwable getThrowable() {
+    return null;
+  }
+
+  /**
+   * Use when there are custom string to be added to default msg.
+   * @param customMessage custom string
+   */
+  private void appendMessage(String customMessage) {
+    this.message += customMessage;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac515d22/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java
index 3743fdd..48de3f7 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java
@@ -46,7 +46,7 @@ package org.apache.hadoop.ozone.audit;
  * **** Auditable ***
  * This is an interface to mark an entity as auditable.
  * This interface must be implemented by entities requiring audit logging.
- * For example - KSMVolumeArgs, KSMBucketArgs.
+ * For example - OMVolumeArgs, OMBucketArgs.
  * The implementing class must override toAuditMap() to return an
  * instance of Map<Key, Value> where both Key and Value are String.
  *
@@ -81,6 +81,11 @@ package org.apache.hadoop.ozone.audit;
  * *** AuditMarker ***
  * Enum to define various Audit Markers used in AuditLogging.
  *
+ * *** AuditMessage ***
+ * Entity to define an audit message to be logged
+ * It will generate a message formatted as:
+ * user=xxx ip=xxx op=XXXX_XXXX {key=val, key1=val1..} ret=XXXXXX
+ *
  * ****************************************************************************
  *                              Usage
  * ****************************************************************************
@@ -88,14 +93,16 @@ package org.apache.hadoop.ozone.audit;
  * 1. Get a logger by specifying the appropriate logger type
  * Example: ExtendedLogger AUDIT = new AuditLogger(AuditLoggerType.OMLogger)
  *
- * 2. Log Read/Write and Success/Failure event as needed.
+ * 2. Construct an instance of AuditMessage
+ *
+ * 3. Log Read/Write and Success/Failure event as needed.
  * Example
- * AUDIT.logWriteSuccess(AuditAction type, Map<String, String> data, Level
- * level)
+ * AUDIT.logWriteSuccess(Level level, AuditMessage msg)
  *
  * If logging is done without specifying Level, then Level implicitly
- * defaults to INFO
- * AUDIT.logWriteSuccess(AuditAction type, Map<String, String> data)
+ * defaults to INFO for xxxxSuccess() and ERROR for xxxxFailure()
+ * AUDIT.logWriteSuccess(AuditMessage msg)
+ * AUDIT.logWriteFailure(AuditMessage msg)
  *
  * See sample invocations in src/test in the following class:
  * org.apache.hadoop.ozone.audit.TestOzoneAuditLogger

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac515d22/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/TestOzoneAuditLogger.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/TestOzoneAuditLogger.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/TestOzoneAuditLogger.java
index 57a7d9e..6c59de6 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/TestOzoneAuditLogger.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/TestOzoneAuditLogger.java
@@ -28,6 +28,7 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.io.IOException;
 import java.util.List;
+import java.util.Map;
 
 import static org.junit.Assert.assertTrue;
 
@@ -36,10 +37,29 @@ import static org.junit.Assert.assertTrue;
  */
 public class TestOzoneAuditLogger {
 
-  private static final Logger LOG = LoggerFactory.getLogger
-      (TestOzoneAuditLogger.class.getName());
-  private static AuditLogger AUDIT = new AuditLogger(AuditLoggerType.OMLOGGER);
-  public DummyEntity auditableObj = new DummyEntity();
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOzoneAuditLogger.class.getName());
+
+  private static final AuditLogger AUDIT =
+      new AuditLogger(AuditLoggerType.OMLOGGER);
+
+  private static final String SUCCESS = AuditEventStatus.SUCCESS.name();
+  private static final String FAILURE = AuditEventStatus.FAILURE.name();
+
+  private static final Map<String, String> PARAMS =
+      new DummyEntity().toAuditMap();
+
+  private static final AuditMessage WRITE_FAIL_MSG = new AuditMessage("john",
+      "192.168.0.1", DummyAction.CREATE_VOLUME.name(), PARAMS, FAILURE);
+
+  private static final AuditMessage WRITE_SUCCESS_MSG = new AuditMessage("john",
+      "192.168.0.1", DummyAction.CREATE_VOLUME.name(), PARAMS, SUCCESS);
+
+  private static final AuditMessage READ_FAIL_MSG = new AuditMessage("john",
+      "192.168.0.1", DummyAction.READ_VOLUME.name(), PARAMS, FAILURE);
+
+  private static final AuditMessage READ_SUCCESS_MSG = new AuditMessage("john",
+      "192.168.0.1", DummyAction.READ_VOLUME.name(), PARAMS, SUCCESS);
 
   @BeforeClass
   public static void setUp(){
@@ -48,13 +68,13 @@ public class TestOzoneAuditLogger {
 
   @AfterClass
   public static void tearDown() {
-      File file = new File("audit.log");
-      if (FileUtils.deleteQuietly(file)) {
-        LOG.info(file.getName() +
-            " has been deleted as all tests have completed.");
-      } else {
-        LOG.info("audit.log could not be deleted.");
-      }
+    File file = new File("audit.log");
+    if (FileUtils.deleteQuietly(file)) {
+      LOG.info(file.getName() +
+          " has been deleted as all tests have completed.");
+    } else {
+      LOG.info("audit.log could not be deleted.");
+    }
   }
 
   /**
@@ -62,20 +82,31 @@ public class TestOzoneAuditLogger {
    */
   @Test
   public void logInfoWriteSuccess() throws IOException {
-    AUDIT.logWriteSuccess(DummyAction.CREATE_VOLUME, auditableObj.toAuditMap(), Level.INFO);
-    String expected = "[INFO ] OMAudit - CREATE_VOLUME [ key1=\"value1\" " +
-        "key2=\"value2\"] SUCCESS";
+    AUDIT.logWriteSuccess(Level.INFO, WRITE_SUCCESS_MSG);
+    String expected =
+        "[INFO ] OMAudit - " + WRITE_SUCCESS_MSG.getFormattedMessage();
+    verifyLog(expected);
+  }
+
+  /**
+   * Test to verify default log level is INFO when logging success events.
+   */
+  @Test
+  public void verifyDefaultLogLevelForSuccess() throws IOException {
+    AUDIT.logWriteSuccess(WRITE_SUCCESS_MSG);
+    String expected =
+        "[INFO ] OMAudit - " + WRITE_SUCCESS_MSG.getFormattedMessage();
     verifyLog(expected);
   }
 
   /**
-   * Test to verify default log level is INFO
+   * Test to verify default log level is ERROR when logging failure events.
    */
   @Test
-  public void verifyDefaultLogLevel() throws IOException {
-    AUDIT.logWriteSuccess(DummyAction.CREATE_VOLUME, auditableObj.toAuditMap());
-    String expected = "[INFO ] OMAudit - CREATE_VOLUME [ key1=\"value1\" " +
-        "key2=\"value2\"] SUCCESS";
+  public void verifyDefaultLogLevelForFailure() throws IOException {
+    AUDIT.logWriteFailure(WRITE_FAIL_MSG);
+    String expected =
+        "[ERROR] OMAudit - " + WRITE_FAIL_MSG.getFormattedMessage();
     verifyLog(expected);
   }
 
@@ -84,9 +115,9 @@ public class TestOzoneAuditLogger {
    */
   @Test
   public void logErrorWriteFailure() throws IOException {
-    AUDIT.logWriteFailure(DummyAction.CREATE_VOLUME, auditableObj.toAuditMap(), Level.ERROR);
-    String expected = "[ERROR] OMAudit - CREATE_VOLUME [ key1=\"value1\" " +
-        "key2=\"value2\"] FAILURE";
+    AUDIT.logWriteFailure(Level.ERROR, WRITE_FAIL_MSG);
+    String expected =
+        "[ERROR] OMAudit - " + WRITE_FAIL_MSG.getFormattedMessage();
     verifyLog(expected);
   }
 
@@ -95,11 +126,10 @@ public class TestOzoneAuditLogger {
    */
   @Test
   public void notLogReadEvents() throws IOException {
-    AUDIT.logReadSuccess(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.INFO);
-    AUDIT.logReadFailure(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.INFO);
-    AUDIT.logReadFailure(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.ERROR);
-    AUDIT.logReadFailure(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.ERROR,
-        new Exception("test"));
+    AUDIT.logReadSuccess(Level.INFO, READ_SUCCESS_MSG);
+    AUDIT.logReadFailure(Level.INFO, READ_FAIL_MSG);
+    AUDIT.logReadFailure(Level.ERROR, READ_FAIL_MSG);
+    AUDIT.logReadFailure(Level.ERROR, READ_FAIL_MSG, new Exception("test"));
     verifyNoLog();
   }
 
@@ -108,34 +138,34 @@ public class TestOzoneAuditLogger {
    */
   @Test
   public void notLogDebugEvents() throws IOException {
-    AUDIT.logWriteSuccess(DummyAction.CREATE_VOLUME, auditableObj.toAuditMap(), Level.DEBUG);
-    AUDIT.logReadSuccess(DummyAction.READ_VOLUME, auditableObj.toAuditMap(), Level.DEBUG);
+    AUDIT.logWriteSuccess(Level.DEBUG, WRITE_SUCCESS_MSG);
+    AUDIT.logReadSuccess(Level.DEBUG, READ_SUCCESS_MSG);
     verifyNoLog();
   }
 
   private void verifyLog(String expected) throws IOException {
     File file = new File("audit.log");
     List<String> lines = FileUtils.readLines(file, (String)null);
-      final int retry = 5;
-      int i = 0;
-      while (lines.isEmpty() && i < retry) {
-        lines = FileUtils.readLines(file, (String)null);
-        try {
-          Thread.sleep( 500 * (i + 1));
-        } catch(InterruptedException ie) {
-          Thread.currentThread().interrupt();
-          break;
-        }
-        i++;
+    final int retry = 5;
+    int i = 0;
+    while (lines.isEmpty() && i < retry) {
+      lines = FileUtils.readLines(file, (String)null);
+      try {
+        Thread.sleep(500 * (i + 1));
+      } catch(InterruptedException ie) {
+        Thread.currentThread().interrupt();
+        break;
       }
+      i++;
+    }
 
-      // When log entry is expected, the log file will contain one line and
-      // that must be equal to the expected string
-      assertTrue(lines.size() != 0);
-      assertTrue(expected.equalsIgnoreCase(lines.get(0)));
-      //empty the file
-      lines.remove(0);
-      FileUtils.writeLines(file, lines, false);
+    // When log entry is expected, the log file will contain one line and
+    // that must be equal to the expected string
+    assertTrue(lines.size() != 0);
+    assertTrue(expected.equalsIgnoreCase(lines.get(0)));
+    //empty the file
+    lines.remove(0);
+    FileUtils.writeLines(file, lines, false);
   }
 
   private void verifyNoLog() throws IOException {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[26/50] [abbrv] hadoop git commit: HDDS-280. Support ozone dist-start-stitching on openbsd/osx. Contributed by Elek, Marton.

Posted by eh...@apache.org.
HDDS-280. Support ozone dist-start-stitching on openbsd/osx. Contributed by Elek, Marton.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/692736f7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/692736f7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/692736f7

Branch: refs/heads/HDFS-12090
Commit: 692736f7cfb72b8932dc2eb4f4faa995dc6521f8
Parents: 7362516
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Thu Aug 30 02:21:24 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Thu Aug 30 02:21:24 2018 +0530

----------------------------------------------------------------------
 dev-support/bin/ozone-dist-layout-stitching               |  6 +++---
 dev-support/bin/ozone-dist-tar-stitching                  |  9 ++-------
 hadoop-ozone/acceptance-test/dev-support/bin/robot-all.sh |  2 +-
 .../acceptance-test/dev-support/bin/robot-dnd-all.sh      | 10 ++--------
 hadoop-ozone/acceptance-test/dev-support/bin/robot.sh     |  7 ++++---
 hadoop-ozone/acceptance-test/pom.xml                      |  7 +++----
 .../src/test/acceptance/basic/ozone-shell.robot           |  1 -
 .../acceptance-test/src/test/acceptance/commonlib.robot   |  2 +-
 hadoop-ozone/common/pom.xml                               |  5 +----
 hadoop-ozone/docs/content/GettingStarted.md               |  3 ++-
 hadoop-ozone/pom.xml                                      |  5 +++++
 11 files changed, 24 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/692736f7/dev-support/bin/ozone-dist-layout-stitching
----------------------------------------------------------------------
diff --git a/dev-support/bin/ozone-dist-layout-stitching b/dev-support/bin/ozone-dist-layout-stitching
index 2ba7791..1ba652c 100755
--- a/dev-support/bin/ozone-dist-layout-stitching
+++ b/dev-support/bin/ozone-dist-layout-stitching
@@ -117,9 +117,9 @@ ROOT=$(cd "${BASEDIR}"/../..;pwd)
 echo
 echo "Current directory $(pwd)"
 echo
-run rm -rf "ozone"
-run mkdir "ozone"
-run cd "ozone"
+run rm -rf "ozone-${HDDS_VERSION}"
+run mkdir "ozone-${HDDS_VERSION}"
+run cd "ozone-${HDDS_VERSION}"
 run cp -p "${ROOT}/LICENSE.txt" .
 run cp -p "${ROOT}/NOTICE.txt" .
 run cp -p "${ROOT}/README.txt" .

http://git-wip-us.apache.org/repos/asf/hadoop/blob/692736f7/dev-support/bin/ozone-dist-tar-stitching
----------------------------------------------------------------------
diff --git a/dev-support/bin/ozone-dist-tar-stitching b/dev-support/bin/ozone-dist-tar-stitching
index d1116e4..93d0525 100755
--- a/dev-support/bin/ozone-dist-tar-stitching
+++ b/dev-support/bin/ozone-dist-tar-stitching
@@ -36,13 +36,8 @@ function run()
   fi
 }
 
-#To make the final dist directory easily mountable from docker we don't use
-#version name in the directory name.
-#To include the version name in the root directory of the tar file
-# we create a symbolic link and dereference it during the tar creation
-ln -s -f ozone ozone-${VERSION}
-run tar -c --dereference -f "ozone-${VERSION}.tar" "ozone-${VERSION}"
+run tar -c -f "ozone-${VERSION}.tar" "ozone-${VERSION}"
 run gzip -f "ozone-${VERSION}.tar"
 echo
 echo "Ozone dist tar available at: ${BASEDIR}/ozone-${VERSION}.tar.gz"
-echo
\ No newline at end of file
+echo

http://git-wip-us.apache.org/repos/asf/hadoop/blob/692736f7/hadoop-ozone/acceptance-test/dev-support/bin/robot-all.sh
----------------------------------------------------------------------
diff --git a/hadoop-ozone/acceptance-test/dev-support/bin/robot-all.sh b/hadoop-ozone/acceptance-test/dev-support/bin/robot-all.sh
index ee9c6b8..87b7137 100755
--- a/hadoop-ozone/acceptance-test/dev-support/bin/robot-all.sh
+++ b/hadoop-ozone/acceptance-test/dev-support/bin/robot-all.sh
@@ -15,4 +15,4 @@
 # limitations under the License.
 
 DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
-$DIR/robot.sh $DIR/../../src/test/acceptance
+"$DIR/robot.sh" "$DIR/../../src/test/acceptance"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/692736f7/hadoop-ozone/acceptance-test/dev-support/bin/robot-dnd-all.sh
----------------------------------------------------------------------
diff --git a/hadoop-ozone/acceptance-test/dev-support/bin/robot-dnd-all.sh b/hadoop-ozone/acceptance-test/dev-support/bin/robot-dnd-all.sh
index 9f1d367..052ffb3 100755
--- a/hadoop-ozone/acceptance-test/dev-support/bin/robot-dnd-all.sh
+++ b/hadoop-ozone/acceptance-test/dev-support/bin/robot-dnd-all.sh
@@ -18,15 +18,9 @@ set -x
 
 DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
 
-#Dir od the definition of the dind based test exeucution container
-DOCKERDIR="$DIR/../docker"
-
 #Dir to save the results
 TARGETDIR="$DIR/../../target/dnd"
 
-#Dir to mount the distribution from
-OZONEDIST="$DIR/../../../../hadoop-dist/target/ozone"
-
 #Name and imagename of the temporary, dind based test containers
 DOCKER_IMAGE_NAME=ozoneacceptance
 DOCKER_INSTANCE_NAME="${DOCKER_INSTANCE_NAME:-ozoneacceptance}"
@@ -47,10 +41,10 @@ chmod o+w "$OZONEDIST" || true
 
 rm "$TARGETDIR/docker-compose.log"
 docker rm "$DOCKER_INSTANCE_NAME" || true
-docker build -t "$DOCKER_IMAGE_NAME" $DIR/../docker
+docker build -t "$DOCKER_IMAGE_NAME" "$DIR/../docker"
 
 #Starting the dind based environment
-docker run --rm -v $DIR/../../../..:/opt/hadoop --privileged -d --name "$DOCKER_INSTANCE_NAME" $DOCKER_IMAGE_NAME
+docker run --rm -v "$DIR/../../../..:/opt/hadoop" --privileged -d --name "$DOCKER_INSTANCE_NAME" $DOCKER_IMAGE_NAME
 sleep 5
 
 #Starting the tests

http://git-wip-us.apache.org/repos/asf/hadoop/blob/692736f7/hadoop-ozone/acceptance-test/dev-support/bin/robot.sh
----------------------------------------------------------------------
diff --git a/hadoop-ozone/acceptance-test/dev-support/bin/robot.sh b/hadoop-ozone/acceptance-test/dev-support/bin/robot.sh
index ef2a111..5b85089 100755
--- a/hadoop-ozone/acceptance-test/dev-support/bin/robot.sh
+++ b/hadoop-ozone/acceptance-test/dev-support/bin/robot.sh
@@ -18,7 +18,7 @@ set -x
 
 DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
 
-if [ ! "$(which robot)" ] ; then
+if [ ! "$(command -v robot)" ] ; then
     echo ""
     echo "robot is not on your PATH."
     echo ""
@@ -28,10 +28,11 @@ if [ ! "$(which robot)" ] ; then
     exit -1
 fi
 
-OZONEDISTDIR="$DIR/../../../../hadoop-dist/target/ozone"
+MARKERFILE=$(find "$DIR/../../../../hadoop-dist/target" -name hadoop-ozone.sh)
+OZONEDISTDIR="$(dirname "$(dirname "$(dirname "$MARKERFILE")")")"
 if [ ! -d "$OZONEDISTDIR" ]; then
    echo "Ozone can't be found in the $OZONEDISTDIR."
    echo "You may need a full build with -Phdds and -Pdist profiles"
    exit -1
 fi
-robot -x junit-results.xml "$@"
+robot --variable "OZONEDIR:$OZONEDISTDIR" -x junit-results.xml "$@"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/692736f7/hadoop-ozone/acceptance-test/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-ozone/acceptance-test/pom.xml b/hadoop-ozone/acceptance-test/pom.xml
index fee41f1..59d34ef 100644
--- a/hadoop-ozone/acceptance-test/pom.xml
+++ b/hadoop-ozone/acceptance-test/pom.xml
@@ -19,9 +19,8 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <parent>
     <groupId>org.apache.hadoop</groupId>
-    <artifactId>hadoop-project</artifactId>
-    <version>3.2.0-SNAPSHOT</version>
-    <relativePath>../../hadoop-project</relativePath>
+    <artifactId>hadoop-ozone</artifactId>
+    <version>0.2.1-SNAPSHOT</version>
   </parent>
   <artifactId>hadoop-ozone-acceptance-test</artifactId>
   <version>3.2.0-SNAPSHOT</version>
@@ -45,7 +44,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                 <configuration>
                   <testCasesDirectory>src/test/acceptance</testCasesDirectory>
                   <variables>
-                    <variable>basedir:${project.basedir}/../..</variable>
+                    <variable>OZONEDIR:${project.basedir}/../../hadoop-dist/target/ozone-${ozone.version}</variable>
                   </variables>
                   <skip>false</skip>
                   <skipTests>false</skipTests>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/692736f7/hadoop-ozone/acceptance-test/src/test/acceptance/basic/ozone-shell.robot
----------------------------------------------------------------------
diff --git a/hadoop-ozone/acceptance-test/src/test/acceptance/basic/ozone-shell.robot b/hadoop-ozone/acceptance-test/src/test/acceptance/basic/ozone-shell.robot
index cc4b035..4751841 100644
--- a/hadoop-ozone/acceptance-test/src/test/acceptance/basic/ozone-shell.robot
+++ b/hadoop-ozone/acceptance-test/src/test/acceptance/basic/ozone-shell.robot
@@ -22,7 +22,6 @@ Resource            ../commonlib.robot
 Test Timeout        2 minute
 
 *** Variables ***
-${basedir}
 ${COMPOSEFILE}          ${CURDIR}/docker-compose.yaml
 ${PROJECTDIR}           ${CURDIR}/../../../../../..
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/692736f7/hadoop-ozone/acceptance-test/src/test/acceptance/commonlib.robot
----------------------------------------------------------------------
diff --git a/hadoop-ozone/acceptance-test/src/test/acceptance/commonlib.robot b/hadoop-ozone/acceptance-test/src/test/acceptance/commonlib.robot
index 9235cd9..e1c55d5 100644
--- a/hadoop-ozone/acceptance-test/src/test/acceptance/commonlib.robot
+++ b/hadoop-ozone/acceptance-test/src/test/acceptance/commonlib.robot
@@ -71,7 +71,7 @@ Execute on
 Run docker compose
     [arguments]                     ${command}
                                     Set Environment Variable    COMPOSE_INTERACTIVE_NO_CLI             1
-                                    Set Environment Variable    OZONEDIR      ${PROJECTDIR}/hadoop-dist/target/ozone
+                                    Set Environment Variable    OZONEDIR      ${OZONEDIR}
     ${rc}                           ${output} =                 Run And Return Rc And Output           docker-compose -f ${COMPOSEFILE} ${command}
     Log                             ${output}
     Should Be Equal As Integers     ${rc}                       0

http://git-wip-us.apache.org/repos/asf/hadoop/blob/692736f7/hadoop-ozone/common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/pom.xml b/hadoop-ozone/common/pom.xml
index ea5eb46..75600ba 100644
--- a/hadoop-ozone/common/pom.xml
+++ b/hadoop-ozone/common/pom.xml
@@ -29,11 +29,8 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <packaging>jar</packaging>
 
   <properties>
-    <ozone.version>0.2.1-SNAPSHOT</ozone.version>
-    <ozone.release>Acadia</ozone.release>
     <hadoop.component>ozone</hadoop.component>
     <is.hadoop.component>true</is.hadoop.component>
-    <declared.ozone.version>${ozone.version}</declared.ozone.version>
   </properties>
 
   <dependencies>
@@ -120,4 +117,4 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       </plugin>
     </plugins>
   </build>
-</project>
\ No newline at end of file
+</project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/692736f7/hadoop-ozone/docs/content/GettingStarted.md
----------------------------------------------------------------------
diff --git a/hadoop-ozone/docs/content/GettingStarted.md b/hadoop-ozone/docs/content/GettingStarted.md
index 4a57ada..9b9d49f 100644
--- a/hadoop-ozone/docs/content/GettingStarted.md
+++ b/hadoop-ozone/docs/content/GettingStarted.md
@@ -80,8 +80,9 @@ on the machine.
 Go to the directory where the docker compose files exist and tell
 `docker-compose` to start Ozone. This will start SCM, OM and a single datanode
 in the background.
+
 ```
-cd hadoop-dist/target/ozone/compose/ozone
+cd hadoop-dist/target/ozone-*/compose/ozone
 
 docker-compose up -d
 ```

http://git-wip-us.apache.org/repos/asf/hadoop/blob/692736f7/hadoop-ozone/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-ozone/pom.xml b/hadoop-ozone/pom.xml
index 20ee71f..cc0f030 100644
--- a/hadoop-ozone/pom.xml
+++ b/hadoop-ozone/pom.xml
@@ -29,6 +29,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <name>Apache Hadoop Ozone</name>
   <packaging>pom</packaging>
 
+  <properties>
+    <ozone.version>0.2.1-SNAPSHOT</ozone.version>
+    <ozone.release>Acadia</ozone.release>
+    <declared.ozone.version>${ozone.version}</declared.ozone.version>
+  </properties>
   <modules>
     <module>common</module>
     <module>client</module>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[25/50] [abbrv] hadoop git commit: YARN-8642. Add support for tmpfs mounts with the Docker runtime. Contributed by Craig Condit

Posted by eh...@apache.org.
YARN-8642. Add support for tmpfs mounts with the Docker runtime. Contributed by Craig Condit


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/73625168
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/73625168
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/73625168

Branch: refs/heads/HDFS-12090
Commit: 73625168c0f29aa646d7a715c9fb15e43d6c7e05
Parents: a0ebb6b
Author: Shane Kumpf <sk...@apache.org>
Authored: Wed Aug 29 07:08:37 2018 -0600
Committer: Shane Kumpf <sk...@apache.org>
Committed: Wed Aug 29 07:08:37 2018 -0600

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   5 +
 .../src/main/resources/yarn-default.xml         |   7 +
 .../runtime/DockerLinuxContainerRuntime.java    |  38 +++++
 .../linux/runtime/docker/DockerRunCommand.java  |   5 +
 .../container-executor/impl/utils/docker-util.c |  42 ++++++
 .../container-executor/impl/utils/docker-util.h |   3 +-
 .../test/utils/test_docker_util.cc              |  64 ++++++++
 .../runtime/TestDockerContainerRuntime.java     | 149 +++++++++++++++++++
 .../runtime/docker/TestDockerRunCommand.java    |   5 +-
 .../src/site/markdown/DockerContainers.md       |   1 +
 10 files changed, 317 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/73625168/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 148edb9..d525e4d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2012,6 +2012,11 @@ public class YarnConfiguration extends Configuration {
   public static final String NM_DOCKER_DEFAULT_RW_MOUNTS =
       DOCKER_CONTAINER_RUNTIME_PREFIX + "default-rw-mounts";
 
+  /** The default list of tmpfs mounts to be mounted into all
+   *  Docker containers that use DockerContainerRuntime. */
+  public static final String NM_DOCKER_DEFAULT_TMPFS_MOUNTS =
+      DOCKER_CONTAINER_RUNTIME_PREFIX + "default-tmpfs-mounts";
+
   /** The mode in which the Java Container Sandbox should run detailed by
    *  the JavaSandboxLinuxContainerRuntime. */
   public static final String YARN_CONTAINER_SANDBOX =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73625168/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 72e42d8..4262436 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -1828,6 +1828,13 @@
   </property>
 
   <property>
+    <description>The default list of tmpfs mounts to be mounted into all Docker
+      containers that use DockerContainerRuntime.</description>
+    <name>yarn.nodemanager.runtime.linux.docker.default-tmpfs-mounts</name>
+    <value></value>
+  </property>
+
+  <property>
     <description>The mode in which the Java Container Sandbox should run detailed by
       the JavaSandboxLinuxContainerRuntime.</description>
     <name>yarn.nodemanager.runtime.linux.sandbox-mode</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73625168/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
index 00771ff..0ae3d0f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/DockerLinuxContainerRuntime.java
@@ -165,6 +165,11 @@ import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.r
  *     {@code docker.allowed.ro-mounts} and {@code docker.allowed.rw-mounts}.
  *   </li>
  *   <li>
+ *     {@code YARN_CONTAINER_RUNTIME_DOCKER_TMPFS_MOUNTS} allows users to
+ *     specify additional tmpfs mounts for the Docker container. The value of
+ *     the environment variable should be a comma-separated list of mounts.
+ *   </li>
+ *   <li>
  *     {@code YARN_CONTAINER_RUNTIME_DOCKER_DELAYED_REMOVAL} allows a user
  *     to request delayed deletion of the Docker containers on a per
  *     container basis. If true, Docker containers will not be removed until
@@ -195,6 +200,8 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
   private static final Pattern USER_MOUNT_PATTERN = Pattern.compile(
       "(?<=^|,)([^:\\x00]+):([^:\\x00]+)" +
           "(:(r[ow]|(r[ow][+])?(r?shared|r?slave|r?private)))?(?:,|$)");
+  private static final Pattern TMPFS_MOUNT_PATTERN = Pattern.compile(
+      "^/[^:\\x00]+$");
   private static final int HOST_NAME_LENGTH = 64;
   private static final String DEFAULT_PROCFS = "/proc";
 
@@ -220,6 +227,9 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
   public static final String ENV_DOCKER_CONTAINER_MOUNTS =
       "YARN_CONTAINER_RUNTIME_DOCKER_MOUNTS";
   @InterfaceAudience.Private
+  public static final String ENV_DOCKER_CONTAINER_TMPFS_MOUNTS =
+      "YARN_CONTAINER_RUNTIME_DOCKER_TMPFS_MOUNTS";
+  @InterfaceAudience.Private
   public static final String ENV_DOCKER_CONTAINER_DELAYED_REMOVAL =
       "YARN_CONTAINER_RUNTIME_DOCKER_DELAYED_REMOVAL";
   private Configuration conf;
@@ -238,6 +248,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
   private int dockerStopGracePeriod;
   private Set<String> defaultROMounts = new HashSet<>();
   private Set<String> defaultRWMounts = new HashSet<>();
+  private Set<String> defaultTmpfsMounts = new HashSet<>();
 
   /**
    * Return whether the given environment variables indicate that the operation
@@ -302,6 +313,7 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
     allowedNetworks.clear();
     defaultROMounts.clear();
     defaultRWMounts.clear();
+    defaultTmpfsMounts.clear();
     allowedNetworks.addAll(Arrays.asList(
         conf.getTrimmedStrings(
             YarnConfiguration.NM_DOCKER_ALLOWED_CONTAINER_NETWORKS,
@@ -355,6 +367,10 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
     defaultRWMounts.addAll(Arrays.asList(
         conf.getTrimmedStrings(
         YarnConfiguration.NM_DOCKER_DEFAULT_RW_MOUNTS)));
+
+    defaultTmpfsMounts.addAll(Arrays.asList(
+        conf.getTrimmedStrings(
+        YarnConfiguration.NM_DOCKER_DEFAULT_TMPFS_MOUNTS)));
   }
 
   private Set<String> getDockerCapabilitiesFromConf() throws
@@ -907,6 +923,28 @@ public class DockerLinuxContainerRuntime implements LinuxContainerRuntime {
       }
     }
 
+    if (environment.containsKey(ENV_DOCKER_CONTAINER_TMPFS_MOUNTS)) {
+      String[] tmpfsMounts = environment.get(ENV_DOCKER_CONTAINER_TMPFS_MOUNTS)
+          .split(",");
+      for (String mount : tmpfsMounts) {
+        if (!TMPFS_MOUNT_PATTERN.matcher(mount).matches()) {
+          throw new ContainerExecutionException("Invalid tmpfs mount : " +
+              mount);
+        }
+        runCommand.addTmpfsMount(mount);
+      }
+    }
+
+    if (defaultTmpfsMounts != null && !defaultTmpfsMounts.isEmpty()) {
+      for (String mount : defaultTmpfsMounts) {
+        if (!TMPFS_MOUNT_PATTERN.matcher(mount).matches()) {
+          throw new ContainerExecutionException("Invalid tmpfs mount : " +
+              mount);
+        }
+        runCommand.addTmpfsMount(mount);
+      }
+    }
+
     if (allowHostPidNamespace(container)) {
       runCommand.setPidNamespace("host");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73625168/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.java
index 48ada5a..395c1e1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/DockerRunCommand.java
@@ -104,6 +104,11 @@ public class DockerRunCommand extends DockerCommand {
     return this;
   }
 
+  public DockerRunCommand addTmpfsMount(String mount) {
+    super.addCommandArguments("tmpfs", mount);
+    return this;
+  }
+
   public DockerRunCommand setVolumeDriver(String volumeDriver) {
     super.addCommandArguments("volume-driver", volumeDriver);
     return this;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73625168/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c
index 899c46a..3884abf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c
@@ -155,6 +155,12 @@ static int is_regex(const char *str) {
   return (strncmp(str, "regex:", 6) == 0);
 }
 
+static int is_valid_tmpfs_mount(const char *mount) {
+  const char *regex_str = "^/[^:]+$";
+  // execute_regex_match return 0 is matched success
+  return execute_regex_match(regex_str, mount) == 0;
+}
+
 static int is_volume_name(const char *volume_name) {
   const char *regex_str = "^[a-zA-Z0-9]([a-zA-Z0-9_.-]*)$";
   // execute_regex_match return 0 is matched success
@@ -334,6 +340,8 @@ const char *get_docker_error_message(const int error_code) {
       return "Invalid pid namespace";
     case INVALID_DOCKER_IMAGE_TRUST:
       return "Docker image is not trusted";
+    case INVALID_DOCKER_TMPFS_MOUNT:
+      return "Invalid docker tmpfs mount";
     default:
       return "Unknown error";
   }
@@ -1129,6 +1137,35 @@ static char* get_mount_type(const char *mount) {
   return mount_type;
 }
 
+static int add_tmpfs_mounts(const struct configuration *command_config, args *args) {
+  char **values = get_configuration_values_delimiter("tmpfs", DOCKER_COMMAND_FILE_SECTION, command_config, ",");
+  int i = 0, ret = 0;
+  if (values == NULL) {
+    goto free_and_exit;
+  }
+  for (i = 0; values[i] != NULL; i++) {
+    if (!is_valid_tmpfs_mount(values[i])) {
+      fprintf(ERRORFILE, "Invalid docker tmpfs mount '%s'\n", values[i]);
+      ret = INVALID_DOCKER_TMPFS_MOUNT;
+      goto free_and_exit;
+    }
+    ret = add_to_args(args, "--tmpfs");
+    if (ret != 0) {
+      ret = BUFFER_TOO_SMALL;
+      goto free_and_exit;
+    }
+    ret = add_to_args(args, values[i]);
+    if (ret != 0) {
+      ret = BUFFER_TOO_SMALL;
+      goto free_and_exit;
+    }
+  }
+
+free_and_exit:
+  free_values(values);
+  return ret;
+}
+
 static int add_mounts(const struct configuration *command_config, const struct configuration *conf, args *args) {
   const char *tmp_path_buffer[2] = {NULL, NULL};
   char *mount_src = NULL;
@@ -1469,6 +1506,11 @@ int get_docker_run_command(const char *command_file, const struct configuration
     goto free_and_exit;
   }
 
+  ret = add_tmpfs_mounts(&command_config, args);
+  if (ret != 0) {
+    goto free_and_exit;
+  }
+
   ret = set_cgroup_parent(&command_config, args);
   if (ret != 0) {
     goto free_and_exit;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73625168/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.h
index 864acd9..278dc53 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.h
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.h
@@ -66,7 +66,8 @@ enum docker_error_codes {
     INVALID_DOCKER_VOLUME_COMMAND,
     PID_HOST_DISABLED,
     INVALID_PID_NAMESPACE,
-    INVALID_DOCKER_IMAGE_TRUST
+    INVALID_DOCKER_IMAGE_TRUST,
+    INVALID_DOCKER_TMPFS_MOUNT
 };
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73625168/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test_docker_util.cc
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test_docker_util.cc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test_docker_util.cc
index 7e18146..b289857 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test_docker_util.cc
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test_docker_util.cc
@@ -1185,6 +1185,70 @@ namespace ContainerExecutor {
     free_configuration(&container_cfg);
   }
 
+  TEST_F(TestDockerUtil, test_add_tmpfs_mounts) {
+    struct configuration cmd_cfg;
+    struct args buff = ARGS_INITIAL_VALUE;
+    int ret = 0;
+    std::string container_executor_cfg_contents = "[docker]\n  docker.trusted.registries=hadoop\n";
+    std::vector<std::pair<std::string, std::string> > file_cmd_vec;
+    file_cmd_vec.push_back(std::make_pair<std::string, std::string>(
+        "[docker-command-execution]\n  docker-command=run\n image=hadoop/image\n  tmpfs=/run",
+        "--tmpfs /run"));
+    file_cmd_vec.push_back(std::make_pair<std::string, std::string>(
+        "[docker-command-execution]\n  docker-command=run\n  image=hadoop/image\n  tmpfs=/run,/run2",
+        "--tmpfs /run --tmpfs /run2"));
+    write_container_executor_cfg(container_executor_cfg_contents);
+
+    ret = create_ce_file();
+    if (ret != 0) {
+      std::cerr << "Could not create ce file, skipping test" << std::endl;
+      return;
+    }
+
+    std::vector<std::pair<std::string, std::string> >::const_iterator itr;
+    for (itr = file_cmd_vec.begin(); itr != file_cmd_vec.end(); ++itr) {
+      write_command_file(itr->first);
+      ret = read_config(docker_command_file.c_str(), &cmd_cfg);
+      if (ret != 0) {
+        FAIL();
+      }
+      ret = add_tmpfs_mounts(&cmd_cfg, &buff);
+      char *actual = flatten(&buff);
+      ASSERT_EQ(0, ret);
+      ASSERT_STREQ(itr->second.c_str(), actual);
+      reset_args(&buff);
+      free(actual);
+      free_configuration(&cmd_cfg);
+    }
+
+    std::vector<std::pair<std::string, int> > bad_file_cmds_vec;
+    bad_file_cmds_vec.push_back(std::make_pair<std::string, int>(
+        "[docker-command-execution]\n  docker-command=run\n  image=hadoop/image\n  tmpfs=run",
+        static_cast<int>(INVALID_DOCKER_TMPFS_MOUNT)));
+    bad_file_cmds_vec.push_back(std::make_pair<std::string, int>(
+        "[docker-command-execution]\n  docker-command=run\n  image=hadoop/image\n  tmpfs=/ru:n",
+        static_cast<int>(INVALID_DOCKER_TMPFS_MOUNT)));
+    bad_file_cmds_vec.push_back(std::make_pair<std::string, int>(
+        "[docker-command-execution]\n  docker-command=run\n  image=hadoop/image\n  tmpfs=/run:",
+        static_cast<int>(INVALID_DOCKER_TMPFS_MOUNT)));
+
+    std::vector<std::pair<std::string, int> >::const_iterator itr2;
+    for (itr2 = bad_file_cmds_vec.begin(); itr2 != bad_file_cmds_vec.end(); ++itr2) {
+      write_command_file(itr2->first);
+      ret = read_config(docker_command_file.c_str(), &cmd_cfg);
+      if (ret != 0) {
+        FAIL();
+      }
+      ret = add_tmpfs_mounts(&cmd_cfg, &buff);
+      char *actual = flatten(&buff);
+      ASSERT_EQ(itr2->second, ret);
+      ASSERT_STREQ("", actual);
+      reset_args(&buff);
+      free(actual);
+      free_configuration(&cmd_cfg);
+    }
+  }
+
   TEST_F(TestDockerUtil, test_docker_run_privileged) {
 
     std::string container_executor_contents = "[docker]\n  docker.allowed.ro-mounts=/var,/etc,/usr/bin/cut\n"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73625168/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
index e098778..2f17167 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/TestDockerContainerRuntime.java
@@ -86,6 +86,7 @@ import java.util.concurrent.ConcurrentMap;
 
 import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_DOCKER_DEFAULT_RO_MOUNTS;
 import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_DOCKER_DEFAULT_RW_MOUNTS;
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_DOCKER_DEFAULT_TMPFS_MOUNTS;
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_RUN_PRIVILEGED_CONTAINER;
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.APPID;
 import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.APPLICATION_LOCAL_DIRS;
@@ -1328,6 +1329,154 @@ public class TestDockerContainerRuntime {
   }
 
   @Test
+  public void testTmpfsMount()
+      throws ContainerExecutionException, PrivilegedOperationException,
+      IOException {
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+        mockExecutor, mockCGroupsHandler);
+    runtime.initialize(conf, nmContext);
+
+    env.put(
+        DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_TMPFS_MOUNTS,
+        "/run");
+
+    runtime.launchContainer(builder.build());
+    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
+    List<String> args = op.getArguments();
+    String dockerCommandFile = args.get(11);
+
+    List<String> dockerCommands = Files.readAllLines(
+        Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
+
+    Assert.assertTrue(dockerCommands.contains("  tmpfs=/run"));
+  }
+
+  @Test
+  public void testTmpfsMountMulti()
+      throws ContainerExecutionException, PrivilegedOperationException,
+      IOException {
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+        mockExecutor, mockCGroupsHandler);
+    runtime.initialize(conf, nmContext);
+
+    env.put(
+        DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_TMPFS_MOUNTS,
+        "/run,/tmp");
+
+    runtime.launchContainer(builder.build());
+    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
+    List<String> args = op.getArguments();
+    String dockerCommandFile = args.get(11);
+
+    List<String> dockerCommands = Files.readAllLines(
+        Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
+
+    Assert.assertTrue(dockerCommands.contains("  tmpfs=/run,/tmp"));
+  }
+
+  @Test
+  public void testDefaultTmpfsMounts()
+      throws ContainerExecutionException, PrivilegedOperationException,
+      IOException {
+    conf.setStrings(NM_DOCKER_DEFAULT_TMPFS_MOUNTS, "/run,/var/run");
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+        mockExecutor, mockCGroupsHandler);
+    runtime.initialize(conf, nmContext);
+
+    env.put(
+        DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_TMPFS_MOUNTS,
+        "/tmpfs");
+
+    runtime.launchContainer(builder.build());
+    PrivilegedOperation op = capturePrivilegedOperationAndVerifyArgs();
+    List<String> args = op.getArguments();
+    String dockerCommandFile = args.get(11);
+
+    List<String> dockerCommands = Files.readAllLines(
+        Paths.get(dockerCommandFile), Charset.forName("UTF-8"));
+
+    Assert.assertTrue(dockerCommands.contains("  tmpfs=/tmpfs,/run,/var/run"));
+  }
+
+  @Test
+  public void testDefaultTmpfsMountsInvalid()
+      throws ContainerExecutionException {
+    conf.setStrings(NM_DOCKER_DEFAULT_TMPFS_MOUNTS, "run,var/run");
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+        mockExecutor, mockCGroupsHandler);
+    runtime.initialize(conf, nmContext);
+
+    env.put(
+        DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_TMPFS_MOUNTS,
+        "/tmpfs");
+
+    try {
+      runtime.launchContainer(builder.build());
+      Assert.fail(
+          "Expected a launch container failure due to non-absolute path.");
+    } catch (ContainerExecutionException e) {
+      LOG.info("Caught expected exception : " + e);
+    }
+  }
+
+  @Test
+  public void testTmpfsRelativeInvalid() throws ContainerExecutionException {
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+        mockExecutor, mockCGroupsHandler);
+    runtime.initialize(conf, nmContext);
+
+    env.put(
+        DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_TMPFS_MOUNTS,
+        "run");
+
+    try {
+      runtime.launchContainer(builder.build());
+      Assert.fail(
+          "Expected a launch container failure due to non-absolute path.");
+    } catch (ContainerExecutionException e) {
+      LOG.info("Caught expected exception : " + e);
+    }
+  }
+
+  @Test
+  public void testTmpfsColonInvalid() throws ContainerExecutionException {
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+        mockExecutor, mockCGroupsHandler);
+    runtime.initialize(conf, nmContext);
+
+    env.put(
+        DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_TMPFS_MOUNTS,
+        "/run:");
+
+    try {
+      runtime.launchContainer(builder.build());
+      Assert.fail(
+          "Expected a launch container failure due to invalid character.");
+    } catch (ContainerExecutionException e) {
+      LOG.info("Caught expected exception : " + e);
+    }
+  }
+
+  @Test
+  public void testTmpfsNulInvalid() throws ContainerExecutionException {
+    DockerLinuxContainerRuntime runtime = new DockerLinuxContainerRuntime(
+            mockExecutor, mockCGroupsHandler);
+    runtime.initialize(conf, nmContext);
+
+    env.put(
+        DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_TMPFS_MOUNTS,
+        "/ru\0n");
+
+    try {
+      runtime.launchContainer(builder.build());
+      Assert.fail(
+          "Expected a launch container failure due to NUL in tmpfs mount.");
+    } catch (ContainerExecutionException e) {
+      LOG.info("Caught expected exception : " + e);
+    }
+  }
+
+  @Test
   public void testDefaultROMounts()
       throws ContainerExecutionException, PrivilegedOperationException,
       IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73625168/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/TestDockerRunCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/TestDockerRunCommand.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/TestDockerRunCommand.java
index 19b1544..8dc37d4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/TestDockerRunCommand.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/linux/runtime/docker/TestDockerRunCommand.java
@@ -58,6 +58,7 @@ public class TestDockerRunCommand {
     commands.add("launch_command");
     dockerRunCommand.setOverrideCommandWithArgs(commands);
     dockerRunCommand.removeContainerOnExit();
+    dockerRunCommand.addTmpfsMount("/run");
 
     assertEquals("run", StringUtils.join(",",
         dockerRunCommand.getDockerCommandWithArguments()
@@ -76,7 +77,9 @@ public class TestDockerRunCommand {
     assertEquals("launch_command", StringUtils.join(",",
         dockerRunCommand.getDockerCommandWithArguments()
             .get("launch-command")));
-    assertEquals(7, dockerRunCommand.getDockerCommandWithArguments().size());
+    assertEquals("/run", StringUtils.join(",",
+        dockerRunCommand.getDockerCommandWithArguments().get("tmpfs")));
+    assertEquals(8, dockerRunCommand.getDockerCommandWithArguments().size());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73625168/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md
index 447155c..a3ce93f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/DockerContainers.md
@@ -350,6 +350,7 @@ environment variables in the application's environment:
 | `YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_PID_NAMESPACE` | Controls which PID namespace will be used by the Docker container. By default, each Docker container has its own PID namespace. To share the namespace of the host, the yarn.nodemanager.runtime.linux.docker.host-pid-namespace.allowed property must be set to true. If the host PID namespace is allowed and this environment variable is set to host, the Docker container will share the host's PID namespace. No other value is allowed. |
 | `YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER` | Controls whether the Docker container is a privileged container. In order to use privileged containers, the yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed property must be set to true, and the application owner must appear in the value of the yarn.nodemanager.runtime.linux.docker.privileged-containers.acl property. If this environment variable is set to true, a privileged Docker container will be used if allowed. No other value is allowed, so the environment variable should be left unset rather than setting it to false. |
 | `YARN_CONTAINER_RUNTIME_DOCKER_MOUNTS` | Adds additional volume mounts to the Docker container. The value of the environment variable should be a comma-separated list of mounts. All such mounts must be given as `source:dest[:mode]` and the mode must be "ro" (read-only) or "rw" (read-write) to specify the type of access being requested. If neither is specified, read-write will be  assumed. The mode may include a bind propagation option. In that case, the mode should either be of the form `[option]`, `rw+[option]`, or `ro+[option]`. Valid bind propagation options are shared, rshared, slave, rslave, private, and rprivate. The requested mounts will be validated by container-executor based on the values set in container-executor.cfg for `docker.allowed.ro-mounts` and `docker.allowed.rw-mounts`. |
+| `YARN_CONTAINER_RUNTIME_DOCKER_TMPFS_MOUNTS` | Adds additional tmpfs mounts to the Docker container. The value of the environment variable should be a comma-separated list of absolute mount points within the container. |
 | `YARN_CONTAINER_RUNTIME_DOCKER_DELAYED_REMOVAL` | Allows a user to request delayed deletion of the Docker container on a per container basis. If true, Docker containers will not be removed until the duration defined by yarn.nodemanager.delete.debug-delay-sec has elapsed. Administrators can disable this feature through the yarn-site property yarn.nodemanager.runtime.linux.docker.delayed-removal.allowed. This feature is disabled by default. When this feature is disabled or set to false, the container will be removed as soon as it exits. |
 
 The first two are required. The remainder can be set as needed. While


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[29/50] [abbrv] hadoop git commit: HADOOP-15698. KMS log4j is not initialized properly at startup. Contributed by Kitti Nanasi.

Posted by eh...@apache.org.
HADOOP-15698. KMS log4j is not initialized properly at startup. Contributed by Kitti Nanasi.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/781437c2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/781437c2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/781437c2

Branch: refs/heads/HDFS-12090
Commit: 781437c219dc3422797a32dc7ba72cd4f5ee38e2
Parents: 582cb10
Author: Kitti Nanasi <kn...@cloudera.com>
Authored: Wed Aug 29 22:06:36 2018 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Wed Aug 29 22:07:49 2018 -0700

----------------------------------------------------------------------
 .../crypto/key/kms/server/KMSConfiguration.java | 31 ++++++++++++++++
 .../hadoop/crypto/key/kms/server/KMSWebApp.java | 38 +-------------------
 .../crypto/key/kms/server/KMSWebServer.java     |  1 +
 3 files changed, 33 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/781437c2/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java
index 18eec19..35ffb42 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.crypto.key.kms.server;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.log4j.PropertyConfigurator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -103,6 +104,8 @@ public class KMSConfiguration {
 
   public static final boolean KEY_AUTHORIZATION_ENABLE_DEFAULT = true;
 
+  private static final String LOG4J_PROPERTIES = "kms-log4j.properties";
+
   static {
     Configuration.addDefaultResource(KMS_DEFAULT_XML);
     Configuration.addDefaultResource(KMS_SITE_XML);
@@ -159,4 +162,32 @@ public class KMSConfiguration {
     }
     return newer;
   }
+
+  public static void initLogging() {
+    String confDir = System.getProperty(KMS_CONFIG_DIR);
+    if (confDir == null) {
+      throw new RuntimeException("System property '" +
+          KMSConfiguration.KMS_CONFIG_DIR + "' not defined");
+    }
+    if (System.getProperty("log4j.configuration") == null) {
+      System.setProperty("log4j.defaultInitOverride", "true");
+      boolean fromClasspath = true;
+      File log4jConf = new File(confDir, LOG4J_PROPERTIES).getAbsoluteFile();
+      if (log4jConf.exists()) {
+        PropertyConfigurator.configureAndWatch(log4jConf.getPath(), 1000);
+        fromClasspath = false;
+      } else {
+        ClassLoader cl = Thread.currentThread().getContextClassLoader();
+        URL log4jUrl = cl.getResource(LOG4J_PROPERTIES);
+        if (log4jUrl != null) {
+          PropertyConfigurator.configure(log4jUrl);
+        }
+      }
+      LOG.debug("KMS log starting");
+      if (fromClasspath) {
+        LOG.warn("Log4j configuration file '{}' not found", LOG4J_PROPERTIES);
+        LOG.warn("Logging with INFO level to standard output");
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/781437c2/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
index cb4bf7e..0640e25 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
@@ -17,10 +17,8 @@
  */
 package org.apache.hadoop.crypto.key.kms.server;
 
-import java.io.File;
 import java.io.IOException;
 import java.net.URI;
-import java.net.URL;
 
 import javax.servlet.ServletContextEvent;
 import javax.servlet.ServletContextListener;
@@ -37,14 +35,13 @@ import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.VersionInfo;
-import org.apache.log4j.PropertyConfigurator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @InterfaceAudience.Private
 public class KMSWebApp implements ServletContextListener {
 
-  private static final String LOG4J_PROPERTIES = "kms-log4j.properties";
+  private static final Logger LOG = LoggerFactory.getLogger(KMSWebApp.class);
 
   private static final String METRICS_PREFIX = "hadoop.kms.";
   private static final String ADMIN_CALLS_METER = METRICS_PREFIX +
@@ -66,7 +63,6 @@ public class KMSWebApp implements ServletContextListener {
   private static final String REENCRYPT_EEK_BATCH_METER = METRICS_PREFIX +
       "reencrypt_eek_batch.calls.meter";
 
-  private static Logger LOG;
   private static MetricRegistry metricRegistry;
 
   private JmxReporter jmxReporter;
@@ -84,42 +80,10 @@ public class KMSWebApp implements ServletContextListener {
   private static KMSAudit kmsAudit;
   private static KeyProviderCryptoExtension keyProviderCryptoExtension;
 
-  private void initLogging(String confDir) {
-    if (System.getProperty("log4j.configuration") == null) {
-      System.setProperty("log4j.defaultInitOverride", "true");
-      boolean fromClasspath = true;
-      File log4jConf = new File(confDir, LOG4J_PROPERTIES).getAbsoluteFile();
-      if (log4jConf.exists()) {
-        PropertyConfigurator.configureAndWatch(log4jConf.getPath(), 1000);
-        fromClasspath = false;
-      } else {
-        ClassLoader cl = Thread.currentThread().getContextClassLoader();
-        URL log4jUrl = cl.getResource(LOG4J_PROPERTIES);
-        if (log4jUrl != null) {
-          PropertyConfigurator.configure(log4jUrl);
-        }
-      }
-      LOG = LoggerFactory.getLogger(KMSWebApp.class);
-      LOG.debug("KMS log starting");
-      if (fromClasspath) {
-        LOG.warn("Log4j configuration file '{}' not found", LOG4J_PROPERTIES);
-        LOG.warn("Logging with INFO level to standard output");
-      }
-    } else {
-      LOG = LoggerFactory.getLogger(KMSWebApp.class);
-    }
-  }
-
   @Override
   public void contextInitialized(ServletContextEvent sce) {
     try {
-      String confDir = System.getProperty(KMSConfiguration.KMS_CONFIG_DIR);
-      if (confDir == null) {
-        throw new RuntimeException("System property '" +
-            KMSConfiguration.KMS_CONFIG_DIR + "' not defined");
-      }
       kmsConf = KMSConfiguration.getKMSConf();
-      initLogging(confDir);
       UserGroupInformation.setConfiguration(kmsConf);
       LOG.info("-------------------------------------------------------------");
       LOG.info("  Java runtime version : {}", System.getProperty(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/781437c2/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebServer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebServer.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebServer.java
index 19f7227..036231d 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebServer.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebServer.java
@@ -166,6 +166,7 @@ public class KMSWebServer {
   }
 
   public static void main(String[] args) throws Exception {
+    KMSConfiguration.initLogging();
     StringUtils.startupShutdownMessage(KMSWebServer.class, args, LOG);
     Configuration conf = new ConfigurationWithLogging(
         KMSConfiguration.getKMSConf());


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[40/50] [abbrv] hadoop git commit: HDDS-392. Incomplete description about auditMap#key in AuditLogging Framework. Contributed by Dinesh Chitlangia.

Posted by eh...@apache.org.
HDDS-392. Incomplete description about auditMap#key in AuditLogging Framework.
Contributed by  Dinesh Chitlangia.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/19abaacd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/19abaacd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/19abaacd

Branch: refs/heads/HDFS-12090
Commit: 19abaacdad84b03fc790341b4b5bcf1c4d41f1fb
Parents: 76bae4c
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Aug 31 22:24:30 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Fri Aug 31 22:24:30 2018 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/ozone/audit/package-info.java  | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/19abaacd/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java
index 48de3f7..9c00ef7 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/package-info.java
@@ -50,8 +50,10 @@ package org.apache.hadoop.ozone.audit;
  * The implementing class must override toAuditMap() to return an
  * instance of Map<Key, Value> where both Key and Value are String.
  *
- * Key: must not contain any spaces. If the key is multi word then use
- * camel case.
+ * Key: must contain printable US ASCII characters
+ * May not contain a space, =, ], or "
+ * If the key is multi word then use camel case.
+ *
  * Value: if it is a collection/array, then it must be converted to a comma
  * delimited string
  *


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[19/50] [abbrv] hadoop git commit: YARN-8697. LocalityMulticastAMRMProxyPolicy should fallback to random sub-cluster when cannot resolve resource. Contributed by Botong Huang.

Posted by eh...@apache.org.
YARN-8697. LocalityMulticastAMRMProxyPolicy should fallback to random sub-cluster when cannot resolve resource. Contributed by Botong Huang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7ed458b2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7ed458b2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7ed458b2

Branch: refs/heads/HDFS-12090
Commit: 7ed458b255e492fd5bc2ca36f216ff1b16054db7
Parents: 3e18b95
Author: Giovanni Matteo Fumarola <gi...@apache.org>
Authored: Tue Aug 28 16:01:35 2018 -0700
Committer: Giovanni Matteo Fumarola <gi...@apache.org>
Committed: Tue Aug 28 16:01:35 2018 -0700

----------------------------------------------------------------------
 .../LocalityMulticastAMRMProxyPolicy.java       | 105 +++++++++++++++----
 .../TestLocalityMulticastAMRMProxyPolicy.java   |  53 ++++++++--
 2 files changed, 125 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ed458b2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
index 1ccd61c..e5f26d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
@@ -21,8 +21,11 @@ package org.apache.hadoop.yarn.server.federation.policies.amrmproxy;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
@@ -123,6 +126,8 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
   public static final Logger LOG =
       LoggerFactory.getLogger(LocalityMulticastAMRMProxyPolicy.class);
 
+  private static Random rand = new Random();
+
   private Map<SubClusterId, Float> weights;
   private SubClusterResolver resolver;
 
@@ -275,26 +280,18 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
       }
 
       // Handle node/rack requests that the SubClusterResolver cannot map to
-      // any cluster. Defaulting to home subcluster.
+      // any cluster. Pick a random sub-cluster from active and enabled ones.
+      targetId = getSubClusterForUnResolvedRequest(bookkeeper,
+          rr.getAllocationRequestId());
       if (LOG.isDebugEnabled()) {
         LOG.debug("ERROR resolving sub-cluster for resourceName: "
-            + rr.getResourceName() + " we are falling back to homeSubCluster:"
-            + homeSubcluster);
+            + rr.getResourceName() + ", picked a random subcluster to forward:"
+            + targetId);
       }
-
-      // If home-subcluster is not active, ignore node/rack request
-      if (bookkeeper.isActiveAndEnabled(homeSubcluster)) {
-        if (targetIds != null && targetIds.size() > 0) {
-          bookkeeper.addRackRR(homeSubcluster, rr);
-        } else {
-          bookkeeper.addLocalizedNodeRR(homeSubcluster, rr);
-        }
+      if (targetIds != null && targetIds.size() > 0) {
+        bookkeeper.addRackRR(targetId, rr);
       } else {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("The homeSubCluster (" + homeSubcluster + ") we are "
-              + "defaulting to is not active, the ResourceRequest "
-              + "will be ignored.");
-        }
+        bookkeeper.addLocalizedNodeRR(targetId, rr);
       }
     }
 
@@ -314,6 +311,14 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
   }
 
   /**
+   * For unit test to override.
+   */
+  protected SubClusterId getSubClusterForUnResolvedRequest(
+      AllocationBookkeeper bookKeeper, long allocationId) {
+    return bookKeeper.getSubClusterForUnResolvedRequest(allocationId);
+  }
+
+  /**
    * It splits a list of non-localized resource requests among sub-clusters.
    */
   private void splitAnyRequests(List<ResourceRequest> originalResourceRequests,
@@ -512,10 +517,11 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
    * This helper class is used to book-keep the requests made to each
    * subcluster, and maintain useful statistics to split ANY requests.
    */
-  private final class AllocationBookkeeper {
+  protected final class AllocationBookkeeper {
 
     // the answer being accumulated
     private Map<SubClusterId, List<ResourceRequest>> answer = new TreeMap<>();
+    private Map<SubClusterId, Set<Long>> maskForRackDeletion = new HashMap<>();
 
     // stores how many containers we have allocated in each RM for localized
     // asks, used to correctly "spread" the corresponding ANY
@@ -523,6 +529,10 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
         new HashMap<>();
     private Map<Long, AtomicLong> totNumLocalizedContainers = new HashMap<>();
 
+    // Store the randomly selected subClusterId for unresolved resource requests
+    // keyed by requestId
+    private Map<Long, SubClusterId> unResolvedRequestLocation = new HashMap<>();
+
     private Set<SubClusterId> activeAndEnabledSC = new HashSet<>();
     private float totHeadroomMemory = 0;
     private int totHeadRoomEnabledRMs = 0;
@@ -538,6 +548,7 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
 
       // reset data structures
       answer.clear();
+      maskForRackDeletion.clear();
       countContainersPerRM.clear();
       totNumLocalizedContainers.clear();
       activeAndEnabledSC.clear();
@@ -628,16 +639,16 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
             .addAndGet(rr.getNumContainers());
       }
 
-      internalAddToAnswer(targetId, rr);
+      internalAddToAnswer(targetId, rr, false);
     }
 
     /**
      * Add a rack-local request to the final asnwer.
      */
-    public void addRackRR(SubClusterId targetId, ResourceRequest rr) {
+    private void addRackRR(SubClusterId targetId, ResourceRequest rr) {
       Preconditions
           .checkArgument(!ResourceRequest.isAnyLocation(rr.getResourceName()));
-      internalAddToAnswer(targetId, rr);
+      internalAddToAnswer(targetId, rr, true);
     }
 
     /**
@@ -646,11 +657,18 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
     private void addAnyRR(SubClusterId targetId, ResourceRequest rr) {
       Preconditions
           .checkArgument(ResourceRequest.isAnyLocation(rr.getResourceName()));
-      internalAddToAnswer(targetId, rr);
+      internalAddToAnswer(targetId, rr, false);
     }
 
     private void internalAddToAnswer(SubClusterId targetId,
-        ResourceRequest partialRR) {
+        ResourceRequest partialRR, boolean isRack) {
+      if (!isRack) {
+        if (!maskForRackDeletion.containsKey(targetId)) {
+          maskForRackDeletion.put(targetId, new HashSet<Long>());
+        }
+        maskForRackDeletion.get(targetId)
+            .add(partialRR.getAllocationRequestId());
+      }
       if (!answer.containsKey(targetId)) {
         answer.put(targetId, new ArrayList<ResourceRequest>());
       }
@@ -658,6 +676,27 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
     }
 
     /**
+     * For requests whose location cannot be resolved, choose an active and
+     * enabled sub-cluster to forward this requestId to.
+     */
+    private SubClusterId getSubClusterForUnResolvedRequest(long allocationId) {
+      if (unResolvedRequestLocation.containsKey(allocationId)) {
+        return unResolvedRequestLocation.get(allocationId);
+      }
+      int id = rand.nextInt(activeAndEnabledSC.size());
+      for (SubClusterId subclusterId : activeAndEnabledSC) {
+        if (id == 0) {
+          unResolvedRequestLocation.put(allocationId, subclusterId);
+          return subclusterId;
+        }
+        id--;
+      }
+      throw new RuntimeException(
+          "Should not be here. activeAndEnabledSC size = "
+              + activeAndEnabledSC.size() + " id = " + id);
+    }
+
+    /**
      * Return all known subclusters associated with an allocation id.
      *
      * @param allocationId the allocation id considered
@@ -678,6 +717,28 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
      * @return the answer
      */
     private Map<SubClusterId, List<ResourceRequest>> getAnswer() {
+      Iterator<Entry<SubClusterId, List<ResourceRequest>>> answerIter =
+          answer.entrySet().iterator();
+      // Remove redundant rack RR before returning the answer
+      while (answerIter.hasNext()) {
+        Entry<SubClusterId, List<ResourceRequest>> entry = answerIter.next();
+        SubClusterId scId = entry.getKey();
+        Set<Long> mask = maskForRackDeletion.get(scId);
+        if (mask != null) {
+          Iterator<ResourceRequest> rrIter = entry.getValue().iterator();
+          while (rrIter.hasNext()) {
+            ResourceRequest rr = rrIter.next();
+            if (!mask.contains(rr.getAllocationRequestId())) {
+              rrIter.remove();
+            }
+          }
+        }
+        if (mask == null || entry.getValue().size() == 0) {
+          answerIter.remove();
+          LOG.info("removing {} from output because it has only rack RR",
+              scId);
+        }
+      }
       return answer;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ed458b2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
index cf9ac53..c49ab60 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
@@ -69,12 +69,12 @@ public class TestLocalityMulticastAMRMProxyPolicy
 
   @Before
   public void setUp() throws Exception {
-    setPolicy(new LocalityMulticastAMRMProxyPolicy());
+    setPolicy(new TestableLocalityMulticastAMRMProxyPolicy());
     setPolicyInfo(new WeightedPolicyInfo());
     Map<SubClusterIdInfo, Float> routerWeights = new HashMap<>();
     Map<SubClusterIdInfo, Float> amrmWeights = new HashMap<>();
 
-    // simulate 20 subclusters with a 5% chance of being inactive
+    // Six sub-clusters with one inactive and one disabled
     for (int i = 0; i < 6; i++) {
       SubClusterIdInfo sc = new SubClusterIdInfo("subcluster" + i);
       // sub-cluster 3 is not active
@@ -207,6 +207,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
     getPolicyInfo().setHeadroomAlpha(1.0f);
 
     initializePolicy();
+    addHomeSubClusterAsActive();
 
     int numRR = 1000;
     List<ResourceRequest> resourceRequests = createLargeRandomList(numRR);
@@ -324,14 +325,11 @@ public class TestLocalityMulticastAMRMProxyPolicy
         null, Collections.<NMToken> emptyList());
   }
 
-  @Test
-  public void testSplitAllocateRequest() throws Exception {
-
-    // Test a complex List<ResourceRequest> is split correctly
-    initializePolicy();
-
-    // modify default initialization to include a "homesubcluster"
-    // which we will use as the default for when nodes or racks are unknown
+  /**
+   * modify default initialization to include a "homesubcluster" which we will
+   * use as the default for when nodes or racks are unknown.
+   */
+  private void addHomeSubClusterAsActive() {
     SubClusterInfo sci = mock(SubClusterInfo.class);
     when(sci.getState()).thenReturn(SubClusterState.SC_RUNNING);
     when(sci.getSubClusterId()).thenReturn(getHomeSubCluster());
@@ -340,6 +338,14 @@ public class TestLocalityMulticastAMRMProxyPolicy
 
     getPolicyInfo().getRouterPolicyWeights().put(sc, 0.1f);
     getPolicyInfo().getAMRMPolicyWeights().put(sc, 0.1f);
+  }
+
+  @Test
+  public void testSplitAllocateRequest() throws Exception {
+
+    // Test a complex List<ResourceRequest> is split correctly
+    initializePolicy();
+    addHomeSubClusterAsActive();
 
     FederationPoliciesTestUtil.initializePolicyContext(
         getFederationPolicyContext(), getPolicy(), getPolicyInfo(),
@@ -502,7 +508,8 @@ public class TestLocalityMulticastAMRMProxyPolicy
 
     // Test target Ids
     for (SubClusterId targetId : split.keySet()) {
-      Assert.assertTrue("Target subclusters should be in the active set",
+      Assert.assertTrue(
+          "Target subcluster " + targetId + " should be in the active set",
           getActiveSubclusters().containsKey(targetId));
       Assert.assertTrue(
           "Target subclusters (" + targetId + ") should have weight >0 in "
@@ -787,4 +794,28 @@ public class TestLocalityMulticastAMRMProxyPolicy
     checkTotalContainerAllocation(response, 100);
   }
 
+  /**
+   * A testable version of LocalityMulticastAMRMProxyPolicy that
+   * deterministically falls back to home sub-cluster for unresolved requests.
+   */
+  private class TestableLocalityMulticastAMRMProxyPolicy
+      extends LocalityMulticastAMRMProxyPolicy {
+    @Override
+    protected SubClusterId getSubClusterForUnResolvedRequest(
+        AllocationBookkeeper bookkeeper, long allocationId) {
+      SubClusterId originalResult =
+          super.getSubClusterForUnResolvedRequest(bookkeeper, allocationId);
+      Map<SubClusterId, SubClusterInfo> activeClusters = null;
+      try {
+        activeClusters = getActiveSubclusters();
+      } catch (YarnException e) {
+        throw new RuntimeException(e);
+      }
+      // The randomly selected sub-cluster should at least be active
+      Assert.assertTrue(activeClusters.containsKey(originalResult));
+
+      // Alwasy use home sub-cluster so that unit test is deterministic
+      return getHomeSubCluster();
+    }
+  }
 }
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[32/50] [abbrv] hadoop git commit: HADOOP-15107. Stabilize/tune S3A committers; review correctness & docs. Contributed by Steve Loughran.

Posted by eh...@apache.org.
HADOOP-15107. Stabilize/tune S3A committers; review correctness & docs.
Contributed by Steve Loughran.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5a0babf7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5a0babf7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5a0babf7

Branch: refs/heads/HDFS-12090
Commit: 5a0babf76550f63dad4c17173c4da2bf335c6532
Parents: e8d138c
Author: Steve Loughran <st...@apache.org>
Authored: Thu Aug 30 14:49:53 2018 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Aug 30 14:49:53 2018 +0100

----------------------------------------------------------------------
 .../lib/output/PathOutputCommitter.java         |  12 +-
 .../java/org/apache/hadoop/fs/s3a/Invoker.java  |  15 +-
 .../fs/s3a/commit/AbstractS3ACommitter.java     |  16 +-
 .../fs/s3a/commit/S3ACommitterFactory.java      |  18 +-
 .../s3a/commit/magic/MagicS3GuardCommitter.java |   7 +
 .../staging/DirectoryStagingCommitter.java      |   8 +-
 .../staging/PartitionedStagingCommitter.java    |   9 +-
 .../hadoop/fs/s3a/commit/staging/Paths.java     |  14 +-
 .../fs/s3a/commit/staging/StagingCommitter.java |  50 ++++-
 .../tools/hadoop-aws/committer_architecture.md  |  94 ++++++---
 .../markdown/tools/hadoop-aws/committers.md     |   2 +-
 .../fs/s3a/commit/AbstractCommitITest.java      |  19 ++
 .../fs/s3a/commit/AbstractITCommitMRJob.java    |   5 +-
 .../fs/s3a/commit/AbstractITCommitProtocol.java |  63 ++++--
 .../fs/s3a/commit/ITestS3ACommitterFactory.java | 200 +++++++++++++++++++
 .../fs/s3a/commit/magic/ITMagicCommitMRJob.java |   6 +-
 .../commit/magic/ITestMagicCommitProtocol.java  |  25 ++-
 .../ITStagingCommitMRJobBadDest.java            |  62 ++++++
 .../integration/ITestStagingCommitProtocol.java |  13 ++
 19 files changed, 542 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitter.java
index 3679d9f..5e25f50 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitter.java
@@ -57,8 +57,8 @@ public abstract class PathOutputCommitter extends OutputCommitter {
   protected PathOutputCommitter(Path outputPath,
       TaskAttemptContext context) throws IOException {
     this.context = Preconditions.checkNotNull(context, "Null context");
-    LOG.debug("Creating committer with output path {} and task context"
-        + " {}", outputPath, context);
+    LOG.debug("Instantiating committer {} with output path {} and task context"
+        + " {}", this, outputPath, context);
   }
 
   /**
@@ -71,8 +71,8 @@ public abstract class PathOutputCommitter extends OutputCommitter {
   protected PathOutputCommitter(Path outputPath,
       JobContext context) throws IOException {
     this.context = Preconditions.checkNotNull(context, "Null context");
-    LOG.debug("Creating committer with output path {} and job context"
-        + " {}", outputPath, context);
+    LOG.debug("Instantiating committer {} with output path {} and job context"
+        + " {}", this, outputPath, context);
   }
 
   /**
@@ -103,6 +103,8 @@ public abstract class PathOutputCommitter extends OutputCommitter {
 
   @Override
   public String toString() {
-    return "PathOutputCommitter{context=" + context + '}';
+    return "PathOutputCommitter{context=" + context
+        + "; " + super.toString()
+        + '}';
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
index a007ba1..45912a0 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
@@ -130,8 +130,9 @@ public class Invoker {
   }
 
   /**
-   * Execute an operation and ignore all raised IOExceptions; log at INFO.
-   * @param log log to log at info.
+   * Execute an operation and ignore all raised IOExceptions; log at INFO;
+   * full stack only at DEBUG.
+   * @param log log to use.
    * @param action action to include in log
    * @param path optional path to include in log
    * @param operation operation to execute
@@ -145,13 +146,17 @@ public class Invoker {
     try {
       once(action, path, operation);
     } catch (IOException e) {
-      log.info("{}: {}", toDescription(action, path), e.toString(), e);
+      String description = toDescription(action, path);
+      String error = e.toString();
+      log.info("{}: {}", description, error);
+      log.debug("{}", description, e);
     }
   }
 
   /**
-   * Execute an operation and ignore all raised IOExceptions; log at INFO.
-   * @param log log to log at info.
+   * Execute an operation and ignore all raised IOExceptions; log at INFO;
+   * full stack only at DEBUG.
+   * @param log log to use.
    * @param action action to include in log
    * @param path optional path to include in log
    * @param operation operation to execute

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java
index 5f1ddfa..d2501da 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java
@@ -292,7 +292,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
     final StringBuilder sb = new StringBuilder(
         "AbstractS3ACommitter{");
     sb.append("role=").append(role);
-    sb.append(", name").append(getName());
+    sb.append(", name=").append(getName());
     sb.append(", outputPath=").append(getOutputPath());
     sb.append(", workPath=").append(workPath);
     sb.append('}');
@@ -532,8 +532,14 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
              new DurationInfo(LOG, "Aborting all pending commits under %s",
                  dest)) {
       CommitOperations ops = getCommitOperations();
-      List<MultipartUpload> pending = ops
-          .listPendingUploadsUnderPath(dest);
+      List<MultipartUpload> pending;
+      try {
+        pending = ops.listPendingUploadsUnderPath(dest);
+      } catch (IOException e) {
+        // raised if the listPendingUploads call failed.
+        maybeIgnore(suppressExceptions, "aborting pending uploads", e);
+        return;
+      }
       Tasks.foreach(pending)
           .executeWith(buildThreadPool(getJobContext()))
           .suppressExceptions(suppressExceptions)
@@ -656,7 +662,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
   }
 
   /**
-   * Execute an operation; maybe suppress any raised IOException.
+   * Log or rethrow a caught IOException.
    * @param suppress should raised IOEs be suppressed?
    * @param action action (for logging when the IOE is suppressed.
    * @param ex  exception
@@ -667,7 +673,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
       String action,
       IOException ex) throws IOException {
     if (suppress) {
-      LOG.info(action, ex);
+      LOG.debug(action, ex);
     } else {
       throw ex;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/S3ACommitterFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/S3ACommitterFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/S3ACommitterFactory.java
index 6b170f9..36d0af1 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/S3ACommitterFactory.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/S3ACommitterFactory.java
@@ -77,9 +77,20 @@ public class S3ACommitterFactory extends AbstractS3ACommitterFactory {
     AbstractS3ACommitterFactory factory = chooseCommitterFactory(fileSystem,
         outputPath,
         context.getConfiguration());
-    return factory != null ?
-      factory.createTaskCommitter(fileSystem, outputPath, context)
-      : createFileOutputCommitter(outputPath, context);
+    if (factory != null) {
+      PathOutputCommitter committer = factory.createTaskCommitter(
+          fileSystem, outputPath, context);
+      LOG.info("Using committer {} to output data to {}",
+          (committer instanceof AbstractS3ACommitter
+              ? ((AbstractS3ACommitter) committer).getName()
+              : committer.toString()),
+          outputPath);
+      return committer;
+    } else {
+      LOG.warn("Using standard FileOutputCommitter to commit work."
+          + " This is slow and potentially unsafe.");
+      return createFileOutputCommitter(outputPath, context);
+    }
   }
 
   /**
@@ -104,6 +115,7 @@ public class S3ACommitterFactory extends AbstractS3ACommitterFactory {
 
     String name = fsConf.getTrimmed(FS_S3A_COMMITTER_NAME, COMMITTER_NAME_FILE);
     name = taskConf.getTrimmed(FS_S3A_COMMITTER_NAME, name);
+    LOG.debug("Committer option is {}", name);
     switch (name) {
     case COMMITTER_NAME_FILE:
       factory = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java
index c305141..c956a98 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java
@@ -285,4 +285,11 @@ public class MagicS3GuardCommitter extends AbstractS3ACommitter {
     return CommitUtilsWithMR.getTempTaskAttemptPath(context, getOutputPath());
   }
 
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder(
+        "MagicCommitter{");
+    sb.append('}');
+    return sb.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java
index 3eda24f..23bb06b 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java
@@ -27,13 +27,11 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathExistsException;
 import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
 import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
-import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.*;
 
 /**
  * This commits to a directory.
@@ -70,10 +68,8 @@ public class DirectoryStagingCommitter extends StagingCommitter {
     if (getConflictResolutionMode(context, fs.getConf())
         == ConflictResolution.FAIL
         && fs.exists(outputPath)) {
-      LOG.debug("Failing commit by task attempt {} to write"
-              + " to existing output path {}",
-          context.getJobID(), getOutputPath());
-      throw new PathExistsException(outputPath.toString(), E_DEST_EXISTS);
+      throw failDestinationExists(outputPath,
+          "Setting job as " + getRole());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java
index bfaf443..b51bcb5 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java
@@ -31,14 +31,12 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathExistsException;
 import org.apache.hadoop.fs.s3a.commit.PathCommitException;
 import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
 import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
-import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.*;
 
 /**
  * Partitioned committer.
@@ -100,11 +98,8 @@ public class PartitionedStagingCommitter extends StagingCommitter {
         Path partitionPath = getFinalPath(partition + "/file",
             context).getParent();
         if (fs.exists(partitionPath)) {
-          LOG.debug("Failing commit by task attempt {} to write"
-              + " to existing path {} under {}",
-              context.getTaskAttemptID(), partitionPath, getOutputPath());
-          throw new PathExistsException(partitionPath.toString(),
-              E_DEST_EXISTS);
+          throw failDestinationExists(partitionPath,
+              "Committing task " + context.getTaskAttemptID());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/Paths.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/Paths.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/Paths.java
index d5d256a..a941572 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/Paths.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/Paths.java
@@ -167,13 +167,15 @@ public final class Paths {
             return FileSystem.getLocal(conf).makeQualified(
                 allocator.getLocalPathForWrite(uuid, conf));
           });
-    } catch (ExecutionException e) {
-      throw new RuntimeException(e.getCause());
-    } catch (UncheckedExecutionException e) {
-      if (e.getCause() instanceof RuntimeException) {
-        throw (RuntimeException) e.getCause();
+    } catch (ExecutionException | UncheckedExecutionException e) {
+      Throwable cause = e.getCause();
+      if (cause instanceof RuntimeException) {
+        throw (RuntimeException) cause;
       }
-      throw new RuntimeException(e);
+      if (cause instanceof IOException) {
+        throw (IOException) cause;
+      }
+      throw new IOException(e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java
index 2182eaa..6d02e86 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java
@@ -36,6 +36,8 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathExistsException;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter;
 import org.apache.hadoop.fs.s3a.commit.CommitConstants;
@@ -500,6 +502,10 @@ public class StagingCommitter extends AbstractS3ACommitter {
           listAndFilter(attemptFS,
               wrappedJobAttemptPath, false,
               HIDDEN_FILE_FILTER));
+    } catch (FileNotFoundException e) {
+      // this can mean the job was aborted early on, so don't confuse people
+      // with long stack traces that aren't the underlying problem.
+      maybeIgnore(suppressExceptions, "Pending upload directory not found", e);
     } catch (IOException e) {
       // unable to work with endpoint, if suppressing errors decide our actions
       maybeIgnore(suppressExceptions, "Listing pending uploads", e);
@@ -565,13 +571,13 @@ public class StagingCommitter extends AbstractS3ACommitter {
   }
 
   /**
-   * Delete the working paths of a job. Does not attempt to clean up
-   * the work of the wrapped committer.
+   * Delete the working paths of a job.
    * <ol>
    *   <li>The job attempt path</li>
-   *   <li>$dest/__temporary</li>
+   *   <li>{@code $dest/__temporary}</li>
    *   <li>the local working directory for staged files</li>
    * </ol>
+   * Does not attempt to clean up the work of the wrapped committer.
    * @param context job context
    * @throws IOException IO failure
    */
@@ -836,6 +842,44 @@ public class StagingCommitter extends AbstractS3ACommitter {
   }
 
   /**
+   * Generate a {@link PathExistsException} because the destination exists.
+   * Lists some of the child entries first, to help diagnose the problem.
+   * @param path path which exists
+   * @param description description (usually task/job ID)
+   * @return an exception to throw
+   */
+  protected PathExistsException failDestinationExists(final Path path,
+      final String description) {
+
+    LOG.error("{}: Failing commit by job {} to write"
+            + " to existing output path {}.",
+        description,
+        getJobContext().getJobID(), path);
+    // List the first 10 descendants, to give some details
+    // on what is wrong but not overload things if there are many files.
+    try {
+      int limit = 10;
+      RemoteIterator<LocatedFileStatus> lf
+          = getDestFS().listFiles(path, true);
+      LOG.info("Partial Directory listing");
+      while (limit > 0 && lf.hasNext()) {
+        limit--;
+        LocatedFileStatus status = lf.next();
+        LOG.info("{}: {}",
+            status.getPath(),
+            status.isDirectory()
+                ? " dir"
+                : ("file size " + status.getLen() + " bytes"));
+      }
+    } catch (IOException e) {
+      LOG.info("Discarding exception raised when listing {}: " + e, path);
+      LOG.debug("stack trace ", e);
+    }
+    return new PathExistsException(path.toString(),
+        description + ": " + InternalCommitterConstants.E_DEST_EXISTS);
+  }
+
+  /**
    * Get the conflict mode option string.
    * @param context context with the config
    * @param fsConf filesystem config

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committer_architecture.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committer_architecture.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committer_architecture.md
index e4ba75d..3071754 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committer_architecture.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committer_architecture.md
@@ -230,7 +230,6 @@ None: directories are created on demand.
 Rename task attempt path to task committed path.
 
 ```python
-
 def needsTaskCommit(fs, jobAttemptPath, taskAttemptPath, dest):
   return fs.exists(taskAttemptPath)
 
@@ -276,12 +275,12 @@ def commitJob(fs, jobAttemptDir, dest):
 (See below for details on `mergePaths()`)
 
 
-A failure during job abort cannot be recovered from except by re-executing
+A failure during job commit cannot be recovered from except by re-executing
 the entire query:
 
 ```python
 def isCommitJobRepeatable() :
-  return True
+  return False
 ```
 
 Accordingly, it is a failure point in the protocol. With a low number of files
@@ -307,12 +306,28 @@ def cleanupJob(fs, dest):
 ```
 
 
-### Job Recovery
+### Job Recovery Before `commitJob()`
 
-1. Data under task committed paths is retained
-1. All directories under `$dest/_temporary/$appAttemptId/_temporary/` are deleted.
+For all committers, the recovery process takes place in the application
+master.
+1. The job history file of the previous attempt is loaded and scanned
+to determine which tasks were recorded as having succeeded.
+1. For each successful task, the job committer has its `recoverTask()` method
+invoked with a `TaskAttemptContext` built from the previous attempt's details.
+1. If the method does not raise an exception, it is considered to have been
+recovered, and not to be re-executed.
+1. All other tasks are queued for execution.
 
-Uncommitted/unexecuted tasks are (re)executed.
+For the v1 committer, task recovery is straightforward.
+The directory of the committed task from the previous attempt is
+moved under the directory of the current application attempt.
+
+```python
+def recoverTask(tac):
+  oldAttemptId = appAttemptId - 1
+  fs.rename('$dest/_temporary/oldAttemptId/${tac.taskId}',
+    '$dest/_temporary/appAttemptId/${tac.taskId}')
+```
 
 This significantly improves time to recover from Job driver (here MR AM) failure.
 The only lost work is that of all tasks in progress -those which had generated
@@ -330,6 +345,11 @@ failure simply by rerunning the entire job. This is implicitly the strategy
 in Spark, which does not attempt to recover any in-progress jobs. The faster
 your queries, the simpler your recovery strategy needs to be.
 
+### Job Recovery During `commitJob()`
+
+This is not possible; a failure during job commit requires the entire job
+to be re-executed after cleaning up the destination directory.
+
 ### `mergePaths(FileSystem fs, FileStatus src, Path dest)` Algorithm
 
 `mergePaths()` is the core algorithm to merge data; it is somewhat confusing
@@ -352,24 +372,23 @@ def mergePathsV1(fs, src, dest) :
       fs.delete(dest, recursive = True)
     fs.rename(src.getPath, dest)
   else :
-    # destination is directory, choose action on source type
-    if src.isDirectory :
-      if not toStat is None :
-        if not toStat.isDirectory :
-          # Destination exists and is not a directory
-          fs.delete(dest)
-          fs.rename(src.getPath(), dest)
-        else :
-          # Destination exists and is a directory
-          # merge all children under destination directory
-          for child in fs.listStatus(src.getPath) :
-            mergePathsV1(fs, child, dest + child.getName)
-      else :
-        # destination does not exist
+    # src is directory, choose action on dest type
+    if not toStat is None :
+      if not toStat.isDirectory :
+        # Destination exists and is not a directory
+        fs.delete(dest)
         fs.rename(src.getPath(), dest)
+      else :
+        # Destination exists and is a directory
+        # merge all children under destination directory
+        for child in fs.listStatus(src.getPath) :
+          mergePathsV1(fs, child, dest + child.getName)
+    else :
+      # destination does not exist
+      fs.rename(src.getPath(), dest)
 ```
 
-## v2 commit algorithm
+## The v2 Commit Algorithm
 
 
 The v2 algorithm directly commits task output into the destination directory.
@@ -506,12 +525,31 @@ Cost: `O(1)` for normal filesystems, `O(files)` for object stores.
 As no data is written to the destination directory, a task can be cleaned up
 by deleting the task attempt directory.
 
-### v2 Job Recovery
+### v2 Job Recovery Before `commitJob()`
+
+
+Because the data has been renamed into the destination directory, all tasks
+recorded as having being committed have no recovery needed at all:
+
+```python
+def recoverTask(tac):
+```
+
+All active and queued tasks are scheduled for execution.
+
+There is a weakness here, the same one on a failure during `commitTask()`:
+it is only safe to repeat a task which failed during that commit operation
+if the name of all generated files are constant across all task attempts.
+
+If the Job AM fails while a task attempt has been instructed to commit,
+and that commit is not recorded as having completed, the state of that
+in-progress task is unknown...really it isn't be safe to recover the
+job at this point.
+
 
-Because the data has been renamed into the destination directory, it is nominally
-recoverable. However, this assumes that the number and name of generated
-files are constant on retried tasks.
+### v2 Job Recovery During `commitJob()`
 
+This is straightforward: `commitJob()` is re-invoked.
 
 ## How MapReduce uses the committer in a task
 
@@ -896,7 +934,7 @@ and metadata.
 
         POST bucket.s3.aws.com/path?uploads
 
-    An UploadId is returned
+    An `UploadId` is returned
 
 1. Caller uploads one or more parts.
 
@@ -994,7 +1032,7 @@ Task outputs are directed to the local FS by `getTaskAttemptPath` and `getWorkPa
 
 The single-directory and partitioned committers handle conflict resolution by
 checking whether target paths exist in S3 before uploading any data.
-There are 3 conflict resolution modes, controlled by setting `fs.s3a.committer.staging.conflict-mode`:
+There are three conflict resolution modes, controlled by setting `fs.s3a.committer.staging.conflict-mode`:
 
 * `fail`: Fail a task if an output directory or partition already exists. (Default)
 * `append`: Upload data files without checking whether directories or partitions already exist.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committers.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committers.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committers.md
index 392cde2..09e123d 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committers.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committers.md
@@ -371,7 +371,7 @@ Put differently: start with the Directory Committer.
 
 To use an S3A committer, the property `mapreduce.outputcommitter.factory.scheme.s3a`
 must be set to the S3A committer factory, `org.apache.hadoop.fs.s3a.commit.staging.S3ACommitterFactory`.
-This is done in `core-default.xml`
+This is done in `mapred-default.xml`
 
 ```xml
 <property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java
index 90e8894..246bf9d 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java
@@ -174,6 +174,25 @@ public abstract class AbstractCommitITest extends AbstractS3ATestBase {
   }
 
   /**
+   * Create a random Job ID using the fork ID as part of the number.
+   * @return fork ID string in a format parseable by Jobs
+   * @throws Exception failure
+   */
+  protected String randomJobId() throws Exception {
+    String testUniqueForkId = System.getProperty(TEST_UNIQUE_FORK_ID, "0001");
+    int l = testUniqueForkId.length();
+    String trailingDigits = testUniqueForkId.substring(l - 4, l);
+    try {
+      int digitValue = Integer.valueOf(trailingDigits);
+      return String.format("20070712%04d_%04d",
+          (long)(Math.random() * 1000),
+          digitValue);
+    } catch (NumberFormatException e) {
+      throw new Exception("Failed to parse " + trailingDigits, e);
+    }
+  }
+
+  /**
    * Teardown waits for the consistency delay and resets failure count, so
    * FS is stable, before the superclass teardown is called. This
    * should clean things up better.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitMRJob.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitMRJob.java
index 13dfd83..161db85 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitMRJob.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitMRJob.java
@@ -38,7 +38,6 @@ import org.junit.rules.TemporaryFolder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -266,9 +265,9 @@ public abstract class AbstractITCommitMRJob extends AbstractCommitITest {
 
   /**
    * Override point to let implementations tune the MR Job conf.
-   * @param c configuration
+   * @param jobConf configuration
    */
-  protected void applyCustomConfigOptions(Configuration c) {
+  protected void applyCustomConfigOptions(JobConf jobConf) throws IOException {
 
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java
index 4d7f524..5ae8f54 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java
@@ -159,25 +159,6 @@ public abstract class AbstractITCommitProtocol extends AbstractCommitITest {
     cleanupDestDir();
   }
 
-  /**
-   * Create a random Job ID using the fork ID as part of the number.
-   * @return fork ID string in a format parseable by Jobs
-   * @throws Exception failure
-   */
-  private String randomJobId() throws Exception {
-    String testUniqueForkId = System.getProperty(TEST_UNIQUE_FORK_ID, "0001");
-    int l = testUniqueForkId.length();
-    String trailingDigits = testUniqueForkId.substring(l - 4, l);
-    try {
-      int digitValue = Integer.valueOf(trailingDigits);
-      return String.format("20070712%04d_%04d",
-          (long)(Math.random() * 1000),
-          digitValue);
-    } catch (NumberFormatException e) {
-      throw new Exception("Failed to parse " + trailingDigits, e);
-    }
-  }
-
   @Override
   public void teardown() throws Exception {
     describe("teardown");
@@ -765,6 +746,7 @@ public abstract class AbstractITCommitProtocol extends AbstractCommitITest {
     JobContext jContext = jobData.jContext;
     TaskAttemptContext tContext = jobData.tContext;
     AbstractS3ACommitter committer = jobData.committer;
+    validateTaskAttemptWorkingDirectory(committer, tContext);
 
     // write output
     describe("1. Writing output");
@@ -1360,12 +1342,55 @@ public abstract class AbstractITCommitProtocol extends AbstractCommitITest {
 
   }
 
+  @Test
+  public void testS3ACommitterFactoryBinding() throws Throwable {
+    describe("Verify that the committer factory returns this "
+        + "committer when configured to do so");
+    Job job = newJob();
+    FileOutputFormat.setOutputPath(job, outDir);
+    Configuration conf = job.getConfiguration();
+    conf.set(MRJobConfig.TASK_ATTEMPT_ID, attempt0);
+    conf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 1);
+    TaskAttemptContext tContext = new TaskAttemptContextImpl(conf,
+        taskAttempt0);
+    String name = getCommitterName();
+    S3ACommitterFactory factory = new S3ACommitterFactory();
+    assertEquals("Wrong committer from factory",
+        createCommitter(outDir, tContext).getClass(),
+        factory.createOutputCommitter(outDir, tContext).getClass());
+  }
+
+  /**
+   * Validate the path of a file being written to during the write
+   * itself.
+   * @param p path
+   * @throws IOException IO failure
+   */
   protected void validateTaskAttemptPathDuringWrite(Path p) throws IOException {
 
   }
 
+  /**
+   * Validate the path of a file being written to after the write
+   * operation has completed.
+   * @param p path
+   * @throws IOException IO failure
+   */
   protected void validateTaskAttemptPathAfterWrite(Path p) throws IOException {
 
   }
 
+  /**
+   * Perform any actions needed to validate the working directory of
+   * a committer.
+   * For example: filesystem, path attributes
+   * @param committer committer instance
+   * @param context task attempt context
+   * @throws IOException IO failure
+   */
+  protected void validateTaskAttemptWorkingDirectory(
+      AbstractS3ACommitter committer,
+      TaskAttemptContext context) throws IOException {
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java
new file mode 100644
index 0000000..a8547d6
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java
@@ -0,0 +1,200 @@
+/*
+ * 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.fs.s3a.commit;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitter;
+import org.apache.hadoop.fs.s3a.commit.staging.DirectoryStagingCommitter;
+import org.apache.hadoop.fs.s3a.commit.staging.PartitionedStagingCommitter;
+import org.apache.hadoop.fs.s3a.commit.staging.StagingCommitter;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
+
+/**
+ * Tests for some aspects of the committer factory.
+ * All tests are grouped into one single test so that only one
+ * S3A FS client is set up and used for the entire run.
+ * Saves time and money.
+ */
+public class ITestS3ACommitterFactory extends AbstractCommitITest {
+
+
+  protected static final String INVALID_NAME = "invalid-name";
+
+  /**
+   * Counter to guarantee that even in parallel test runs, no job has the same
+   * ID.
+   */
+
+  private String jobId;
+
+  // A random task attempt id for testing.
+  private String attempt0;
+
+  private TaskAttemptID taskAttempt0;
+
+  private Path outDir;
+
+  private S3ACommitterFactory factory;
+
+  private TaskAttemptContext tContext;
+
+  /**
+   * Parameterized list of bindings of committer name in config file to
+   * expected class instantiated.
+   */
+  private static final Object[][] bindings = {
+      {COMMITTER_NAME_FILE, FileOutputCommitter.class},
+      {COMMITTER_NAME_DIRECTORY, DirectoryStagingCommitter.class},
+      {COMMITTER_NAME_PARTITIONED, PartitionedStagingCommitter.class},
+      {InternalCommitterConstants.COMMITTER_NAME_STAGING,
+          StagingCommitter.class},
+      {COMMITTER_NAME_MAGIC, MagicS3GuardCommitter.class}
+  };
+
+  /**
+   * This is a ref to the FS conf, so changes here are visible
+   * to callers querying the FS config.
+   */
+  private Configuration filesystemConfRef;
+
+  private Configuration taskConfRef;
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    jobId = randomJobId();
+    attempt0 = "attempt_" + jobId + "_m_000000_0";
+    taskAttempt0 = TaskAttemptID.forName(attempt0);
+
+    outDir = path(getMethodName());
+    factory = new S3ACommitterFactory();
+    Configuration conf = new Configuration();
+    conf.set(FileOutputFormat.OUTDIR, outDir.toUri().toString());
+    conf.set(MRJobConfig.TASK_ATTEMPT_ID, attempt0);
+    conf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 1);
+    filesystemConfRef = getFileSystem().getConf();
+    tContext = new TaskAttemptContextImpl(conf, taskAttempt0);
+    taskConfRef = tContext.getConfiguration();
+  }
+
+  @Test
+  public void testEverything() throws Throwable {
+    testImplicitFileBinding();
+    testBindingsInTask();
+    testBindingsInFSConfig();
+    testInvalidFileBinding();
+    testInvalidTaskBinding();
+  }
+
+  /**
+   * Verify that if all config options are unset, the FileOutputCommitter
+   *
+   * is returned.
+   */
+  public void testImplicitFileBinding() throws Throwable {
+    taskConfRef.unset(FS_S3A_COMMITTER_NAME);
+    filesystemConfRef.unset(FS_S3A_COMMITTER_NAME);
+    assertFactoryCreatesExpectedCommitter(FileOutputCommitter.class);
+  }
+
+  /**
+   * Verify that task bindings are picked up.
+   */
+  public void testBindingsInTask() throws Throwable {
+    // set this to an invalid value to be confident it is not
+    // being checked.
+    filesystemConfRef.set(FS_S3A_COMMITTER_NAME, "INVALID");
+    taskConfRef.set(FS_S3A_COMMITTER_NAME, COMMITTER_NAME_FILE);
+    assertFactoryCreatesExpectedCommitter(FileOutputCommitter.class);
+    for (Object[] binding : bindings) {
+      taskConfRef.set(FS_S3A_COMMITTER_NAME,
+          (String) binding[0]);
+      assertFactoryCreatesExpectedCommitter((Class) binding[1]);
+    }
+  }
+
+  /**
+   * Verify that FS bindings are picked up.
+   */
+  public void testBindingsInFSConfig() throws Throwable {
+    taskConfRef.unset(FS_S3A_COMMITTER_NAME);
+    filesystemConfRef.set(FS_S3A_COMMITTER_NAME, COMMITTER_NAME_FILE);
+    assertFactoryCreatesExpectedCommitter(FileOutputCommitter.class);
+    for (Object[] binding : bindings) {
+      taskConfRef.set(FS_S3A_COMMITTER_NAME, (String) binding[0]);
+      assertFactoryCreatesExpectedCommitter((Class) binding[1]);
+    }
+  }
+
+  /**
+   * Create an invalid committer via the FS binding,
+   */
+  public void testInvalidFileBinding() throws Throwable {
+    taskConfRef.unset(FS_S3A_COMMITTER_NAME);
+    filesystemConfRef.set(FS_S3A_COMMITTER_NAME, INVALID_NAME);
+    LambdaTestUtils.intercept(PathCommitException.class, INVALID_NAME,
+        () -> createCommitter());
+  }
+
+  /**
+   * Create an invalid committer via the task attempt.
+   */
+  public void testInvalidTaskBinding() throws Throwable {
+    filesystemConfRef.unset(FS_S3A_COMMITTER_NAME);
+    taskConfRef.set(FS_S3A_COMMITTER_NAME, INVALID_NAME);
+    LambdaTestUtils.intercept(PathCommitException.class, INVALID_NAME,
+        () -> createCommitter());
+  }
+
+  /**
+   * Assert that the factory creates the expected committer.
+   * @param expected expected committer class.
+   * @throws IOException IO failure.
+   */
+  protected void assertFactoryCreatesExpectedCommitter(
+      final Class expected)
+      throws IOException {
+    assertEquals("Wrong Committer from factory",
+        expected,
+        createCommitter().getClass());
+  }
+
+  /**
+   * Create a committer.
+   * @return the committer
+   * @throws IOException IO failure.
+   */
+  private PathOutputCommitter createCommitter() throws IOException {
+    return factory.createOutputCommitter(outDir, tContext);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITMagicCommitMRJob.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITMagicCommitMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITMagicCommitMRJob.java
index 57eb8b2..b7be17a 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITMagicCommitMRJob.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITMagicCommitMRJob.java
@@ -18,10 +18,10 @@
 
 package org.apache.hadoop.fs.s3a.commit.magic;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.commit.AbstractITCommitMRJob;
 import org.apache.hadoop.fs.s3a.commit.files.SuccessData;
+import org.apache.hadoop.mapred.JobConf;
 
 import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
 
@@ -30,7 +30,7 @@ import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
  *
  * There's no need to disable the committer setting for the filesystem here,
  * because the committers are being instantiated in their own processes;
- * the settings in {@link #applyCustomConfigOptions(Configuration)} are
+ * the settings in {@link AbstractITCommitMRJob#applyCustomConfigOptions(JobConf)} are
  * passed down to these processes.
  */
 public class ITMagicCommitMRJob extends AbstractITCommitMRJob {
@@ -54,7 +54,7 @@ public class ITMagicCommitMRJob extends AbstractITCommitMRJob {
    * @param conf configuration
    */
   @Override
-  protected void applyCustomConfigOptions(Configuration conf) {
+  protected void applyCustomConfigOptions(JobConf conf) {
     conf.setBoolean(MAGIC_COMMITTER_ENABLED, true);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java
index 74c1d9d..057adf5 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java
@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.fs.s3a.commit.magic;
 
+import java.io.IOException;
+import java.net.URI;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
@@ -32,9 +35,8 @@ import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobStatus;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-import java.io.IOException;
-
 import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
+import static org.hamcrest.CoreMatchers.containsString;
 
 /**
  * Test the magic committer's commit protocol.
@@ -116,6 +118,25 @@ public class ITestMagicCommitProtocol extends AbstractITCommitProtocol {
   }
 
   /**
+   * The magic committer paths are always on S3, and always have
+   * "__magic" in the path.
+   * @param committer committer instance
+   * @param context task attempt context
+   * @throws IOException IO failure
+   */
+  @Override
+  protected void validateTaskAttemptWorkingDirectory(
+      final AbstractS3ACommitter committer,
+      final TaskAttemptContext context) throws IOException {
+    URI wd = committer.getWorkPath().toUri();
+    assertEquals("Wrong schema for working dir " + wd
+        + " with committer " + committer,
+        "s3a", wd.getScheme());
+    assertThat(wd.getPath(),
+        containsString('/' + CommitConstants.MAGIC + '/'));
+  }
+
+  /**
    * The class provides a overridden implementation of commitJobInternal which
    * causes the commit failed for the first time then succeed.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITStagingCommitMRJobBadDest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITStagingCommitMRJobBadDest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITStagingCommitMRJobBadDest.java
new file mode 100644
index 0000000..be477a7
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITStagingCommitMRJobBadDest.java
@@ -0,0 +1,62 @@
+/*
+ * 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.fs.s3a.commit.staging.integration;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.commit.AbstractITCommitMRJob;
+import org.apache.hadoop.fs.s3a.commit.staging.StagingCommitter;
+import org.apache.hadoop.mapred.FileAlreadyExistsException;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+/**
+ * This is a test to verify that the committer will fail if the destination
+ * directory exists, and that this happens in job setup.
+ */
+public class ITStagingCommitMRJobBadDest extends AbstractITCommitMRJob {
+
+  @Override
+  protected String committerName() {
+    return StagingCommitter.NAME;
+  }
+
+  /**
+   * create the destination directory and expect a failure.
+   * @param conf configuration
+   */
+  @Override
+  protected void applyCustomConfigOptions(JobConf conf) throws IOException {
+    // This is the destination in the S3 FS
+    String outdir = conf.get(FileOutputFormat.OUTDIR);
+    S3AFileSystem fs = getFileSystem();
+    Path outputPath = new Path(outdir);
+    fs.mkdirs(outputPath);
+  }
+
+  @Override
+  public void testMRJob() throws Exception {
+    LambdaTestUtils.intercept(FileAlreadyExistsException.class,
+        "Output directory",
+        super::testMRJob);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a0babf7/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java
index 08c572e..180e743 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java
@@ -118,6 +118,19 @@ public class ITestStagingCommitProtocol extends AbstractITCommitProtocol {
   }
 
   /**
+   * The staging committers always have the local FS for their work.
+   * @param committer committer instance
+   * @param context task attempt context
+   * @throws IOException IO failure
+   */
+  @Override
+  protected void validateTaskAttemptWorkingDirectory(final AbstractS3ACommitter committer,
+      final TaskAttemptContext context) throws IOException {
+    Path wd = context.getWorkingDirectory();
+    assertEquals("file", wd.toUri().getScheme());
+  }
+
+  /**
    * The class provides a overridden implementation of commitJobInternal which
    * causes the commit failed for the first time then succeed.
    */


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[46/50] [abbrv] hadoop git commit: HDFS-13867. RBF: Add validation for max arguments for Router admin ls, clrQuota, setQuota, rm and nameservice commands. Contributed by Ayush Saxena.

Posted by eh...@apache.org.
HDFS-13867. RBF: Add validation for max arguments for Router admin ls, clrQuota, setQuota, rm and nameservice commands. Contributed by Ayush Saxena.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/780df903
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/780df903
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/780df903

Branch: refs/heads/HDFS-12090
Commit: 780df9034f265a8e602856b34cc21d9be02f5c48
Parents: 873ef8a
Author: Vinayakumar B <vi...@apache.org>
Authored: Mon Sep 3 14:28:31 2018 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Mon Sep 3 14:28:31 2018 +0530

----------------------------------------------------------------------
 .../hdfs/tools/federation/RouterAdmin.java      | 45 ++++++++++++++++++--
 .../federation/router/TestRouterAdminCLI.java   | 35 +++++++++++++++
 2 files changed, 76 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/780df903/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
index 46be373..ef8d7c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
@@ -146,6 +146,43 @@ public class RouterAdmin extends Configured implements Tool {
     return getUsage(null);
   }
 
+  /**
+   * Usage: validates the maximum number of arguments for a command.
+   * @param arg List of of command line parameters.
+   */
+  private void validateMax(String[] arg) {
+    if (arg[0].equals("-rm")) {
+      if (arg.length > 2) {
+        throw new IllegalArgumentException(
+            "Too many arguments, Max=1 argument allowed");
+      }
+    } else if (arg[0].equals("-ls")) {
+      if (arg.length > 2) {
+        throw new IllegalArgumentException(
+            "Too many arguments, Max=1 argument allowed");
+      }
+    } else if (arg[0].equals("-clrQuota")) {
+      if (arg.length > 2) {
+        throw new IllegalArgumentException(
+            "Too many arguments, Max=1 argument allowed");
+      }
+    } else if (arg[0].equals("-safemode")) {
+      if (arg.length > 2) {
+        throw new IllegalArgumentException(
+            "Too many arguments, Max=1 argument allowed only");
+      }
+    } else if (arg[0].equals("-nameservice")) {
+      if (arg.length > 3) {
+        throw new IllegalArgumentException(
+            "Too many arguments, Max=2 arguments allowed");
+      }
+    } else if (arg[0].equals("-getDisabledNameservices")) {
+      if (arg.length > 1) {
+        throw new IllegalArgumentException("No arguments allowed");
+      }
+    }
+  }
+
   @Override
   public int run(String[] argv) throws Exception {
     if (argv.length < 1) {
@@ -222,6 +259,7 @@ public class RouterAdmin extends Configured implements Tool {
     Exception debugException = null;
     exitCode = 0;
     try {
+      validateMax(argv);
       if ("-add".equals(cmd)) {
         if (addMount(argv, i)) {
           System.out.println("Successfully added mount point " + argv[i]);
@@ -251,10 +289,6 @@ public class RouterAdmin extends Configured implements Tool {
               "Successfully clear quota for mount point " + argv[i]);
         }
       } else if ("-safemode".equals(cmd)) {
-        if (argv.length > 2) {
-          throw new IllegalArgumentException(
-              "Too many arguments, Max=1 argument allowed only");
-        }
         manageSafeMode(argv[i]);
       } else if ("-nameservice".equals(cmd)) {
         String subcmd = argv[i];
@@ -641,6 +675,9 @@ public class RouterAdmin extends Configured implements Tool {
           throw new IllegalArgumentException(
               "Cannot parse ssQuota: " + parameters[i]);
         }
+      } else {
+        throw new IllegalArgumentException(
+            "Invalid argument : " + parameters[i]);
       }
 
       i++;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/780df903/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
index 0c7321f..fa29cd9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
@@ -267,6 +267,13 @@ public class TestRouterAdminCLI {
     assertEquals(0, ToolRunner.run(admin, argv));
     assertTrue(out.toString().contains(src));
 
+    // Test with wrong number of arguments
+    argv = new String[] {"-ls", srcWithSlash, "check", "check2"};
+    System.setErr(new PrintStream(err));
+    ToolRunner.run(admin, argv);
+    assertTrue(
+        err.toString().contains("Too many arguments, Max=1 argument allowed"));
+
     out.reset();
     GetMountTableEntriesRequest getRequest = GetMountTableEntriesRequest
         .newInstance("/");
@@ -317,6 +324,13 @@ public class TestRouterAdminCLI {
     assertEquals(0, ToolRunner.run(admin, argv));
     assertTrue(out.toString().contains(
         "Cannot remove mount point " + invalidPath));
+
+    // test wrong number of arguments
+    System.setErr(new PrintStream(err));
+    argv = new String[] {"-rm", src, "check" };
+    ToolRunner.run(admin, argv);
+    assertTrue(err.toString()
+        .contains("Too many arguments, Max=1 argument allowed"));
   }
 
   @Test
@@ -572,6 +586,18 @@ public class TestRouterAdminCLI {
     // verify if quota unset successfully
     assertEquals(HdfsConstants.QUOTA_RESET, quotaUsage.getQuota());
     assertEquals(HdfsConstants.QUOTA_RESET, quotaUsage.getSpaceQuota());
+
+    // verify wrong arguments
+    System.setErr(new PrintStream(err));
+    argv = new String[] {"-clrQuota", src, "check"};
+    ToolRunner.run(admin, argv);
+    assertTrue(err.toString(),
+        err.toString().contains("Too many arguments, Max=1 argument allowed"));
+
+    argv = new String[] {"-setQuota", src, "check", "check2"};
+    err.reset();
+    ToolRunner.run(admin, argv);
+    assertTrue(err.toString().contains("Invalid argument : check"));
   }
 
   @Test
@@ -686,6 +712,15 @@ public class TestRouterAdminCLI {
         new String[] {"-nameservice", "wrong", "ns0"}));
     assertTrue("Got error: " + err.toString(),
         err.toString().startsWith("nameservice: Unknown command: wrong"));
+
+    err.reset();
+    ToolRunner.run(admin,
+        new String[] {"-nameservice", "enable", "ns0", "check"});
+    assertTrue(
+        err.toString().contains("Too many arguments, Max=2 arguments allowed"));
+    err.reset();
+    ToolRunner.run(admin, new String[] {"-getDisabledNameservices", "check"});
+    assertTrue(err.toString().contains("No arguments allowed"));
   }
 
   /**


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[44/50] [abbrv] hadoop git commit: HDDS-357. Use DBStore and TableStore for OzoneManager non-background service. Contributed by Nandakumar.

Posted by eh...@apache.org.
HDDS-357. Use DBStore and TableStore for OzoneManager non-background service.
Contributed by Nandakumar.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ff036e49
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ff036e49
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ff036e49

Branch: refs/heads/HDFS-12090
Commit: ff036e49ff967d5dacf4b2d9d5376e57578ef391
Parents: eed8415
Author: Anu Engineer <ae...@apache.org>
Authored: Sun Sep 2 11:47:32 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Sun Sep 2 11:47:32 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/ozone/OzoneConsts.java    |   6 +-
 .../org/apache/hadoop/utils/RocksDBStore.java   |   2 +-
 .../org/apache/hadoop/utils/db/DBStore.java     |  22 +
 .../org/apache/hadoop/utils/db/RDBStore.java    |  26 +-
 .../common/src/main/resources/ozone-default.xml |   2 +-
 .../apache/hadoop/hdds/server/ServerUtils.java  |   5 +
 .../ozone/client/io/ChunkGroupOutputStream.java |   4 +-
 .../hadoop/ozone/om/helpers/OpenKeySession.java |   6 +-
 .../ozone/om/protocol/OzoneManagerProtocol.java |  11 +-
 ...neManagerProtocolClientSideTranslatorPB.java |   8 +-
 .../src/main/proto/OzoneManagerProtocol.proto   |   6 +-
 .../rpc/TestCloseContainerHandlingByClient.java |  37 +-
 .../ozone/client/rpc/TestOzoneRpcClient.java    |   4 +
 .../apache/hadoop/ozone/om/TestOmSQLCli.java    |   7 +-
 .../hadoop/ozone/om/TestOzoneManager.java       |  37 +-
 .../hadoop/ozone/web/client/TestVolume.java     |   6 +
 .../hadoop/ozone/om/BucketManagerImpl.java      |  57 ++-
 .../org/apache/hadoop/ozone/om/KeyManager.java  |   6 +-
 .../apache/hadoop/ozone/om/KeyManagerImpl.java  | 276 +++++-----
 .../hadoop/ozone/om/OMMetadataManager.java      | 222 ++++----
 .../hadoop/ozone/om/OmMetadataManagerImpl.java  | 509 +++++++++++--------
 .../apache/hadoop/ozone/om/OzoneManager.java    | 209 ++++----
 .../hadoop/ozone/om/VolumeManagerImpl.java      | 156 +++---
 ...neManagerProtocolServerSideTranslatorPB.java |   7 +-
 .../hadoop/ozone/om/TestBucketManagerImpl.java  | 208 ++++----
 .../org/apache/hadoop/ozone/scm/cli/SQLCLI.java |  12 +-
 26 files changed, 978 insertions(+), 873 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index 15366fb..8ea4d7f 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -92,7 +92,6 @@ public final class OzoneConsts {
   public static final String CONTAINER_DB_SUFFIX = "container.db";
   public static final String SCM_CONTAINER_DB = "scm-" + CONTAINER_DB_SUFFIX;
   public static final String DN_CONTAINER_DB = "-dn-"+ CONTAINER_DB_SUFFIX;
-  public static final String BLOCK_DB = "block.db";
   public static final String OPEN_CONTAINERS_DB = "openContainers.db";
   public static final String DELETED_BLOCK_DB = "deletedBlock.db";
   public static final String OM_DB_NAME = "om.db";
@@ -113,8 +112,6 @@ public final class OzoneConsts {
   public static final String DELETING_KEY_PREFIX = "#deleting#";
   public static final String DELETED_KEY_PREFIX = "#deleted#";
   public static final String DELETE_TRANSACTION_KEY_PREFIX = "#delTX#";
-  public static final String OPEN_KEY_PREFIX = "#open#";
-  public static final String OPEN_KEY_ID_DELIMINATOR = "#";
 
   /**
    * OM LevelDB prefixes.
@@ -138,8 +135,7 @@ public final class OzoneConsts {
    *  | #deleting#/volumeName/bucketName/keyName |  KeyInfo    |
    *  ----------------------------------------------------------
    */
-  public static final String OM_VOLUME_PREFIX = "/#";
-  public static final String OM_BUCKET_PREFIX = "/#";
+
   public static final String OM_KEY_PREFIX = "/";
   public static final String OM_USER_PREFIX = "$";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java
index b243e3d..379d9e9 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/RocksDBStore.java
@@ -94,7 +94,7 @@ public class RocksDBStore implements MetadataStore {
     }
   }
 
-  private IOException toIOException(String msg, RocksDBException e) {
+  public static IOException toIOException(String msg, RocksDBException e) {
     String statusCode = e.getStatus() == null ? "N/A" :
         e.getStatus().getCodeString();
     String errMessage = e.getMessage() == null ? "Unknown error" :

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStore.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStore.java
index a817f4f..6947a83 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStore.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/DBStore.java
@@ -20,6 +20,7 @@
 package org.apache.hadoop.utils.db;
 
 import org.apache.hadoop.classification.InterfaceStability;
+import org.rocksdb.WriteBatch;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -83,11 +84,32 @@ public interface DBStore extends AutoCloseable {
       throws IOException;
 
   /**
+   * Moves a key from the Source Table to the destination Table and updates the
+   * destination with the new key name and value.
+   * This is similar to deleting an entry in one table and adding an entry in
+   * another table, here it is done atomically.
+   *
+   * @param sourceKey - Key to move.
+   * @param destKey - Destination key name.
+   * @param value - new value to write to the destination table.
+   * @param source - Source Table.
+   * @param dest - Destination Table.
+   * @throws IOException on Failure
+   */
+  void move(byte[] sourceKey, byte[] destKey, byte[] value,
+            Table source, Table dest) throws IOException;
+
+  /**
    * Returns an estimated count of keys in this DB.
    *
    * @return long, estimate of keys in the DB.
    */
   long getEstimatedKeyCount() throws IOException;
 
+  /**
+   * Writes a transaction into the DB using the default write Options.
+   * @param batch - Batch to write.
+   */
+  void write(WriteBatch batch) throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
index 85508d5..5078b3e 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/db/RDBStore.java
@@ -189,9 +189,16 @@ public class RDBStore implements DBStore {
     }
   }
 
+
   @Override
   public void move(byte[] key, byte[] value, Table source,
       Table dest) throws IOException {
+    move(key, key, value, source, dest);
+  }
+
+  @Override
+  public void move(byte[] sourceKey, byte[] destKey, byte[] value, Table source,
+      Table dest) throws IOException {
     RDBTable sourceTable;
     RDBTable destTable;
     if (source instanceof RDBTable) {
@@ -210,13 +217,13 @@ public class RDBStore implements DBStore {
           + "RocksDBTable.");
     }
     try (WriteBatch batch = new WriteBatch()) {
-      batch.put(destTable.getHandle(), key, value);
-      batch.delete(sourceTable.getHandle(), key);
+      batch.put(destTable.getHandle(), destKey, value);
+      batch.delete(sourceTable.getHandle(), sourceKey);
       db.write(writeOptions, batch);
     } catch (RocksDBException rockdbException) {
-      LOG.error("Move of key failed. Key:{}", DFSUtil.bytes2String(key));
-      throw toIOException("Unable to move key: " + DFSUtil.bytes2String(key),
-          rockdbException);
+      LOG.error("Move of key failed. Key:{}", DFSUtil.bytes2String(sourceKey));
+      throw toIOException("Unable to move key: " +
+              DFSUtil.bytes2String(sourceKey), rockdbException);
     }
   }
 
@@ -229,6 +236,15 @@ public class RDBStore implements DBStore {
     }
   }
 
+  @Override
+  public void write(WriteBatch batch) throws IOException {
+    try {
+      db.write(writeOptions, batch);
+    } catch (RocksDBException e) {
+      throw toIOException("Unable to write the batch.", e);
+    }
+  }
+
   @VisibleForTesting
   protected ObjectName getStatMBeanName() {
     return statMBeanName;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-hdds/common/src/main/resources/ozone-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index d3ec4a5..6f296c6 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -1101,7 +1101,7 @@
 
   <property>
     <name>hdds.db.profile</name>
-    <value>DBProfile.SSD</value>
+    <value>SSD</value>
     <tag>OZONE, OM, PERFORMANCE, REQUIRED</tag>
     <description>This property allows user to pick a configuration
     that tunes the RocksDB settings for the hardware it is running

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/ServerUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/ServerUtils.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/ServerUtils.java
index a0e78dc..c6d85d8 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/ServerUtils.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/server/ServerUtils.java
@@ -136,4 +136,9 @@ public final class ServerUtils {
     return dirPath;
   }
 
+  public static void setOzoneMetaDirPath(OzoneConfiguration conf,
+      String path) {
+    conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, path);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
index 00624d5..c632df6 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
@@ -74,7 +74,7 @@ public class ChunkGroupOutputStream extends OutputStream {
   private final
       StorageContainerLocationProtocolClientSideTranslatorPB scmClient;
   private final OmKeyArgs keyArgs;
-  private final int openID;
+  private final long openID;
   private final XceiverClientManager xceiverClientManager;
   private final int chunkSize;
   private final String requestID;
@@ -115,7 +115,7 @@ public class ChunkGroupOutputStream extends OutputStream {
   }
 
   @VisibleForTesting
-  public int getOpenID() {
+  public long getOpenID() {
     return openID;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OpenKeySession.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OpenKeySession.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OpenKeySession.java
index bc364e6..11ee622 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OpenKeySession.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OpenKeySession.java
@@ -23,14 +23,14 @@ package org.apache.hadoop.ozone.om.helpers;
  * that servers can recognize this client, and thus know how to close the key.
  */
 public class OpenKeySession {
-  private final int id;
+  private final long id;
   private final OmKeyInfo keyInfo;
   // the version of the key when it is being opened in this session.
   // a block that has a create version equals to open version means it will
   // be committed only when this open session is closed.
   private long openVersion;
 
-  public OpenKeySession(int id, OmKeyInfo info, long version) {
+  public OpenKeySession(long id, OmKeyInfo info, long version) {
     this.id = id;
     this.keyInfo = info;
     this.openVersion = version;
@@ -44,7 +44,7 @@ public class OpenKeySession {
     return keyInfo;
   }
 
-  public int getId() {
+  public long getId() {
     return id;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java
index b7a099d..edb260a 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java
@@ -148,7 +148,7 @@ public interface OzoneManagerProtocol {
    * @param clientID the client identification
    * @throws IOException
    */
-  void commitKey(OmKeyArgs args, int clientID) throws IOException;
+  void commitKey(OmKeyArgs args, long clientID) throws IOException;
 
   /**
    * Allocate a new block, it is assumed that the client is having an open key
@@ -159,7 +159,7 @@ public interface OzoneManagerProtocol {
    * @return an allocated block
    * @throws IOException
    */
-  OmKeyLocationInfo allocateBlock(OmKeyArgs args, int clientID)
+  OmKeyLocationInfo allocateBlock(OmKeyArgs args, long clientID)
       throws IOException;
 
   /**
@@ -172,9 +172,10 @@ public interface OzoneManagerProtocol {
   OmKeyInfo lookupKey(OmKeyArgs args) throws IOException;
 
   /**
-   * Rename an existing key within a bucket
+   * Rename an existing key within a bucket.
    * @param args the args of the key.
    * @param toKeyName New name to be used for the Key
+   * @throws IOException
    */
   void renameKey(OmKeyArgs args, String toKeyName) throws IOException;
 
@@ -214,7 +215,7 @@ public interface OzoneManagerProtocol {
    * @throws IOException
    */
   List<OmBucketInfo> listBuckets(String volumeName,
-                                 String startBucketName, String bucketPrefix, int maxNumOfBuckets)
+      String startBucketName, String bucketPrefix, int maxNumOfBuckets)
       throws IOException;
 
   /**
@@ -239,7 +240,7 @@ public interface OzoneManagerProtocol {
    * @throws IOException
    */
   List<OmKeyInfo> listKeys(String volumeName,
-                           String bucketName, String startKeyName, String keyPrefix, int maxKeys)
+      String bucketName, String startKeyName, String keyPrefix, int maxKeys)
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
index e557ac5..c0829fa 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
@@ -488,7 +488,7 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
    */
   @Override
   public List<OmBucketInfo> listBuckets(String volumeName,
-                                        String startKey, String prefix, int count) throws IOException {
+      String startKey, String prefix, int count) throws IOException {
     List<OmBucketInfo> buckets = new ArrayList<>();
     ListBucketsRequest.Builder reqBuilder = ListBucketsRequest.newBuilder();
     reqBuilder.setVolumeName(volumeName);
@@ -554,7 +554,7 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
   }
 
   @Override
-  public OmKeyLocationInfo allocateBlock(OmKeyArgs args, int clientID)
+  public OmKeyLocationInfo allocateBlock(OmKeyArgs args, long clientID)
       throws IOException {
     AllocateBlockRequest.Builder req = AllocateBlockRequest.newBuilder();
     KeyArgs keyArgs = KeyArgs.newBuilder()
@@ -579,7 +579,7 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
   }
 
   @Override
-  public void commitKey(OmKeyArgs args, int clientID)
+  public void commitKey(OmKeyArgs args, long clientID)
       throws IOException {
     CommitKeyRequest.Builder req = CommitKeyRequest.newBuilder();
     List<OmKeyLocationInfo> locationInfoList = args.getLocationInfoList();
@@ -708,7 +708,7 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
    */
   @Override
   public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
-                                  String startKey, String prefix, int maxKeys) throws IOException {
+      String startKey, String prefix, int maxKeys) throws IOException {
     List<OmKeyInfo> keys = new ArrayList<>();
     ListKeysRequest.Builder reqBuilder = ListKeysRequest.newBuilder();
     reqBuilder.setVolumeName(volumeName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto b/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
index 51a0a7f..242e3b5 100644
--- a/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
+++ b/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
@@ -273,7 +273,7 @@ message LocateKeyResponse {
     optional KeyInfo keyInfo = 2;
     // clients' followup request may carry this ID for stateful operations (similar
     // to a cookie).
-    optional uint32 ID = 3;
+    optional uint64 ID = 3;
     // TODO : allow specifiying a particular version to read.
     optional uint64 openVersion = 4;
 }
@@ -319,7 +319,7 @@ message ListKeysResponse {
 
 message AllocateBlockRequest {
     required KeyArgs keyArgs = 1;
-    required uint32 clientID = 2;
+    required uint64 clientID = 2;
 }
 
 message AllocateBlockResponse {
@@ -329,7 +329,7 @@ message AllocateBlockResponse {
 
 message CommitKeyRequest {
     required KeyArgs keyArgs = 1;
-    required uint32 clientID = 2;
+    required uint64 clientID = 2;
 }
 
 message CommitKeyResponse {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
index ffdba7e..50d7ec5 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
@@ -1,19 +1,18 @@
 /**
- * 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
+ * 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.
+ * 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.client.rpc;
@@ -69,7 +68,6 @@ public class TestCloseContainerHandlingByClient {
   private static String bucketName;
   private static String keyString;
 
-
   /**
    * Create a MiniDFSCluster for testing.
    * <p>
@@ -80,7 +78,7 @@ public class TestCloseContainerHandlingByClient {
   @BeforeClass
   public static void init() throws Exception {
     conf = new OzoneConfiguration();
-    chunkSize = (int)OzoneConsts.MB;
+    chunkSize = (int) OzoneConsts.MB;
     blockSize = 4 * chunkSize;
     conf.setInt(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY, chunkSize);
     conf.setLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB, (4));
@@ -108,7 +106,7 @@ public class TestCloseContainerHandlingByClient {
   }
 
   private static String fixedLengthString(String string, int length) {
-    return String.format("%1$"+length+ "s", string);
+    return String.format("%1$" + length + "s", string);
   }
 
   @Test
@@ -288,13 +286,13 @@ public class TestCloseContainerHandlingByClient {
 
     ChunkGroupOutputStream groupOutputStream =
         (ChunkGroupOutputStream) outputStream.getOutputStream();
-    int clientId = groupOutputStream.getOpenID();
+    long clientId = groupOutputStream.getOpenID();
     OMMetadataManager metadataManager =
         cluster.getOzoneManager().getMetadataManager();
-    String objectKey =
-        metadataManager.getKeyWithDBPrefix(volumeName, bucketName, keyName);
-    byte[] openKey = metadataManager.getOpenKeyNameBytes(objectKey, clientId);
-    byte[] openKeyData = metadataManager.get(openKey);
+    byte[] openKey =
+        metadataManager.getOpenKeyBytes(
+            volumeName, bucketName, keyName, clientId);
+    byte[] openKeyData = metadataManager.getOpenKeyTable().get(openKey);
     OmKeyInfo keyInfo = OmKeyInfo.getFromProtobuf(
         OzoneManagerProtocolProtos.KeyInfo.parseFrom(openKeyData));
     List<OmKeyLocationInfo> locationInfoList =
@@ -361,7 +359,6 @@ public class TestCloseContainerHandlingByClient {
     is.close();
   }
 
-
   @Test
   public void testBlockWriteViaRatis() throws Exception {
     String keyName = "ratis";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
index 45b3843..f8ad32e 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.util.Time;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -601,6 +602,9 @@ public class TestOzoneRpcClient {
     Assert.assertEquals(toKeyName, key.getName());
   }
 
+  // Listing all volumes in the cluster feature has to be fixed after HDDS-357.
+  // TODO: fix this
+  @Ignore
   @Test
   public void testListVolume() throws IOException, OzoneException {
     String volBase = "vol-" + RandomStringUtils.randomNumeric(3);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSQLCli.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSQLCli.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSQLCli.java
index ab26c00..a3ff6c8 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSQLCli.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSQLCli.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -82,7 +83,8 @@ public class TestOmSQLCli {
   @Parameterized.Parameters
   public static Collection<Object[]> data() {
     return Arrays.asList(new Object[][] {
-        {OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB},
+        // Uncomment the below line if we support leveldb in future.
+        //{OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB},
         {OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_ROCKSDB}
     });
   }
@@ -161,6 +163,9 @@ public class TestOmSQLCli {
     }
   }
 
+  // After HDDS-357, we have to fix SQLCli.
+  // TODO: fix SQLCli
+  @Ignore
   @Test
   public void testOmDB() throws Exception {
     String dbOutPath =  GenericTestUtils.getTempPath(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
index 4908c4d..b6ade60 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
@@ -56,8 +56,8 @@ import org.apache.hadoop.ozone.web.response.ListBuckets;
 import org.apache.hadoop.ozone.web.response.ListKeys;
 import org.apache.hadoop.ozone.web.response.ListVolumes;
 import org.apache.hadoop.util.Time;
-import org.apache.hadoop.utils.MetadataKeyFilters;
-import org.apache.hadoop.utils.MetadataStore;
+import org.apache.hadoop.utils.db.Table;
+import org.apache.hadoop.utils.db.TableIterator;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -75,7 +75,6 @@ import java.nio.file.Paths;
 import java.net.InetSocketAddress;
 import java.text.ParseException;
 import java.util.LinkedList;
-import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 import java.util.List;
@@ -83,8 +82,8 @@ import java.util.UUID;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS;
-import static org.apache.hadoop.ozone.OzoneConsts.DELETING_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConfigKeys
+    .OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
     .OZONE_SCM_CLIENT_ADDRESS_KEY;
@@ -631,13 +630,16 @@ public class TestOzoneManager {
     storageHandler.deleteKey(keyArgs);
     Assert.assertEquals(1 + numKeyDeletes, omMetrics.getNumKeyDeletes());
 
-    // Make sure the deleted key has been renamed.
-    MetadataStore store = cluster.getOzoneManager().
-        getMetadataManager().getStore();
-    List<Map.Entry<byte[], byte[]>> list = store.getRangeKVs(null, 10,
-        new MetadataKeyFilters.KeyPrefixFilter()
-            .addFilter(DELETING_KEY_PREFIX));
-    Assert.assertEquals(1, list.size());
+    // Make sure the deleted key has been moved to the deleted table.
+    OMMetadataManager manager = cluster.getOzoneManager().
+        getMetadataManager();
+
+    try(TableIterator<Table.KeyValue> iter =
+            manager.getDeletedTable().iterator()) {
+      iter.seekToFirst();
+      Table.KeyValue kv = iter.next();
+      Assert.assertNotNull(kv);
+    }
 
     // Delete the key again to test deleting non-existing key.
     try {
@@ -1016,13 +1018,14 @@ public class TestOzoneManager {
       storageHandler.createVolume(createVolumeArgs);
     }
 
-    // Test list all volumes
+    // Test list all volumes - Removed Support for this operation for time
+    // being. TODO: we will need to bring this back if needed.
     UserArgs userArgs0 = new UserArgs(user0, OzoneUtils.getRequestID(),
         null, null, null, null);
-    listVolumeArgs = new ListArgs(userArgs0, "Vol-testListVolumes", 100, null);
-    listVolumeArgs.setRootScan(true);
-    volumes = storageHandler.listVolumes(listVolumeArgs);
-    Assert.assertEquals(20, volumes.getVolumes().size());
+    //listVolumeArgs = new ListArgs(userArgs0,"Vol-testListVolumes", 100, null);
+    // listVolumeArgs.setRootScan(true);
+    // volumes = storageHandler.listVolumes(listVolumeArgs);
+    // Assert.assertEquals(20, volumes.getVolumes().size());
 
     // Test list all volumes belongs to an user
     listVolumeArgs = new ListArgs(userArgs0, null, 100, null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
index 31f9214..3765bc8 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
@@ -221,6 +221,9 @@ public class TestVolume {
     assertTrue(newVol.getCreationTime() > 0);
   }
 
+  // Listing all volumes in the cluster feature has to be fixed after HDDS-357.
+  // TODO: fix this
+  @Ignore
   @Test
   public void testListVolume() throws OzoneException, IOException {
     runTestListVolume(client);
@@ -305,6 +308,9 @@ public class TestVolume {
     assertEquals(volCount / step, pagecount);
   }
 
+  // Listing all volumes in the cluster feature has to be fixed after HDDS-357.
+  // TODO: fix this
+  @Ignore
   @Test
   public void testListVolumes() throws Exception {
     runTestListVolumes(client);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManagerImpl.java
index 4bbce81..d54addd 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManagerImpl.java
@@ -18,12 +18,11 @@ package org.apache.hadoop.ozone.om;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
-import org.apache.hadoop.ozone.om.exceptions.OMException;
-import org.apache.hadoop.ozone.protocol.proto
-    .OzoneManagerProtocolProtos.BucketInfo;
-import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketInfo;
 import org.apache.hadoop.util.Time;
 import org.iq80.leveldb.DBException;
 import org.slf4j.Logger;
@@ -46,9 +45,10 @@ public class BucketManagerImpl implements BucketManager {
 
   /**
    * Constructs BucketManager.
+   *
    * @param metadataManager
    */
-  public BucketManagerImpl(OMMetadataManager metadataManager){
+  public BucketManagerImpl(OMMetadataManager metadataManager) {
     this.metadataManager = metadataManager;
   }
 
@@ -73,6 +73,7 @@ public class BucketManagerImpl implements BucketManager {
 
   /**
    * Creates a bucket.
+   *
    * @param bucketInfo - OmBucketInfo.
    */
   @Override
@@ -86,13 +87,13 @@ public class BucketManagerImpl implements BucketManager {
       byte[] bucketKey = metadataManager.getBucketKey(volumeName, bucketName);
 
       //Check if the volume exists
-      if (metadataManager.get(volumeKey) == null) {
+      if (metadataManager.getVolumeTable().get(volumeKey) == null) {
         LOG.debug("volume: {} not found ", volumeName);
         throw new OMException("Volume doesn't exist",
             OMException.ResultCodes.FAILED_VOLUME_NOT_FOUND);
       }
       //Check if bucket already exists
-      if (metadataManager.get(bucketKey) != null) {
+      if (metadataManager.getBucketTable().get(bucketKey) != null) {
         LOG.debug("bucket: {} already exists ", bucketName);
         throw new OMException("Bucket already exist",
             OMException.ResultCodes.FAILED_BUCKET_ALREADY_EXISTS);
@@ -106,7 +107,8 @@ public class BucketManagerImpl implements BucketManager {
           .setIsVersionEnabled(bucketInfo.getIsVersionEnabled())
           .setCreationTime(Time.now())
           .build();
-      metadataManager.put(bucketKey, omBucketInfo.getProtobuf().toByteArray());
+      metadataManager.getBucketTable().put(bucketKey,
+          omBucketInfo.getProtobuf().toByteArray());
 
       LOG.debug("created bucket: {} in volume: {}", bucketName, volumeName);
     } catch (IOException | DBException ex) {
@@ -134,7 +136,7 @@ public class BucketManagerImpl implements BucketManager {
     metadataManager.readLock().lock();
     try {
       byte[] bucketKey = metadataManager.getBucketKey(volumeName, bucketName);
-      byte[] value = metadataManager.get(bucketKey);
+      byte[] value = metadataManager.getBucketTable().get(bucketKey);
       if (value == null) {
         LOG.debug("bucket: {} not found in volume: {}.", bucketName,
             volumeName);
@@ -155,8 +157,9 @@ public class BucketManagerImpl implements BucketManager {
 
   /**
    * Sets bucket property from args.
+   *
    * @param args - BucketArgs.
-   * @throws IOException
+   * @throws IOException - On Failure.
    */
   @Override
   public void setBucketProperty(OmBucketArgs args) throws IOException {
@@ -167,15 +170,15 @@ public class BucketManagerImpl implements BucketManager {
     try {
       byte[] bucketKey = metadataManager.getBucketKey(volumeName, bucketName);
       //Check if volume exists
-      if(metadataManager.get(metadataManager.getVolumeKey(volumeName)) ==
-          null) {
+      if (metadataManager.getVolumeTable()
+          .get(metadataManager.getVolumeKey(volumeName)) == null) {
         LOG.debug("volume: {} not found ", volumeName);
         throw new OMException("Volume doesn't exist",
             OMException.ResultCodes.FAILED_VOLUME_NOT_FOUND);
       }
-      byte[] value = metadataManager.get(bucketKey);
+      byte[] value = metadataManager.getBucketTable().get(bucketKey);
       //Check if bucket exist
-      if(value == null) {
+      if (value == null) {
         LOG.debug("bucket: {} not found ", bucketName);
         throw new OMException("Bucket doesn't exist",
             OMException.ResultCodes.FAILED_BUCKET_NOT_FOUND);
@@ -187,7 +190,7 @@ public class BucketManagerImpl implements BucketManager {
           .setBucketName(oldBucketInfo.getBucketName());
 
       //Check ACLs to update
-      if(args.getAddAcls() != null || args.getRemoveAcls() != null) {
+      if (args.getAddAcls() != null || args.getRemoveAcls() != null) {
         bucketInfoBuilder.setAcls(getUpdatedAclList(oldBucketInfo.getAcls(),
             args.getRemoveAcls(), args.getAddAcls()));
         LOG.debug("Updating ACLs for bucket: {} in volume: {}",
@@ -218,7 +221,7 @@ public class BucketManagerImpl implements BucketManager {
       }
       bucketInfoBuilder.setCreationTime(oldBucketInfo.getCreationTime());
 
-      metadataManager.put(bucketKey,
+      metadataManager.getBucketTable().put(bucketKey,
           bucketInfoBuilder.build().getProtobuf().toByteArray());
     } catch (IOException | DBException ex) {
       if (!(ex instanceof OMException)) {
@@ -242,10 +245,10 @@ public class BucketManagerImpl implements BucketManager {
    */
   private List<OzoneAcl> getUpdatedAclList(List<OzoneAcl> existingAcls,
       List<OzoneAcl> removeAcls, List<OzoneAcl> addAcls) {
-    if(removeAcls != null && !removeAcls.isEmpty()) {
+    if (removeAcls != null && !removeAcls.isEmpty()) {
       existingAcls.removeAll(removeAcls);
     }
-    if(addAcls != null && !addAcls.isEmpty()) {
+    if (addAcls != null && !addAcls.isEmpty()) {
       addAcls.stream().filter(acl -> !existingAcls.contains(acl)).forEach(
           existingAcls::add);
     }
@@ -254,9 +257,10 @@ public class BucketManagerImpl implements BucketManager {
 
   /**
    * Deletes an existing empty bucket from volume.
+   *
    * @param volumeName - Name of the volume.
    * @param bucketName - Name of the bucket.
-   * @throws IOException
+   * @throws IOException - on Failure.
    */
   public void deleteBucket(String volumeName, String bucketName)
       throws IOException {
@@ -264,16 +268,17 @@ public class BucketManagerImpl implements BucketManager {
     Preconditions.checkNotNull(bucketName);
     metadataManager.writeLock().lock();
     try {
-      byte[] bucketKey = metadataManager.getBucketKey(volumeName, bucketName);
       //Check if volume exists
-      if (metadataManager.get(metadataManager.getVolumeKey(volumeName))
-          == null) {
+      if (metadataManager.getVolumeTable()
+          .get(metadataManager.getVolumeKey(volumeName)) == null) {
         LOG.debug("volume: {} not found ", volumeName);
         throw new OMException("Volume doesn't exist",
             OMException.ResultCodes.FAILED_VOLUME_NOT_FOUND);
       }
-      //Check if bucket exist
-      if (metadataManager.get(bucketKey) == null) {
+
+      //Check if bucket exists
+      byte[] bucketKey = metadataManager.getBucketKey(volumeName, bucketName);
+      if (metadataManager.getBucketTable().get(bucketKey) == null) {
         LOG.debug("bucket: {} not found ", bucketName);
         throw new OMException("Bucket doesn't exist",
             OMException.ResultCodes.FAILED_BUCKET_NOT_FOUND);
@@ -284,7 +289,7 @@ public class BucketManagerImpl implements BucketManager {
         throw new OMException("Bucket is not empty",
             OMException.ResultCodes.FAILED_BUCKET_NOT_EMPTY);
       }
-      metadataManager.delete(bucketKey);
+      metadataManager.getBucketTable().delete(bucketKey);
     } catch (IOException ex) {
       if (!(ex instanceof OMException)) {
         LOG.error("Delete bucket failed for bucket:{} in volume:{}", bucketName,
@@ -301,7 +306,7 @@ public class BucketManagerImpl implements BucketManager {
    */
   @Override
   public List<OmBucketInfo> listBuckets(String volumeName,
-                                        String startBucket, String bucketPrefix, int maxNumOfBuckets)
+      String startBucket, String bucketPrefix, int maxNumOfBuckets)
       throws IOException {
     Preconditions.checkNotNull(volumeName);
     metadataManager.readLock().lock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java
index 226c07d..a512d7b 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java
@@ -49,7 +49,7 @@ public interface KeyManager {
    * @param clientID the client that is committing.
    * @throws IOException
    */
-  void commitKey(OmKeyArgs args, int clientID) throws IOException;
+  void commitKey(OmKeyArgs args, long clientID) throws IOException;
 
   /**
    * A client calls this on an open key, to request to allocate a new block,
@@ -60,7 +60,7 @@ public interface KeyManager {
    * @return the reference to the new block.
    * @throws IOException
    */
-  OmKeyLocationInfo allocateBlock(OmKeyArgs args, int clientID)
+  OmKeyLocationInfo allocateBlock(OmKeyArgs args, long clientID)
       throws IOException;
   /**
    * Given the args of a key to put, write an open key entry to meta data.
@@ -128,7 +128,7 @@ public interface KeyManager {
    * @throws IOException
    */
   List<OmKeyInfo> listKeys(String volumeName,
-                           String bucketName, String startKey, String keyPrefix, int maxKeys)
+      String bucketName, String startKey, String keyPrefix, int maxKeys)
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
index d0561d6..d585523 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
@@ -17,24 +17,25 @@
 package org.apache.hadoop.ozone.om;
 
 import com.google.common.base.Preconditions;
-import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
+import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
+import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.common.BlockGroup;
-import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
-import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
-import org.apache.hadoop.ozone.protocol.proto
-    .OzoneManagerProtocolProtos.KeyInfo;
-import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
-import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
 import org.apache.hadoop.util.Time;
-import org.apache.hadoop.utils.BatchOperation;
-import org.iq80.leveldb.DBException;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.WriteBatch;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -42,25 +43,13 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
-import java.util.Random;
-
-import static org.apache.hadoop.ozone
-    .OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT;
-import static org.apache.hadoop.ozone
-    .OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY;
-import static org.apache.hadoop.ozone
-    .OzoneConfigKeys.OZONE_KEY_PREALLOCATION_MAXSIZE;
-import static org.apache.hadoop.ozone
-    .OzoneConfigKeys.OZONE_KEY_PREALLOCATION_MAXSIZE_DEFAULT;
-import static org.apache.hadoop.ozone
-    .OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT;
-import static org.apache.hadoop.ozone
-    .OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB;
-import org.apache.hadoop.hdds.protocol
-    .proto.HddsProtos.ReplicationType;
-import org.apache.hadoop.hdds.protocol
-    .proto.HddsProtos.ReplicationFactor;
 
+import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_KEY_PREALLOCATION_MAXSIZE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_KEY_PREALLOCATION_MAXSIZE_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB;
 
 /**
  * Implementation of keyManager.
@@ -78,13 +67,12 @@ public class KeyManagerImpl implements KeyManager {
   private final boolean useRatis;
 
   private final long preallocateMax;
-  private final Random random;
   private final String omId;
 
   public KeyManagerImpl(ScmBlockLocationProtocol scmBlockClient,
-                        OMMetadataManager metadataManager,
-                        OzoneConfiguration conf,
-                        String omId) {
+      OMMetadataManager metadataManager,
+      OzoneConfiguration conf,
+      String omId) {
     this.scmBlockClient = scmBlockClient;
     this.metadataManager = metadataManager;
     this.scmBlockSize = conf.getLong(OZONE_SCM_BLOCK_SIZE_IN_MB,
@@ -94,11 +82,9 @@ public class KeyManagerImpl implements KeyManager {
     this.preallocateMax = conf.getLong(
         OZONE_KEY_PREALLOCATION_MAXSIZE,
         OZONE_KEY_PREALLOCATION_MAXSIZE_DEFAULT);
-    random = new Random();
     this.omId = omId;
   }
 
-
   @Override
   public void start() {
   }
@@ -113,13 +99,13 @@ public class KeyManagerImpl implements KeyManager {
     byte[] bucketKey = metadataManager.getBucketKey(volumeName, bucketName);
 
     //Check if the volume exists
-    if(metadataManager.get(volumeKey) == null) {
+    if (metadataManager.getVolumeTable().get(volumeKey) == null) {
       LOG.error("volume not found: {}", volumeName);
       throw new OMException("Volume not found",
           OMException.ResultCodes.FAILED_VOLUME_NOT_FOUND);
     }
     //Check if bucket already exists
-    if(metadataManager.get(bucketKey) == null) {
+    if (metadataManager.getBucketTable().get(bucketKey) == null) {
       LOG.error("bucket not found: {}/{} ", volumeName, bucketName);
       throw new OMException("Bucket not found",
           OMException.ResultCodes.FAILED_BUCKET_NOT_FOUND);
@@ -127,7 +113,7 @@ public class KeyManagerImpl implements KeyManager {
   }
 
   @Override
-  public OmKeyLocationInfo allocateBlock(OmKeyArgs args, int clientID)
+  public OmKeyLocationInfo allocateBlock(OmKeyArgs args, long clientID)
       throws IOException {
     Preconditions.checkNotNull(args);
     metadataManager.writeLock().lock();
@@ -137,13 +123,13 @@ public class KeyManagerImpl implements KeyManager {
 
     try {
       validateBucket(volumeName, bucketName);
-      String objectKey = metadataManager.getKeyWithDBPrefix(
-          volumeName, bucketName, keyName);
-      byte[] openKey = metadataManager.getOpenKeyNameBytes(objectKey, clientID);
-      byte[] keyData = metadataManager.get(openKey);
+      byte[] openKey = metadataManager.getOpenKeyBytes(
+          volumeName, bucketName, keyName, clientID);
+
+      byte[] keyData = metadataManager.getOpenKeyTable().get(openKey);
       if (keyData == null) {
-        LOG.error("Allocate block for a key not in open status in meta store " +
-            objectKey + " with ID " + clientID);
+        LOG.error("Allocate block for a key not in open status in meta store" +
+            " /{}/{}/{} with ID {}", volumeName, bucketName, keyName, clientID);
         throw new OMException("Open Key not found",
             OMException.ResultCodes.FAILED_KEY_NOT_FOUND);
       }
@@ -162,7 +148,8 @@ public class KeyManagerImpl implements KeyManager {
       // the same version
       keyInfo.appendNewBlocks(Collections.singletonList(info));
       keyInfo.updateModifcationTime();
-      metadataManager.put(openKey, keyInfo.getProtobuf().toByteArray());
+      metadataManager.getOpenKeyTable().put(openKey,
+          keyInfo.getProtobuf().toByteArray());
       return info;
     } finally {
       metadataManager.writeLock().unlock();
@@ -172,28 +159,30 @@ public class KeyManagerImpl implements KeyManager {
   @Override
   public OpenKeySession openKey(OmKeyArgs args) throws IOException {
     Preconditions.checkNotNull(args);
-    metadataManager.writeLock().lock();
     String volumeName = args.getVolumeName();
     String bucketName = args.getBucketName();
+    validateBucket(volumeName, bucketName);
+
+    metadataManager.writeLock().lock();
     String keyName = args.getKeyName();
     ReplicationFactor factor = args.getFactor();
     ReplicationType type = args.getType();
+    long currentTime = Time.monotonicNowNanos();
 
     // If user does not specify a replication strategy or
     // replication factor, OM will use defaults.
-    if(factor == null) {
-      factor = useRatis ? ReplicationFactor.THREE: ReplicationFactor.ONE;
+    if (factor == null) {
+      factor = useRatis ? ReplicationFactor.THREE : ReplicationFactor.ONE;
     }
 
-    if(type == null) {
+    if (type == null) {
       type = useRatis ? ReplicationType.RATIS : ReplicationType.STAND_ALONE;
     }
 
     try {
-      validateBucket(volumeName, bucketName);
       long requestedSize = Math.min(preallocateMax, args.getDataSize());
       List<OmKeyLocationInfo> locations = new ArrayList<>();
-      String objectKey = metadataManager.getKeyWithDBPrefix(
+      byte[] objectKey = metadataManager.getOzoneKeyBytes(
           volumeName, bucketName, keyName);
       // requested size is not required but more like a optimization:
       // SCM looks at the requested, if it 0, no block will be allocated at
@@ -218,9 +207,7 @@ public class KeyManagerImpl implements KeyManager {
       // value, then this value is used, otherwise, we allocate a single block
       // which is the current size, if read by the client.
       long size = args.getDataSize() >= 0 ? args.getDataSize() : scmBlockSize;
-      byte[] keyKey = metadataManager.getDBKeyBytes(
-          volumeName, bucketName, keyName);
-      byte[] value = metadataManager.get(keyKey);
+      byte[] value = metadataManager.getKeyTable().get(objectKey);
       OmKeyInfo keyInfo;
       long openVersion;
       if (value != null) {
@@ -233,7 +220,7 @@ public class KeyManagerImpl implements KeyManager {
       } else {
         // the key does not exist, create a new object, the new blocks are the
         // version 0
-        long currentTime = Time.now();
+
         keyInfo = new OmKeyInfo.Builder()
             .setVolumeName(args.getVolumeName())
             .setBucketName(args.getBucketName())
@@ -248,31 +235,31 @@ public class KeyManagerImpl implements KeyManager {
             .build();
         openVersion = 0;
       }
-      // Generate a random ID which is not already in meta db.
-      int id = -1;
-      // in general this should finish in a couple times at most. putting some
-      // arbitrary large number here to avoid dead loop.
-      for (int j = 0; j < 10000; j++) {
-        id = random.nextInt();
-        byte[] openKey = metadataManager.getOpenKeyNameBytes(objectKey, id);
-        if (metadataManager.get(openKey) == null) {
-          metadataManager.put(openKey, keyInfo.getProtobuf().toByteArray());
-          break;
-        }
-      }
-      if (id == -1) {
-        throw new IOException("Failed to find a usable id for " + objectKey);
+      byte[] openKey = metadataManager.getOpenKeyBytes(
+          volumeName, bucketName, keyName, currentTime);
+      if (metadataManager.getOpenKeyTable().get(openKey) != null) {
+        // This should not happen. If this condition is satisfied, it means
+        // that we have generated a same openKeyId (i.e. currentTime) for two
+        // different client who are trying to write the same key at the same
+        // time. The chance of this happening is very, very minimal.
+
+        // Do we really need this check? Can we avoid this to gain some
+        // minor performance improvement?
+        LOG.warn("Cannot allocate key. The generated open key id is already" +
+            "used for the same key which is currently being written.");
+        throw new OMException("Cannot allocate key. Not able to get a valid" +
+            "open key id.", OMException.ResultCodes.FAILED_KEY_ALLOCATION);
       }
+      metadataManager.getOpenKeyTable().put(openKey,
+          keyInfo.getProtobuf().toByteArray());
       LOG.debug("Key {} allocated in volume {} bucket {}",
           keyName, volumeName, bucketName);
-      return new OpenKeySession(id, keyInfo, openVersion);
+      return new OpenKeySession(currentTime, keyInfo, openVersion);
     } catch (OMException e) {
       throw e;
     } catch (IOException ex) {
-      if (!(ex instanceof OMException)) {
-        LOG.error("Key open failed for volume:{} bucket:{} key:{}",
-            volumeName, bucketName, keyName, ex);
-      }
+      LOG.error("Key open failed for volume:{} bucket:{} key:{}",
+          volumeName, bucketName, keyName, ex);
       throw new OMException(ex.getMessage(),
           OMException.ResultCodes.FAILED_KEY_ALLOCATION);
     } finally {
@@ -281,7 +268,7 @@ public class KeyManagerImpl implements KeyManager {
   }
 
   @Override
-  public void commitKey(OmKeyArgs args, int clientID) throws IOException {
+  public void commitKey(OmKeyArgs args, long clientID) throws IOException {
     Preconditions.checkNotNull(args);
     metadataManager.writeLock().lock();
     String volumeName = args.getVolumeName();
@@ -289,15 +276,14 @@ public class KeyManagerImpl implements KeyManager {
     String keyName = args.getKeyName();
     try {
       validateBucket(volumeName, bucketName);
-      String objectKey = metadataManager.getKeyWithDBPrefix(
+      byte[] openKey = metadataManager.getOpenKeyBytes(volumeName, bucketName,
+          keyName, clientID);
+      byte[] objectKey = metadataManager.getOzoneKeyBytes(
           volumeName, bucketName, keyName);
-      byte[] objectKeyBytes = metadataManager.getDBKeyBytes(volumeName,
-          bucketName, keyName);
-      byte[] openKey = metadataManager.getOpenKeyNameBytes(objectKey, clientID);
-      byte[] openKeyData = metadataManager.get(openKey);
+      byte[] openKeyData = metadataManager.getOpenKeyTable().get(openKey);
       if (openKeyData == null) {
         throw new OMException("Commit a key without corresponding entry " +
-            DFSUtil.bytes2String(openKey), ResultCodes.FAILED_KEY_NOT_FOUND);
+            DFSUtil.bytes2String(objectKey), ResultCodes.FAILED_KEY_NOT_FOUND);
       }
       OmKeyInfo keyInfo =
           OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(openKeyData));
@@ -305,12 +291,13 @@ public class KeyManagerImpl implements KeyManager {
       keyInfo.setModificationTime(Time.now());
       List<OmKeyLocationInfo> locationInfoList = args.getLocationInfoList();
       Preconditions.checkNotNull(locationInfoList);
+
       //update the block length for each block
       keyInfo.updateLocationInfoList(locationInfoList);
-      BatchOperation batch = new BatchOperation();
-      batch.delete(openKey);
-      batch.put(objectKeyBytes, keyInfo.getProtobuf().toByteArray());
-      metadataManager.writeBatch(batch);
+      metadataManager.getStore().move(openKey, objectKey,
+          keyInfo.getProtobuf().toByteArray(),
+          metadataManager.getOpenKeyTable(),
+          metadataManager.getKeyTable());
     } catch (OMException e) {
       throw e;
     } catch (IOException ex) {
@@ -331,9 +318,9 @@ public class KeyManagerImpl implements KeyManager {
     String bucketName = args.getBucketName();
     String keyName = args.getKeyName();
     try {
-      byte[] keyKey = metadataManager.getDBKeyBytes(
+      byte[] keyBytes = metadataManager.getOzoneKeyBytes(
           volumeName, bucketName, keyName);
-      byte[] value = metadataManager.get(keyKey);
+      byte[] value = metadataManager.getKeyTable().get(keyBytes);
       if (value == null) {
         LOG.debug("volume:{} bucket:{} Key:{} not found",
             volumeName, bucketName, keyName);
@@ -341,7 +328,7 @@ public class KeyManagerImpl implements KeyManager {
             OMException.ResultCodes.FAILED_KEY_NOT_FOUND);
       }
       return OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(value));
-    } catch (DBException ex) {
+    } catch (IOException ex) {
       LOG.error("Get key failed for volume:{} bucket:{} key:{}",
           volumeName, bucketName, keyName, ex);
       throw new OMException(ex.getMessage(),
@@ -368,9 +355,9 @@ public class KeyManagerImpl implements KeyManager {
     metadataManager.writeLock().lock();
     try {
       // fromKeyName should exist
-      byte[] fromKey = metadataManager.getDBKeyBytes(
+      byte[] fromKey = metadataManager.getOzoneKeyBytes(
           volumeName, bucketName, fromKeyName);
-      byte[] fromKeyValue = metadataManager.get(fromKey);
+      byte[] fromKeyValue = metadataManager.getKeyTable().get(fromKey);
       if (fromKeyValue == null) {
         // TODO: Add support for renaming open key
         LOG.error(
@@ -381,10 +368,20 @@ public class KeyManagerImpl implements KeyManager {
             OMException.ResultCodes.FAILED_KEY_NOT_FOUND);
       }
 
+      // A rename is a no-op if the target and source name is same.
+      // TODO: Discuss if we need to throw?.
+      // TODO: Define the semantics of rename more clearly. Today this code
+      // will allow rename of a Key across volumes. This should *not* be
+      // allowed. The documentation of Ozone says that rename is permitted only
+      // within a volume.
+      if (fromKeyName.equals(toKeyName)) {
+        return;
+      }
+
       // toKeyName should not exist
       byte[] toKey =
-          metadataManager.getDBKeyBytes(volumeName, bucketName, toKeyName);
-      byte[] toKeyValue = metadataManager.get(toKey);
+          metadataManager.getOzoneKeyBytes(volumeName, bucketName, toKeyName);
+      byte[] toKeyValue = metadataManager.getKeyTable().get(toKey);
       if (toKeyValue != null) {
         LOG.error(
             "Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}. "
@@ -394,19 +391,18 @@ public class KeyManagerImpl implements KeyManager {
             OMException.ResultCodes.FAILED_KEY_ALREADY_EXISTS);
       }
 
-      if (fromKeyName.equals(toKeyName)) {
-        return;
-      }
 
       OmKeyInfo newKeyInfo =
           OmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(fromKeyValue));
       newKeyInfo.setKeyName(toKeyName);
       newKeyInfo.updateModifcationTime();
-      BatchOperation batch = new BatchOperation();
-      batch.delete(fromKey);
-      batch.put(toKey, newKeyInfo.getProtobuf().toByteArray());
-      metadataManager.writeBatch(batch);
-    } catch (DBException ex) {
+      try (WriteBatch batch = new WriteBatch()) {
+        batch.delete(metadataManager.getKeyTable().getHandle(), fromKey);
+        batch.put(metadataManager.getKeyTable().getHandle(), toKey,
+            newKeyInfo.getProtobuf().toByteArray());
+        metadataManager.getStore().write(batch);
+      }
+    } catch (RocksDBException | IOException ex) {
       LOG.error("Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}",
           volumeName, bucketName, fromKeyName, toKeyName, ex);
       throw new OMException(ex.getMessage(),
@@ -424,19 +420,19 @@ public class KeyManagerImpl implements KeyManager {
     String bucketName = args.getBucketName();
     String keyName = args.getKeyName();
     try {
-      byte[] objectKey = metadataManager.getDBKeyBytes(
+      byte[] objectKey = metadataManager.getOzoneKeyBytes(
           volumeName, bucketName, keyName);
-      byte[] objectValue = metadataManager.get(objectKey);
+      byte[] objectValue = metadataManager.getKeyTable().get(objectKey);
       if (objectValue == null) {
         throw new OMException("Key not found",
             OMException.ResultCodes.FAILED_KEY_NOT_FOUND);
       }
-      byte[] deletingKey = metadataManager.getDeletedKeyName(objectKey);
-      BatchOperation batch = new BatchOperation();
-      batch.put(deletingKey, objectValue);
-      batch.delete(objectKey);
-      metadataManager.writeBatch(batch);
-    } catch (DBException ex) {
+      metadataManager.getStore().move(objectKey,
+          metadataManager.getKeyTable(),
+          metadataManager.getDeletedTable());
+    } catch (OMException ex) {
+      throw ex;
+    } catch (IOException ex) {
       LOG.error(String.format("Delete key failed for volume:%s "
           + "bucket:%s key:%s", volumeName, bucketName, keyName), ex);
       throw new OMException(ex.getMessage(), ex,
@@ -448,53 +444,30 @@ public class KeyManagerImpl implements KeyManager {
 
   @Override
   public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
-                                  String startKey, String keyPrefix,
+      String startKey, String keyPrefix,
       int maxKeys) throws IOException {
     Preconditions.checkNotNull(volumeName);
     Preconditions.checkNotNull(bucketName);
 
-    metadataManager.readLock().lock();
-    try {
-      return metadataManager.listKeys(volumeName, bucketName,
-          startKey, keyPrefix, maxKeys);
-    } finally {
-      metadataManager.readLock().unlock();
-    }
+    // We don't take a lock in this path, since we walk the
+    // underlying table using an iterator. That automatically creates a
+    // snapshot of the data, so we don't need these locks at a higher level
+    // when we iterate.
+    return metadataManager.listKeys(volumeName, bucketName,
+        startKey, keyPrefix, maxKeys);
   }
 
   @Override
   public List<BlockGroup> getPendingDeletionKeys(final int count)
       throws IOException {
-    metadataManager.readLock().lock();
-    try {
-      return metadataManager.getPendingDeletionKeys(count);
-    } finally {
-      metadataManager.readLock().unlock();
-    }
+    //TODO: Fix this in later patches.
+    return null;
   }
 
   @Override
   public void deletePendingDeletionKey(String objectKeyName)
-      throws IOException{
-    Preconditions.checkNotNull(objectKeyName);
-    if (!objectKeyName.startsWith(OzoneConsts.DELETING_KEY_PREFIX)) {
-      throw new IllegalArgumentException("Invalid key name,"
-          + " the name should be the key name with deleting prefix");
-    }
-
-    // Simply removes the entry from OM DB.
-    metadataManager.writeLock().lock();
-    try {
-      byte[] pendingDelKey = DFSUtil.string2Bytes(objectKeyName);
-      byte[] delKeyValue = metadataManager.get(pendingDelKey);
-      if (delKeyValue == null) {
-        throw new IOException("Failed to delete key " + objectKeyName
-            + " because it is not found in DB");
-      }
-      metadataManager.delete(pendingDelKey);
-    } finally {
-      metadataManager.writeLock().unlock();
-    }
+      throws IOException {
+    // TODO : Fix in later patches.
   }
 
   @Override
@@ -510,23 +483,6 @@ public class KeyManagerImpl implements KeyManager {
   @Override
   public void deleteExpiredOpenKey(String objectKeyName) throws IOException {
     Preconditions.checkNotNull(objectKeyName);
-    if (!objectKeyName.startsWith(OzoneConsts.OPEN_KEY_PREFIX)) {
-      throw new IllegalArgumentException("Invalid key name,"
-          + " the name should be the key name with open key prefix");
-    }
-
-    // Simply removes the entry from OM DB.
-    metadataManager.writeLock().lock();
-    try {
-      byte[] openKey = DFSUtil.string2Bytes(objectKeyName);
-      byte[] delKeyValue = metadataManager.get(openKey);
-      if (delKeyValue == null) {
-        throw new IOException("Failed to delete key " + objectKeyName
-            + " because it is not found in DB");
-      }
-      metadataManager.delete(openKey);
-    } finally {
-      metadataManager.writeLock().unlock();
-    }
+    // TODO: Fix this in later patches.
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff036e49/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
index f2e78e6..0e9ae42 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
@@ -17,12 +17,12 @@
 package org.apache.hadoop.ozone.om;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.common.BlockGroup;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
-import org.apache.hadoop.utils.BatchOperation;
-import org.apache.hadoop.utils.MetadataStore;
+import org.apache.hadoop.utils.db.DBStore;
+import org.apache.hadoop.utils.db.Table;
 
 import java.io.IOException;
 import java.util.List;
@@ -40,68 +40,47 @@ public interface OMMetadataManager {
   /**
    * Stop metadata manager.
    */
-  void stop() throws IOException;
+  void stop() throws Exception;
 
   /**
    * Get metadata store.
+   *
    * @return metadata store.
    */
   @VisibleForTesting
-  MetadataStore getStore();
+  DBStore getStore();
 
   /**
    * Returns the read lock used on Metadata DB.
+   *
    * @return readLock
    */
   Lock readLock();
 
   /**
    * Returns the write lock used on Metadata DB.
+   *
    * @return writeLock
    */
   Lock writeLock();
 
   /**
-   * Returns the value associated with this key.
-   * @param key - key
-   * @return value
-   */
-  byte[] get(byte[] key) throws IOException;
-
-  /**
-   * Puts a Key into Metadata DB.
-   * @param key   - key
-   * @param value - value
-   */
-  void put(byte[] key, byte[] value) throws IOException;
-
-  /**
-   * Deletes a Key from Metadata DB.
-   * @param key   - key
-   */
-  void delete(byte[] key) throws IOException;
-
-  /**
-   * Atomic write a batch of operations.
-   * @param batch
-   * @throws IOException
-   */
-  void writeBatch(BatchOperation batch) throws IOException;
-
-  /**
    * Given a volume return the corresponding DB key.
+   *
    * @param volume - Volume name
    */
   byte[] getVolumeKey(String volume);
 
   /**
    * Given a user return the corresponding DB key.
+   *
    * @param user - User name
    */
   byte[] getUserKey(String user);
 
   /**
    * Given a volume and bucket, return the corresponding DB key.
+   *
    * @param volume - User name
    * @param bucket - Bucket name
    */
@@ -109,131 +88,103 @@ public interface OMMetadataManager {
 
   /**
    * Given a volume, bucket and a key, return the corresponding DB key.
+   *
    * @param volume - volume name
    * @param bucket - bucket name
    * @param key - key name
    * @return bytes of DB key.
    */
-  byte[] getDBKeyBytes(String volume, String bucket, String key);
-
-  /**
-   * Returns the DB key name of a deleted key in OM metadata store.
-   * The name for a deleted key has prefix #deleting# followed by
-   * the actual key name.
-   * @param keyName - key name
-   * @return bytes of DB key.
-   */
-  byte[] getDeletedKeyName(byte[] keyName);
+  byte[] getOzoneKeyBytes(String volume, String bucket, String key);
 
   /**
-   * Returns the DB key name of a open key in OM metadata store.
-   * Should be #open# prefix followed by actual key name.
-   * @param keyName - key name
+   * Returns the DB key name of a open key in OM metadata store. Should be
+   * #open# prefix followed by actual key name.
+   *
+   * @param volume - volume name
+   * @param bucket - bucket name
+   * @param key - key name
    * @param id - the id for this open
    * @return bytes of DB key.
    */
-  byte[] getOpenKeyNameBytes(String keyName, int id);
+  byte[] getOpenKeyBytes(String volume, String bucket, String key, long id);
 
   /**
-   * Returns the full name of a key given volume name, bucket name and key name.
-   * Generally done by padding certain delimiters.
+   * Given a volume, check if it is empty, i.e there are no buckets inside it.
    *
-   * @param volumeName - volume name
-   * @param bucketName - bucket name
-   * @param keyName - key name
-   * @return the full key name.
-   */
-  String getKeyWithDBPrefix(String volumeName, String bucketName,
-      String keyName);
-
-  /**
-   * Given a volume, check if it is empty,
-   * i.e there are no buckets inside it.
    * @param volume - Volume name
    */
   boolean isVolumeEmpty(String volume) throws IOException;
 
   /**
-   * Given a volume/bucket, check if it is empty,
-   * i.e there are no keys inside it.
+   * Given a volume/bucket, check if it is empty, i.e there are no keys inside
+   * it.
+   *
    * @param volume - Volume name
-   * @param  bucket - Bucket name
+   * @param bucket - Bucket name
    * @return true if the bucket is empty
    */
   boolean isBucketEmpty(String volume, String bucket) throws IOException;
 
   /**
-   * Returns a list of buckets represented by {@link OmBucketInfo}
-   * in the given volume.
-   *
-   * @param volumeName
-   *   the name of the volume. This argument is required,
-   *   this method returns buckets in this given volume.
-   * @param startBucket
-   *   the start bucket name. Only the buckets whose name is
-   *   after this value will be included in the result.
-   *   This key is excluded from the result.
-   * @param bucketPrefix
-   *   bucket name prefix. Only the buckets whose name has
-   *   this prefix will be included in the result.
-   * @param maxNumOfBuckets
-   *   the maximum number of buckets to return. It ensures
-   *   the size of the result will not exceed this limit.
+   * Returns a list of buckets represented by {@link OmBucketInfo} in the given
+   * volume.
+   *
+   * @param volumeName the name of the volume. This argument is required, this
+   * method returns buckets in this given volume.
+   * @param startBucket the start bucket name. Only the buckets whose name is
+   * after this value will be included in the result. This key is excluded from
+   * the result.
+   * @param bucketPrefix bucket name prefix. Only the buckets whose name has
+   * this prefix will be included in the result.
+   * @param maxNumOfBuckets the maximum number of buckets to return. It ensures
+   * the size of the result will not exceed this limit.
    * @return a list of buckets.
    * @throws IOException
    */
   List<OmBucketInfo> listBuckets(String volumeName, String startBucket,
-                                 String bucketPrefix, int maxNumOfBuckets) throws IOException;
-
-  /**
-   * Returns a list of keys represented by {@link OmKeyInfo}
-   * in the given bucket.
-   *
-   * @param volumeName
-   *   the name of the volume.
-   * @param bucketName
-   *   the name of the bucket.
-   * @param startKey
-   *   the start key name, only the keys whose name is
-   *   after this value will be included in the result.
-   *   This key is excluded from the result.
-   * @param keyPrefix
-   *   key name prefix, only the keys whose name has
-   *   this prefix will be included in the result.
-   * @param maxKeys
-   *   the maximum number of keys to return. It ensures
-   *   the size of the result will not exceed this limit.
+      String bucketPrefix, int maxNumOfBuckets)
+      throws IOException;
+
+  /**
+   * Returns a list of keys represented by {@link OmKeyInfo} in the given
+   * bucket.
+   *
+   * @param volumeName the name of the volume.
+   * @param bucketName the name of the bucket.
+   * @param startKey the start key name, only the keys whose name is after this
+   * value will be included in the result. This key is excluded from the
+   * result.
+   * @param keyPrefix key name prefix, only the keys whose name has this prefix
+   * will be included in the result.
+   * @param maxKeys the maximum number of keys to return. It ensures the size of
+   * the result will not exceed this limit.
    * @return a list of keys.
    * @throws IOException
    */
   List<OmKeyInfo> listKeys(String volumeName,
-                           String bucketName, String startKey, String keyPrefix, int maxKeys)
+      String bucketName, String startKey, String keyPrefix, int maxKeys)
       throws IOException;
 
   /**
-   * Returns a list of volumes owned by a given user; if user is null,
-   * returns all volumes.
+   * Returns a list of volumes owned by a given user; if user is null, returns
+   * all volumes.
    *
-   * @param userName
-   *   volume owner
-   * @param prefix
-   *   the volume prefix used to filter the listing result.
-   * @param startKey
-   *   the start volume name determines where to start listing from,
-   *   this key is excluded from the result.
-   * @param maxKeys
-   *   the maximum number of volumes to return.
+   * @param userName volume owner
+   * @param prefix the volume prefix used to filter the listing result.
+   * @param startKey the start volume name determines where to start listing
+   * from, this key is excluded from the result.
+   * @param maxKeys the maximum number of volumes to return.
    * @return a list of {@link OmVolumeArgs}
    * @throws IOException
    */
   List<OmVolumeArgs> listVolumes(String userName, String prefix,
-                                 String startKey, int maxKeys) throws IOException;
+      String startKey, int maxKeys) throws IOException;
 
   /**
    * Returns a list of pending deletion key info that ups to the given count.
-   * Each entry is a {@link BlockGroup}, which contains the info about the
-   * key name and all its associated block IDs. A pending deletion key is
-   * stored with #deleting# prefix in OM DB.
+   * Each entry is a {@link BlockGroup}, which contains the info about the key
+   * name and all its associated block IDs. A pending deletion key is stored
+   * with #deleting# prefix in OM DB.
    *
    * @param count max number of keys to return.
    * @return a list of {@link BlockGroup} represent keys and blocks.
@@ -250,4 +201,47 @@ public interface OMMetadataManager {
    * @throws IOException
    */
   List<BlockGroup> getExpiredOpenKeys() throws IOException;
+
+  /**
+   * Returns the user Table.
+   *
+   * @return UserTable.
+   */
+  Table getUserTable();
+
+  /**
+   * Returns the Volume Table.
+   *
+   * @return VolumeTable.
+   */
+  Table getVolumeTable();
+
+  /**
+   * Returns the BucketTable.
+   *
+   * @return BucketTable.
+   */
+  Table getBucketTable();
+
+  /**
+   * Returns the KeyTable.
+   *
+   * @return KeyTable.
+   */
+  Table getKeyTable();
+
+  /**
+   * Get Deleted Table.
+   *
+   * @return Deleted Table.
+   */
+  Table getDeletedTable();
+
+  /**
+   * Gets the OpenKeyTable.
+   *
+   * @return Table.
+   */
+  Table getOpenKeyTable();
+
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[27/50] [abbrv] hadoop git commit: HADOOP-15705. Typo in the definition of "stable" in the interface classification

Posted by eh...@apache.org.
HADOOP-15705. Typo in the definition of "stable" in the interface classification

Change-Id: I3eae2143400a534903db4f186400561fc8d2bd56


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d53a10b0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d53a10b0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d53a10b0

Branch: refs/heads/HDFS-12090
Commit: d53a10b0a552155de700e396fd7f450a4c5f9c22
Parents: 692736f
Author: Daniel Templeton <te...@apache.org>
Authored: Wed Aug 29 13:59:32 2018 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Wed Aug 29 13:59:32 2018 -0700

----------------------------------------------------------------------
 .../hadoop-common/src/site/markdown/InterfaceClassification.md     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d53a10b0/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md b/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
index a21e28b..7348044 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
@@ -124,7 +124,7 @@ hence serves as a safe development target. A Stable interface may evolve
 compatibly between minor releases.
 
 Incompatible changes allowed: major (X.0.0)
-Compatible changes allowed: maintenance (x.Y.0)
+Compatible changes allowed: maintenance (x.y.Z)
 
 #### Evolving
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[20/50] [abbrv] hadoop git commit: HDDS-365. Implement flushStateMachineData for containerStateMachine. Contributed by Shashikant Banerjee.

Posted by eh...@apache.org.
HDDS-365. Implement flushStateMachineData for containerStateMachine. Contributed by Shashikant Banerjee.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2651e2c4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2651e2c4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2651e2c4

Branch: refs/heads/HDFS-12090
Commit: 2651e2c43d0825912669a87afc256bad9f1ea6ed
Parents: 7ed458b
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Wed Aug 29 07:57:57 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Wed Aug 29 07:58:30 2018 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hdds/scm/XceiverClientGrpc.java |  2 +-
 .../transport/server/XceiverServerGrpc.java       |  2 +-
 .../server/ratis/ContainerStateMachine.java       | 18 ++++++++++++++++++
 hadoop-project/pom.xml                            |  2 +-
 4 files changed, 21 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2651e2c4/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
index e2416c2..1622ddb 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java
@@ -87,7 +87,7 @@ public class XceiverClientGrpc extends XceiverClientSpi {
     }
     LOG.debug("Connecting to server Port : " + leader.getIpAddress());
     channel = NettyChannelBuilder.forAddress(leader.getIpAddress(), port)
-        .usePlaintext(true)
+        .usePlaintext()
         .maxInboundMessageSize(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE)
         .build();
     asyncStub = XceiverClientProtocolServiceGrpc.newStub(channel);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2651e2c4/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java
index f4f3f6f..4dc232d 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java
@@ -77,7 +77,7 @@ public final class XceiverServerGrpc implements XceiverServerSpi {
     datanodeDetails.setPort(
         DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, port));
     server = ((NettyServerBuilder) ServerBuilder.forPort(port))
-        .maxMessageSize(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE)
+        .maxInboundMessageSize(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE)
         .addService(new GrpcXceiverService(dispatcher))
         .build();
     storageContainer = dispatcher;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2651e2c4/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
index ede87f4..68d6d5b 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
@@ -59,6 +59,7 @@ import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ThreadPoolExecutor;
+import java.util.stream.Collectors;
 
 /** A {@link org.apache.ratis.statemachine.StateMachine} for containers.
  *
@@ -316,6 +317,23 @@ public class ContainerStateMachine extends BaseStateMachine {
     return LogEntryProto.newBuilder().setSmLogEntry(log).build();
   }
 
+  /**
+   * Returns the combined future of all the writeChunks till the given log
+   * index. The Raft log worker will wait for the stateMachineData to complete
+   * flush as well.
+   *
+   * @param index log index till which the stateMachine data needs to be flushed
+   * @return Combined future of all writeChunks till the log index given.
+   */
+  @Override
+  public CompletableFuture<Void> flushStateMachineData(long index) {
+    List<CompletableFuture<Message>> futureList =
+        writeChunkFutureMap.entrySet().stream().filter(x -> x.getKey() <= index)
+            .map(x -> x.getValue()).collect(Collectors.toList());
+    CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(
+        futureList.toArray(new CompletableFuture[futureList.size()]));
+    return combinedFuture;
+  }
   /*
    * This api is used by the leader while appending logs to the follower
    * This allows the leader to read the state machine data from the

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2651e2c4/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index b45b495..18b907f 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -97,7 +97,7 @@
     <ldap-api.version>1.0.0-M33</ldap-api.version>
 
     <!-- Apache Ratis version -->
-    <ratis.version>0.3.0-e4a016f-SNAPSHOT</ratis.version>
+    <ratis.version>0.3.0-e6fd494-SNAPSHOT</ratis.version>
     <jcache.version>1.0-alpha-1</jcache.version>
     <ehcache.version>3.3.1</ehcache.version>
     <hikari.version>2.4.12</hikari.version>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[42/50] [abbrv] hadoop git commit: YARN-8535. Fix DistributedShell unit tests. Contributed by Abhishek Modi.

Posted by eh...@apache.org.
YARN-8535. Fix DistributedShell unit tests. Contributed by Abhishek Modi.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/eed8415d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/eed8415d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/eed8415d

Branch: refs/heads/HDFS-12090
Commit: eed8415dc18fa7415ebd105350bd0532b3b1b6bb
Parents: 6edf3d2
Author: bibinchundatt <bi...@apache.org>
Authored: Sun Sep 2 13:35:52 2018 +0530
Committer: bibinchundatt <bi...@apache.org>
Committed: Sun Sep 2 13:35:52 2018 +0530

----------------------------------------------------------------------
 .../yarn/applications/distributedshell/ApplicationMaster.java    | 2 +-
 .../yarn/applications/distributedshell/TestDistributedShell.java | 1 +
 .../test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java | 4 +++-
 3 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/eed8415d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
index 76fa38f..ecf07b1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
@@ -944,7 +944,7 @@ public class ApplicationMaster {
 
     // When the application completes, it should send a finish application
     // signal to the RM
-    LOG.info("Application completed. Signalling finish to RM");
+    LOG.info("Application completed. Signalling finished to RM");
 
     FinalApplicationStatus appStatus;
     boolean success = true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eed8415d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
index 3a98a22..c7e1cf1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
@@ -624,6 +624,7 @@ public class TestDistributedShell {
       String entityfileName) {
     String outputDirPathForEntity =
         basePath + File.separator + entityType + File.separator;
+    LOG.info(outputDirPathForEntity);
     File outputDirForEntity = new File(outputDirPathForEntity);
     Assert.assertTrue(outputDirForEntity.isDirectory());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eed8415d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
index 0395138..fa69f18 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
@@ -577,7 +577,9 @@ public class MiniYARNCluster extends CompositeService {
       config.set(YarnConfiguration.NM_LOCALIZER_ADDRESS,
           MiniYARNCluster.getHostname() + ":0");
       config.set(YarnConfiguration.NM_COLLECTOR_SERVICE_ADDRESS,
-          MiniYARNCluster.getHostname() + ":0");
+          MiniYARNCluster.getHostname() + ":" +
+              ServerSocketUtil.getPort(
+                  YarnConfiguration.DEFAULT_NM_COLLECTOR_SERVICE_PORT, 10));
       WebAppUtils
           .setNMWebAppHostNameAndPort(config,
               MiniYARNCluster.getHostname(), 0);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[36/50] [abbrv] hadoop git commit: HDDS-98. Adding Ozone Manager Audit Log. Contributed by Dinesh Chitlangia.

Posted by eh...@apache.org.
HDDS-98. Adding Ozone Manager Audit Log.
Contributed by Dinesh Chitlangia.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/630b64ec
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/630b64ec
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/630b64ec

Branch: refs/heads/HDFS-12090
Commit: 630b64ec7e963968a5bdcd1d625fc78746950137
Parents: 8aa6c4f
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Aug 31 14:20:56 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Fri Aug 31 14:20:56 2018 -0700

----------------------------------------------------------------------
 .../src/main/compose/ozone/docker-config        |  37 ++++
 .../org/apache/hadoop/ozone/OzoneConsts.java    |  32 +++
 hadoop-ozone/common/src/main/bin/ozone          |   2 +
 .../src/main/conf/om-audit-log4j2.properties    |  86 ++++++++
 .../org/apache/hadoop/ozone/audit/OMAction.java |  25 ++-
 .../hadoop/ozone/om/helpers/OmBucketArgs.java   |  25 ++-
 .../hadoop/ozone/om/helpers/OmBucketInfo.java   |  21 +-
 .../hadoop/ozone/om/helpers/OmKeyArgs.java      |  22 +-
 .../hadoop/ozone/om/helpers/OmVolumeArgs.java   |  16 +-
 .../apache/hadoop/ozone/om/OzoneManager.java    | 218 ++++++++++++++++++-
 10 files changed, 466 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/630b64ec/hadoop-dist/src/main/compose/ozone/docker-config
----------------------------------------------------------------------
diff --git a/hadoop-dist/src/main/compose/ozone/docker-config b/hadoop-dist/src/main/compose/ozone/docker-config
index a1828a3..21127f8 100644
--- a/hadoop-dist/src/main/compose/ozone/docker-config
+++ b/hadoop-dist/src/main/compose/ozone/docker-config
@@ -31,3 +31,40 @@ LOG4J.PROPERTIES_log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
 LOG4J.PROPERTIES_log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n
 #Enable this variable to print out all hadoop rpc traffic to the stdout. See http://byteman.jboss.org/ to define your own instrumentation.
 #BYTEMAN_SCRIPT_URL=https://raw.githubusercontent.com/apache/hadoop/trunk/dev-support/byteman/hadooprpc.btm
+
+#LOG4J2.PROPERTIES_* are for Ozone Audit Logging
+LOG4J2.PROPERTIES_monitorInterval=30
+LOG4J2.PROPERTIES_filter=read,write
+LOG4J2.PROPERTIES_filter.read.type=MarkerFilter
+LOG4J2.PROPERTIES_filter.read.marker=READ
+LOG4J2.PROPERTIES_filter.read.onMatch=DENY
+LOG4J2.PROPERTIES_filter.read.onMismatch=NEUTRAL
+LOG4J2.PROPERTIES_filter.write.type=MarkerFilter
+LOG4J2.PROPERTIES_filter.write.marker=WRITE
+LOG4J2.PROPERTIES_filter.write.onMatch=NEUTRAL
+LOG4J2.PROPERTIES_filter.write.onMismatch=NEUTRAL
+LOG4J2.PROPERTIES_appenders=console, rolling
+LOG4J2.PROPERTIES_appender.console.type=Console
+LOG4J2.PROPERTIES_appender.console.name=STDOUT
+LOG4J2.PROPERTIES_appender.console.layout.type=PatternLayout
+LOG4J2.PROPERTIES_appender.console.layout.pattern=%d{DEFAULT} | %-5level | %c{1} | %msg | %throwable{3} %n
+LOG4J2.PROPERTIES_appender.rolling.type=RollingFile
+LOG4J2.PROPERTIES_appender.rolling.name=RollingFile
+LOG4J2.PROPERTIES_appender.rolling.fileName =${sys:hadoop.log.dir}/om-audit-${hostName}.log
+LOG4J2.PROPERTIES_appender.rolling.filePattern=${sys:hadoop.log.dir}/om-audit-${hostName}-%d{yyyy-MM-dd-HH-mm-ss}-%i.log.gz
+LOG4J2.PROPERTIES_appender.rolling.layout.type=PatternLayout
+LOG4J2.PROPERTIES_appender.rolling.layout.pattern=%d{DEFAULT} | %-5level | %c{1} | %msg | %throwable{3} %n
+LOG4J2.PROPERTIES_appender.rolling.policies.type=Policies
+LOG4J2.PROPERTIES_appender.rolling.policies.time.type=TimeBasedTriggeringPolicy
+LOG4J2.PROPERTIES_appender.rolling.policies.time.interval=86400
+LOG4J2.PROPERTIES_appender.rolling.policies.size.type=SizeBasedTriggeringPolicy
+LOG4J2.PROPERTIES_appender.rolling.policies.size.size=64MB
+LOG4J2.PROPERTIES_loggers=audit
+LOG4J2.PROPERTIES_logger.audit.type=AsyncLogger
+LOG4J2.PROPERTIES_logger.audit.name=OMAudit
+LOG4J2.PROPERTIES_logger.audit.level=INFO
+LOG4J2.PROPERTIES_logger.audit.appenderRefs=rolling
+LOG4J2.PROPERTIES_logger.audit.appenderRef.file.ref=RollingFile
+LOG4J2.PROPERTIES_rootLogger.level=INFO
+LOG4J2.PROPERTIES_rootLogger.appenderRefs=stdout
+LOG4J2.PROPERTIES_rootLogger.appenderRef.stdout.ref=STDOUT

http://git-wip-us.apache.org/repos/asf/hadoop/blob/630b64ec/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index 15366fb..9645c02 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -184,4 +184,36 @@ public final class OzoneConsts {
   public static final String CHUNKS_PATH = "chunksPath";
   public static final String CONTAINER_DB_TYPE = "containerDBType";
   public static final String CHECKSUM = "checksum";
+
+  // For OM Audit usage
+  public static final String VOLUME = "volume";
+  public static final String BUCKET = "bucket";
+  public static final String KEY = "key";
+  public static final String QUOTA = "quota";
+  public static final String QUOTA_IN_BYTES = "quotaInBytes";
+  public static final String CLIENT_ID = "clientID";
+  public static final String OWNER = "owner";
+  public static final String ADMIN = "admin";
+  public static final String USERNAME = "username";
+  public static final String PREV_KEY = "prevKey";
+  public static final String START_KEY = "startKey";
+  public static final String MAX_KEYS = "maxKeys";
+  public static final String PREFIX = "prefix";
+  public static final String KEY_PREFIX = "keyPrefix";
+  public static final String ACLS = "acls";
+  public static final String USER_ACL = "userAcl";
+  public static final String ADD_ACLS = "addAcls";
+  public static final String REMOVE_ACLS = "removeAcls";
+  public static final String MAX_NUM_OF_BUCKETS = "maxNumOfBuckets";
+  public static final String TO_KEY_NAME = "toKeyName";
+  public static final String STORAGE_TYPE = "storageType";
+  public static final String IS_VERSION_ENABLED = "isVersionEnabled";
+  public static final String CREATION_TIME = "creationTime";
+  public static final String DATA_SIZE = "dataSize";
+  public static final String REPLICATION_TYPE = "replicationType";
+  public static final String REPLICATION_FACTOR = "replicationFactor";
+  public static final String KEY_LOCATION_INFO = "keyLocationInfo";
+
+
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/630b64ec/hadoop-ozone/common/src/main/bin/ozone
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/bin/ozone b/hadoop-ozone/common/src/main/bin/ozone
index 75ceeb7..6a30433 100755
--- a/hadoop-ozone/common/src/main/bin/ozone
+++ b/hadoop-ozone/common/src/main/bin/ozone
@@ -97,6 +97,8 @@ function ozonecmd_case
     om)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
       HADOOP_CLASSNAME=org.apache.hadoop.ozone.om.OzoneManager
+      HDFS_OM_OPTS="${HDFS_OM_OPTS} -Dlog4j.configurationFile=${HADOOP_CONF_DIR}/om-audit-log4j2.properties"
+      HADOOP_OPTS="${HADOOP_OPTS} ${HDFS_OM_OPTS}"
     ;;
     oz)
       HADOOP_CLASSNAME=org.apache.hadoop.ozone.web.ozShell.Shell

http://git-wip-us.apache.org/repos/asf/hadoop/blob/630b64ec/hadoop-ozone/common/src/main/conf/om-audit-log4j2.properties
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/conf/om-audit-log4j2.properties b/hadoop-ozone/common/src/main/conf/om-audit-log4j2.properties
new file mode 100644
index 0000000..7d097a0
--- /dev/null
+++ b/hadoop-ozone/common/src/main/conf/om-audit-log4j2.properties
@@ -0,0 +1,86 @@
+#
+# 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.
+#
+name=PropertiesConfig
+
+# Checks for config change periodically and reloads
+monitorInterval=30
+
+filter=read,write
+# filter.read.onMatch=DENY avoids logging all READ events
+# filter.read.onMatch=ACCEPT permits logging all READ events
+# The above two settings ignore the log levels in configuration
+# filter.read.onMatch=NEUTRAL permits logging of only those READ events
+# which are attempted at log level equal or greater than log level specified
+# in the configuration
+filter.read.type=MarkerFilter
+filter.read.marker=READ
+filter.read.onMatch=DENY
+filter.read.onMismatch=NEUTRAL
+
+# filter.write.onMatch=DENY avoids logging all WRITE events
+# filter.write.onMatch=ACCEPT permits logging all WRITE events
+# The above two settings ignore the log levels in configuration
+# filter.write.onMatch=NEUTRAL permits logging of only those WRITE events
+# which are attempted at log level equal or greater than log level specified
+# in the configuration
+filter.write.type=MarkerFilter
+filter.write.marker=WRITE
+filter.write.onMatch=NEUTRAL
+filter.write.onMismatch=NEUTRAL
+
+# Log Levels are organized from most specific to least:
+# OFF (most specific, no logging)
+# FATAL (most specific, little data)
+# ERROR
+# WARN
+# INFO
+# DEBUG
+# TRACE (least specific, a lot of data)
+# ALL (least specific, all data)
+
+appenders=console, rolling
+appender.console.type=Console
+appender.console.name=STDOUT
+appender.console.layout.type=PatternLayout
+appender.console.layout.pattern=%d{DEFAULT} | %-5level | %c{1} | %msg | %throwable{3} %n
+
+#Rolling File Appender with size & time thresholds.
+#Rolling is triggered when either threshold is breached.
+#The rolled over file is compressed by default
+#Time interval is specified in seconds 86400s=1 day
+appender.rolling.type=RollingFile
+appender.rolling.name=RollingFile
+appender.rolling.fileName =${sys:hadoop.log.dir}/om-audit-${hostName}.log
+appender.rolling.filePattern=${sys:hadoop.log.dir}/om-audit-${hostName}-%d{yyyy-MM-dd-HH-mm-ss}-%i.log.gz
+appender.rolling.layout.type=PatternLayout
+appender.rolling.layout.pattern=%d{DEFAULT} | %-5level | %c{1} | %msg | %throwable{3} %n
+appender.rolling.policies.type=Policies
+appender.rolling.policies.time.type=TimeBasedTriggeringPolicy
+appender.rolling.policies.time.interval=86400
+appender.rolling.policies.size.type=SizeBasedTriggeringPolicy
+appender.rolling.policies.size.size=64MB
+
+loggers=audit
+logger.audit.type=AsyncLogger
+logger.audit.name=OMAudit
+logger.audit.level=INFO
+logger.audit.appenderRefs=rolling
+logger.audit.appenderRef.file.ref=RollingFile
+
+rootLogger.level=INFO
+rootLogger.appenderRefs=stdout
+rootLogger.appenderRef.stdout.ref=STDOUT

http://git-wip-us.apache.org/repos/asf/hadoop/blob/630b64ec/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
index d780ea2..6488f5e 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
@@ -18,24 +18,33 @@
 package org.apache.hadoop.ozone.audit;
 
 /**
- * Enum to define OM Action types for Audit.
+ * Enum to define Audit Action types for OzoneManager.
  */
 public enum OMAction implements AuditAction {
 
+  // WRITE Actions
+  ALLOCATE_BLOCK("ALLOCATE_BLOCK"),
+  ALLOCATE_KEY("ALLOCATE_KEY"),
+  COMMIT_KEY("COMMIT_KEY"),
   CREATE_VOLUME("CREATE_VOLUME"),
   CREATE_BUCKET("CREATE_BUCKET"),
   CREATE_KEY("CREATE_KEY"),
-  READ_VOLUME("READ_VOLUME"),
-  READ_BUCKET("READ_BUCKET"),
-  READ_KEY("READ_BUCKET"),
-  UPDATE_VOLUME("UPDATE_VOLUME"),
-  UPDATE_BUCKET("UPDATE_BUCKET"),
-  UPDATE_KEY("UPDATE_KEY"),
   DELETE_VOLUME("DELETE_VOLUME"),
   DELETE_BUCKET("DELETE_BUCKET"),
   DELETE_KEY("DELETE_KEY"),
+  RENAME_KEY("RENAME_KEY"),
   SET_OWNER("SET_OWNER"),
-  SET_QUOTA("SET_QUOTA");
+  SET_QUOTA("SET_QUOTA"),
+  UPDATE_VOLUME("UPDATE_VOLUME"),
+  UPDATE_BUCKET("UPDATE_BUCKET"),
+  UPDATE_KEY("UPDATE_KEY"),
+  // READ Actions
+  LIST_BUCKETS("LIST_BUCKETS"),
+  LIST_VOLUMES("LIST_VOLUMES"),
+  LIST_KEYS("LIST_KEYS"),
+  READ_VOLUME("READ_VOLUME"),
+  READ_BUCKET("READ_BUCKET"),
+  READ_KEY("READ_BUCKET");
 
   private String action;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/630b64ec/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
index 6aabfef..1bd258e 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
@@ -17,13 +17,17 @@
  */
 package org.apache.hadoop.ozone.om.helpers;
 
+import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.stream.Collectors;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.Auditable;
 import org.apache.hadoop.ozone.protocol.proto
     .OzoneManagerProtocolProtos.BucketArgs;
 import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
@@ -31,7 +35,7 @@ import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
 /**
  * A class that encapsulates Bucket Arguments.
  */
-public final class OmBucketArgs {
+public final class OmBucketArgs implements Auditable {
   /**
    * Name of the volume in which the bucket belongs to.
    */
@@ -135,6 +139,25 @@ public final class OmBucketArgs {
     return new Builder();
   }
 
+  @Override
+  public Map<String, String> toAuditMap() {
+    Map<String, String> auditMap = new LinkedHashMap<>();
+    auditMap.put(OzoneConsts.VOLUME, this.volumeName);
+    auditMap.put(OzoneConsts.BUCKET, this.bucketName);
+    if(this.addAcls != null){
+      auditMap.put(OzoneConsts.ADD_ACLS, this.addAcls.toString());
+    }
+    if(this.removeAcls != null){
+      auditMap.put(OzoneConsts.REMOVE_ACLS, this.removeAcls.toString());
+    }
+    auditMap.put(OzoneConsts.IS_VERSION_ENABLED,
+                String.valueOf(this.isVersionEnabled));
+    if(this.storageType != null){
+      auditMap.put(OzoneConsts.STORAGE_TYPE, this.storageType.name());
+    }
+    return auditMap;
+  }
+
   /**
    * Builder for OmBucketArgs.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/630b64ec/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
index bf5abdd..0a136a7 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
@@ -21,18 +21,22 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.Auditable;
 import org.apache.hadoop.ozone.protocol.proto
     .OzoneManagerProtocolProtos.BucketInfo;
 import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
 
+import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.stream.Collectors;
 
 /**
  * A class that encapsulates Bucket Info.
  */
-public final class OmBucketInfo {
+public final class OmBucketInfo implements Auditable {
   /**
    * Name of the volume in which the bucket belongs to.
    */
@@ -137,6 +141,21 @@ public final class OmBucketInfo {
     return new Builder();
   }
 
+  @Override
+  public Map<String, String> toAuditMap() {
+    Map<String, String> auditMap = new LinkedHashMap<>();
+    auditMap.put(OzoneConsts.VOLUME, this.volumeName);
+    auditMap.put(OzoneConsts.BUCKET, this.bucketName);
+    auditMap.put(OzoneConsts.ACLS,
+        (this.acls != null) ? this.acls.toString() : null);
+    auditMap.put(OzoneConsts.IS_VERSION_ENABLED,
+        String.valueOf(this.isVersionEnabled));
+    auditMap.put(OzoneConsts.STORAGE_TYPE,
+        (this.storageType != null) ? this.storageType.name() : null);
+    auditMap.put(OzoneConsts.CREATION_TIME, String.valueOf(this.creationTime));
+    return auditMap;
+  }
+
   /**
    * Builder for OmBucketInfo.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/630b64ec/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java
index aab35c5..d8d41d5 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java
@@ -18,14 +18,18 @@
 package org.apache.hadoop.ozone.om.helpers;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.Auditable;
 
+import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
 
 /**
  * Args for key. Client use this to specify key's attributes on  key creation
  * (putKey()).
  */
-public final class OmKeyArgs {
+public final class OmKeyArgs implements Auditable {
   private final String volumeName;
   private final String bucketName;
   private final String keyName;
@@ -82,6 +86,22 @@ public final class OmKeyArgs {
     return locationInfoList;
   }
 
+  @Override
+  public Map<String, String> toAuditMap() {
+    Map<String, String> auditMap = new LinkedHashMap<>();
+    auditMap.put(OzoneConsts.VOLUME, this.volumeName);
+    auditMap.put(OzoneConsts.BUCKET, this.bucketName);
+    auditMap.put(OzoneConsts.KEY, this.keyName);
+    auditMap.put(OzoneConsts.DATA_SIZE, String.valueOf(this.dataSize));
+    auditMap.put(OzoneConsts.REPLICATION_TYPE,
+        (this.type != null) ? this.type.name() : null);
+    auditMap.put(OzoneConsts.REPLICATION_FACTOR,
+        (this.factor != null) ? this.factor.name() : null);
+    auditMap.put(OzoneConsts.KEY_LOCATION_INFO,
+        (this.locationInfoList != null) ? locationInfoList.toString() : null);
+    return auditMap;
+  }
+
   /**
    * Builder class of OmKeyArgs.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/630b64ec/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
index c8b59b6..27e25f9 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.ozone.om.helpers;
 
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.Auditable;
 import org.apache.hadoop.ozone.protocol.proto
     .OzoneManagerProtocolProtos.OzoneAclInfo;
 import org.apache.hadoop.ozone.protocol.proto
@@ -26,6 +28,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.KeyValue;
 
 import java.io.IOException;
 import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -35,7 +38,7 @@ import java.util.stream.Collectors;
 /**
  * A class that encapsulates the OmVolumeArgs Args.
  */
-public final class OmVolumeArgs {
+public final class OmVolumeArgs implements Auditable{
   private final String adminName;
   private final String ownerName;
   private final String volume;
@@ -122,6 +125,17 @@ public final class OmVolumeArgs {
     return new Builder();
   }
 
+  @Override
+  public Map<String, String> toAuditMap() {
+    Map<String, String> auditMap = new LinkedHashMap<>();
+    auditMap.put(OzoneConsts.ADMIN, this.adminName);
+    auditMap.put(OzoneConsts.OWNER, this.ownerName);
+    auditMap.put(OzoneConsts.VOLUME, this.volume);
+    auditMap.put(OzoneConsts.CREATION_TIME, String.valueOf(this.creationTime));
+    auditMap.put(OzoneConsts.QUOTA_IN_BYTES, String.valueOf(this.quotaInBytes));
+    return auditMap;
+  }
+
   /**
    * Builder for OmVolumeArgs.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/630b64ec/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
index 71fa921..887ddd0 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
@@ -27,6 +27,14 @@ import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.hdds.server.ServiceRuntimeInfoImpl;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.AuditAction;
+import org.apache.hadoop.ozone.audit.AuditEventStatus;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.AuditLoggerType;
+import org.apache.hadoop.ozone.audit.AuditMessage;
+import org.apache.hadoop.ozone.audit.OMAction;
 import org.apache.hadoop.ozone.common.Storage.StorageState;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
@@ -69,6 +77,8 @@ import static org.apache.hadoop.hdds.HddsUtils.isHddsEnabled;
 import static org.apache.hadoop.ozone.OmUtils.getOmAddress;
 import static org.apache.hadoop.hdds.server.ServerUtils
     .updateRPCListenAddress;
+
+import org.apache.logging.log4j.Level;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -78,6 +88,7 @@ import java.io.PrintStream;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -104,6 +115,9 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   private static final Logger LOG =
       LoggerFactory.getLogger(OzoneManager.class);
 
+  private static final AuditLogger AUDIT =
+      new AuditLogger(AuditLoggerType.OMLOGGER);
+
   private static final String USAGE =
       "Usage: \n ozone om [genericOptions] " + "[ "
           + StartupOption.CREATEOBJECTSTORE.getName() + " ]\n " + "ozone om [ "
@@ -335,8 +349,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    * @throws IOException in case OM instance creation fails.
    */
 
-  public static OzoneManager createOm(String[] argv,
-                                      OzoneConfiguration conf) throws IOException {
+  public static OzoneManager createOm(String[] argv, OzoneConfiguration conf)
+      throws IOException {
     if (!isHddsEnabled(conf)) {
       System.err.println("OM cannot be started in secure mode or when " +
           OZONE_ENABLED + " is set to false");
@@ -486,8 +500,13 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     try {
       metrics.incNumVolumeCreates();
       volumeManager.createVolume(args);
+      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.CREATE_VOLUME,
+          args.toAuditMap()));
     } catch (Exception ex) {
       metrics.incNumVolumeCreateFails();
+      AUDIT.logWriteFailure(Level.ERROR,
+          buildAuditMessageForFailure(OMAction.CREATE_VOLUME,
+          args.toAuditMap()), ex);
       throw ex;
     }
   }
@@ -501,11 +520,17 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    */
   @Override
   public void setOwner(String volume, String owner) throws IOException {
+    Map<String, String> auditMap = buildAuditMap(volume);
+    auditMap.put(OzoneConsts.OWNER, owner);
     try {
       metrics.incNumVolumeUpdates();
       volumeManager.setOwner(volume, owner);
+      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.SET_OWNER,
+          auditMap));
     } catch (Exception ex) {
       metrics.incNumVolumeUpdateFails();
+      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.SET_OWNER,
+          auditMap), ex);
       throw ex;
     }
   }
@@ -519,15 +544,22 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    */
   @Override
   public void setQuota(String volume, long quota) throws IOException {
+    Map<String, String> auditMap = buildAuditMap(volume);
+    auditMap.put(OzoneConsts.QUOTA, String.valueOf(quota));
     try {
       metrics.incNumVolumeUpdates();
       volumeManager.setQuota(volume, quota);
+      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.SET_QUOTA,
+          auditMap));
     } catch (Exception ex) {
       metrics.incNumVolumeUpdateFails();
+      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.SET_QUOTA,
+          auditMap), ex);
       throw ex;
     }
   }
 
+  //TODO: Should define new OMAction type? How to log OzoneAclInfo ?
   /**
    * Checks if the specified user can access this volume.
    *
@@ -540,12 +572,23 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   @Override
   public boolean checkVolumeAccess(String volume, OzoneAclInfo userAcl)
       throws IOException {
+    boolean auditSuccess = true;
+    Map<String, String> auditMap = buildAuditMap(volume);
+    auditMap.put(OzoneConsts.USER_ACL, userAcl.getName());
     try {
       metrics.incNumVolumeCheckAccesses();
       return volumeManager.checkVolumeAccess(volume, userAcl);
     } catch (Exception ex) {
       metrics.incNumVolumeCheckAccessFails();
+      auditSuccess = false;
+      AUDIT.logReadFailure(buildAuditMessageForFailure(OMAction.READ_VOLUME,
+          auditMap), ex);
       throw ex;
+    } finally {
+      if(auditSuccess){
+        AUDIT.logReadSuccess(buildAuditMessageForSuccess(OMAction.READ_VOLUME,
+            auditMap));
+      }
     }
   }
 
@@ -558,12 +601,22 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    */
   @Override
   public OmVolumeArgs getVolumeInfo(String volume) throws IOException {
+    boolean auditSuccess = true;
+    Map<String, String> auditMap = buildAuditMap(volume);
     try {
       metrics.incNumVolumeInfos();
       return volumeManager.getVolumeInfo(volume);
     } catch (Exception ex) {
       metrics.incNumVolumeInfoFails();
+      auditSuccess = false;
+      AUDIT.logReadFailure(buildAuditMessageForFailure(OMAction.READ_VOLUME,
+          auditMap), ex);
       throw ex;
+    } finally {
+      if(auditSuccess){
+        AUDIT.logReadSuccess(buildAuditMessageForSuccess(OMAction.READ_VOLUME,
+            auditMap));
+      }
     }
   }
 
@@ -578,8 +631,12 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     try {
       metrics.incNumVolumeDeletes();
       volumeManager.deleteVolume(volume);
+      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.DELETE_VOLUME,
+          buildAuditMap(volume)));
     } catch (Exception ex) {
       metrics.incNumVolumeDeleteFails();
+      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.DELETE_VOLUME,
+          buildAuditMap(volume)), ex);
       throw ex;
     }
   }
@@ -597,13 +654,27 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    */
   @Override
   public List<OmVolumeArgs> listVolumeByUser(String userName, String prefix,
-                                             String prevKey, int maxKeys) throws IOException {
+      String prevKey, int maxKeys) throws IOException {
+    boolean auditSuccess = true;
+    Map<String, String> auditMap = new LinkedHashMap<>();
+    auditMap.put(OzoneConsts.PREV_KEY, prevKey);
+    auditMap.put(OzoneConsts.PREFIX, prefix);
+    auditMap.put(OzoneConsts.MAX_KEYS, String.valueOf(maxKeys));
+    auditMap.put(OzoneConsts.USERNAME, userName);
     try {
       metrics.incNumVolumeLists();
       return volumeManager.listVolumes(userName, prefix, prevKey, maxKeys);
     } catch (Exception ex) {
       metrics.incNumVolumeListFails();
+      auditSuccess = false;
+      AUDIT.logReadFailure(buildAuditMessageForFailure(OMAction.LIST_VOLUMES,
+          auditMap), ex);
       throw ex;
+    } finally {
+      if(auditSuccess){
+        AUDIT.logReadSuccess(buildAuditMessageForSuccess(OMAction.LIST_VOLUMES,
+            auditMap));
+      }
     }
   }
 
@@ -620,12 +691,26 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   @Override
   public List<OmVolumeArgs> listAllVolumes(String prefix, String prevKey, int
       maxKeys) throws IOException {
+    boolean auditSuccess = true;
+    Map<String, String> auditMap = new LinkedHashMap<>();
+    auditMap.put(OzoneConsts.PREV_KEY, prevKey);
+    auditMap.put(OzoneConsts.PREFIX, prefix);
+    auditMap.put(OzoneConsts.MAX_KEYS, String.valueOf(maxKeys));
+    auditMap.put(OzoneConsts.USERNAME, null);
     try {
       metrics.incNumVolumeLists();
       return volumeManager.listVolumes(null, prefix, prevKey, maxKeys);
     } catch (Exception ex) {
       metrics.incNumVolumeListFails();
+      auditSuccess = false;
+      AUDIT.logReadFailure(buildAuditMessageForFailure(OMAction.LIST_VOLUMES,
+          auditMap), ex);
       throw ex;
+    } finally {
+      if(auditSuccess){
+        AUDIT.logReadSuccess(buildAuditMessageForSuccess(OMAction.LIST_VOLUMES,
+            auditMap));
+      }
     }
   }
 
@@ -640,8 +725,12 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     try {
       metrics.incNumBucketCreates();
       bucketManager.createBucket(bucketInfo);
+      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.CREATE_BUCKET,
+          bucketInfo.toAuditMap()));
     } catch (Exception ex) {
       metrics.incNumBucketCreateFails();
+      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.CREATE_BUCKET,
+          bucketInfo.toAuditMap()), ex);
       throw ex;
     }
   }
@@ -650,16 +739,30 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    * {@inheritDoc}
    */
   @Override
-  public List<OmBucketInfo> listBuckets(String volumeName,
-                                        String startKey, String prefix, int maxNumOfBuckets)
+  public List<OmBucketInfo> listBuckets(String volumeName, String startKey,
+      String prefix, int maxNumOfBuckets)
       throws IOException {
+    boolean auditSuccess = true;
+    Map<String, String> auditMap = buildAuditMap(volumeName);
+    auditMap.put(OzoneConsts.START_KEY, startKey);
+    auditMap.put(OzoneConsts.PREFIX, prefix);
+    auditMap.put(OzoneConsts.MAX_NUM_OF_BUCKETS,
+                String.valueOf(maxNumOfBuckets));
     try {
       metrics.incNumBucketLists();
       return bucketManager.listBuckets(volumeName,
           startKey, prefix, maxNumOfBuckets);
     } catch (IOException ex) {
       metrics.incNumBucketListFails();
+      auditSuccess = false;
+      AUDIT.logReadFailure(buildAuditMessageForFailure(OMAction.LIST_BUCKETS,
+          auditMap), ex);
       throw ex;
+    } finally {
+      if(auditSuccess){
+        AUDIT.logReadSuccess(buildAuditMessageForSuccess(OMAction.LIST_BUCKETS,
+            auditMap));
+      }
     }
   }
 
@@ -674,12 +777,23 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   @Override
   public OmBucketInfo getBucketInfo(String volume, String bucket)
       throws IOException {
+    boolean auditSuccess = true;
+    Map<String, String> auditMap = buildAuditMap(volume);
+    auditMap.put(OzoneConsts.BUCKET, bucket);
     try {
       metrics.incNumBucketInfos();
       return bucketManager.getBucketInfo(volume, bucket);
     } catch (Exception ex) {
       metrics.incNumBucketInfoFails();
+      auditSuccess = false;
+      AUDIT.logReadFailure(buildAuditMessageForFailure(OMAction.READ_BUCKET,
+          auditMap), ex);
       throw ex;
+    } finally {
+      if(auditSuccess){
+        AUDIT.logReadSuccess(buildAuditMessageForSuccess(OMAction.READ_BUCKET,
+            auditMap));
+      }
     }
   }
 
@@ -692,23 +806,38 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    */
   @Override
   public OpenKeySession openKey(OmKeyArgs args) throws IOException {
+    boolean auditSuccess = true;
     try {
       metrics.incNumKeyAllocates();
       return keyManager.openKey(args);
     } catch (Exception ex) {
       metrics.incNumKeyAllocateFails();
+      auditSuccess = false;
+      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.ALLOCATE_KEY,
+          args.toAuditMap()), ex);
       throw ex;
+    } finally {
+      if(auditSuccess){
+        AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.ALLOCATE_KEY,
+            args.toAuditMap()));
+      }
     }
   }
 
   @Override
   public void commitKey(OmKeyArgs args, int clientID)
       throws IOException {
+    Map<String, String> auditMap = args.toAuditMap();
+    auditMap.put(OzoneConsts.CLIENT_ID, String.valueOf(clientID));
     try {
       metrics.incNumKeyCommits();
       keyManager.commitKey(args, clientID);
+      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.COMMIT_KEY,
+          args.toAuditMap()));
     } catch (Exception ex) {
       metrics.incNumKeyCommitFails();
+      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.COMMIT_KEY,
+          args.toAuditMap()), ex);
       throw ex;
     }
   }
@@ -716,12 +845,24 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   @Override
   public OmKeyLocationInfo allocateBlock(OmKeyArgs args, int clientID)
       throws IOException {
+    boolean auditSuccess = true;
+    Map<String, String> auditMap = args.toAuditMap();
+    auditMap.put(OzoneConsts.CLIENT_ID, String.valueOf(clientID));
     try {
       metrics.incNumBlockAllocateCalls();
       return keyManager.allocateBlock(args, clientID);
     } catch (Exception ex) {
       metrics.incNumBlockAllocateCallFails();
+      auditSuccess = false;
+      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.ALLOCATE_BLOCK,
+          args.toAuditMap()), ex);
       throw ex;
+    } finally {
+      if(auditSuccess){
+        AUDIT.logWriteSuccess(buildAuditMessageForSuccess(
+            OMAction.ALLOCATE_BLOCK,
+            args.toAuditMap()));
+      }
     }
   }
 
@@ -734,22 +875,37 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    */
   @Override
   public OmKeyInfo lookupKey(OmKeyArgs args) throws IOException {
+    boolean auditSuccess = true;
     try {
       metrics.incNumKeyLookups();
       return keyManager.lookupKey(args);
     } catch (Exception ex) {
       metrics.incNumKeyLookupFails();
+      auditSuccess = false;
+      AUDIT.logReadFailure(buildAuditMessageForFailure(OMAction.READ_KEY,
+          args.toAuditMap()), ex);
       throw ex;
+    } finally {
+      if(auditSuccess){
+        AUDIT.logReadSuccess(buildAuditMessageForSuccess(OMAction.READ_KEY,
+            args.toAuditMap()));
+      }
     }
   }
 
   @Override
   public void renameKey(OmKeyArgs args, String toKeyName) throws IOException {
+    Map<String, String> auditMap = args.toAuditMap();
+    auditMap.put(OzoneConsts.TO_KEY_NAME, toKeyName);
     try {
       metrics.incNumKeyRenames();
       keyManager.renameKey(args, toKeyName);
+      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.RENAME_KEY,
+          args.toAuditMap()));
     } catch (IOException e) {
       metrics.incNumKeyRenameFails();
+      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.RENAME_KEY,
+          args.toAuditMap()), e);
       throw e;
     }
   }
@@ -765,22 +921,40 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     try {
       metrics.incNumKeyDeletes();
       keyManager.deleteKey(args);
+      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.DELETE_KEY,
+          args.toAuditMap()));
     } catch (Exception ex) {
       metrics.incNumKeyDeleteFails();
+      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.DELETE_KEY,
+          args.toAuditMap()), ex);
       throw ex;
     }
   }
 
   @Override
   public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
-                                  String startKey, String keyPrefix, int maxKeys) throws IOException {
+      String startKey, String keyPrefix, int maxKeys) throws IOException {
+    boolean auditSuccess = true;
+    Map<String, String> auditMap = buildAuditMap(volumeName);
+    auditMap.put(OzoneConsts.BUCKET, bucketName);
+    auditMap.put(OzoneConsts.START_KEY, startKey);
+    auditMap.put(OzoneConsts.MAX_KEYS, String.valueOf(maxKeys));
+    auditMap.put(OzoneConsts.KEY_PREFIX, keyPrefix);
     try {
       metrics.incNumKeyLists();
       return keyManager.listKeys(volumeName, bucketName,
           startKey, keyPrefix, maxKeys);
     } catch (IOException ex) {
       metrics.incNumKeyListFails();
+      auditSuccess = false;
+      AUDIT.logReadFailure(buildAuditMessageForFailure(OMAction.LIST_KEYS,
+          auditMap), ex);
       throw ex;
+    } finally {
+      if(auditSuccess){
+        AUDIT.logReadSuccess(buildAuditMessageForSuccess(OMAction.LIST_KEYS,
+            auditMap));
+      }
     }
   }
 
@@ -795,8 +969,12 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     try {
       metrics.incNumBucketUpdates();
       bucketManager.setBucketProperty(args);
+      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.UPDATE_BUCKET,
+          args.toAuditMap()));
     } catch (Exception ex) {
       metrics.incNumBucketUpdateFails();
+      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.UPDATE_BUCKET,
+          args.toAuditMap()), ex);
       throw ex;
     }
   }
@@ -809,11 +987,17 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    * @throws IOException
    */
   public void deleteBucket(String volume, String bucket) throws IOException {
+    Map<String, String> auditMap = buildAuditMap(volume);
+    auditMap.put(OzoneConsts.BUCKET, bucket);
     try {
       metrics.incNumBucketDeletes();
       bucketManager.deleteBucket(volume, bucket);
+      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction.DELETE_BUCKET,
+          auditMap));
     } catch (Exception ex) {
       metrics.incNumBucketDeleteFails();
+      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction.DELETE_BUCKET,
+          auditMap), ex);
       throw ex;
     }
   }
@@ -908,4 +1092,26 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     // metrics.incNumGetServiceListFails()
     return services;
   }
+
+  private Map<String, String> buildAuditMap(String volume){
+    Map<String, String> auditMap = new LinkedHashMap<>();
+    auditMap.put(OzoneConsts.VOLUME, volume);
+    return auditMap;
+  }
+
+  // TODO: Temporary method until AuditMessage is simplified
+  private AuditMessage buildAuditMessageForSuccess(AuditAction op,
+      Map<String, String> auditMap) {
+    return new AuditMessage(Server.getRemoteUser().getUserName(),
+        Server.getRemoteIp().getHostAddress(), op.toString(), auditMap,
+        AuditEventStatus.SUCCESS.toString());
+  }
+
+  // TODO: Temporary method until AuditMessage is simplified
+  private AuditMessage buildAuditMessageForFailure(AuditAction op,
+      Map<String, String> auditMap) {
+    return new AuditMessage(Server.getRemoteUser().getUserName(),
+        Server.getRemoteIp().getHostAddress(), op.toString(), auditMap,
+        AuditEventStatus.FAILURE.toString());
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[23/50] [abbrv] hadoop git commit: HDDS-380. Remove synchronization from ChunkGroupOutputStream and ChunkOutputStream. Contributed by Shashikant Banerjee.

Posted by eh...@apache.org.
HDDS-380. Remove synchronization from ChunkGroupOutputStream and ChunkOutputStream. Contributed by Shashikant Banerjee.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0bd42171
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0bd42171
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0bd42171

Branch: refs/heads/HDFS-12090
Commit: 0bd4217194ae50ec30e386b200fcfa54c069f042
Parents: 3fa4639
Author: Nanda kumar <na...@apache.org>
Authored: Wed Aug 29 13:31:19 2018 +0530
Committer: Nanda kumar <na...@apache.org>
Committed: Wed Aug 29 13:31:19 2018 +0530

----------------------------------------------------------------------
 .../hadoop/hdds/scm/storage/ChunkOutputStream.java  | 16 ++++++++--------
 .../ozone/client/io/ChunkGroupOutputStream.java     | 12 ++++++------
 2 files changed, 14 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bd42171/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java
index f2df3fa..8d311d0 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkOutputStream.java
@@ -99,7 +99,7 @@ public class ChunkOutputStream extends OutputStream {
   }
 
   @Override
-  public synchronized void write(int b) throws IOException {
+  public void write(int b) throws IOException {
     checkOpen();
     int rollbackPosition = buffer.position();
     int rollbackLimit = buffer.limit();
@@ -110,7 +110,7 @@ public class ChunkOutputStream extends OutputStream {
   }
 
   @Override
-  public synchronized void write(byte[] b, int off, int len)
+  public void write(byte[] b, int off, int len)
       throws IOException {
     if (b == null) {
       throw new NullPointerException();
@@ -137,7 +137,7 @@ public class ChunkOutputStream extends OutputStream {
   }
 
   @Override
-  public synchronized void flush() throws IOException {
+  public void flush() throws IOException {
     checkOpen();
     if (buffer.position() > 0) {
       int rollbackPosition = buffer.position();
@@ -147,7 +147,7 @@ public class ChunkOutputStream extends OutputStream {
   }
 
   @Override
-  public synchronized void close() throws IOException {
+  public void close() throws IOException {
     if (xceiverClientManager != null && xceiverClient != null
         && buffer != null) {
       if (buffer.position() > 0) {
@@ -164,7 +164,7 @@ public class ChunkOutputStream extends OutputStream {
     }
   }
 
-  public synchronized void cleanup() {
+  public void cleanup() {
     xceiverClientManager.releaseClient(xceiverClient);
     xceiverClientManager = null;
     xceiverClient = null;
@@ -176,7 +176,7 @@ public class ChunkOutputStream extends OutputStream {
    *
    * @throws IOException if stream is closed
    */
-  private synchronized void checkOpen() throws IOException {
+  private void checkOpen() throws IOException {
     if (xceiverClient == null) {
       throw new IOException("ChunkOutputStream has been closed.");
     }
@@ -191,7 +191,7 @@ public class ChunkOutputStream extends OutputStream {
    * @param rollbackLimit limit to restore in buffer if write fails
    * @throws IOException if there is an I/O error while performing the call
    */
-  private synchronized void flushBufferToChunk(int rollbackPosition,
+  private void flushBufferToChunk(int rollbackPosition,
       int rollbackLimit) throws IOException {
     boolean success = false;
     try {
@@ -213,7 +213,7 @@ public class ChunkOutputStream extends OutputStream {
    *
    * @throws IOException if there is an I/O error while performing the call
    */
-  private synchronized void writeChunkToContainer() throws IOException {
+  private void writeChunkToContainer() throws IOException {
     buffer.flip();
     ByteString data = ByteString.copyFrom(buffer);
     ChunkInfo chunk = ChunkInfo

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bd42171/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
index 988af07..00624d5 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ChunkGroupOutputStream.java
@@ -105,7 +105,7 @@ public class ChunkGroupOutputStream extends OutputStream {
    * @param length
    */
   @VisibleForTesting
-  public synchronized void addStream(OutputStream outputStream, long length) {
+  public void addStream(OutputStream outputStream, long length) {
     streamEntries.add(new ChunkOutputStreamEntry(outputStream, length));
   }
 
@@ -227,7 +227,7 @@ public class ChunkGroupOutputStream extends OutputStream {
 
 
   @Override
-  public synchronized void write(int b) throws IOException {
+  public void write(int b) throws IOException {
     byte[] buf = new byte[1];
     buf[0] = (byte) b;
     write(buf, 0, 1);
@@ -246,7 +246,7 @@ public class ChunkGroupOutputStream extends OutputStream {
    * @throws IOException
    */
   @Override
-  public synchronized void write(byte[] b, int off, int len)
+  public void write(byte[] b, int off, int len)
       throws IOException {
     checkNotClosed();
     handleWrite(b, off, len);
@@ -404,7 +404,7 @@ public class ChunkGroupOutputStream extends OutputStream {
   }
 
   @Override
-  public synchronized void flush() throws IOException {
+  public void flush() throws IOException {
     checkNotClosed();
     handleFlushOrClose(false);
   }
@@ -450,7 +450,7 @@ public class ChunkGroupOutputStream extends OutputStream {
    * @throws IOException
    */
   @Override
-  public synchronized void close() throws IOException {
+  public void close() throws IOException {
     if (closed) {
       return;
     }
@@ -585,7 +585,7 @@ public class ChunkGroupOutputStream extends OutputStream {
       return length - currentPosition;
     }
 
-    private synchronized void checkStream() {
+    private void checkStream() {
       if (this.outputStream == null) {
         this.outputStream = new ChunkOutputStream(blockID,
             key, xceiverClientManager, xceiverClient,


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[16/50] [abbrv] hadoop git commit: HDDS-382. Remove RatisTestHelper#RatisTestSuite constructor argument and fix checkstyle in ContainerTestHelper, GenericTestUtils Contributed by Nandakumar.

Posted by eh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRandom.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRandom.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRandom.java
index 3b4426c..b652b6b 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRandom.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRandom.java
@@ -51,9 +51,9 @@ public class TestSCMContainerPlacementRandom {
         .thenReturn(new ArrayList<>(datanodes));
 
     when(mockNodeManager.getNodeStat(anyObject()))
-        .thenReturn(new SCMNodeMetric(100l, 0l, 100l));
+        .thenReturn(new SCMNodeMetric(100L, 0L, 100L));
     when(mockNodeManager.getNodeStat(datanodes.get(2)))
-        .thenReturn(new SCMNodeMetric(100l, 90l, 10l));
+        .thenReturn(new SCMNodeMetric(100L, 90L, 10L));
 
     SCMContainerPlacementRandom scmContainerPlacementRandom =
         new SCMContainerPlacementRandom(mockNodeManager, conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java
index fa87706..da05913 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationManager.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Objects;
-import java.util.UUID;
 
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
@@ -132,7 +131,7 @@ public class TestReplicationManager {
       //WHEN
 
       queue.fireEvent(SCMEvents.REPLICATE_CONTAINER,
-          new ReplicationRequest(1l, (short) 2, System.currentTimeMillis(),
+          new ReplicationRequest(1L, (short) 2, System.currentTimeMillis(),
               (short) 3));
 
       Thread.sleep(500L);
@@ -159,10 +158,8 @@ public class TestReplicationManager {
       leaseManager.start();
 
       ReplicationManager replicationManager =
-          new ReplicationManager(containerPlacementPolicy, containerStateManager,
-
-
-              queue, leaseManager) {
+          new ReplicationManager(containerPlacementPolicy,
+              containerStateManager, queue, leaseManager) {
             @Override
             protected List<DatanodeDetails> getCurrentReplicas(
                 ReplicationRequest request) throws IOException {
@@ -172,7 +169,7 @@ public class TestReplicationManager {
       replicationManager.start();
 
       queue.fireEvent(SCMEvents.REPLICATE_CONTAINER,
-          new ReplicationRequest(1l, (short) 2, System.currentTimeMillis(),
+          new ReplicationRequest(1L, (short) 2, System.currentTimeMillis(),
               (short) 3));
 
       Thread.sleep(500L);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationQueue.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationQueue.java
index a593718..9dd4fe3 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationQueue.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestReplicationQueue.java
@@ -92,8 +92,8 @@ public class TestReplicationQueue {
         1, replicationQueue.size());
     Assert.assertEquals(temp, msg5);
 
-    // Message 2 should be ordered before message 5 as both have same replication
-    // number but message 2 has earlier timestamp.
+    // Message 2 should be ordered before message 5 as both have same
+    // replication number but message 2 has earlier timestamp.
     temp = replicationQueue.take();
     Assert.assertEquals("Should have 0 objects",
         replicationQueue.size(), 0);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
index 52963c0..6f8534d 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
@@ -32,8 +32,6 @@ import org.apache.hadoop.hdds.scm.container.placement.algorithms
     .SCMContainerPlacementCapacity;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto
-    .StorageContainerDatanodeProtocolProtos.StorageReportProto;
 import org.apache.hadoop.hdds.scm.events.SCMEvents;
 import org.apache.hadoop.hdds.server.events.EventQueue;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
@@ -158,7 +156,8 @@ public class TestContainerPlacement {
 
       assertTrue(nodeManager.isOutOfChillMode());
 
-      ContainerWithPipeline containerWithPipeline = containerManager.allocateContainer(
+      ContainerWithPipeline containerWithPipeline = containerManager
+          .allocateContainer(
           xceiverClientManager.getType(),
           xceiverClientManager.getFactor(), "OZONE");
       assertEquals(xceiverClientManager.getFactor().getNumber(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java
index 9d4346a..f438c8b 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeManager.java
@@ -424,7 +424,8 @@ public class TestNodeManager {
       List<DatanodeDetails> nodeList = createNodeSet(nodeManager, nodeCount);
 
 
-      DatanodeDetails staleNode = TestUtils.createRandomDatanodeAndRegister(nodeManager);
+      DatanodeDetails staleNode = TestUtils.createRandomDatanodeAndRegister(
+          nodeManager);
 
       // Heartbeat once
       nodeManager.processHeartbeat(staleNode);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeReportHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeReportHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeReportHandler.java
index e2e89ab..e50b823 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeReportHandler.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeReportHandler.java
@@ -37,13 +37,12 @@ import org.slf4j.LoggerFactory;
 
 public class TestNodeReportHandler implements EventPublisher {
 
-  private static Logger LOG = LoggerFactory
+  private static final Logger LOG = LoggerFactory
       .getLogger(TestNodeReportHandler.class);
   private NodeReportHandler nodeReportHandler;
   private SCMNodeManager nodeManager;
   private String storagePath = GenericTestUtils.getRandomizedTempPath()
       .concat("/" + UUID.randomUUID().toString());
-  ;
 
   @Before
   public void resetEventCollector() throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
index 5efcdd1..a513f6c 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java
@@ -121,7 +121,7 @@ public class TestEndPoint {
     config.set(OZONE_METADATA_DIRS, testDir.getAbsolutePath());
     config
         .setBoolean(OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_RANDOM_PORT, true);
-    config.set(HddsConfigKeys.HDDS_COMMAND_STATUS_REPORT_INTERVAL,"1s");
+    config.set(HddsConfigKeys.HDDS_COMMAND_STATUS_REPORT_INTERVAL, "1s");
   }
 
   @Test
@@ -228,8 +228,8 @@ public class TestEndPoint {
     try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
         nonExistentServerAddress, 1000)) {
       rpcEndPoint.setState(EndpointStateMachine.EndPointStates.GETVERSION);
-      OzoneContainer ozoneContainer = new OzoneContainer(TestUtils.randomDatanodeDetails(),
-          conf, null);
+      OzoneContainer ozoneContainer = new OzoneContainer(
+          TestUtils.randomDatanodeDetails(), conf, null);
       VersionEndpointTask versionTask = new VersionEndpointTask(rpcEndPoint,
           conf, ozoneContainer);
       EndpointStateMachine.EndPointStates newState = versionTask.call();
@@ -405,7 +405,8 @@ public class TestEndPoint {
       assertEquals(0, scmServerImpl.getCommandStatusReportCount());
 
       // Send heartbeat again from heartbeat endpoint task
-      final StateContext stateContext = heartbeatTaskHelper(serverAddress, 3000);
+      final StateContext stateContext = heartbeatTaskHelper(
+          serverAddress, 3000);
       Map<Long, CommandStatus> map = stateContext.getCommandStatusMap();
       assertNotNull(map);
       assertEquals("Should have 3 objects", 3, map.size());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/placement/TestContainerPlacement.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/placement/TestContainerPlacement.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/placement/TestContainerPlacement.java
index 802f2ef..1c80880 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/placement/TestContainerPlacement.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/placement/TestContainerPlacement.java
@@ -87,11 +87,13 @@ public class TestContainerPlacement {
     for (int x = 0; x < opsCount; x++) {
       long containerSize = random.nextInt(100) * OzoneConsts.GB;
       List<DatanodeDetails> nodesCapacity =
-          capacityPlacer.chooseDatanodes(new ArrayList<>(), nodesRequired, containerSize);
+          capacityPlacer.chooseDatanodes(new ArrayList<>(), nodesRequired,
+              containerSize);
       assertEquals(nodesRequired, nodesCapacity.size());
 
       List<DatanodeDetails> nodesRandom =
-          randomPlacer.chooseDatanodes(nodesCapacity, nodesRequired, containerSize);
+          randomPlacer.chooseDatanodes(nodesCapacity, nodesRequired,
+              containerSize);
 
       // One fifth of all calls are delete
       if (x % 5 == 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
index c5f0689..b6005d1 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.ozone.client;
 
-import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
@@ -158,9 +157,9 @@ public class ObjectStore {
     private OzoneVolume currentValue;
 
     /**
-     * Creates an Iterator to iterate over all volumes after prevVolume of the user.
-     * If prevVolume is null it iterates from the first volume. The returned volumes
-     * match volume prefix.
+     * Creates an Iterator to iterate over all volumes after
+     * prevVolume of the user. If prevVolume is null it iterates from the
+     * first volume. The returned volumes match volume prefix.
      * @param user user name
      * @param volPrefix volume prefix to match
      */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java
index 8915a82..eb15396 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineClose.java
@@ -28,10 +28,7 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
 import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
 import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
-import org.apache.hadoop.ozone.HddsDatanodeService;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.container.common.impl.ContainerData;
-import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -39,7 +36,6 @@ import org.junit.Test;
 
 import java.io.IOException;
 import java.util.NavigableSet;
-import java.util.UUID;
 import java.util.concurrent.TimeoutException;
 
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
index 79d1bef..e27de3d 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/RatisTestHelper.java
@@ -23,7 +23,6 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.client.rpc.RpcClient;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.ozone.container.ContainerTestHelper;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.ratis.rpc.RpcType;
 import org.apache.ratis.rpc.SupportedRpcType;
@@ -54,9 +53,9 @@ public interface RatisTestHelper {
      *   OZONE_ENABLED = true
      *   RATIS_ENABLED = true
      */
-    public RatisTestSuite(final Class<?> clazz)
+    public RatisTestSuite()
         throws IOException, TimeoutException, InterruptedException {
-      conf = newOzoneConfiguration(clazz, RPC);
+      conf = newOzoneConfiguration(RPC);
       cluster = newMiniOzoneCluster(NUM_DATANODES, conf);
     }
 
@@ -84,8 +83,7 @@ public interface RatisTestHelper {
     }
   }
 
-  static OzoneConfiguration newOzoneConfiguration(
-      Class<?> clazz, RpcType rpc) {
+  static OzoneConfiguration newOzoneConfiguration(RpcType rpc) {
     final OzoneConfiguration conf = new OzoneConfiguration();
     initRatisConf(rpc, conf);
     return conf;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java
index fff8611..d0e173c 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManagerHelper.java
@@ -27,9 +27,7 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.ozone.container.common.impl.ContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
-import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
index 43bd23e..ffdba7e 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestCloseContainerHandlingByClient.java
@@ -73,8 +73,7 @@ public class TestCloseContainerHandlingByClient {
   /**
    * Create a MiniDFSCluster for testing.
    * <p>
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "distributed"
+   * Ozone is made active by setting OZONE_ENABLED = true
    *
    * @throws IOException
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
index 0eb1ad4..0c86828 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
@@ -38,12 +38,10 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
 import org.apache.hadoop.ozone.container.common.helpers.KeyData;
 import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
-import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.junit.Assert;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
index e49e283..3a36331 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java
@@ -75,7 +75,6 @@ import java.util.UUID;
 
 import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Stage.COMBINED;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
-import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_ROOT_PREFIX;
 import static org.apache.hadoop.ozone.container.ContainerTestHelper.getChunk;
 import static org.apache.hadoop.ozone.container.ContainerTestHelper.getData;
 import static org.apache.hadoop.ozone.container.ContainerTestHelper.setDataChecksum;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerRatis.java
index cf469ec..0f5d21f 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerRatis.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerRatis.java
@@ -53,8 +53,7 @@ public class TestOzoneContainerRatis {
   public Timeout testTimeout = new Timeout(300000);
 
   static OzoneConfiguration newOzoneConfiguration() {
-    final OzoneConfiguration conf = new OzoneConfiguration();
-    return conf;
+    return new OzoneConfiguration();
   }
 
   private static void runTestOzoneContainerViaDataNodeRatis(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestRatisManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestRatisManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestRatisManager.java
index a0c8cfe..1e78ec6 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestRatisManager.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestRatisManager.java
@@ -23,7 +23,6 @@ import org.apache.hadoop.ozone.HddsDatanodeService;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.RatisTestHelper;
-import org.apache.hadoop.ozone.container.ContainerTestHelper;
 import org.apache.ratis.rpc.RpcType;
 import org.apache.ratis.rpc.SupportedRpcType;
 import org.junit.Ignore;
@@ -46,8 +45,7 @@ public class TestRatisManager {
       TestRatisManager.class);
 
   static OzoneConfiguration newOzoneConfiguration() {
-    final OzoneConfiguration conf = new OzoneConfiguration();
-    return conf;
+    return new OzoneConfiguration();
   }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestAllocateContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestAllocateContainer.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestAllocateContainer.java
index 144c562..dfe7894 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestAllocateContainer.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestAllocateContainer.java
@@ -30,8 +30,6 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
-import java.io.IOException;
-
 /**
  * Test allocate container calls.
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
index 9828b95..63215e0 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/TestOzoneWebAccess.java
@@ -23,7 +23,6 @@ import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.rest.headers.Header;
-import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.http.HttpResponse;
 import org.apache.http.client.methods.HttpPost;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
index 2eedb58..e866d20 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBuckets.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.ozone.client.rest.RestClient;
 import org.apache.hadoop.ozone.client.rpc.RpcClient;
 import org.apache.hadoop.ozone.web.request.OzoneQuota;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
-import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -95,8 +94,6 @@ public class TestBuckets {
       InterruptedException {
     conf = new OzoneConfiguration();
 
-    String path = GenericTestUtils
-        .getTempPath(TestBuckets.class.getSimpleName());
     cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(3)
         .build();
@@ -128,7 +125,7 @@ public class TestBuckets {
   }
 
   static void runTestCreateBucket(ClientProtocol client)
-      throws OzoneException, IOException, ParseException {
+      throws IOException {
     String volumeName = OzoneUtils.getRequestID().toLowerCase();
     VolumeArgs volumeArgs = VolumeArgs.newBuilder()
         .setOwner("bilbo")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java
index 68aee44..687e7e6 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestBucketsRatis.java
@@ -61,7 +61,7 @@ public class TestBucketsRatis {
 
   @BeforeClass
   public static void init() throws Exception {
-    suite = new RatisTestHelper.RatisTestSuite(TestBucketsRatis.class);
+    suite = new RatisTestHelper.RatisTestSuite();
     conf = suite.getConf();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
index 5eeeacf..915d0f6 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeysRatis.java
@@ -57,7 +57,7 @@ public class TestKeysRatis {
 
   @BeforeClass
   public static void init() throws Exception {
-    suite = new RatisTestHelper.RatisTestSuite(TestBucketsRatis.class);
+    suite = new RatisTestHelper.RatisTestSuite();
     path = GenericTestUtils.getTempPath(TestKeysRatis.class.getSimpleName());
     ozoneCluster = suite.getCluster();
     ozoneCluster.waitForClusterToBeReady();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
index 83394f0..d0561d6 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
@@ -43,28 +43,15 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Random;
-import java.util.concurrent.TimeUnit;
 
 import static org.apache.hadoop.ozone
     .OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT;
 import static org.apache.hadoop.ozone
     .OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY;
 import static org.apache.hadoop.ozone
-    .OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
-import static org.apache.hadoop.ozone
-    .OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL_DEFAULT;
-import static org.apache.hadoop.ozone
-    .OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT;
-import static org.apache.hadoop.ozone
-    .OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT;
-import static org.apache.hadoop.ozone
     .OzoneConfigKeys.OZONE_KEY_PREALLOCATION_MAXSIZE;
 import static org.apache.hadoop.ozone
     .OzoneConfigKeys.OZONE_KEY_PREALLOCATION_MAXSIZE_DEFAULT;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
-    .OZONE_OPEN_KEY_CLEANUP_SERVICE_INTERVAL_SECONDS;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
-    .OZONE_OPEN_KEY_CLEANUP_SERVICE_INTERVAL_SECONDS_DEFAULT;
 import static org.apache.hadoop.ozone
     .OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT;
 import static org.apache.hadoop.ozone

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
index 6906a9d..b876dc2 100644
--- a/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
+++ b/hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ozone.client.ObjectStore;
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneClient;
@@ -60,7 +59,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.ozone.client.io.OzoneInputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 
 import static org.apache.hadoop.fs.ozone.Constants.OZONE_DEFAULT_USER;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c5629d54/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java
index 3884edd..51018a1 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/scm/cli/SQLCLI.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.ozone.scm.cli;
 
 import com.google.common.base.Preconditions;
 import com.google.common.primitives.Longs;
-import com.google.protobuf.ByteString;
 import org.apache.commons.cli.BasicParser;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
@@ -31,7 +30,6 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneAclInfo;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketInfo;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
@@ -56,8 +54,6 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.SQLException;
 import java.sql.Statement;
-import java.util.HashSet;
-import java.util.Set;
 
 import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB_SUFFIX;
 import static org.apache.hadoop.ozone.OzoneConsts.OM_DB_NAME;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[24/50] [abbrv] hadoop git commit: HDFS-13634. RBF: Configurable value in xml for async connection request queue size. Contributed by CR Hota.

Posted by eh...@apache.org.
HDFS-13634. RBF: Configurable value in xml for async connection request queue size. Contributed by CR Hota.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a0ebb6b3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a0ebb6b3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a0ebb6b3

Branch: refs/heads/HDFS-12090
Commit: a0ebb6b39f2932d3ea2fb5e287f52b841e108428
Parents: 0bd4217
Author: Yiqun Lin <yq...@apache.org>
Authored: Wed Aug 29 16:15:22 2018 +0800
Committer: Yiqun Lin <yq...@apache.org>
Committed: Wed Aug 29 16:15:22 2018 +0800

----------------------------------------------------------------------
 .../federation/router/ConnectionManager.java      | 18 +++++++++++-------
 .../server/federation/router/RBFConfigKeys.java   |  5 +++++
 .../src/main/resources/hdfs-rbf-default.xml       |  8 ++++++++
 3 files changed, 24 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0ebb6b3/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java
index 0b50845..9fb83e4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java
@@ -49,9 +49,6 @@ public class ConnectionManager {
   private static final Logger LOG =
       LoggerFactory.getLogger(ConnectionManager.class);
 
-  /** Number of parallel new connections to create. */
-  protected static final int MAX_NEW_CONNECTIONS = 100;
-
   /** Minimum amount of active connections: 50%. */
   protected static final float MIN_ACTIVE_RATIO = 0.5f;
 
@@ -77,8 +74,10 @@ public class ConnectionManager {
   private final Lock writeLock = readWriteLock.writeLock();
 
   /** Queue for creating new connections. */
-  private final BlockingQueue<ConnectionPool> creatorQueue =
-      new ArrayBlockingQueue<>(MAX_NEW_CONNECTIONS);
+  private final BlockingQueue<ConnectionPool> creatorQueue;
+  /** Max size of queue for creating new connections. */
+  private final int creatorQueueMaxSize;
+
   /** Create new connections asynchronously. */
   private final ConnectionCreator creator;
   /** Periodic executor to remove stale connection pools. */
@@ -106,7 +105,12 @@ public class ConnectionManager {
     this.pools = new HashMap<>();
 
     // Create connections in a thread asynchronously
-    this.creator = new ConnectionCreator(creatorQueue);
+    this.creatorQueueMaxSize = this.conf.getInt(
+        RBFConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_CREATOR_QUEUE_SIZE,
+        RBFConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_CREATOR_QUEUE_SIZE_DEFAULT
+        );
+    this.creatorQueue = new ArrayBlockingQueue<>(this.creatorQueueMaxSize);
+    this.creator = new ConnectionCreator(this.creatorQueue);
     this.creator.setDaemon(true);
 
     // Cleanup periods
@@ -213,7 +217,7 @@ public class ConnectionManager {
     if (conn == null || !conn.isUsable()) {
       if (!this.creatorQueue.offer(pool)) {
         LOG.error("Cannot add more than {} connections at the same time",
-            MAX_NEW_CONNECTIONS);
+            this.creatorQueueMaxSize);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0ebb6b3/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
index 87df5d2..997e1dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
@@ -93,6 +93,11 @@ public class RBFConfigKeys extends CommonConfigurationKeysPublic {
       TimeUnit.SECONDS.toMillis(5);
 
   // HDFS Router NN client
+  public static final String
+      DFS_ROUTER_NAMENODE_CONNECTION_CREATOR_QUEUE_SIZE =
+      FEDERATION_ROUTER_PREFIX + "connection.creator.queue-size";
+  public static final int
+      DFS_ROUTER_NAMENODE_CONNECTION_CREATOR_QUEUE_SIZE_DEFAULT = 100;
   public static final String DFS_ROUTER_NAMENODE_CONNECTION_POOL_SIZE =
       FEDERATION_ROUTER_PREFIX + "connection.pool-size";
   public static final int DFS_ROUTER_NAMENODE_CONNECTION_POOL_SIZE_DEFAULT =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0ebb6b3/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
index 8be5b8a..ed39d4b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
@@ -94,6 +94,14 @@
   </property>
 
   <property>
+    <name>dfs.federation.router.connection.creator.queue-size</name>
+    <value>100</value>
+    <description>
+      Size of async connection creator queue.
+    </description>
+  </property>
+
+  <property>
     <name>dfs.federation.router.connection.pool-size</name>
     <value>1</value>
     <description>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[07/50] [abbrv] hadoop git commit: HDFS-13858. RBF: Add check to have single valid argument to safemode command. Contributed by Ayush Saxena.

Posted by eh...@apache.org.
HDFS-13858. RBF: Add check to have single valid argument to safemode command. Contributed by Ayush Saxena.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/75691ad6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/75691ad6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/75691ad6

Branch: refs/heads/HDFS-12090
Commit: 75691ad600473d4d315434b0876d6d10d3050a6b
Parents: 3974427
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Aug 28 09:21:07 2018 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Tue Aug 28 09:21:07 2018 +0530

----------------------------------------------------------------------
 .../hadoop/hdfs/tools/federation/RouterAdmin.java     |  6 ++++++
 .../server/federation/router/TestRouterAdminCLI.java  | 14 ++++++++++++++
 2 files changed, 20 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/75691ad6/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
index 91e1669..f88d0a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
@@ -218,6 +218,10 @@ public class RouterAdmin extends Configured implements Tool {
               "Successfully clear quota for mount point " + argv[i]);
         }
       } else if ("-safemode".equals(cmd)) {
+        if (argv.length > 2) {
+          throw new IllegalArgumentException(
+              "Too many arguments, Max=1 argument allowed only");
+        }
         manageSafeMode(argv[i]);
       } else if ("-nameservice".equals(cmd)) {
         String subcmd = argv[i];
@@ -712,6 +716,8 @@ public class RouterAdmin extends Configured implements Tool {
     } else if (cmd.equals("get")) {
       boolean result = getSafeMode();
       System.out.println("Safe Mode: " + result);
+    } else {
+      throw new IllegalArgumentException("Invalid argument: " + cmd);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75691ad6/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
index 2da5fb9..2682e9a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
@@ -519,6 +519,7 @@ public class TestRouterAdminCLI {
     assertTrue(routerContext.getRouter().getSafemodeService().isInSafeMode());
 
     System.setOut(new PrintStream(out));
+    System.setErr(new PrintStream(err));
     assertEquals(0, ToolRunner.run(admin,
         new String[] {"-safemode", "get"}));
     assertTrue(out.toString().contains("true"));
@@ -534,6 +535,19 @@ public class TestRouterAdminCLI {
     assertEquals(0, ToolRunner.run(admin,
         new String[] {"-safemode", "get"}));
     assertTrue(out.toString().contains("false"));
+
+    out.reset();
+    assertEquals(-1, ToolRunner.run(admin,
+        new String[] {"-safemode", "get", "-random", "check" }));
+    assertTrue(err.toString(), err.toString()
+        .contains("safemode: Too many arguments, Max=1 argument allowed only"));
+    err.reset();
+
+    assertEquals(-1,
+        ToolRunner.run(admin, new String[] {"-safemode", "check" }));
+    assertTrue(err.toString(),
+        err.toString().contains("safemode: Invalid argument: check"));
+    err.reset();
   }
 
   @Test


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[13/50] [abbrv] hadoop git commit: HDFS-13861. RBF: Illegal Router Admin command leads to printing usage for all commands. Contributed by Ayush Saxena.

Posted by eh...@apache.org.
HDFS-13861. RBF: Illegal Router Admin command leads to printing usage for all commands. Contributed by Ayush Saxena.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/cb9d371a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cb9d371a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cb9d371a

Branch: refs/heads/HDFS-12090
Commit: cb9d371ae2cda1624fc83316ddc09de37d8d0bd3
Parents: fd089ca
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Wed Aug 29 00:29:05 2018 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Wed Aug 29 00:29:05 2018 +0530

----------------------------------------------------------------------
 .../hdfs/tools/federation/RouterAdmin.java      | 92 +++++++++++++-------
 .../federation/router/TestRouterAdminCLI.java   | 68 +++++++++++++++
 2 files changed, 130 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb9d371a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
index f88d0a6..46be373 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
@@ -94,25 +94,58 @@ public class RouterAdmin extends Configured implements Tool {
    * Print the usage message.
    */
   public void printUsage() {
-    String usage = "Federation Admin Tools:\n"
-        + "\t[-add <source> <nameservice1, nameservice2, ...> <destination> "
-        + "[-readonly] [-order HASH|LOCAL|RANDOM|HASH_ALL] "
-        + "-owner <owner> -group <group> -mode <mode>]\n"
-        + "\t[-update <source> <nameservice1, nameservice2, ...> <destination> "
-        + "[-readonly] [-order HASH|LOCAL|RANDOM|HASH_ALL] "
-        + "-owner <owner> -group <group> -mode <mode>]\n"
-        + "\t[-rm <source>]\n"
-        + "\t[-ls <path>]\n"
-        + "\t[-setQuota <path> -nsQuota <nsQuota> -ssQuota "
-        + "<quota in bytes or quota size string>]\n"
-        + "\t[-clrQuota <path>]\n"
-        + "\t[-safemode enter | leave | get]\n"
-        + "\t[-nameservice enable | disable <nameservice>]\n"
-        + "\t[-getDisabledNameservices]\n";
+    String usage = getUsage(null);
+    System.out.println(usage);
+  }
 
+  private void printUsage(String cmd) {
+    String usage = getUsage(cmd);
     System.out.println(usage);
   }
 
+  private String getUsage(String cmd) {
+    if (cmd == null) {
+      String[] commands =
+          {"-add", "-update", "-rm", "-ls", "-setQuota", "-clrQuota",
+              "-safemode", "-nameservice", "-getDisabledNameservices"};
+      StringBuilder usage = new StringBuilder();
+      usage.append("Usage: hdfs routeradmin :\n");
+      for (int i = 0; i < commands.length; i++) {
+        usage.append(getUsage(commands[i]));
+        if (i + 1 < commands.length) {
+          usage.append("\n");
+        }
+      }
+      return usage.toString();
+    }
+    if (cmd.equals("-add")) {
+      return "\t[-add <source> <nameservice1, nameservice2, ...> <destination> "
+          + "[-readonly] [-order HASH|LOCAL|RANDOM|HASH_ALL] "
+          + "-owner <owner> -group <group> -mode <mode>]";
+    } else if (cmd.equals("-update")) {
+      return "\t[-update <source> <nameservice1, nameservice2, ...> "
+          + "<destination> "
+          + "[-readonly] [-order HASH|LOCAL|RANDOM|HASH_ALL] "
+          + "-owner <owner> -group <group> -mode <mode>]";
+    } else if (cmd.equals("-rm")) {
+      return "\t[-rm <source>]";
+    } else if (cmd.equals("-ls")) {
+      return "\t[-ls <path>]";
+    } else if (cmd.equals("-setQuota")) {
+      return "\t[-setQuota <path> -nsQuota <nsQuota> -ssQuota "
+          + "<quota in bytes or quota size string>]";
+    } else if (cmd.equals("-clrQuota")) {
+      return "\t[-clrQuota <path>]";
+    } else if (cmd.equals("-safemode")) {
+      return "\t[-safemode enter | leave | get]";
+    } else if (cmd.equals("-nameservice")) {
+      return "\t[-nameservice enable | disable <nameservice>]";
+    } else if (cmd.equals("-getDisabledNameservices")) {
+      return "\t[-getDisabledNameservices]";
+    }
+    return getUsage(null);
+  }
+
   @Override
   public int run(String[] argv) throws Exception {
     if (argv.length < 1) {
@@ -129,43 +162,43 @@ public class RouterAdmin extends Configured implements Tool {
     if ("-add".equals(cmd)) {
       if (argv.length < 4) {
         System.err.println("Not enough parameters specified for cmd " + cmd);
-        printUsage();
+        printUsage(cmd);
         return exitCode;
       }
     } else if ("-update".equals(cmd)) {
       if (argv.length < 4) {
         System.err.println("Not enough parameters specified for cmd " + cmd);
-        printUsage();
+        printUsage(cmd);
         return exitCode;
       }
-    } else if ("-rm".equalsIgnoreCase(cmd)) {
+    } else if ("-rm".equals(cmd)) {
       if (argv.length < 2) {
         System.err.println("Not enough parameters specified for cmd " + cmd);
-        printUsage();
+        printUsage(cmd);
         return exitCode;
       }
-    } else if ("-setQuota".equalsIgnoreCase(cmd)) {
+    } else if ("-setQuota".equals(cmd)) {
       if (argv.length < 4) {
         System.err.println("Not enough parameters specified for cmd " + cmd);
-        printUsage();
+        printUsage(cmd);
         return exitCode;
       }
-    } else if ("-clrQuota".equalsIgnoreCase(cmd)) {
+    } else if ("-clrQuota".equals(cmd)) {
       if (argv.length < 2) {
         System.err.println("Not enough parameters specified for cmd " + cmd);
-        printUsage();
+        printUsage(cmd);
         return exitCode;
       }
-    } else if ("-safemode".equalsIgnoreCase(cmd)) {
+    } else if ("-safemode".equals(cmd)) {
       if (argv.length < 2) {
         System.err.println("Not enough parameters specified for cmd " + cmd);
-        printUsage();
+        printUsage(cmd);
         return exitCode;
       }
-    } else if ("-nameservice".equalsIgnoreCase(cmd)) {
+    } else if ("-nameservice".equals(cmd)) {
       if (argv.length < 3) {
         System.err.println("Not enough parameters specificed for cmd " + cmd);
-        printUsage();
+        printUsage(cmd);
         return exitCode;
       }
     }
@@ -230,14 +263,13 @@ public class RouterAdmin extends Configured implements Tool {
       } else if ("-getDisabledNameservices".equals(cmd)) {
         getDisabledNameservices();
       } else {
-        printUsage();
-        return exitCode;
+        throw new IllegalArgumentException("Unknown Command: " + cmd);
       }
     } catch (IllegalArgumentException arge) {
       debugException = arge;
       exitCode = -1;
       System.err.println(cmd.substring(1) + ": " + arge.getLocalizedMessage());
-      printUsage();
+      printUsage(cmd);
     } catch (RemoteException e) {
       // This is a error returned by the server.
       // Print out the first line of the error message, ignore the stack trace.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb9d371a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
index 2682e9a..0c7321f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
@@ -437,6 +437,74 @@ public class TestRouterAdminCLI {
   }
 
   @Test
+  public void testInvalidArgumentMessage() throws Exception {
+    String nsId = "ns0";
+    String src = "/testSource";
+    System.setOut(new PrintStream(out));
+    String[] argv = new String[] {"-add", src, nsId};
+    assertEquals(-1, ToolRunner.run(admin, argv));
+    assertTrue(out.toString().contains(
+        "\t[-add <source> <nameservice1, nameservice2, ...> <destination> "
+            + "[-readonly] [-order HASH|LOCAL|RANDOM|HASH_ALL] "
+            + "-owner <owner> -group <group> -mode <mode>]"));
+    out.reset();
+
+    argv = new String[] {"-update", src, nsId};
+    assertEquals(-1, ToolRunner.run(admin, argv));
+    assertTrue(out.toString().contains(
+        "\t[-update <source> <nameservice1, nameservice2, ...> <destination> "
+            + "[-readonly] [-order HASH|LOCAL|RANDOM|HASH_ALL] "
+            + "-owner <owner> -group <group> -mode <mode>]"));
+    out.reset();
+
+    argv = new String[] {"-rm"};
+    assertEquals(-1, ToolRunner.run(admin, argv));
+    assertTrue(out.toString().contains("\t[-rm <source>]"));
+    out.reset();
+
+    argv = new String[] {"-setQuota", src};
+    assertEquals(-1, ToolRunner.run(admin, argv));
+    assertTrue(out.toString()
+        .contains("\t[-setQuota <path> -nsQuota <nsQuota> -ssQuota "
+            + "<quota in bytes or quota size string>]"));
+    out.reset();
+
+    argv = new String[] {"-clrQuota"};
+    assertEquals(-1, ToolRunner.run(admin, argv));
+    assertTrue(out.toString().contains("\t[-clrQuota <path>]"));
+    out.reset();
+
+    argv = new String[] {"-safemode"};
+    assertEquals(-1, ToolRunner.run(admin, argv));
+    assertTrue(out.toString().contains("\t[-safemode enter | leave | get]"));
+    out.reset();
+
+    argv = new String[] {"-nameservice", nsId};
+    assertEquals(-1, ToolRunner.run(admin, argv));
+    assertTrue(out.toString()
+        .contains("\t[-nameservice enable | disable <nameservice>]"));
+    out.reset();
+
+    argv = new String[] {"-Random"};
+    assertEquals(-1, ToolRunner.run(admin, argv));
+    String expected = "Usage: hdfs routeradmin :\n"
+        + "\t[-add <source> <nameservice1, nameservice2, ...> <destination> "
+        + "[-readonly] [-order HASH|LOCAL|RANDOM|HASH_ALL] "
+        + "-owner <owner> -group <group> -mode <mode>]\n"
+        + "\t[-update <source> <nameservice1, nameservice2, ...> "
+        + "<destination> " + "[-readonly] [-order HASH|LOCAL|RANDOM|HASH_ALL] "
+        + "-owner <owner> -group <group> -mode <mode>]\n" + "\t[-rm <source>]\n"
+        + "\t[-ls <path>]\n"
+        + "\t[-setQuota <path> -nsQuota <nsQuota> -ssQuota "
+        + "<quota in bytes or quota size string>]\n" + "\t[-clrQuota <path>]\n"
+        + "\t[-safemode enter | leave | get]\n"
+        + "\t[-nameservice enable | disable <nameservice>]\n"
+        + "\t[-getDisabledNameservices]";
+    assertTrue(out.toString(), out.toString().contains(expected));
+    out.reset();
+  }
+
+  @Test
   public void testSetAndClearQuota() throws Exception {
     String nsId = "ns0";
     String src = "/test-QuotaMounttable";


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[09/50] [abbrv] hadoop git commit: HDDS-332. Remove the ability to configure ozone.handler.type Contributed by Nandakumar and Anu Engineer.

Posted by eh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
index 2200cd8..f56cbe8 100644
--- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
+++ b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/hdfs/server/datanode/ObjectStoreHandler.java
@@ -1,64 +1,58 @@
 /**
- * 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
- *
+ * 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.
+ * 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.hdfs.server.datanode;
 
-import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForBlockClients;
-import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
-import static org.apache.hadoop.ozone.OmUtils.getOmAddress;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.*;
-import static com.sun.jersey.api.core.ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS;
-import static com.sun.jersey.api.core.ResourceConfig.FEATURE_TRACE;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.HashMap;
-import java.util.Map;
-
 import com.sun.jersey.api.container.ContainerFactory;
 import com.sun.jersey.api.core.ApplicationAdapter;
-
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolPB;
+import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ipc.Client;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB;
 import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.web.ObjectStoreApplication;
 import org.apache.hadoop.ozone.web.handlers.ServiceFilter;
+import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
 import org.apache.hadoop.ozone.web.netty.ObjectStoreJerseyContainer;
-import org.apache.hadoop.hdds.scm.protocolPB
-    .ScmBlockLocationProtocolClientSideTranslatorPB;
-import org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolPB;
+import org.apache.hadoop.ozone.web.storage.DistributedStorageHandler;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.Client;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.scm.protocolPB
-    .StorageContainerLocationProtocolClientSideTranslatorPB;
-import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
-import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
-import org.apache.hadoop.ozone.web.storage.DistributedStorageHandler;
-import org.apache.hadoop.ozone.web.localstorage.LocalStorageHandler;
-import org.apache.hadoop.security.UserGroupInformation;
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.Map;
+
+import static com.sun.jersey.api.core.ResourceConfig.FEATURE_TRACE;
+import static com.sun.jersey.api.core.ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS;
+import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForBlockClients;
+import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
+import static org.apache.hadoop.ozone.OmUtils.getOmAddress;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_TRACE_ENABLED_DEFAULT;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_TRACE_ENABLED_KEY;
 
 /**
  * Implements object store handling within the DataNode process.  This class is
@@ -86,69 +80,49 @@ public final class ObjectStoreHandler implements Closeable {
    * @throws IOException if there is an I/O error
    */
   public ObjectStoreHandler(Configuration conf) throws IOException {
-    String shType = conf.getTrimmed(OZONE_HANDLER_TYPE_KEY,
-        OZONE_HANDLER_TYPE_DEFAULT);
-    LOG.info("ObjectStoreHandler initializing with {}: {}",
-        OZONE_HANDLER_TYPE_KEY, shType);
     boolean ozoneTrace = conf.getBoolean(OZONE_TRACE_ENABLED_KEY,
         OZONE_TRACE_ENABLED_DEFAULT);
 
     // Initialize Jersey container for object store web application.
-    if (OzoneConsts.OZONE_HANDLER_DISTRIBUTED.equalsIgnoreCase(shType)) {
-      RPC.setProtocolEngine(conf, StorageContainerLocationProtocolPB.class,
-          ProtobufRpcEngine.class);
-      long scmVersion =
-          RPC.getProtocolVersion(StorageContainerLocationProtocolPB.class);
-
-      InetSocketAddress scmAddress =
-          getScmAddressForClients(conf);
-      this.storageContainerLocationClient =
-          new StorageContainerLocationProtocolClientSideTranslatorPB(
-              RPC.getProxy(StorageContainerLocationProtocolPB.class, scmVersion,
-              scmAddress, UserGroupInformation.getCurrentUser(), conf,
-              NetUtils.getDefaultSocketFactory(conf),
-              Client.getRpcTimeout(conf)));
-
-      InetSocketAddress scmBlockAddress =
-          getScmAddressForBlockClients(conf);
-      this.scmBlockLocationClient =
-          new ScmBlockLocationProtocolClientSideTranslatorPB(
-              RPC.getProxy(ScmBlockLocationProtocolPB.class, scmVersion,
-                  scmBlockAddress, UserGroupInformation.getCurrentUser(), conf,
-                  NetUtils.getDefaultSocketFactory(conf),
-                  Client.getRpcTimeout(conf)));
-
-      RPC.setProtocolEngine(conf, OzoneManagerProtocolPB.class,
-          ProtobufRpcEngine.class);
-      long omVersion =
-          RPC.getProtocolVersion(OzoneManagerProtocolPB.class);
-      InetSocketAddress omAddress = getOmAddress(conf);
-      this.ozoneManagerClient =
-          new OzoneManagerProtocolClientSideTranslatorPB(
-              RPC.getProxy(OzoneManagerProtocolPB.class, omVersion,
-                  omAddress, UserGroupInformation.getCurrentUser(), conf,
-              NetUtils.getDefaultSocketFactory(conf),
-              Client.getRpcTimeout(conf)));
-
-      storageHandler = new DistributedStorageHandler(
-          new OzoneConfiguration(conf),
-          this.storageContainerLocationClient,
-          this.ozoneManagerClient);
-    } else {
-      if (OzoneConsts.OZONE_HANDLER_LOCAL.equalsIgnoreCase(shType)) {
-        storageHandler = new LocalStorageHandler(conf);
-        this.storageContainerLocationClient = null;
-        this.scmBlockLocationClient = null;
-        this.ozoneManagerClient = null;
-      } else {
-        throw new IllegalArgumentException(
-            String.format("Unrecognized value for %s: %s,"
-                + " Allowed values are %s,%s",
-                OZONE_HANDLER_TYPE_KEY, shType,
-                OzoneConsts.OZONE_HANDLER_DISTRIBUTED,
-                OzoneConsts.OZONE_HANDLER_LOCAL));
-      }
-    }
+    RPC.setProtocolEngine(conf, StorageContainerLocationProtocolPB.class,
+        ProtobufRpcEngine.class);
+    long scmVersion =
+        RPC.getProtocolVersion(StorageContainerLocationProtocolPB.class);
+
+    InetSocketAddress scmAddress =
+        getScmAddressForClients(conf);
+    this.storageContainerLocationClient =
+        new StorageContainerLocationProtocolClientSideTranslatorPB(
+            RPC.getProxy(StorageContainerLocationProtocolPB.class, scmVersion,
+                scmAddress, UserGroupInformation.getCurrentUser(), conf,
+                NetUtils.getDefaultSocketFactory(conf),
+                Client.getRpcTimeout(conf)));
+
+    InetSocketAddress scmBlockAddress =
+        getScmAddressForBlockClients(conf);
+    this.scmBlockLocationClient =
+        new ScmBlockLocationProtocolClientSideTranslatorPB(
+            RPC.getProxy(ScmBlockLocationProtocolPB.class, scmVersion,
+                scmBlockAddress, UserGroupInformation.getCurrentUser(), conf,
+                NetUtils.getDefaultSocketFactory(conf),
+                Client.getRpcTimeout(conf)));
+
+    RPC.setProtocolEngine(conf, OzoneManagerProtocolPB.class,
+        ProtobufRpcEngine.class);
+    long omVersion =
+        RPC.getProtocolVersion(OzoneManagerProtocolPB.class);
+    InetSocketAddress omAddress = getOmAddress(conf);
+    this.ozoneManagerClient =
+        new OzoneManagerProtocolClientSideTranslatorPB(
+            RPC.getProxy(OzoneManagerProtocolPB.class, omVersion,
+                omAddress, UserGroupInformation.getCurrentUser(), conf,
+                NetUtils.getDefaultSocketFactory(conf),
+                Client.getRpcTimeout(conf)));
+
+    storageHandler = new DistributedStorageHandler(
+        new OzoneConfiguration(conf),
+        this.storageContainerLocationClient,
+        this.ozoneManagerClient);
     ApplicationAdapter aa =
         new ApplicationAdapter(new ObjectStoreApplication());
     Map<String, Object> settingsMap = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/StorageHandlerBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/StorageHandlerBuilder.java b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/StorageHandlerBuilder.java
index f86f247..b3c3391 100644
--- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/StorageHandlerBuilder.java
+++ b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/StorageHandlerBuilder.java
@@ -19,10 +19,11 @@
 package org.apache.hadoop.ozone.web.handlers;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
-import org.apache.hadoop.ozone.web.localstorage.LocalStorageHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
 
 /**
  * This class is responsible for providing a {@link StorageHandler}
@@ -31,6 +32,9 @@ import org.apache.hadoop.ozone.web.localstorage.LocalStorageHandler;
 @InterfaceAudience.Private
 public final class StorageHandlerBuilder {
 
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StorageHandlerBuilder.class);
   private static final ThreadLocal<StorageHandler>
       STORAGE_HANDLER_THREAD_LOCAL = new ThreadLocal<>();
 
@@ -40,15 +44,15 @@ public final class StorageHandlerBuilder {
    *
    * @return StorageHandler from thread-local storage
    */
-  public static StorageHandler getStorageHandler() {
+  public static StorageHandler getStorageHandler() throws IOException {
     StorageHandler storageHandler = STORAGE_HANDLER_THREAD_LOCAL.get();
     if (storageHandler != null) {
       return storageHandler;
     } else {
-      // This only happens while using mvn jetty:run for testing.
-      Configuration conf = new OzoneConfiguration();
-      return new LocalStorageHandler(conf);
+      LOG.error("No Storage Handler Configured.");
+      throw new IOException("Invalid Handler Configuration");
     }
+
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java
deleted file mode 100644
index 89158cb..0000000
--- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/localstorage/LocalStorageHandler.java
+++ /dev/null
@@ -1,385 +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.ozone.web.localstorage;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ozone.client.io.LengthInputStream;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
-import org.apache.hadoop.ozone.web.handlers.BucketArgs;
-import org.apache.hadoop.ozone.web.handlers.KeyArgs;
-import org.apache.hadoop.ozone.web.handlers.ListArgs;
-import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
-import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
-import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.web.request.OzoneQuota;
-import org.apache.hadoop.ozone.web.response.BucketInfo;
-import org.apache.hadoop.ozone.web.response.KeyInfo;
-import org.apache.hadoop.ozone.web.response.ListBuckets;
-import org.apache.hadoop.ozone.web.response.ListKeys;
-import org.apache.hadoop.ozone.web.response.ListVolumes;
-import org.apache.hadoop.ozone.web.response.VolumeInfo;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-/**
- * PLEASE NOTE : This file is a dummy backend for test purposes and prototyping
- * effort only. It does not handle any Object semantics correctly, neither does
- * it take care of security.
- */
-@InterfaceAudience.Private
-public class LocalStorageHandler implements StorageHandler {
-  private final Configuration conf;
-
-  /**
-   * Constructs LocalStorageHandler.
-   *
-   * @param conf ozone conf.
-   */
-  public LocalStorageHandler(Configuration conf) {
-    this.conf = conf;
-  }
-
-  /**
-   * Creates Storage Volume.
-   *
-   * @param args - volumeArgs
-   * @throws IOException
-   */
-  @Override
-  public void createVolume(VolumeArgs args) throws IOException, OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    oz.createVolume(args);
-
-  }
-
-  /**
-   * setVolumeOwner - sets the owner of the volume.
-   *
-   * @param args volumeArgs
-   * @throws IOException
-   */
-  @Override
-  public void setVolumeOwner(VolumeArgs args)
-      throws IOException, OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    oz.setVolumeProperty(args, OzoneMetadataManager.VolumeProperty.OWNER);
-  }
-
-  /**
-   * Set Volume Quota Info.
-   *
-   * @param args   - volumeArgs
-   * @param remove - true if the request is to remove the quota
-   * @throws IOException
-   */
-  @Override
-  public void setVolumeQuota(VolumeArgs args, boolean remove)
-      throws IOException, OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-
-    if (remove) {
-      OzoneQuota quota = new OzoneQuota();
-      args.setQuota(quota);
-    }
-    oz.setVolumeProperty(args, OzoneMetadataManager.VolumeProperty.QUOTA);
-  }
-
-  /**
-   * Checks if a Volume exists and the user specified has access to the volume.
-   *
-   * @param volume - Volume Name
-   * @param acl - Ozone acl which needs to be compared for access
-   * @return - Boolean - True if the user can modify the volume. This is
-   * possible for owners of the volume and admin users
-   * @throws IOException
-   */
-  @Override
-  public boolean checkVolumeAccess(String volume, OzoneAcl acl)
-      throws IOException, OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    return oz.checkVolumeAccess(volume, acl);
-  }
-
-  /**
-   * Returns Info about the specified Volume.
-   *
-   * @param args - volumeArgs
-   * @return VolumeInfo
-   * @throws IOException
-   */
-  @Override
-  public VolumeInfo getVolumeInfo(VolumeArgs args)
-      throws IOException, OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    return oz.getVolumeInfo(args);
-  }
-
-  /**
-   * Deletes an Empty Volume.
-   *
-   * @param args - Volume Args
-   * @throws IOException
-   */
-  @Override
-  public void deleteVolume(VolumeArgs args) throws IOException, OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    oz.deleteVolume(args);
-
-  }
-
-  /**
-   * Returns the List of Volumes owned by the specific user.
-   *
-   * @param args - ListArgs
-   * @return - List of Volumes
-   * @throws IOException
-   */
-  @Override
-  public ListVolumes listVolumes(ListArgs args)
-      throws IOException, OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    return oz.listVolumes(args);
-  }
-
-  /**
-   * true if the bucket exists and user has read access to the bucket else
-   * throws Exception.
-   *
-   * @param args Bucket args structure
-   * @throws IOException
-   */
-  @Override
-  public void checkBucketAccess(BucketArgs args)
-      throws IOException, OzoneException {
-
-  }
-
-  /**
-   * Creates a Bucket in specified Volume.
-   *
-   * @param args BucketArgs- BucketName, UserName and Acls
-   * @throws IOException
-   */
-  @Override
-  public void createBucket(BucketArgs args) throws IOException, OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    oz.createBucket(args);
-  }
-
-  /**
-   * Adds or Removes ACLs from a Bucket.
-   *
-   * @param args - BucketArgs
-   * @throws IOException
-   */
-  @Override
-  public void setBucketAcls(BucketArgs args)
-      throws IOException, OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    oz.setBucketProperty(args, OzoneMetadataManager.BucketProperty.ACLS);
-  }
-
-  /**
-   * Enables or disables Bucket Versioning.
-   *
-   * @param args - BucketArgs
-   * @throws IOException
-   */
-  @Override
-  public void setBucketVersioning(BucketArgs args)
-      throws IOException, OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    oz.setBucketProperty(args, OzoneMetadataManager.BucketProperty.VERSIONING);
-
-  }
-
-  /**
-   * Sets the Storage Class of a Bucket.
-   *
-   * @param args - BucketArgs
-   * @throws IOException
-   */
-  @Override
-  public void setBucketStorageClass(BucketArgs args)
-      throws IOException, OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    oz.setBucketProperty(args, OzoneMetadataManager.BucketProperty.STORAGETYPE);
-
-  }
-
-  /**
-   * Deletes a bucket if it is empty.
-   *
-   * @param args Bucket args structure
-   * @throws IOException
-   */
-  @Override
-  public void deleteBucket(BucketArgs args) throws IOException, OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    oz.deleteBucket(args);
-  }
-
-  /**
-   * Returns all Buckets of a specified Volume.
-   *
-   * @param args --User Args
-   * @return ListAllBuckets
-   * @throws OzoneException
-   */
-  @Override
-  public ListBuckets listBuckets(ListArgs args)
-      throws IOException, OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    return oz.listBuckets(args);
-  }
-
-  /**
-   * Returns Bucket's Metadata as a String.
-   *
-   * @param args Bucket args structure
-   * @return Info about the bucket
-   * @throws IOException
-   */
-  @Override
-  public BucketInfo getBucketInfo(BucketArgs args)
-      throws IOException, OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    return oz.getBucketInfo(args);
-  }
-
-  /**
-   * Writes a key in an existing bucket.
-   *
-   * @param args KeyArgs
-   * @return InputStream
-   * @throws OzoneException
-   */
-  @Override
-  public OutputStream newKeyWriter(KeyArgs args) throws IOException,
-      OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    return oz.createKey(args);
-  }
-
-  /**
-   * Tells the file system that the object has been written out completely and
-   * it can do any house keeping operation that needs to be done.
-   *
-   * @param args   Key Args
-   * @param stream
-   * @throws IOException
-   */
-  @Override
-  public void commitKey(KeyArgs args, OutputStream stream) throws
-      IOException, OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    oz.commitKey(args, stream);
-
-  }
-
-  /**
-   * Reads a key from an existing bucket.
-   *
-   * @param args KeyArgs
-   * @return LengthInputStream
-   * @throws IOException
-   */
-  @Override
-  public LengthInputStream newKeyReader(KeyArgs args) throws IOException,
-      OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    return oz.newKeyReader(args);
-  }
-
-  /**
-   * Deletes an existing key.
-   *
-   * @param args KeyArgs
-   * @throws OzoneException
-   */
-  @Override
-  public void deleteKey(KeyArgs args) throws IOException, OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    oz.deleteKey(args);
-  }
-
-  @Override
-  public void renameKey(KeyArgs args, String toKeyName)
-      throws IOException, OzoneException {
-    throw new UnsupportedOperationException("Not yet implemented");
-  }
-
-  /**
-   * Returns a list of Key.
-   *
-   * @param args KeyArgs
-   * @return BucketList
-   * @throws IOException
-   */
-  @Override
-  public ListKeys listKeys(ListArgs args) throws IOException, OzoneException {
-    OzoneMetadataManager oz =
-        OzoneMetadataManager.getOzoneMetadataManager(conf);
-    return oz.listKeys(args);
-
-  }
-
-  /**
-   * Get information of the specified Key.
-   *
-   * @param args Key Args
-   *
-   * @return KeyInfo
-   *
-   * @throws IOException
-   * @throws OzoneException
-   */
-  @Override
-  public KeyInfo getKeyInfo(KeyArgs args) throws IOException, OzoneException {
-    OzoneMetadataManager oz = OzoneMetadataManager
-        .getOzoneMetadataManager(conf);
-    return oz.getKeyInfo(args);
-  }
-
-  @Override
-  public void close() {
-    //No resource to close, do nothing.
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
deleted file mode 100644
index 1fe9a18..0000000
--- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/localstorage/OzoneMetadataManager.java
+++ /dev/null
@@ -1,1138 +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.ozone.web.localstorage;
-
-import com.google.common.base.Preconditions;
-import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.client.io.LengthInputStream;
-import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
-import org.apache.hadoop.ozone.web.handlers.BucketArgs;
-import org.apache.hadoop.ozone.web.handlers.KeyArgs;
-import org.apache.hadoop.ozone.web.handlers.ListArgs;
-import org.apache.hadoop.ozone.web.handlers.UserArgs;
-import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
-import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.web.response.BucketInfo;
-import org.apache.hadoop.ozone.web.response.KeyInfo;
-import org.apache.hadoop.ozone.web.response.ListBuckets;
-import org.apache.hadoop.ozone.web.response.ListKeys;
-import org.apache.hadoop.ozone.web.response.ListVolumes;
-import org.apache.hadoop.ozone.web.response.VolumeInfo;
-import org.apache.hadoop.ozone.web.response.VolumeOwner;
-import org.apache.hadoop.utils.MetadataStore;
-import org.apache.hadoop.utils.MetadataStoreBuilder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.charset.Charset;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.Locale;
-import java.util.TimeZone;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-/**
- * A stand alone Ozone implementation that allows us to run Ozone tests in local
- * mode. This acts as the ozone backend when using MiniDFSCluster for testing.
- */
-public final class OzoneMetadataManager {
-
-  /*
-    OzoneMetadataManager manages volume/bucket/object metadata and
-    data.
-
-    Metadata is maintained in 2 level DB files, UserDB and MetadataDB.
-
-    UserDB contains a Name and a List. For example volumes owned by the user
-    bilbo, would be maintained in UserDB as {bilbo}->{shire, rings}
-
-    This list part of mapping is context sensitive.  That is, if you use {user
-    name} as the key, the list you get is a list of volumes. if you use
-    {user/volume} as the key the list you get is list of buckets. if you use
-    {user/volume/bucket} as key the list you get is the list of objects.
-
-    All keys in the UserDB starts with the UserName.
-
-    We also need to maintain a flat namespace for volumes. This is
-    maintained by the MetadataDB. MetadataDB contains the name of an
-    object(volume, bucket or key) and its associated metadata.
-    The keys in the Metadata DB are {volume}, {volume/bucket} or
-    {volume/bucket/key}. User name is absent, so we have a common root name
-    space for the volume.
-
-    The value of part of metadataDB points to corresponding *Info structures.
-    {volume] -> volumeInfo
-    {volume/bucket} -> bucketInfo
-    {volume/bucket/key} -> keyInfo
-
-
-    Here are various work flows :
-
-    CreateVolume -> Check if Volume exists in metadataDB, if not update UserDB
-    with a list of volumes and update metadataDB with VolumeInfo.
-
-    DeleteVolume -> Check the Volume, and check the VolumeInfo->bucketCount.
-    if bucketCount == 0, delete volume from userDB->{List of volumes} and
-    metadataDB.
-
-    Very similar work flows exist for CreateBucket and DeleteBucket.
-
-      // Please note : These database operations are *not* transactional,
-      // which means that failure can lead to inconsistencies.
-      // Only way to recover is to reset to a clean state, or
-      // use rm -rf /tmp/ozone :)
-
-    We have very simple locking policy. We have a ReaderWriter lock that is
-    taken for each action, this lock is aptly named "lock".
-
-    All actions *must* be performed with a lock held, either a read
-    lock or a write lock. Violation of these locking policies can be harmful.
-
-
-      // // IMPORTANT :
-      // //  This is a simulation layer, this is NOT how the real
-      // //  OZONE functions. This is written to so that we can write
-      // //  stand-alone tests for the protocol and client code.
-
-*/
-  static final Logger LOG = LoggerFactory.getLogger(OzoneMetadataManager.class);
-  private static final String USER_DB = "/user.db";
-  private static final String META_DB = "/metadata.db";
-  private static OzoneMetadataManager bm = null;
-  private MetadataStore userDB;
-  private MetadataStore metadataDB;
-  private ReadWriteLock lock;
-  private Charset encoding = Charset.forName("UTF-8");
-  private String storageRoot;
-  private static final String OBJECT_DIR = "/_objects/";
-
-  // This table keeps a pointer to objects whose operations
-  // are in progress but not yet committed to persistent store
-  private ConcurrentHashMap<OutputStream, String> inProgressObjects;
-
-  /**
-   * Constructs OzoneMetadataManager.
-   */
-  private OzoneMetadataManager(Configuration conf) throws IOException {
-
-    lock = new ReentrantReadWriteLock();
-    storageRoot =
-        conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
-            OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
-
-    File file = new File(storageRoot + OBJECT_DIR);
-
-    if (!file.exists() && !file.mkdirs()) {
-      LOG.error("Creation of Ozone root failed. " + file.toString());
-      throw new IOException("Creation of Ozone root failed.");
-    }
-
-    try {
-      userDB = MetadataStoreBuilder.newBuilder()
-          .setDbFile(new File(storageRoot + USER_DB))
-          .setCreateIfMissing(true)
-          .build();
-      metadataDB = MetadataStoreBuilder.newBuilder()
-          .setDbFile(new File(storageRoot + META_DB))
-          .setCreateIfMissing(true)
-          .build();
-      inProgressObjects = new ConcurrentHashMap<>();
-    } catch (IOException ex) {
-      LOG.error("Cannot open db :" + ex.getMessage());
-      throw ex;
-    }
-  }
-
-  /**
-   * Gets Ozone Manager.
-   *
-   * @return OzoneMetadataManager
-   */
-  public static synchronized OzoneMetadataManager
-      getOzoneMetadataManager(Configuration conf) throws IOException {
-    if (bm == null) {
-      bm = new OzoneMetadataManager(conf);
-    }
-    return bm;
-  }
-
-  /**
-   * Creates a volume.
-   *
-   * @param args - VolumeArgs
-   * @throws OzoneException
-   */
-  public void createVolume(VolumeArgs args) throws OzoneException {
-    lock.writeLock().lock();
-    try {
-      SimpleDateFormat format =
-          new SimpleDateFormat(OzoneConsts.OZONE_DATE_FORMAT, Locale.US);
-      format.setTimeZone(TimeZone.getTimeZone(OzoneConsts.OZONE_TIME_ZONE));
-
-      byte[] volumeName =
-          metadataDB.get(args.getVolumeName().getBytes(encoding));
-
-      if (volumeName != null) {
-        LOG.debug("Volume {} already exists.", volumeName);
-        throw ErrorTable.newError(ErrorTable.VOLUME_ALREADY_EXISTS, args);
-      }
-
-      VolumeInfo newVInfo = new VolumeInfo(args.getVolumeName(), format
-          .format(new Date(System.currentTimeMillis())), args.getAdminName());
-
-      newVInfo.setQuota(args.getQuota());
-      VolumeOwner owner = new VolumeOwner(args.getUserName());
-      newVInfo.setOwner(owner);
-
-      ListVolumes volumeList;
-      byte[] userVolumes = userDB.get(args.getUserName().getBytes(encoding));
-      if (userVolumes == null) {
-        volumeList = new ListVolumes();
-      } else {
-        volumeList = ListVolumes.parse(new String(userVolumes, encoding));
-      }
-
-      volumeList.addVolume(newVInfo);
-      volumeList.sort();
-
-      // Please note : These database operations are *not* transactional,
-      // which means that failure can lead to inconsistencies.
-      // Only way to recover is to reset to a clean state, or
-      // use rm -rf /tmp/ozone :)
-
-
-      userDB.put(args.getUserName().getBytes(encoding),
-          volumeList.toDBString().getBytes(encoding));
-
-      metadataDB.put(args.getVolumeName().getBytes(encoding),
-          newVInfo.toDBString().getBytes(encoding));
-
-    } catch (IOException ex) {
-      throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, ex);
-    } finally {
-      lock.writeLock().unlock();
-    }
-  }
-
-  /**
-   * Updates the Volume properties like Owner Name and Quota.
-   *
-   * @param args     - Volume Args
-   * @param property - Flag which tells us what property to upgrade
-   * @throws OzoneException
-   */
-  public void setVolumeProperty(VolumeArgs args, VolumeProperty property)
-      throws OzoneException {
-    lock.writeLock().lock();
-    try {
-      byte[] volumeInfo =
-          metadataDB.get(args.getVolumeName().getBytes(encoding));
-      if (volumeInfo == null) {
-        throw ErrorTable.newError(ErrorTable.VOLUME_NOT_FOUND, args);
-      }
-      VolumeInfo info = VolumeInfo.parse(new String(volumeInfo, encoding));
-
-      byte[] userBytes = userDB.get(args.getResourceName().getBytes(encoding));
-      ListVolumes volumeList;
-      if (userBytes == null) {
-        volumeList = new ListVolumes();
-      } else {
-        volumeList = ListVolumes.parse(new String(userBytes, encoding));
-      }
-
-      switch (property) {
-      case OWNER:
-        // needs new owner, we delete the volume object from the
-        // old user's volume list
-        removeOldOwner(info);
-        VolumeOwner owner = new VolumeOwner(args.getUserName());
-        // set the new owner
-        info.setOwner(owner);
-        break;
-      case QUOTA:
-        // if this is quota update we just remove the old object from the
-        // current users list and update the same object later.
-        volumeList.getVolumes().remove(info);
-        info.setQuota(args.getQuota());
-        break;
-      default:
-        OzoneException ozEx =
-            ErrorTable.newError(ErrorTable.BAD_PROPERTY, args);
-        ozEx.setMessage("Volume property is not recognized");
-        throw ozEx;
-      }
-
-      volumeList.addVolume(info);
-
-      metadataDB.put(args.getVolumeName().getBytes(encoding),
-          info.toDBString().getBytes(encoding));
-
-      // if this is an owner change this put will create a new owner or update
-      // the owner's volume list.
-      userDB.put(args.getResourceName().getBytes(encoding),
-          volumeList.toDBString().getBytes(encoding));
-
-    } catch (IOException ex) {
-      throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, ex);
-    } finally {
-      lock.writeLock().unlock();
-    }
-  }
-
-  /**
-   * Removes the old owner from the volume.
-   *
-   * @param info - VolumeInfo
-   * @throws IOException
-   */
-  private void removeOldOwner(VolumeInfo info) throws IOException {
-    // We need to look the owner that we know is the current owner
-    byte[] volumeBytes =
-        userDB.get(info.getOwner().getName().getBytes(encoding));
-    ListVolumes volumeList =
-        ListVolumes.parse(new String(volumeBytes, encoding));
-    volumeList.getVolumes().remove(info);
-
-    // Write the new list info to the old user data
-    userDB.put(info.getOwner().getName().getBytes(encoding),
-        volumeList.toDBString().getBytes(encoding));
-  }
-
-  /**
-   * Checks if you are the owner of a specific volume.
-   *
-   * @param volume - Volume Name whose access permissions needs to be checked
-   * @param acl - requested acls which needs to be checked for access
-   * @return - True if you are the owner, false otherwise
-   * @throws OzoneException
-   */
-  public boolean checkVolumeAccess(String volume, OzoneAcl acl)
-      throws OzoneException {
-    lock.readLock().lock();
-    try {
-      byte[] volumeInfo =
-          metadataDB.get(volume.getBytes(encoding));
-      if (volumeInfo == null) {
-        throw ErrorTable.newError(ErrorTable.VOLUME_NOT_FOUND, null);
-      }
-
-      VolumeInfo info = VolumeInfo.parse(new String(volumeInfo, encoding));
-      return info.getOwner().getName().equals(acl.getName());
-    } catch (IOException ex) {
-      throw ErrorTable.newError(ErrorTable.SERVER_ERROR, null, ex);
-    } finally {
-      lock.readLock().unlock();
-    }
-  }
-
-  /**
-   * getVolumeInfo returns the Volume Info of a specific volume.
-   *
-   * @param args - Volume args
-   * @return VolumeInfo
-   * @throws OzoneException
-   */
-  public VolumeInfo getVolumeInfo(VolumeArgs args) throws OzoneException {
-    lock.readLock().lock();
-    try {
-      byte[] volumeInfo =
-          metadataDB.get(args.getVolumeName().getBytes(encoding));
-      if (volumeInfo == null) {
-        throw ErrorTable.newError(ErrorTable.VOLUME_NOT_FOUND, args);
-      }
-
-      return VolumeInfo.parse(new String(volumeInfo, encoding));
-    } catch (IOException ex) {
-      throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, ex);
-    } finally {
-      lock.readLock().unlock();
-    }
-  }
-
-  /**
-   * Returns all the volumes owned by a specific user.
-   *
-   * @param args - User Args
-   * @return - ListVolumes
-   * @throws OzoneException
-   */
-  public ListVolumes listVolumes(ListArgs args) throws OzoneException {
-    lock.readLock().lock();
-    try {
-      if (args.isRootScan()) {
-        return listAllVolumes(args);
-      }
-
-      UserArgs uArgs = (UserArgs) args.getArgs();
-      byte[] volumeList = userDB.get(uArgs.getUserName().getBytes(encoding));
-      if (volumeList == null) {
-        throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, uArgs);
-      }
-
-      String prefix = args.getPrefix();
-      int maxCount = args.getMaxKeys();
-      String prevKey = args.getPrevKey();
-      if (prevKey != null) {
-        // Format is username/volumeName, in local mode we don't use the
-        // user name since we have a userName DB.
-        String[] volName = args.getPrevKey().split("/");
-        if (volName.length < 2) {
-          throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, uArgs);
-        }
-        prevKey = volName[1];
-      }
-      return getFilteredVolumes(volumeList, prefix, prevKey, maxCount);
-    } catch (IOException ex) {
-      throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args.getArgs(), ex);
-    } finally {
-      lock.readLock().unlock();
-    }
-  }
-
-  /**
-   * Returns a List of Volumes that meet the prefix, prevkey and maxCount
-   * constraints.
-   *
-   * @param volumeList - Byte Array of Volume Info.
-   * @param prefix     - prefix string.
-   * @param prevKey    - PrevKey
-   * @param maxCount   - Maximum Count.
-   * @return ListVolumes.
-   * @throws IOException
-   */
-  private ListVolumes getFilteredVolumes(byte[] volumeList, String prefix,
-                                         String prevKey, int maxCount) throws
-      IOException {
-    ListVolumes volumes = ListVolumes.parse(new String(volumeList,
-        encoding));
-    int currentCount = 0;
-    ListIterator<VolumeInfo> iter = volumes.getVolumes().listIterator();
-    ListVolumes filteredVolumes = new ListVolumes();
-    while (currentCount < maxCount && iter.hasNext()) {
-      VolumeInfo vInfo = iter.next();
-      if (isMatchingPrefix(prefix, vInfo) && isAfterKey(prevKey, vInfo)) {
-        filteredVolumes.addVolume(vInfo);
-        currentCount++;
-      }
-    }
-    return filteredVolumes;
-  }
-
-  /**
-   * Returns all volumes in a cluster.
-   *
-   * @param args - ListArgs.
-   * @return ListVolumes.
-   * @throws OzoneException
-   */
-  public ListVolumes listAllVolumes(ListArgs args)
-      throws OzoneException, IOException {
-    String prefix = args.getPrefix();
-    final String prevKey;
-    int maxCount = args.getMaxKeys();
-    String userName = null;
-
-    if (args.getPrevKey() != null) {
-      // Format is username/volumeName
-      String[] volName = args.getPrevKey().split("/");
-      if (volName.length < 2) {
-        throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, args.getArgs());
-      }
-
-      byte[] userNameBytes = userDB.get(volName[0].getBytes(encoding));
-      userName = new String(userNameBytes, encoding);
-      prevKey = volName[1];
-    } else {
-      userName = new String(userDB.peekAround(0, null).getKey(), encoding);
-      prevKey = null;
-    }
-
-    if (userName == null || userName.isEmpty()) {
-      throw ErrorTable.newError(ErrorTable.USER_NOT_FOUND, args.getArgs());
-    }
-
-    ListVolumes returnSet = new ListVolumes();
-    // we need to iterate through users until we get maxcount volumes
-    // or no more volumes are left.
-    userDB.iterate(null, (key, value) -> {
-      int currentSize = returnSet.getVolumes().size();
-      if (currentSize < maxCount) {
-        String name = new String(key, encoding);
-        byte[] volumeList = userDB.get(name.getBytes(encoding));
-        if (volumeList == null) {
-          throw new IOException(
-              ErrorTable.newError(ErrorTable.USER_NOT_FOUND, args.getArgs()));
-        }
-        returnSet.getVolumes().addAll(
-            getFilteredVolumes(volumeList, prefix, prevKey,
-                maxCount - currentSize).getVolumes());
-        return true;
-      } else {
-        return false;
-      }
-    });
-
-    return returnSet;
-  }
-
-  /**
-   * Checks if a name starts with a matching prefix.
-   *
-   * @param prefix - prefix string.
-   * @param vInfo  - volume info.
-   * @return true or false.
-   */
-  private boolean isMatchingPrefix(String prefix, VolumeInfo vInfo) {
-    if (prefix == null || prefix.isEmpty()) {
-      return true;
-    }
-    return vInfo.getVolumeName().startsWith(prefix);
-  }
-
-  /**
-   * Checks if the key is after the prevKey.
-   *
-   * @param prevKey - String prevKey.
-   * @param vInfo   - volume Info.
-   * @return - true or false.
-   */
-  private boolean isAfterKey(String prevKey, VolumeInfo vInfo) {
-    if (prevKey == null || prevKey.isEmpty()) {
-      return true;
-    }
-    return prevKey.compareTo(vInfo.getVolumeName()) < 0;
-  }
-
-  /**
-   * Deletes a volume if it exists and is empty.
-   *
-   * @param args - volume args
-   * @throws OzoneException
-   */
-  public void deleteVolume(VolumeArgs args) throws OzoneException {
-    lock.writeLock().lock();
-    try {
-      byte[] volumeName =
-          metadataDB.get(args.getVolumeName().getBytes(encoding));
-      if (volumeName == null) {
-        throw ErrorTable.newError(ErrorTable.VOLUME_NOT_FOUND, args);
-      }
-
-      VolumeInfo vInfo = VolumeInfo.parse(new String(volumeName, encoding));
-
-      // Only remove volumes if they are empty.
-      if (vInfo.getBucketCount() > 0) {
-        throw ErrorTable.newError(ErrorTable.VOLUME_NOT_EMPTY, args);
-      }
-
-      ListVolumes volumeList;
-      String user = vInfo.getOwner().getName();
-      byte[] userVolumes = userDB.get(user.getBytes(encoding));
-      if (userVolumes == null) {
-        throw ErrorTable.newError(ErrorTable.VOLUME_NOT_FOUND, args);
-      }
-
-      volumeList = ListVolumes.parse(new String(userVolumes, encoding));
-      volumeList.getVolumes().remove(vInfo);
-
-      metadataDB.delete(args.getVolumeName().getBytes(encoding));
-      userDB.put(user.getBytes(encoding),
-          volumeList.toDBString().getBytes(encoding));
-    } catch (IOException ex) {
-      throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, ex);
-    } finally {
-      lock.writeLock().unlock();
-    }
-  }
-
-  /**
-   * Create a bucket if it does not exist.
-   *
-   * @param args - BucketArgs
-   * @throws OzoneException
-   */
-  public void createBucket(BucketArgs args) throws OzoneException {
-    lock.writeLock().lock();
-    try {
-      // check if volume exists, buckets cannot be created without volumes
-      byte[] volumeName = metadataDB.get(args.getVolumeName()
-          .getBytes(encoding));
-      if (volumeName == null) {
-        throw ErrorTable.newError(ErrorTable.VOLUME_NOT_FOUND, args);
-      }
-
-      // A resource name is volume/bucket -- That is the key in metadata table
-      byte[] bucketName = metadataDB.get(args.getResourceName()
-          .getBytes(encoding));
-      if (bucketName != null) {
-        throw ErrorTable.newError(ErrorTable.BUCKET_ALREADY_EXISTS, args);
-      }
-
-      BucketInfo bucketInfo =
-          new BucketInfo(args.getVolumeName(), args.getBucketName());
-
-      if (args.getRemoveAcls() != null) {
-        OzoneException ex = ErrorTable.newError(ErrorTable.MALFORMED_ACL, args);
-        ex.setMessage("Remove ACLs specified in bucket create. Please remove "
-            + "them and retry.");
-        throw ex;
-      }
-
-      VolumeInfo volInfo = VolumeInfo.parse(new String(volumeName, encoding));
-      volInfo.setBucketCount(volInfo.getBucketCount() + 1);
-
-      bucketInfo.setAcls(args.getAddAcls());
-      bucketInfo.setStorageType(args.getStorageType());
-      bucketInfo.setVersioning(args.getVersioning());
-      ListBuckets bucketList;
-
-      // get bucket list from user/volume -> bucketList
-      byte[] volumeBuckets = userDB.get(args.getParentName()
-          .getBytes(encoding));
-      if (volumeBuckets == null) {
-        bucketList = new ListBuckets();
-      } else {
-        bucketList = ListBuckets.parse(new String(volumeBuckets, encoding));
-      }
-
-      bucketList.addBucket(bucketInfo);
-      bucketList.sort();
-
-      // Update Volume->bucketCount
-      userDB.put(args.getVolumeName().getBytes(encoding),
-          volInfo.toDBString().getBytes(encoding));
-
-      // Now update the userDB with user/volume -> bucketList
-      userDB.put(args.getParentName().getBytes(encoding),
-          bucketList.toDBString().getBytes(encoding));
-
-      // Update userDB with volume/bucket -> empty key list
-      userDB.put(args.getResourceName().getBytes(encoding),
-          new ListKeys().toDBString().getBytes(encoding));
-
-      // and update the metadataDB with volume/bucket->BucketInfo
-      metadataDB.put(args.getResourceName().getBytes(encoding),
-          bucketInfo.toDBString().getBytes(encoding));
-
-    } catch (IOException ex) {
-      throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, ex);
-    } finally {
-      lock.writeLock().unlock();
-    }
-  }
-
-  /**
-   * Updates the Bucket properties like ACls and Storagetype.
-   *
-   * @param args     - Bucket Args
-   * @param property - Flag which tells us what property to upgrade
-   * @throws OzoneException
-   */
-  public void setBucketProperty(BucketArgs args, BucketProperty property)
-      throws OzoneException {
-
-    lock.writeLock().lock();
-    try {
-      // volume/bucket-> bucketInfo
-      byte[] bucketInfo = metadataDB.get(args.getResourceName().
-          getBytes(encoding));
-      if (bucketInfo == null) {
-        throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
-      }
-
-      BucketInfo info = BucketInfo.parse(new String(bucketInfo, encoding));
-      byte[] volumeBuckets = userDB.get(args.getParentName()
-          .getBytes(encoding));
-      ListBuckets bucketList = ListBuckets.parse(new String(volumeBuckets,
-          encoding));
-      bucketList.getBuckets().remove(info);
-
-      switch (property) {
-      case ACLS:
-        processRemoveAcls(args, info);
-        processAddAcls(args, info);
-        break;
-      case STORAGETYPE:
-        info.setStorageType(args.getStorageType());
-        break;
-      case VERSIONING:
-        info.setVersioning(args.getVersioning());
-        break;
-      default:
-        OzoneException ozEx =
-            ErrorTable.newError(ErrorTable.BAD_PROPERTY, args);
-        ozEx.setMessage("Bucket property is not recognized.");
-        throw ozEx;
-      }
-
-      bucketList.addBucket(info);
-      metadataDB.put(args.getResourceName().getBytes(encoding),
-          info.toDBString().getBytes(encoding));
-
-      userDB.put(args.getParentName().getBytes(encoding),
-          bucketList.toDBString().getBytes(encoding));
-    } catch (IOException ex) {
-      throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, ex);
-    } finally {
-      lock.writeLock().unlock();
-    }
-  }
-
-  /**
-   * Process Remove Acls and remove them from the bucket.
-   *
-   * @param args - BucketArgs
-   * @param info - BucketInfo
-   */
-  private void processRemoveAcls(BucketArgs args, BucketInfo info) {
-    List<OzoneAcl> removeAcls = args.getRemoveAcls();
-    if ((removeAcls == null) || (info.getAcls() == null)) {
-      return;
-    }
-    for (OzoneAcl racl : args.getRemoveAcls()) {
-      ListIterator<OzoneAcl> aclIter = info.getAcls().listIterator();
-      while (aclIter.hasNext()) {
-        if (racl.equals(aclIter.next())) {
-          aclIter.remove();
-          break;
-        }
-      }
-    }
-  }
-
-  /**
-   * Process Add Acls and Add them to the bucket.
-   *
-   * @param args - BucketArgs
-   * @param info - BucketInfo
-   */
-  private void processAddAcls(BucketArgs args, BucketInfo info) {
-    List<OzoneAcl> addAcls = args.getAddAcls();
-    if ((addAcls == null)) {
-      return;
-    }
-
-    if (info.getAcls() == null) {
-      info.setAcls(addAcls);
-      return;
-    }
-
-    for (OzoneAcl newacl : addAcls) {
-      ListIterator<OzoneAcl> aclIter = info.getAcls().listIterator();
-      while (aclIter.hasNext()) {
-        if (newacl.equals(aclIter.next())) {
-          continue;
-        }
-      }
-      info.getAcls().add(newacl);
-    }
-  }
-
-  /**
-   * Deletes a given bucket.
-   *
-   * @param args - BucketArgs
-   * @throws OzoneException
-   */
-  public void deleteBucket(BucketArgs args) throws OzoneException {
-    lock.writeLock().lock();
-    try {
-      byte[] bucketInfo = metadataDB.get(args.getResourceName()
-          .getBytes(encoding));
-      if (bucketInfo == null) {
-        throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
-      }
-
-      BucketInfo bInfo = BucketInfo.parse(new String(bucketInfo, encoding));
-
-      // Only remove buckets if they are empty.
-      if (bInfo.getKeyCount() > 0) {
-        throw ErrorTable.newError(ErrorTable.BUCKET_NOT_EMPTY, args);
-      }
-
-      byte[] bucketBytes = userDB.get(args.getParentName().getBytes(encoding));
-      if (bucketBytes == null) {
-        throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
-      }
-
-      ListBuckets bucketList =
-          ListBuckets.parse(new String(bucketBytes, encoding));
-      bucketList.getBuckets().remove(bInfo);
-
-      metadataDB.delete(args.getResourceName().getBytes(encoding));
-      userDB.put(args.getParentName().getBytes(encoding),
-          bucketList.toDBString().getBytes(encoding));
-    } catch (IOException ex) {
-      throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, ex);
-    } finally {
-      lock.writeLock().unlock();
-    }
-  }
-
-  /**
-   * Returns the Bucket info for a given bucket.
-   *
-   * @param args - Bucket Args
-   * @return BucketInfo   -  Bucket Information
-   * @throws OzoneException
-   */
-  public BucketInfo getBucketInfo(BucketArgs args) throws OzoneException {
-    lock.readLock().lock();
-    try {
-      byte[] bucketBytes = metadataDB.get(args.getResourceName()
-          .getBytes(encoding));
-      if (bucketBytes == null) {
-        throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
-      }
-
-      return BucketInfo.parse(new String(bucketBytes, encoding));
-    } catch (IOException ex) {
-      throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, ex);
-    } finally {
-      lock.readLock().unlock();
-    }
-  }
-
-  /**
-   * Returns a list of buckets for a given volume.
-   *
-   * @param args - volume args
-   * @return List of buckets
-   * @throws OzoneException
-   */
-  public ListBuckets listBuckets(ListArgs args) throws OzoneException {
-    lock.readLock().lock();
-    try {
-      Preconditions.checkState(args.getArgs() instanceof VolumeArgs);
-      VolumeArgs vArgs = (VolumeArgs) args.getArgs();
-      String userVolKey = vArgs.getUserName() + "/" + vArgs.getVolumeName();
-
-      // TODO : Query using Prefix and PrevKey
-      byte[] bucketBytes = userDB.get(userVolKey.getBytes(encoding));
-      if (bucketBytes == null) {
-        throw ErrorTable.newError(ErrorTable.INVALID_VOLUME_NAME,
-            args.getArgs());
-      }
-      return ListBuckets.parse(new String(bucketBytes, encoding));
-    } catch (IOException ex) {
-      throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args.getArgs(), ex);
-    } finally {
-      lock.readLock().unlock();
-    }
-  }
-
-  /**
-   * Creates a key and returns a stream to which this key can be written to.
-   *
-   * @param args KeyArgs
-   * @return - A stream into which key can be written to.
-   * @throws OzoneException
-   */
-  public OutputStream createKey(KeyArgs args) throws OzoneException {
-    lock.writeLock().lock();
-    try {
-      String fileNameHash = DigestUtils.sha256Hex(args.getResourceName());
-
-      // Please don't try trillion objects unless the physical file system
-      // is capable of doing that in a single directory.
-
-      String fullPath = storageRoot + OBJECT_DIR + fileNameHash;
-      File f = new File(fullPath);
-
-      // In real ozone it would not be this way, a file will be overwritten
-      // only if the upload is successful.
-      if (f.exists()) {
-        LOG.debug("we are overwriting a file. This is by design.");
-        if (!f.delete()) {
-          LOG.error("Unable to delete the file: {}", fullPath);
-          throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args);
-        }
-      }
-
-      // f.createNewFile();
-      FileOutputStream fsStream = new FileOutputStream(f);
-      inProgressObjects.put(fsStream, fullPath);
-
-      return fsStream;
-    } catch (IOException e) {
-      throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, e);
-    } finally {
-      lock.writeLock().unlock();
-    }
-  }
-
-  /**
-   * commit keys moves an In progress object into the metadata store so that key
-   * is visible in the metadata operations from that point onwards.
-   *
-   * @param args Object args
-   * @throws OzoneException
-   */
-  public void commitKey(KeyArgs args, OutputStream stream)
-      throws OzoneException {
-    SimpleDateFormat format =
-        new SimpleDateFormat("EEE, dd MMM yyyy HH:mm:ss ZZZ", Locale.US);
-    lock.writeLock().lock();
-
-    try {
-      byte[] bucketInfo = metadataDB.get(args.getParentName()
-          .getBytes(encoding));
-      if (bucketInfo == null) {
-        throw ErrorTable.newError(ErrorTable.INVALID_RESOURCE_NAME, args);
-      }
-      BucketInfo bInfo = BucketInfo.parse(new String(bucketInfo, encoding));
-      bInfo.setKeyCount(bInfo.getKeyCount() + 1);
-
-      String fileNameHash = inProgressObjects.get(stream);
-      inProgressObjects.remove(stream);
-      if (fileNameHash == null) {
-        throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args);
-      }
-
-      ListKeys keyList;
-      byte[] bucketListBytes = userDB.get(args.getParentName()
-          .getBytes(encoding));
-      keyList = ListKeys.parse(new String(bucketListBytes, encoding));
-      KeyInfo keyInfo;
-
-      byte[] objectBytes = metadataDB.get(args.getResourceName()
-          .getBytes(encoding));
-
-      if (objectBytes != null) {
-        // we are overwriting an existing object.
-        // TODO : Emit info for Accounting
-        keyInfo = KeyInfo.parse(new String(objectBytes, encoding));
-        keyList.getKeyList().remove(keyInfo);
-      } else {
-        keyInfo = new KeyInfo();
-      }
-
-      keyInfo.setCreatedOn(format.format(new Date(System.currentTimeMillis())));
-
-      // TODO : support version, we need to check if versioning
-      // is switched on the bucket and make appropriate calls.
-      keyInfo.setVersion(0);
-
-      keyInfo.setDataFileName(fileNameHash);
-      keyInfo.setKeyName(args.getKeyName());
-      keyInfo.setMd5hash(args.getHash());
-      keyInfo.setSize(args.getSize());
-
-      keyList.getKeyList().add(keyInfo);
-
-      // if the key exists, we overwrite happily :). since the
-      // earlier call - createObject -  has overwritten the data.
-
-      metadataDB.put(args.getResourceName().getBytes(encoding),
-          keyInfo.toDBString().getBytes(encoding));
-
-      metadataDB.put(args.getParentName().getBytes(encoding),
-          bInfo.toDBString().getBytes(encoding));
-
-      userDB.put(args.getParentName().getBytes(encoding),
-          keyList.toDBString().getBytes(encoding));
-
-    } catch (IOException e) {
-      throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, e);
-    } finally {
-      lock.writeLock().unlock();
-    }
-  }
-
-  /**
-   * deletes an key from a given bucket.
-   *
-   * @param args - ObjectArgs
-   * @throws OzoneException
-   */
-  public void deleteKey(KeyArgs args) throws OzoneException {
-    lock.writeLock().lock();
-    try {
-      byte[] bucketInfo = metadataDB.get(args.getParentName()
-          .getBytes(encoding));
-      if (bucketInfo == null) {
-        throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
-      }
-      BucketInfo bInfo = BucketInfo.parse(new String(bucketInfo, encoding));
-      bInfo.setKeyCount(bInfo.getKeyCount() - 1);
-
-
-      byte[] bucketListBytes = userDB.get(args.getParentName()
-          .getBytes(encoding));
-      if (bucketListBytes == null) {
-        throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
-      }
-      ListKeys keyList = ListKeys.parse(new String(bucketListBytes, encoding));
-
-
-      byte[] objectBytes = metadataDB.get(args.getResourceName()
-          .getBytes(encoding));
-      if (objectBytes == null) {
-        throw ErrorTable.newError(ErrorTable.INVALID_KEY, args);
-      }
-
-      KeyInfo oInfo = KeyInfo.parse(new String(objectBytes, encoding));
-      keyList.getKeyList().remove(oInfo);
-
-      String fileNameHash = DigestUtils.sha256Hex(args.getResourceName());
-
-      String fullPath = storageRoot + OBJECT_DIR + fileNameHash;
-      File f = new File(fullPath);
-
-      if (f.exists()) {
-        if (!f.delete()) {
-          throw ErrorTable.newError(ErrorTable.KEY_OPERATION_CONFLICT, args);
-        }
-      } else {
-        throw ErrorTable.newError(ErrorTable.INVALID_KEY, args);
-      }
-
-
-      metadataDB.delete(args.getResourceName().getBytes(encoding));
-      metadataDB.put(args.getParentName().getBytes(encoding),
-          bInfo.toDBString().getBytes(encoding));
-      userDB.put(args.getParentName().getBytes(encoding),
-          keyList.toDBString().getBytes(encoding));
-    } catch (IOException e) {
-      throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, e);
-    } finally {
-      lock.writeLock().unlock();
-    }
-  }
-
-  /**
-   * Returns a Stream for the file.
-   *
-   * @param args - Object args
-   * @return Stream
-   * @throws IOException
-   * @throws OzoneException
-   */
-  public LengthInputStream newKeyReader(KeyArgs args)
-      throws IOException, OzoneException {
-    lock.readLock().lock();
-    try {
-      String fileNameHash = DigestUtils.sha256Hex(args.getResourceName());
-      String fullPath = storageRoot + OBJECT_DIR + fileNameHash;
-      File f = new File(fullPath);
-      if (!f.exists()) {
-        throw ErrorTable.newError(ErrorTable.INVALID_RESOURCE_NAME, args);
-      }
-      long size = f.length();
-
-      FileInputStream fileStream = new FileInputStream(f);
-      return new LengthInputStream(fileStream, size);
-    } finally {
-      lock.readLock().unlock();
-    }
-  }
-
-  /**
-   * Returns keys in a bucket.
-   *
-   * @param args
-   * @return List of keys.
-   * @throws IOException
-   * @throws OzoneException
-   */
-  public ListKeys listKeys(ListArgs args) throws IOException, OzoneException {
-    lock.readLock().lock();
-    // TODO : Support Prefix and PrevKey lookup.
-    try {
-      Preconditions.checkState(args.getArgs() instanceof BucketArgs);
-      BucketArgs bArgs = (BucketArgs) args.getArgs();
-      byte[] bucketInfo = metadataDB.get(bArgs.getResourceName()
-          .getBytes(encoding));
-      if (bucketInfo == null) {
-        throw ErrorTable.newError(ErrorTable.INVALID_RESOURCE_NAME, bArgs);
-      }
-
-      byte[] bucketListBytes = userDB.get(bArgs.getResourceName()
-          .getBytes(encoding));
-      if (bucketListBytes == null) {
-        throw ErrorTable.newError(ErrorTable.INVALID_RESOURCE_NAME, bArgs);
-      }
-      return ListKeys.parse(new String(bucketListBytes, encoding));
-    } finally {
-      lock.readLock().unlock();
-    }
-  }
-
-  /**
-   * Get the Key information for a given key.
-   *
-   * @param args - Key Args
-   * @return KeyInfo - Key Information
-   * @throws OzoneException
-   */
-  public KeyInfo getKeyInfo(KeyArgs args) throws IOException, OzoneException {
-    lock.readLock().lock();
-    try {
-      byte[] bucketInfo = metadataDB
-          .get(args.getParentName().getBytes(encoding));
-      if (bucketInfo == null) {
-        throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
-      }
-
-      byte[] bucketListBytes = userDB
-          .get(args.getParentName().getBytes(encoding));
-      if (bucketListBytes == null) {
-        throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, args);
-      }
-
-      byte[] objectBytes = metadataDB
-          .get(args.getResourceName().getBytes(encoding));
-      if (objectBytes == null) {
-        throw ErrorTable.newError(ErrorTable.INVALID_KEY, args);
-      }
-
-      return KeyInfo.parse(new String(objectBytes, encoding));
-    } catch (IOException e) {
-      throw ErrorTable.newError(ErrorTable.SERVER_ERROR, args, e);
-    } finally {
-      lock.readLock().unlock();
-    }
-  }
-
-  /**
-   * This is used in updates to volume metadata.
-   */
-  public enum VolumeProperty {
-    OWNER, QUOTA
-  }
-
-  /**
-   * Bucket Properties.
-   */
-  public enum BucketProperty {
-    ACLS, STORAGETYPE, VERSIONING
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df21e1b1/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java b/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java
index ad21f28..d3bc857 100644
--- a/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java
+++ b/hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSInputStream.java
@@ -57,8 +57,7 @@ public class TestOzoneFSInputStream {
   /**
    * Create a MiniDFSCluster for testing.
    * <p>
-   * Ozone is made active by setting OZONE_ENABLED = true and
-   * OZONE_HANDLER_TYPE_KEY = "distributed"
+   * Ozone is made active by setting OZONE_ENABLED = true
    *
    * @throws IOException
    */
@@ -90,9 +89,6 @@ public class TestOzoneFSInputStream {
     // Fetch the host and port for File System init
     DatanodeDetails datanodeDetails = cluster.getHddsDatanodes().get(0)
         .getDatanodeDetails();
-    int port = datanodeDetails
-        .getPort(DatanodeDetails.Port.Name.REST).getValue();
-    String host = datanodeDetails.getHostName();
 
     // Set the fs.defaultFS and start the filesystem
     String uri = String.format("%s://%s.%s/",


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[35/50] [abbrv] hadoop git commit: Revert "HDFS-13838. WebHdfsFileSystem.getFileStatus() won't return correct "snapshot enabled" status. Contributed by Siyao Meng."

Posted by eh...@apache.org.
Revert "HDFS-13838. WebHdfsFileSystem.getFileStatus() won't return correct "snapshot enabled" status. Contributed by Siyao Meng."

This reverts commit 26c2a97c566969f50eb8e8432009724c51152a98.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8aa6c4f0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8aa6c4f0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8aa6c4f0

Branch: refs/heads/HDFS-12090
Commit: 8aa6c4f079fd38a3230bc070c2ce837fefbc5301
Parents: c36d69a
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Thu Aug 30 11:44:20 2018 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Thu Aug 30 11:44:20 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/web/JsonUtilClient.java |  4 ----
 .../java/org/apache/hadoop/hdfs/web/TestWebHDFS.java    | 12 ------------
 2 files changed, 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aa6c4f0/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
index a685573..9bb1846 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
@@ -133,7 +133,6 @@ class JsonUtilClient {
     Boolean aclBit = (Boolean) m.get("aclBit");
     Boolean encBit = (Boolean) m.get("encBit");
     Boolean erasureBit  = (Boolean) m.get("ecBit");
-    Boolean snapshotEnabledBit  = (Boolean) m.get("snapshotEnabled");
     EnumSet<HdfsFileStatus.Flags> f =
         EnumSet.noneOf(HdfsFileStatus.Flags.class);
     if (aclBit != null && aclBit) {
@@ -145,9 +144,6 @@ class JsonUtilClient {
     if (erasureBit != null && erasureBit) {
       f.add(HdfsFileStatus.Flags.HAS_EC);
     }
-    if (snapshotEnabledBit != null && snapshotEnabledBit) {
-      f.add(HdfsFileStatus.Flags.SNAPSHOT_ENABLED);
-    }
 
     Map<String, Object> ecPolicyObj = (Map) m.get("ecPolicyObj");
     ErasureCodingPolicy ecPolicy = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aa6c4f0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
index 9152636..cbc428a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
@@ -482,9 +482,6 @@ public class TestWebHDFS {
 
       // allow snapshots on /bar using webhdfs
       webHdfs.allowSnapshot(bar);
-      // check if snapshot status is enabled
-      assertTrue(dfs.getFileStatus(bar).isSnapshotEnabled());
-      assertTrue(webHdfs.getFileStatus(bar).isSnapshotEnabled());
       webHdfs.createSnapshot(bar, "s1");
       final Path s1path = SnapshotTestHelper.getSnapshotRoot(bar, "s1");
       Assert.assertTrue(webHdfs.exists(s1path));
@@ -494,24 +491,15 @@ public class TestWebHDFS {
       assertEquals(bar, snapshottableDirs[0].getFullPath());
       dfs.deleteSnapshot(bar, "s1");
       dfs.disallowSnapshot(bar);
-      // check if snapshot status is disabled
-      assertFalse(dfs.getFileStatus(bar).isSnapshotEnabled());
-      assertFalse(webHdfs.getFileStatus(bar).isSnapshotEnabled());
       snapshottableDirs = dfs.getSnapshottableDirListing();
       assertNull(snapshottableDirs);
 
       // disallow snapshots on /bar using webhdfs
       dfs.allowSnapshot(bar);
-      // check if snapshot status is enabled, again
-      assertTrue(dfs.getFileStatus(bar).isSnapshotEnabled());
-      assertTrue(webHdfs.getFileStatus(bar).isSnapshotEnabled());
       snapshottableDirs = dfs.getSnapshottableDirListing();
       assertEquals(1, snapshottableDirs.length);
       assertEquals(bar, snapshottableDirs[0].getFullPath());
       webHdfs.disallowSnapshot(bar);
-      // check if snapshot status is disabled, again
-      assertFalse(dfs.getFileStatus(bar).isSnapshotEnabled());
-      assertFalse(webHdfs.getFileStatus(bar).isSnapshotEnabled());
       snapshottableDirs = dfs.getSnapshottableDirListing();
       assertNull(snapshottableDirs);
       try {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[38/50] [abbrv] hadoop git commit: HDDS-379. Simplify and improve the cli arg parsing of ozone scmcli. Contributed by Elek, Marton.

Posted by eh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java
deleted file mode 100644
index 722c1a5..0000000
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMCli.java
+++ /dev/null
@@ -1,518 +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.ozone.scm;
-
-import com.google.common.primitives.Longs;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
-import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.ozone.container.ContainerTestHelper;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.scm.cli.ResultCode;
-import org.apache.hadoop.hdds.scm.cli.SCMCLI;
-import org.apache.hadoop.hdds.scm.XceiverClientManager;
-import org.apache.hadoop.hdds.scm.client.ContainerOperationClient;
-import org.apache.hadoop.hdds.scm.client.ScmClient;
-import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
-import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
-import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
-
-import org.apache.hadoop.ozone.container.common.impl.ContainerData;
-import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
-import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.OPEN;
-
-import static org.apache.hadoop.hdds.scm.cli.ResultCode.EXECUTION_ERROR;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
-
-/**
- * This class tests the CLI of SCM.
- */
-@Ignore ("Needs to be fixed for new SCM and Storage design")
-public class TestSCMCli {
-  private static SCMCLI cli;
-
-  private static MiniOzoneCluster cluster;
-  private static OzoneConfiguration conf;
-  private static StorageContainerLocationProtocolClientSideTranslatorPB
-      storageContainerLocationClient;
-
-  private static StorageContainerManager scm;
-  private static ScmClient containerOperationClient;
-
-  private static ByteArrayOutputStream outContent;
-  private static PrintStream outStream;
-  private static ByteArrayOutputStream errContent;
-  private static PrintStream errStream;
-  private static XceiverClientManager xceiverClientManager;
-  private static String containerOwner = "OZONE";
-
-  @Rule
-  public Timeout globalTimeout = new Timeout(30000);
-
-  @BeforeClass
-  public static void setup() throws Exception {
-    conf = new OzoneConfiguration();
-    cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(3).build();
-    cluster.waitForClusterToBeReady();
-    xceiverClientManager = new XceiverClientManager(conf);
-    storageContainerLocationClient =
-        cluster.getStorageContainerLocationClient();
-    containerOperationClient = new ContainerOperationClient(
-        storageContainerLocationClient, new XceiverClientManager(conf));
-    outContent = new ByteArrayOutputStream();
-    outStream = new PrintStream(outContent);
-    errContent = new ByteArrayOutputStream();
-    errStream = new PrintStream(errContent);
-    cli = new SCMCLI(containerOperationClient, outStream, errStream);
-    scm = cluster.getStorageContainerManager();
-  }
-
-  private int runCommandAndGetOutput(String[] cmd,
-      ByteArrayOutputStream out,
-      ByteArrayOutputStream err) throws Exception {
-    PrintStream cmdOutStream = System.out;
-    PrintStream cmdErrStream = System.err;
-    if(out != null) {
-      cmdOutStream = new PrintStream(out);
-    }
-    if (err != null) {
-      cmdErrStream = new PrintStream(err);
-    }
-    ScmClient client = new ContainerOperationClient(
-        storageContainerLocationClient, new XceiverClientManager(conf));
-    SCMCLI scmCLI = new SCMCLI(client, cmdOutStream, cmdErrStream);
-    return scmCLI.run(cmd);
-  }
-
-  @AfterClass
-  public static void shutdown() throws InterruptedException {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-    IOUtils.cleanupWithLogger(null, storageContainerLocationClient);
-  }
-
-  @Test
-  public void testCreateContainer() throws Exception {
-    ByteArrayOutputStream testContent = new ByteArrayOutputStream();
-    PrintStream testPrintOut = new PrintStream(testContent);
-    System.setOut(testPrintOut);
-    String[] args = {"-container", "-create"};
-    assertEquals(ResultCode.SUCCESS, cli.run(args));
-    assertEquals("", testContent.toString());
-  }
-
-  private boolean containerExist(long containerID) {
-    try {
-      ContainerInfo container = scm.getClientProtocolServer()
-          .getContainerWithPipeline(containerID).getContainerInfo();
-      return container != null
-          && containerID == container.getContainerID();
-    } catch (IOException e) {
-      return false;
-    }
-  }
-
-  @Test
-  public void testDeleteContainer() throws Exception {
-    String containerName;
-    ContainerData containerData;
-    Pipeline pipeline;
-    String[] delCmd;
-    ByteArrayOutputStream testErr;
-    int exitCode;
-
-    // ****************************************
-    // 1. Test to delete a non-empty container.
-    // ****************************************
-    // Create an non-empty container
-    ContainerWithPipeline container = containerOperationClient
-        .createContainer(xceiverClientManager.getType(),
-            HddsProtos.ReplicationFactor.ONE, containerOwner);
-    KeyValueContainerData kvData = KeyValueContainerData
-        .getFromProtoBuf(containerOperationClient.readContainer(
-            container.getContainerInfo().getContainerID(), container
-                .getPipeline()));
-    KeyUtils.getDB(kvData, conf)
-        .put(Longs.toByteArray(container.getContainerInfo().getContainerID()),
-            "someKey".getBytes());
-    Assert.assertTrue(containerExist(container.getContainerInfo()
-        .getContainerID()));
-
-    // Gracefully delete a container should fail because it is open.
-    delCmd = new String[]{"-container", "-delete", "-c",
-        Long.toString(container.getContainerInfo().getContainerID())};
-    testErr = new ByteArrayOutputStream();
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    exitCode = runCommandAndGetOutput(delCmd, out, testErr);
-    assertEquals(EXECUTION_ERROR, exitCode);
-    assertTrue(testErr.toString()
-        .contains("Deleting an open container is not allowed."));
-    Assert.assertTrue(
-        containerExist(container.getContainerInfo().getContainerID()));
-
-    // Close the container
-    containerOperationClient.closeContainer(
-        container.getContainerInfo().getContainerID());
-
-    // Gracefully delete a container should fail because it is not empty.
-    testErr = new ByteArrayOutputStream();
-    int exitCode2 = runCommandAndGetOutput(delCmd, out, testErr);
-    assertEquals(EXECUTION_ERROR, exitCode2);
-    assertTrue(testErr.toString()
-        .contains("Container cannot be deleted because it is not empty."));
-    Assert.assertTrue(
-        containerExist(container.getContainerInfo().getContainerID()));
-
-    // Try force delete again.
-    delCmd = new String[]{"-container", "-delete", "-c",
-        Long.toString(container.getContainerInfo().getContainerID()), "-f"};
-    exitCode = runCommandAndGetOutput(delCmd, out, null);
-    assertEquals("Expected success, found:", ResultCode.SUCCESS, exitCode);
-    assertFalse(containerExist(container.getContainerInfo().getContainerID()));
-
-    // ****************************************
-    // 2. Test to delete an empty container.
-    // ****************************************
-    // Create an empty container
-    ContainerWithPipeline emptyContainer = containerOperationClient
-        .createContainer(xceiverClientManager.getType(),
-            HddsProtos.ReplicationFactor.ONE, containerOwner);
-    containerOperationClient
-        .closeContainer(emptyContainer.getContainerInfo().getContainerID());
-    Assert.assertTrue(
-        containerExist(emptyContainer.getContainerInfo().getContainerID()));
-
-    // Successfully delete an empty container.
-    delCmd = new String[]{"-container", "-delete", "-c",
-        Long.toString(emptyContainer.getContainerInfo().getContainerID())};
-    exitCode = runCommandAndGetOutput(delCmd, out, null);
-    assertEquals(ResultCode.SUCCESS, exitCode);
-    assertFalse(
-        containerExist(emptyContainer.getContainerInfo().getContainerID()));
-
-    // After the container is deleted,
-    // another container can now be recreated.
-    ContainerWithPipeline newContainer = containerOperationClient.
-        createContainer(xceiverClientManager.getType(),
-            HddsProtos.ReplicationFactor.ONE, containerOwner);
-    Assert.assertTrue(
-        containerExist(newContainer.getContainerInfo().getContainerID()));
-
-    // ****************************************
-    // 3. Test to delete a non-exist container.
-    // ****************************************
-    long nonExistContainerID = ContainerTestHelper.getTestContainerID();
-    delCmd = new String[]{"-container", "-delete", "-c",
-        Long.toString(nonExistContainerID)};
-    testErr = new ByteArrayOutputStream();
-    exitCode = runCommandAndGetOutput(delCmd, out, testErr);
-    assertEquals(EXECUTION_ERROR, exitCode);
-    assertTrue(testErr.toString()
-        .contains("Specified key does not exist."));
-  }
-
-  @Test
-  public void testInfoContainer() throws Exception {
-    // The cluster has one Datanode server.
-    DatanodeDetails datanodeDetails = cluster.getHddsDatanodes().get(0)
-        .getDatanodeDetails();
-    String formatStr =
-        "Container id: %s\n" +
-        "Container State: %s\n" +
-        "Container DB Path: %s\n" +
-        "Container Path: %s\n" +
-        "Container Metadata: {%s}\n" +
-        "LeaderID: %s\n" +
-        "Datanodes: [%s]\n";
-
-    // Test a non-exist container
-    String containerID =
-        Long.toString(ContainerTestHelper.getTestContainerID());
-    String[] info = {"-container", "-info", containerID};
-    int exitCode = runCommandAndGetOutput(info, null, null);
-    assertEquals("Expected Execution Error, Did not find that.",
-        EXECUTION_ERROR, exitCode);
-
-    // Create an empty container.
-    ContainerWithPipeline container = containerOperationClient
-        .createContainer(xceiverClientManager.getType(),
-            HddsProtos.ReplicationFactor.ONE, containerOwner);
-    KeyValueContainerData data = KeyValueContainerData
-        .getFromProtoBuf(containerOperationClient.
-            readContainer(container.getContainerInfo().getContainerID(),
-                container.getPipeline()));
-    info = new String[]{"-container", "-info", "-c",
-        Long.toString(container.getContainerInfo().getContainerID())};
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    exitCode = runCommandAndGetOutput(info, out, null);
-    assertEquals("Expected Success, did not find it.", ResultCode.SUCCESS,
-        exitCode);
-
-    String openStatus = data.isOpen() ? "OPEN" : "CLOSED";
-    String expected =
-        String.format(formatStr, container.getContainerInfo().getContainerID
-                (), openStatus, data.getDbFile().getPath(), data
-                .getContainerPath(), "", datanodeDetails.getHostName(),
-            datanodeDetails.getHostName());
-
-    assertEquals(expected, out.toString());
-
-    out.reset();
-
-    // Create an non-empty container
-    container = containerOperationClient
-        .createContainer(xceiverClientManager.getType(),
-            HddsProtos.ReplicationFactor.ONE, containerOwner);
-    data = KeyValueContainerData
-        .getFromProtoBuf(containerOperationClient.readContainer(
-            container.getContainerInfo().getContainerID(), container
-                .getPipeline()));
-    KeyUtils.getDB(data, conf)
-        .put(containerID.getBytes(), "someKey".getBytes());
-
-    info = new String[]{"-container", "-info", "-c",
-        Long.toString(container.getContainerInfo().getContainerID())};
-    exitCode = runCommandAndGetOutput(info, out, null);
-    assertEquals(ResultCode.SUCCESS, exitCode);
-
-    openStatus = data.isOpen() ? "OPEN" : "CLOSED";
-
-    expected = String.format(formatStr, container.getContainerInfo()
-            .getContainerID(), openStatus, data.getDbFile().getPath(), data
-            .getContainerPath(), "", datanodeDetails.getHostName(),
-        datanodeDetails.getHostName());
-    assertEquals(expected, out.toString());
-
-    out.reset();
-
-    // Close last container and test info again.
-    containerOperationClient
-        .closeContainer(container.getContainerInfo().getContainerID());
-
-    info = new String[]{"-container", "-info", "-c",
-        Long.toString(container.getContainerInfo().getContainerID())};
-    exitCode = runCommandAndGetOutput(info, out, null);
-    assertEquals(ResultCode.SUCCESS, exitCode);
-    data = KeyValueContainerData
-        .getFromProtoBuf(containerOperationClient.readContainer(
-            container.getContainerInfo().getContainerID(), container
-                .getPipeline()));
-
-    openStatus = data.isOpen() ? "OPEN" : "CLOSED";
-    expected = String
-        .format(formatStr, container.getContainerInfo().getContainerID(),
-            openStatus, data.getDbFile().getPath(), data.getContainerPath(), "",
-            datanodeDetails.getHostName(), datanodeDetails.getHostName());
-    assertEquals(expected, out.toString());
-  }
-
-  @Test
-  public void testNonExistCommand() throws Exception {
-    PrintStream init = System.out;
-    ByteArrayOutputStream testContent = new ByteArrayOutputStream();
-    PrintStream testPrintOut = new PrintStream(testContent);
-    System.setOut(testPrintOut);
-    String[] args = {"-nothingUseful"};
-    assertEquals(ResultCode.UNRECOGNIZED_CMD, cli.run(args));
-    assertTrue(errContent.toString()
-        .contains("Unrecognized options:[-nothingUseful]"));
-    String expectedOut =
-        "usage: hdfs scmcli <commands> [<options>]\n" +
-        "where <commands> can be one of the following\n" +
-        " -container   Container related options\n";
-    assertEquals(expectedOut, testContent.toString());
-    System.setOut(init);
-  }
-
-  @Test
-  public void testListContainerCommand() throws Exception {
-    // Create 20 containers for testing.
-    List<ContainerInfo> containers = new ArrayList<>();
-    for (int index = 0; index < 20; index++) {
-      ContainerWithPipeline container = containerOperationClient.createContainer(
-          xceiverClientManager.getType(), HddsProtos.ReplicationFactor.ONE,
-          containerOwner);
-      containers.add(container.getContainerInfo());
-    }
-
-    ByteArrayOutputStream out = new ByteArrayOutputStream();
-    ByteArrayOutputStream err = new ByteArrayOutputStream();
-
-    // Test without -start, -prefix and -count
-    String[] args = new String[] {"-container", "-list"};
-    int exitCode = runCommandAndGetOutput(args, out, err);
-    assertEquals(EXECUTION_ERROR, exitCode);
-    assertTrue(err.toString()
-        .contains("Expecting container count"));
-
-    out.reset();
-    err.reset();
-
-    long startContainerID = containers.get(0).getContainerID();
-    String startContainerIDStr = Long.toString(startContainerID);
-    // Test with -start and -count, the value of -count is negative.
-    args = new String[] {"-container", "-list",
-        "-start", startContainerIDStr, "-count", "-1"};
-    exitCode = runCommandAndGetOutput(args, out, err);
-    assertEquals(EXECUTION_ERROR, exitCode);
-    assertTrue(err.toString()
-        .contains("-count should not be negative"));
-
-    out.reset();
-    err.reset();
-
-    // Test with -start and -count.
-    args = new String[] {"-container", "-list", "-start",
-        startContainerIDStr, "-count", "10"};
-    exitCode = runCommandAndGetOutput(args, out, err);
-    assertEquals(ResultCode.SUCCESS, exitCode);
-    for (int index = 1; index < 10; index++) {
-      String containerID = Long.toString(
-          containers.get(index).getContainerID());
-      assertTrue(out.toString().contains(containerID));
-    }
-
-    out.reset();
-    err.reset();
-
-    // Test with -start, while -count doesn't exist.
-    args = new String[] {"-container", "-list", "-start",
-        startContainerIDStr};
-    exitCode = runCommandAndGetOutput(args, out, err);
-    assertEquals(ResultCode.EXECUTION_ERROR, exitCode);
-    assertTrue(err.toString().contains(
-        "java.io.IOException: Expecting container count"));
-  }
-
-  @Test
-  public void testCloseContainer() throws Exception {
-    long containerID = containerOperationClient.createContainer(
-        xceiverClientManager.getType(), HddsProtos.ReplicationFactor.ONE,
-        containerOwner).getContainerInfo().getContainerID();
-    ContainerInfo container = scm.getClientProtocolServer()
-        .getContainerWithPipeline(containerID).getContainerInfo();
-    assertNotNull(container);
-    assertEquals(containerID, container.getContainerID());
-
-    ContainerInfo containerInfo = scm.getContainerInfo(containerID);
-    assertEquals(OPEN, containerInfo.getState());
-
-    String[] args1 = {"-container", "-close", "-c",
-        Long.toString(containerID)};
-    assertEquals(ResultCode.SUCCESS, cli.run(args1));
-
-    containerInfo = scm.getContainerInfo(containerID);
-    assertEquals(CLOSED, containerInfo.getState());
-
-    // closing this container again will trigger an error.
-    assertEquals(EXECUTION_ERROR, cli.run(args1));
-  }
-
-  @Test
-  public void testHelp() throws Exception {
-    // TODO : this test assertion may break for every new help entry added
-    // may want to disable this test some time later. For now, mainly to show
-    // case the format of help output.
-    PrintStream init = System.out;
-    ByteArrayOutputStream testContent = new ByteArrayOutputStream();
-    PrintStream testPrintOut = new PrintStream(testContent);
-    System.setOut(testPrintOut);
-    String[] args = {"-help"};
-    assertEquals(ResultCode.SUCCESS, cli.run(args));
-    String expected =
-        "usage: hdfs scmcli <commands> [<options>]\n" +
-        "where <commands> can be one of the following\n" +
-        " -container   Container related options\n";
-    assertEquals(expected, testContent.toString());
-    testContent.reset();
-
-    String[] args1 = {"-container", "-help"};
-    assertEquals(ResultCode.SUCCESS, cli.run(args1));
-    String expected1 =
-        "usage: hdfs scm -container <commands> <options>\n" +
-        "where <commands> can be one of the following\n" +
-        " -close    Close container\n" +
-        " -create   Create container\n" +
-        " -delete   Delete container\n" +
-        " -info     Info container\n" +
-        " -list     List container\n";
-
-    assertEquals(expected1, testContent.toString());
-    testContent.reset();
-
-    String[] args2 = {"-container", "-create", "-help"};
-    assertEquals(ResultCode.SUCCESS, cli.run(args2));
-    String expected2 =
-        "usage: hdfs scm -container -create\n\n";
-    assertEquals(expected2, testContent.toString());
-    testContent.reset();
-
-    String[] args3 = {"-container", "-delete", "-help"};
-    assertEquals(ResultCode.SUCCESS, cli.run(args3));
-    String expected3 =
-        "usage: hdfs scm -container -delete <option>\n" +
-        "where <option> is\n" +
-        " -c <arg>   Specify container id\n" +
-        " -f         forcibly delete a container\n";
-    assertEquals(expected3, testContent.toString());
-    testContent.reset();
-
-    String[] args4 = {"-container", "-info", "-help"};
-    assertEquals(ResultCode.SUCCESS, cli.run(args4));
-    String expected4 =
-        "usage: hdfs scm -container -info <option>\n" +
-        "where <option> is\n" +
-        " -c <arg>   Specify container id\n";
-    assertEquals(expected4, testContent.toString());
-    testContent.reset();
-
-    String[] args5 = {"-container", "-list", "-help"};
-    assertEquals(ResultCode.SUCCESS, cli.run(args5));
-    String expected5 = "usage: hdfs scm -container -list <option>\n"
-        + "where <option> can be the following\n"
-        + " -count <arg>   Specify count number, required\n"
-        + " -start <arg>   Specify start container id\n";
-    assertEquals(expected5, testContent.toString());
-    testContent.reset();
-
-    System.setOut(init);
-  }
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[48/50] [abbrv] hadoop git commit: HDDS-336. Print out container location information for a specific ozone key . Contributed by LiXin Ge.

Posted by eh...@apache.org.
HDDS-336. Print out container location information for a specific ozone key . Contributed by LiXin Ge.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/211034a6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/211034a6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/211034a6

Branch: refs/heads/HDFS-12090
Commit: 211034a6c22dd4ebe697481ea4d57b5eb932fa08
Parents: 3801436
Author: Márton Elek <el...@apache.org>
Authored: Mon Sep 3 13:32:55 2018 +0200
Committer: Márton Elek <el...@apache.org>
Committed: Mon Sep 3 13:32:55 2018 +0200

----------------------------------------------------------------------
 .../apache/hadoop/ozone/client/OzoneBucket.java |   4 +-
 .../hadoop/ozone/client/OzoneClientUtils.java   |  29 ++++-
 .../hadoop/ozone/client/OzoneKeyDetails.java    |  58 ++++++++++
 .../hadoop/ozone/client/OzoneKeyLocation.java   |  82 ++++++++++++++
 .../ozone/client/protocol/ClientProtocol.java   |  10 +-
 .../hadoop/ozone/client/rest/RestClient.java    |  27 ++---
 .../hadoop/ozone/client/rpc/RpcClient.java      |  22 ++--
 .../ozone/client/rest/headers/Header.java       |   1 +
 .../client/rest/response/KeyInfoDetails.java    | 107 +++++++++++++++++++
 .../ozone/client/rest/response/KeyLocation.java |  89 +++++++++++++++
 .../ozone/web/response/KeyInfoDetails.java      |  80 ++++++++++++++
 .../hadoop/ozone/web/response/KeyLocation.java  |  82 ++++++++++++++
 .../ozone/client/rest/TestOzoneRestClient.java  |  86 +++++++++++++--
 .../ozone/client/rpc/TestOzoneRpcClient.java    | 101 +++++++++++++++--
 .../hadoop/ozone/ozShell/TestOzoneShell.java    |   6 +-
 .../hadoop/ozone/web/handlers/KeyHandler.java   |  12 +++
 .../ozone/web/interfaces/StorageHandler.java    |  12 +++
 .../web/storage/DistributedStorageHandler.java  |  33 ++++--
 .../ozone/web/ozShell/keys/InfoKeyHandler.java  |  10 +-
 19 files changed, 779 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
index 2f3cff6..97bd682 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
@@ -258,10 +258,10 @@ public class OzoneBucket {
   /**
    * Returns information about the key.
    * @param key Name of the key.
-   * @return OzoneKey Information about the key.
+   * @return OzoneKeyDetails Information about the key.
    * @throws IOException
    */
-  public OzoneKey getKey(String key) throws IOException {
+  public OzoneKeyDetails getKey(String key) throws IOException {
     return proxy.getKeyDetails(volumeName, name, key);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java
index 5d57753..40e4d83 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java
@@ -25,10 +25,10 @@ import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.client.rest.response.BucketInfo;
-import org.apache.hadoop.ozone.client.rest.response.KeyInfo;
-import org.apache.hadoop.ozone.client.rest.response.VolumeInfo;
-import org.apache.hadoop.ozone.client.rest.response.VolumeOwner;
+import org.apache.hadoop.ozone.client.rest.response.*;
+
+import java.util.ArrayList;
+import java.util.List;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -112,4 +112,25 @@ public final class OzoneClientUtils {
             exceptionToPolicyMap);
     return retryPolicy;
   }
+  /**
+   * Returns a KeyInfoDetails object constructed using fields of the input
+   * OzoneKeyDetails object.
+   *
+   * @param key OzoneKeyDetails instance from which KeyInfo object needs to
+   *            be created.
+   * @return KeyInfoDetails instance
+   */
+  public static KeyInfoDetails asKeyInfoDetails(OzoneKeyDetails key) {
+    KeyInfoDetails keyInfo = new KeyInfoDetails();
+    keyInfo.setKeyName(key.getName());
+    keyInfo.setCreatedOn(HddsClientUtils.formatDateTime(key.getCreationTime()));
+    keyInfo.setModifiedOn(
+        HddsClientUtils.formatDateTime(key.getModificationTime()));
+    keyInfo.setSize(key.getDataSize());
+    List<KeyLocation> keyLocations = new ArrayList<>();
+    key.getOzoneKeyLocations().forEach((a) -> keyLocations.add(new KeyLocation(
+        a.getContainerID(), a.getLocalID(), a.getLength(), a.getOffset())));
+    keyInfo.setKeyLocation(keyLocations);
+    return keyInfo;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyDetails.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyDetails.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyDetails.java
new file mode 100644
index 0000000..e7709dd
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyDetails.java
@@ -0,0 +1,58 @@
+/**
+ * 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.client;
+
+import java.util.List;
+
+/**
+ * A class that encapsulates OzoneKeyLocation.
+ */
+public class OzoneKeyDetails extends OzoneKey {
+
+  /**
+   * A list of block location information to specify replica locations.
+   */
+  private List<OzoneKeyLocation> ozoneKeyLocations;
+
+  /**
+   * Constructs OzoneKeyDetails from OmKeyInfo.
+   */
+  public OzoneKeyDetails(String volumeName, String bucketName, String keyName,
+                  long size, long creationTime, long modificationTime,
+                         List<OzoneKeyLocation> ozoneKeyLocations) {
+    super(volumeName, bucketName, keyName, size, creationTime,
+        modificationTime);
+    this.ozoneKeyLocations = ozoneKeyLocations;
+  }
+
+  /**
+   * Returns the location detail information of the specific Key.
+   */
+  public List<OzoneKeyLocation> getOzoneKeyLocations() {
+    return ozoneKeyLocations;
+  }
+
+  /**
+   * Set details of key location.
+   * @param ozoneKeyLocations - details of key location
+   */
+  public void setOzoneKeyLocations(List<OzoneKeyLocation> ozoneKeyLocations) {
+    this.ozoneKeyLocations = ozoneKeyLocations;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyLocation.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyLocation.java
new file mode 100644
index 0000000..0ff8ba7
--- /dev/null
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneKeyLocation.java
@@ -0,0 +1,82 @@
+/**
+ * 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.client;
+
+/**
+ * One key can be stored in one or more containers as one or more blocks.
+ * This class represents one such block instance.
+ */
+public class OzoneKeyLocation {
+  /**
+   * Which container this key stored.
+   */
+  private final long containerID;
+  /**
+   * Which block this key stored inside a container.
+   */
+  private final long localID;
+  /**
+   * Data length of this key replica.
+   */
+  private final long length;
+  /**
+   * Offset of this key.
+   */
+  private final long offset;
+
+  /**
+   * Constructs OzoneKeyLocation.
+   */
+  public OzoneKeyLocation(long containerID, long localID,
+                  long length, long offset) {
+    this.containerID = containerID;
+    this.localID = localID;
+    this.length = length;
+    this.offset = offset;
+  }
+
+  /**
+   * Returns the containerID of this Key.
+   */
+  public long getContainerID() {
+    return containerID;
+  }
+
+  /**
+   * Returns the localID of this Key.
+   */
+  public long getLocalID() {
+    return localID;
+  }
+
+  /**
+   * Returns the length of this Key.
+   */
+  public long getLength() {
+    return length;
+  }
+
+  /**
+   * Returns the offset of this Key.
+   */
+  public long getOffset() {
+    return offset;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
index 94cc257..008b69d 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
@@ -20,14 +20,10 @@ package org.apache.hadoop.ozone.client.protocol;
 
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.client.BucketArgs;
-import org.apache.hadoop.ozone.client.OzoneBucket;
-import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.*;
 import org.apache.hadoop.hdds.client.OzoneQuota;
-import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
-import org.apache.hadoop.ozone.client.VolumeArgs;
 import org.apache.hadoop.ozone.client.io.OzoneInputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 
@@ -321,8 +317,8 @@ public interface ClientProtocol {
    * @return {@link OzoneKey}
    * @throws IOException
    */
-  OzoneKey getKeyDetails(String volumeName, String bucketName,
-                         String keyName)
+  OzoneKeyDetails getKeyDetails(String volumeName, String bucketName,
+                                String keyName)
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java
index 78fbe8d..fdd049a 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java
@@ -30,12 +30,8 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.client.BucketArgs;
-import org.apache.hadoop.ozone.client.VolumeArgs;
-import org.apache.hadoop.ozone.client.OzoneBucket;
-import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.*;
 import org.apache.hadoop.hdds.client.OzoneQuota;
-import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.ozone.client.io.OzoneInputStream;
@@ -43,7 +39,7 @@ import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.client.rest.headers.Header;
 import org.apache.hadoop.ozone.client.rest.response.BucketInfo;
-import org.apache.hadoop.ozone.client.rest.response.KeyInfo;
+import org.apache.hadoop.ozone.client.rest.response.KeyInfoDetails;
 import org.apache.hadoop.ozone.client.rest.response.VolumeInfo;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
@@ -80,6 +76,7 @@ import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.text.ParseException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.FutureTask;
@@ -788,7 +785,7 @@ public class RestClient implements ClientProtocol {
   }
 
   @Override
-  public OzoneKey getKeyDetails(
+  public OzoneKeyDetails getKeyDetails(
       String volumeName, String bucketName, String keyName)
       throws IOException {
     try {
@@ -798,18 +795,24 @@ public class RestClient implements ClientProtocol {
       builder.setPath(PATH_SEPARATOR + volumeName +
           PATH_SEPARATOR + bucketName + PATH_SEPARATOR + keyName);
       builder.setParameter(Header.OZONE_INFO_QUERY_TAG,
-          Header.OZONE_INFO_QUERY_KEY);
+          Header.OZONE_INFO_QUERY_KEY_DETAIL);
       HttpGet httpGet = new HttpGet(builder.build());
       addOzoneHeaders(httpGet);
       HttpEntity response = executeHttpRequest(httpGet);
-      KeyInfo keyInfo =
-          KeyInfo.parse(EntityUtils.toString(response));
-      OzoneKey key = new OzoneKey(volumeName,
+      KeyInfoDetails keyInfo =
+          KeyInfoDetails.parse(EntityUtils.toString(response));
+
+      List<OzoneKeyLocation> ozoneKeyLocations = new ArrayList<>();
+      keyInfo.getKeyLocations().forEach((a) -> ozoneKeyLocations.add(
+          new OzoneKeyLocation(a.getContainerID(), a.getLocalID(),
+              a.getLength(), a.getOffset())));
+      OzoneKeyDetails key = new OzoneKeyDetails(volumeName,
           bucketName,
           keyInfo.getKeyName(),
           keyInfo.getSize(),
           HddsClientUtils.formatDateTime(keyInfo.getCreatedOn()),
-          HddsClientUtils.formatDateTime(keyInfo.getModifiedOn()));
+          HddsClientUtils.formatDateTime(keyInfo.getModifiedOn()),
+          ozoneKeyLocations);
       EntityUtils.consume(response);
       return key;
     } catch (URISyntaxException | ParseException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
index 387f41f..e9a684e 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
@@ -30,11 +30,8 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ozone.OmUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.client.BucketArgs;
-import org.apache.hadoop.ozone.client.OzoneBucket;
-import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.*;
 import org.apache.hadoop.hdds.client.OzoneQuota;
-import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.ozone.client.VolumeArgs;
@@ -73,10 +70,7 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.UUID;
+import java.util.*;
 import java.util.stream.Collectors;
 
 /**
@@ -548,7 +542,7 @@ public class RpcClient implements ClientProtocol {
   }
 
   @Override
-  public OzoneKey getKeyDetails(
+  public OzoneKeyDetails getKeyDetails(
       String volumeName, String bucketName, String keyName)
       throws IOException {
     Preconditions.checkNotNull(volumeName);
@@ -560,12 +554,18 @@ public class RpcClient implements ClientProtocol {
         .setKeyName(keyName)
         .build();
     OmKeyInfo keyInfo = ozoneManagerClient.lookupKey(keyArgs);
-    return new OzoneKey(keyInfo.getVolumeName(),
+
+    List<OzoneKeyLocation> ozoneKeyLocations = new ArrayList<>();
+    keyInfo.getLatestVersionLocations().getBlocksLatestVersionOnly().forEach(
+        (a) -> ozoneKeyLocations.add(new OzoneKeyLocation(a.getContainerID(),
+            a.getLocalID(), a.getLength(), a.getOffset())));
+    return new OzoneKeyDetails(keyInfo.getVolumeName(),
                         keyInfo.getBucketName(),
                         keyInfo.getKeyName(),
                         keyInfo.getDataSize(),
                         keyInfo.getCreationTime(),
-                        keyInfo.getModificationTime());
+                        keyInfo.getModificationTime(),
+                        ozoneKeyLocations);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/headers/Header.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/headers/Header.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/headers/Header.java
index ebfc0a9..3e40493 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/headers/Header.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/headers/Header.java
@@ -44,6 +44,7 @@ public final class Header {
   public static final String OZONE_INFO_QUERY_VOLUME = "volume";
   public static final String OZONE_INFO_QUERY_BUCKET = "bucket";
   public static final String OZONE_INFO_QUERY_KEY = "key";
+  public static final String OZONE_INFO_QUERY_KEY_DETAIL = "key-detail";
 
   public static final String OZONE_REQUEST_ID = "x-ozone-request-id";
   public static final String OZONE_SERVER_NAME = "x-ozone-server-name";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/response/KeyInfoDetails.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/response/KeyInfoDetails.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/response/KeyInfoDetails.java
new file mode 100644
index 0000000..98506f0
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/response/KeyInfoDetails.java
@@ -0,0 +1,107 @@
+/**
+ * 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.client.rest.response;
+
+import java.io.IOException;
+import java.util.List;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+
+/**
+ * KeyInfoDetails class is used for parsing json response
+ * when KeyInfoDetails Call is made.
+ */
+public class KeyInfoDetails extends KeyInfo {
+
+  private static final ObjectReader READER =
+      new ObjectMapper().readerFor(KeyInfoDetails.class);
+
+  /**
+   * a list of Map which maps localID to ContainerID
+   * to specify replica locations.
+   */
+  private List<KeyLocation> keyLocations;
+
+  /**
+   * Constructor needed for json serialization.
+   */
+  public KeyInfoDetails() {
+  }
+
+  /**
+   * Set details of key location.
+   *
+   * @param locations - details of key location
+   */
+  public void setKeyLocation(List<KeyLocation> locations) {
+    this.keyLocations = locations;
+  }
+
+  /**
+   * Returns details of key location.
+   *
+   * @return volumeName
+   */
+  public List<KeyLocation> getKeyLocations() {
+    return keyLocations;
+  }
+
+  /**
+   * Parse a string to return KeyInfoDetails Object.
+   *
+   * @param jsonString Json String
+   * @return KeyInfoDetails
+   * @throws IOException
+   */
+  public static KeyInfoDetails parse(String jsonString) throws IOException {
+    return READER.readValue(jsonString);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    KeyInfoDetails that = (KeyInfoDetails) o;
+
+    return new EqualsBuilder()
+        .append(getVersion(), that.getVersion())
+        .append(getKeyName(), that.getKeyName())
+        .append(keyLocations, that.keyLocations)
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(21, 33)
+        .append(getVersion())
+        .append(getKeyName())
+        .append(keyLocations)
+        .toHashCode();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/response/KeyLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/response/KeyLocation.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/response/KeyLocation.java
new file mode 100644
index 0000000..e5f4698
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/rest/response/KeyLocation.java
@@ -0,0 +1,89 @@
+/**
+ * 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.client.rest.response;
+
+/**
+ * KeyLocation class is used used for parsing json response
+ * when KeyInfoDetails Call is made.
+ */
+public class KeyLocation {
+  /**
+   * Which container this key stored.
+   */
+  private long containerID;
+  /**
+   * Which block this key stored inside a container.
+   */
+  private long localID;
+  /**
+   * Data length of this key replica.
+   */
+  private long length;
+  /**
+   * Offset of this key.
+   */
+  private long offset;
+
+  /**
+   * Empty constructor for Json serialization.
+   */
+  public KeyLocation() {
+
+  }
+
+  /**
+   * Constructs KeyLocation.
+   */
+  public KeyLocation(long containerID, long localID,
+                          long length, long offset) {
+    this.containerID = containerID;
+    this.localID = localID;
+    this.length = length;
+    this.offset = offset;
+  }
+
+  /**
+   * Returns the containerID of this Key.
+   */
+  public long getContainerID() {
+    return containerID;
+  }
+
+  /**
+   * Returns the localID of this Key.
+   */
+  public long getLocalID() {
+    return localID;
+  }
+
+  /**
+   * Returns the length of this Key.
+   */
+  public long getLength() {
+    return length;
+  }
+
+  /**
+   * Returns the offset of this Key.
+   */
+  public long getOffset() {
+    return offset;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/response/KeyInfoDetails.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/response/KeyInfoDetails.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/response/KeyInfoDetails.java
new file mode 100644
index 0000000..7f2ba09
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/response/KeyInfoDetails.java
@@ -0,0 +1,80 @@
+/*
+ * 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.web.response;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+
+import java.util.List;
+
+/**
+ * Represents an Ozone key Object with detail information of location.
+ */
+public class KeyInfoDetails extends KeyInfo {
+  /**
+   * a list of Map which maps localID to ContainerID
+   * to specify replica locations.
+   */
+  private List<KeyLocation> keyLocations;
+
+  /**
+   * Set details of key location.
+   *
+   * @param keyLocations - details of key location
+   */
+  public void setKeyLocations(List<KeyLocation> keyLocations) {
+    this.keyLocations = keyLocations;
+  }
+
+  /**
+   * Returns details of key location.
+   *
+   * @return volumeName
+   */
+  public List<KeyLocation> getKeyLocations() {
+    return keyLocations;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    KeyInfoDetails that = (KeyInfoDetails) o;
+
+    return new EqualsBuilder()
+        .append(getVersion(), that.getVersion())
+        .append(getKeyName(), that.getKeyName())
+        .append(keyLocations, that.getKeyLocations())
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37)
+        .append(getVersion())
+        .append(getKeyName())
+        .append(keyLocations)
+        .toHashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/response/KeyLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/response/KeyLocation.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/response/KeyLocation.java
new file mode 100644
index 0000000..d03eff7
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/web/response/KeyLocation.java
@@ -0,0 +1,82 @@
+/**
+ * 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.web.response;
+
+/**
+ * KeyLocation class is used used for parsing json response
+ * when KeyInfoDetails Call is made.
+ */
+public class KeyLocation {
+  /**
+   * Which container this key stored.
+   */
+  private final long containerID;
+  /**
+   * Which block this key stored inside a container.
+   */
+  private final long localID;
+  /**
+   * Data length of this key replica.
+   */
+  private final long length;
+  /**
+   * Offset of this key.
+   */
+  private final long offset;
+
+  /**
+   * Constructs KeyLocation.
+   */
+  public KeyLocation(long containerID, long localID,
+                     long length, long offset) {
+    this.containerID = containerID;
+    this.localID = localID;
+    this.length = length;
+    this.offset = offset;
+  }
+
+  /**
+   * Returns the containerID of this Key.
+   */
+  public long getContainerID() {
+    return containerID;
+  }
+
+  /**
+   * Returns the localID of this Key.
+   */
+  public long getLocalID() {
+    return localID;
+  }
+
+  /**
+   * Returns the length of this Key.
+   */
+  public long getLength() {
+    return length;
+  }
+
+  /**
+   * Returns the offset of this Key.
+   */
+  public long getOffset() {
+    return offset;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java
index 73bb32d..ddff0c5 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java
@@ -18,23 +18,22 @@
 
 package org.apache.hadoop.ozone.client.rest;
 
+import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.client.BucketArgs;
-import org.apache.hadoop.ozone.client.ObjectStore;
-import org.apache.hadoop.ozone.client.OzoneBucket;
-import org.apache.hadoop.ozone.client.OzoneClient;
-import org.apache.hadoop.ozone.client.OzoneClientFactory;
-import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.ozone.*;
+import org.apache.hadoop.ozone.client.*;
 import org.apache.hadoop.hdds.client.OzoneQuota;
-import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
-import org.apache.hadoop.ozone.client.VolumeArgs;
 import org.apache.hadoop.ozone.client.io.OzoneInputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.container.common.helpers.KeyData;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueBlockIterator;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -42,6 +41,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
+import java.io.File;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
@@ -414,6 +414,72 @@ public class TestOzoneRestClient {
     bucket.getKey(fromKeyName);
   }
 
+  @Test
+  public void testGetKeyDetails() throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+    String keyName = UUID.randomUUID().toString();
+    String keyValue = RandomStringUtils.random(128);
+    OzoneOutputStream out = bucket.createKey(keyName,
+        keyValue.getBytes().length, ReplicationType.STAND_ALONE,
+        ReplicationFactor.ONE);
+    out.write(keyValue.getBytes());
+    out.close();
+
+    // Get the containerID and localID.
+    OzoneKeyDetails keyDetails = (OzoneKeyDetails)bucket.getKey(keyName);
+    Assert.assertEquals(keyName, keyDetails.getName());
+    List<OzoneKeyLocation> keyLocations = keyDetails.getOzoneKeyLocations();
+    Assert.assertEquals(1, keyLocations.size());
+    Long containerID = keyLocations.get(0).getContainerID();
+    Long localID = keyLocations.get(0).getLocalID();
+
+    // Make sure that the data size matched.
+    Assert.assertEquals(keyValue.getBytes().length,
+        keyLocations.get(0).getLength());
+
+    // Sum the data size from chunks in Container via containerID
+    // and localID, make sure the size equals to the actually value size.
+    Pipeline pipeline = cluster.getStorageContainerManager()
+        .getScmContainerManager().getContainerWithPipeline(containerID)
+        .getPipeline();
+    List<DatanodeDetails> datanodes = pipeline.getMachines();
+    Assert.assertEquals(datanodes.size(), 1);
+
+    DatanodeDetails datanodeDetails = datanodes.get(0);
+    Assert.assertNotNull(datanodeDetails);
+    HddsDatanodeService datanodeService = null;
+    for (HddsDatanodeService datanodeServiceItr : cluster.getHddsDatanodes()) {
+      if (datanodeDetails.equals(datanodeServiceItr.getDatanodeDetails())) {
+        datanodeService = datanodeServiceItr;
+        break;
+      }
+    }
+    KeyValueContainerData containerData =
+        (KeyValueContainerData)(datanodeService.getDatanodeStateMachine()
+            .getContainer().getContainerSet().getContainer(containerID)
+            .getContainerData());
+    String containerPath = new File(containerData.getMetadataPath())
+        .getParent();
+    KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
+        containerID, new File(containerPath));
+    long valueLength = 0;
+    while (keyValueBlockIterator.hasNext()) {
+      KeyData keyData = keyValueBlockIterator.nextBlock();
+      if (keyData.getBlockID().getLocalID() == localID) {
+        List<ContainerProtos.ChunkInfo> chunks = keyData.getChunks();
+        for (ContainerProtos.ChunkInfo chunk : chunks) {
+          valueLength += chunk.getLen();
+        }
+      }
+    }
+    Assert.assertEquals(keyValue.getBytes().length, valueLength);
+  }
+
   /**
    * Close OzoneClient and shutdown MiniDFSCluster.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
index f8ad32e..bf1eba6 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java
@@ -21,23 +21,21 @@ package org.apache.hadoop.ozone.client.rpc;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
-import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.ozone.*;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.ozone.client.BucketArgs;
-import org.apache.hadoop.ozone.client.ObjectStore;
-import org.apache.hadoop.ozone.client.OzoneBucket;
-import org.apache.hadoop.ozone.client.OzoneClient;
-import org.apache.hadoop.ozone.client.OzoneClientFactory;
-import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.*;
 import org.apache.hadoop.hdds.client.OzoneQuota;
-import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
-import org.apache.hadoop.ozone.client.VolumeArgs;
 import org.apache.hadoop.ozone.client.io.OzoneInputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.container.common.helpers.KeyData;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueBlockIterator;
+import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.om.OzoneManager;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
@@ -56,6 +54,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Iterator;
@@ -536,6 +535,88 @@ public class TestOzoneRpcClient {
   }
 
   @Test
+  public void testGetKeyDetails() throws IOException, OzoneException {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+    String keyName = UUID.randomUUID().toString();
+    String keyValue = RandomStringUtils.random(128);
+    //String keyValue = "this is a test value.glx";
+    // create the initial key with size 0, write will allocate the first block.
+    OzoneOutputStream out = bucket.createKey(keyName,
+        keyValue.getBytes().length, ReplicationType.STAND_ALONE,
+        ReplicationFactor.ONE);
+    out.write(keyValue.getBytes());
+    out.close();
+
+    OzoneInputStream is = bucket.readKey(keyName);
+    byte[] fileContent = new byte[32];
+    is.read(fileContent);
+
+    // First, confirm the key info from the client matches the info in OM.
+    OmKeyArgs.Builder builder = new OmKeyArgs.Builder();
+    builder.setVolumeName(volumeName).setBucketName(bucketName)
+        .setKeyName(keyName);
+    OmKeyLocationInfo keyInfo = ozoneManager.lookupKey(builder.build()).
+        getKeyLocationVersions().get(0).getBlocksLatestVersionOnly().get(0);
+    long containerID = keyInfo.getContainerID();
+    long localID = keyInfo.getLocalID();
+    OzoneKeyDetails keyDetails = (OzoneKeyDetails)bucket.getKey(keyName);
+    Assert.assertEquals(keyName, keyDetails.getName());
+
+    List<OzoneKeyLocation> keyLocations = keyDetails.getOzoneKeyLocations();
+    Assert.assertEquals(1, keyLocations.size());
+    Assert.assertEquals(containerID, keyLocations.get(0).getContainerID());
+    Assert.assertEquals(localID, keyLocations.get(0).getLocalID());
+
+    // Make sure that the data size matched.
+    Assert.assertEquals(keyValue.getBytes().length,
+        keyLocations.get(0).getLength());
+
+    // Second, sum the data size from chunks in Container via containerID
+    // and localID, make sure the size equals to the size from keyDetails.
+    Pipeline pipeline = cluster.getStorageContainerManager()
+        .getScmContainerManager().getContainerWithPipeline(containerID)
+        .getPipeline();
+    List<DatanodeDetails> datanodes = pipeline.getMachines();
+    Assert.assertEquals(datanodes.size(), 1);
+
+    DatanodeDetails datanodeDetails = datanodes.get(0);
+    Assert.assertNotNull(datanodeDetails);
+    HddsDatanodeService datanodeService = null;
+    for (HddsDatanodeService datanodeServiceItr : cluster.getHddsDatanodes()) {
+      if (datanodeDetails.equals(datanodeServiceItr.getDatanodeDetails())) {
+        datanodeService = datanodeServiceItr;
+        break;
+      }
+    }
+    KeyValueContainerData containerData =
+        (KeyValueContainerData)(datanodeService.getDatanodeStateMachine()
+        .getContainer().getContainerSet().getContainer(containerID)
+        .getContainerData());
+    String containerPath = new File(containerData.getMetadataPath())
+        .getParent();
+    KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
+        containerID, new File(containerPath));
+    while (keyValueBlockIterator.hasNext()) {
+      KeyData keyData = keyValueBlockIterator.nextBlock();
+      if (keyData.getBlockID().getLocalID() == localID) {
+        long length = 0;
+        List<ContainerProtos.ChunkInfo> chunks = keyData.getChunks();
+        for (ContainerProtos.ChunkInfo chunk : chunks) {
+          length += chunk.getLen();
+        }
+        Assert.assertEquals(length, keyValue.getBytes().length);
+        break;
+      }
+    }
+  }
+
+  @Test
   public void testDeleteKey()
       throws IOException, OzoneException {
     thrown.expectMessage("Lookup key failed, error");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
index 528828b..b4740b4 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
@@ -773,7 +773,7 @@ public class TestOzoneShell {
   }
 
   @Test
-  public void testInfoKey() throws Exception {
+  public void testInfoKeyDetails() throws Exception {
     LOG.info("Running testInfoKey");
     String keyName = "key" + RandomStringUtils.randomNumeric(5);
     OzoneBucket bucket = creatBucket();
@@ -797,7 +797,9 @@ public class TestOzoneShell {
     assertTrue(
         output.contains("createdOn") && output.contains("modifiedOn") && output
             .contains(OzoneConsts.OZONE_TIME_ZONE));
-
+    assertTrue(
+        output.contains("containerID") && output.contains("localID") && output
+            .contains("length") && output.contains("offset"));
     // reset stream
     out.reset();
     err.reset();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyHandler.java b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyHandler.java
index 8c0b103..97e2a1a 100644
--- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyHandler.java
+++ b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyHandler.java
@@ -88,6 +88,8 @@ public class KeyHandler implements Keys {
           return getKey(args);
         } else if (info.equals(Header.OZONE_INFO_QUERY_KEY)) {
           return getKeyInfo(args);
+        } else if (info.equals(Header.OZONE_INFO_QUERY_KEY_DETAIL)) {
+          return getKeyInfoDetail(args);
         }
 
         OzoneException ozException = ErrorTable
@@ -119,6 +121,16 @@ public class KeyHandler implements Keys {
   }
 
   /**
+   * Gets the Key detail information if it exists.
+   */
+  private Response getKeyInfoDetail(KeyArgs args)
+      throws IOException, OzoneException {
+    StorageHandler fs = StorageHandlerBuilder.getStorageHandler();
+    KeyInfo keyInfo = fs.getKeyInfoDetails(args);
+    return OzoneRestUtils.getResponse(args, HTTP_OK, keyInfo.toJsonString());
+  }
+
+  /**
    * Adds a key to an existing bucket. If the object already exists this call
    * will overwrite or add with new version number if the bucket versioning is
    * turned on.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
index 338ff63..9c115a8 100644
--- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
+++ b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/interfaces/StorageHandler.java
@@ -298,6 +298,18 @@ public interface StorageHandler extends Closeable{
   KeyInfo getKeyInfo(KeyArgs args) throws IOException, OzoneException;
 
   /**
+   * Get detail information of the specified Key.
+   *
+   * @param args Key Args
+   *
+   * @return KeyInfo
+   *
+   * @throws IOException
+   * @throws OzoneException
+   */
+  KeyInfo getKeyInfoDetails(KeyArgs args) throws IOException, OzoneException;
+
+  /**
    * Closes all the opened resources.
    */
   void close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
index 0d62432..88f2d6e 100644
--- a/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
+++ b/hadoop-ozone/objectstore-service/src/main/java/org/apache/hadoop/ozone/web/storage/DistributedStorageHandler.java
@@ -54,18 +54,13 @@ import org.apache.hadoop.ozone.web.handlers.ListArgs;
 import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
-import org.apache.hadoop.ozone.web.response.ListVolumes;
-import org.apache.hadoop.ozone.web.response.VolumeInfo;
-import org.apache.hadoop.ozone.web.response.VolumeOwner;
-import org.apache.hadoop.ozone.web.response.ListBuckets;
-import org.apache.hadoop.ozone.web.response.BucketInfo;
-import org.apache.hadoop.ozone.web.response.KeyInfo;
-import org.apache.hadoop.ozone.web.response.ListKeys;
+import org.apache.hadoop.ozone.web.response.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.io.OutputStream;
+import java.util.ArrayList;
 import java.util.List;
 
 /**
@@ -495,6 +490,30 @@ public final class DistributedStorageHandler implements StorageHandler {
   }
 
   @Override
+  public KeyInfo getKeyInfoDetails(KeyArgs args) throws IOException{
+    OmKeyArgs keyArgs = new OmKeyArgs.Builder()
+        .setVolumeName(args.getVolumeName())
+        .setBucketName(args.getBucketName())
+        .setKeyName(args.getKeyName())
+        .build();
+    OmKeyInfo omKeyInfo = ozoneManagerClient.lookupKey(keyArgs);
+    List<KeyLocation> keyLocations = new ArrayList<>();
+    omKeyInfo.getLatestVersionLocations().getBlocksLatestVersionOnly()
+        .forEach((a) -> keyLocations.add(new KeyLocation(a.getContainerID(),
+            a.getLocalID(), a.getLength(), a.getOffset())));
+    KeyInfoDetails keyInfoDetails = new KeyInfoDetails();
+    keyInfoDetails.setVersion(0);
+    keyInfoDetails.setKeyName(omKeyInfo.getKeyName());
+    keyInfoDetails.setSize(omKeyInfo.getDataSize());
+    keyInfoDetails.setCreatedOn(
+        HddsClientUtils.formatDateTime(omKeyInfo.getCreationTime()));
+    keyInfoDetails.setModifiedOn(
+        HddsClientUtils.formatDateTime(omKeyInfo.getModificationTime()));
+    keyInfoDetails.setKeyLocations(keyLocations);
+    return keyInfoDetails;
+  }
+
+  @Override
   public ListKeys listKeys(ListArgs args) throws IOException, OzoneException {
     ListKeys result = new ListKeys();
     UserArgs userArgs = args.getArgs();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/211034a6/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java
index 3fcdda9..fe87756 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/InfoKeyHandler.java
@@ -25,11 +25,7 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 
 import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.ozone.client.OzoneClientException;
-import org.apache.hadoop.ozone.client.OzoneClientUtils;
-import org.apache.hadoop.ozone.client.OzoneBucket;
-import org.apache.hadoop.ozone.client.OzoneKey;
-import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.*;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.web.ozShell.Handler;
 import org.apache.hadoop.ozone.web.ozShell.Shell;
@@ -79,9 +75,9 @@ public class InfoKeyHandler extends Handler {
 
     OzoneVolume vol = client.getObjectStore().getVolume(volumeName);
     OzoneBucket bucket = vol.getBucket(bucketName);
-    OzoneKey key = bucket.getKey(keyName);
+    OzoneKeyDetails key = bucket.getKey(keyName);
 
     System.out.printf("%s%n", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
-        JsonUtils.toJsonString(OzoneClientUtils.asKeyInfo(key))));
+        JsonUtils.toJsonString(OzoneClientUtils.asKeyInfoDetails(key))));
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[39/50] [abbrv] hadoop git commit: HDDS-379. Simplify and improve the cli arg parsing of ozone scmcli. Contributed by Elek, Marton.

Posted by eh...@apache.org.
HDDS-379. Simplify and improve the cli arg parsing of ozone scmcli.
Contributed by Elek, Marton.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/76bae4cc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/76bae4cc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/76bae4cc

Branch: refs/heads/HDFS-12090
Commit: 76bae4ccb1d929260038b1869be8070c2320b617
Parents: 50d2e3e
Author: Anu Engineer <ae...@apache.org>
Authored: Fri Aug 31 18:11:01 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Fri Aug 31 18:11:01 2018 -0700

----------------------------------------------------------------------
 .../common/dev-support/findbugsExcludeFile.xml  |   4 +
 .../org/apache/hadoop/hdds/cli/GenericCli.java  |  82 +++
 .../hadoop/hdds/cli/HddsVersionProvider.java    |  35 ++
 .../apache/hadoop/hdds/cli/package-info.java    |  22 +
 hadoop-hdds/pom.xml                             |   5 +
 .../hadoop/hdds/scm/cli/OzoneBaseCLI.java       |  43 --
 .../hdds/scm/cli/OzoneCommandHandler.java       |  87 ----
 .../apache/hadoop/hdds/scm/cli/ResultCode.java  |  31 --
 .../org/apache/hadoop/hdds/scm/cli/SCMCLI.java  | 246 +++------
 .../cli/container/CloseContainerHandler.java    |  85 ---
 .../hdds/scm/cli/container/CloseSubcommand.java |  54 ++
 .../cli/container/ContainerCommandHandler.java  | 128 -----
 .../cli/container/CreateContainerHandler.java   |  67 ---
 .../scm/cli/container/CreateSubcommand.java     |  65 +++
 .../cli/container/DeleteContainerHandler.java   |  95 ----
 .../scm/cli/container/DeleteSubcommand.java     |  60 +++
 .../scm/cli/container/InfoContainerHandler.java | 114 ----
 .../hdds/scm/cli/container/InfoSubcommand.java  |  94 ++++
 .../scm/cli/container/ListContainerHandler.java | 117 -----
 .../hdds/scm/cli/container/ListSubcommand.java  |  83 +++
 .../hdds/scm/cli/container/package-info.java    |   3 +
 .../hadoop/hdds/scm/cli/package-info.java       |  12 +-
 hadoop-ozone/common/src/main/bin/ozone          |   2 +-
 .../org/apache/hadoop/ozone/scm/TestSCMCli.java | 518 -------------------
 24 files changed, 596 insertions(+), 1456 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/common/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/dev-support/findbugsExcludeFile.xml b/hadoop-hdds/common/dev-support/findbugsExcludeFile.xml
index daf6fec..c7db679 100644
--- a/hadoop-hdds/common/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdds/common/dev-support/findbugsExcludeFile.xml
@@ -21,4 +21,8 @@
   <Match>
     <Package name="org.apache.hadoop.hdds.protocol.datanode.proto"/>
   </Match>
+  <Match>
+    <Class name="org.apache.hadoop.hdds.cli.GenericCli"></Class>
+    <Bug pattern="DM_EXIT" />
+  </Match>
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
new file mode 100644
index 0000000..2b3e6c0
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
@@ -0,0 +1,82 @@
+/**
+ * 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.cli;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+
+import picocli.CommandLine;
+import picocli.CommandLine.ExecutionException;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.ParameterException;
+import picocli.CommandLine.RunLast;
+
+/**
+ * This is a generic parent class for all the ozone related cli tools.
+ */
+public class GenericCli implements Callable<Void> {
+
+  @Option(names = {"--verbose"},
+      description = "More verbose output. Show the stack trace of the errors.")
+  private boolean verbose;
+
+  @Option(names = {"-D", "--set"})
+  private Map<String, String> configurationOverrides = new HashMap<>();
+
+  private final CommandLine cmd;
+
+  public GenericCli() {
+    cmd = new CommandLine(this);
+  }
+
+  public void run(String[] argv) {
+    try {
+      cmd.parseWithHandler(new RunLast(), argv);
+    } catch (ExecutionException ex) {
+      printError(ex.getCause());
+      System.exit(-1);
+    }
+  }
+
+  private void printError(Throwable error) {
+    if (verbose) {
+      error.printStackTrace(System.err);
+    } else {
+      System.err.println(error.getMessage().split("\n")[0]);
+    }
+  }
+
+  @Override
+  public Void call() throws Exception {
+    throw new ParameterException(cmd, "Please choose a subcommand");
+  }
+
+  public OzoneConfiguration createOzoneConfiguration() {
+    OzoneConfiguration ozoneConf = new OzoneConfiguration();
+    if (configurationOverrides != null) {
+      for (Entry<String, String> entry : configurationOverrides.entrySet()) {
+        ozoneConf
+            .set(entry.getKey(), configurationOverrides.get(entry.getValue()));
+      }
+    }
+    return ozoneConf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/HddsVersionProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/HddsVersionProvider.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/HddsVersionProvider.java
new file mode 100644
index 0000000..7110839
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/HddsVersionProvider.java
@@ -0,0 +1,35 @@
+/**
+ * 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.cli;
+
+import org.apache.hadoop.utils.HddsVersionInfo;
+
+import picocli.CommandLine.IVersionProvider;
+
+/**
+ * Version provider for the CLI interface.
+ */
+public class HddsVersionProvider implements IVersionProvider {
+  @Override
+  public String[] getVersion() throws Exception {
+    String[] result = new String[] {
+        HddsVersionInfo.getBuildVersion()
+    };
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/package-info.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/package-info.java
new file mode 100644
index 0000000..8dcc1d1
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/package-info.java
@@ -0,0 +1,22 @@
+/**
+ * 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.
+ */
+
+/**
+ * Generic helper class to make instantiate picocli based cli tools.
+ */
+package org.apache.hadoop.hdds.cli;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/pom.xml b/hadoop-hdds/pom.xml
index 428fd11..d97e6d8 100644
--- a/hadoop-hdds/pom.xml
+++ b/hadoop-hdds/pom.xml
@@ -69,6 +69,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <type>test-jar</type>
     </dependency>
     <dependency>
+      <groupId>info.picocli</groupId>
+      <artifactId>picocli</artifactId>
+      <version>3.5.2</version>
+    </dependency>
+    <dependency>
       <groupId>com.google.protobuf</groupId>
       <artifactId>protobuf-java</artifactId>
       <scope>compile</scope>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/OzoneBaseCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/OzoneBaseCLI.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/OzoneBaseCLI.java
deleted file mode 100644
index 7828445..0000000
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/OzoneBaseCLI.java
+++ /dev/null
@@ -1,43 +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.scm.cli;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.util.Tool;
-
-import java.io.IOException;
-import java.net.URISyntaxException;
-
-/**
- * This class is the base CLI for scm, om and scmadm.
- */
-public abstract class OzoneBaseCLI extends Configured implements Tool {
-
-  protected abstract int dispatch(CommandLine cmd, Options opts)
-      throws IOException, URISyntaxException;
-
-  protected abstract CommandLine parseArgs(String[] argv, Options opts)
-      throws ParseException;
-
-  protected abstract Options getOptions();
-
-  protected abstract void displayHelp();
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/OzoneCommandHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/OzoneCommandHandler.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/OzoneCommandHandler.java
deleted file mode 100644
index 641dd0e..0000000
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/OzoneCommandHandler.java
+++ /dev/null
@@ -1,87 +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.scm.cli;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.hadoop.hdds.scm.client.ScmClient;
-
-import java.io.IOException;
-import java.io.PrintStream;
-
-/**
- * The abstract class of all SCM CLI commands.
- */
-public abstract class OzoneCommandHandler {
-
-  private ScmClient scmClient;
-  private PrintStream out = System.out;
-  private PrintStream err = System.err;
-
-  /**
-   * Constructs a handler object.
-   */
-  public OzoneCommandHandler(ScmClient scmClient) {
-    this.scmClient = scmClient;
-  }
-
-  protected ScmClient getScmClient() {
-    return scmClient;
-  }
-
-  /**
-   * Sets customized output stream to redirect the stdout to somewhere else.
-   * @param out
-   */
-  public void setOut(PrintStream out) {
-    this.out = out;
-  }
-
-  /**
-   * Sets customized error stream to redirect the stderr to somewhere else.
-   * @param err
-   */
-  public void setErr(PrintStream err) {
-    this.err = err;
-  }
-
-  public void logOut(String msg, String... variable) {
-    this.out.println(String.format(msg, variable));
-  }
-
-  /**
-   * Executes the Client command.
-   *
-   * @param cmd - CommandLine.
-   * @throws IOException throws exception.
-   */
-  public abstract void execute(CommandLine cmd) throws IOException;
-
-  /**
-   * Display a help message describing the options the command takes.
-   * TODO : currently only prints to standard out, may want to change this.
-   */
-  public abstract void displayHelp();
-
-  public PrintStream getOut() {
-    return out;
-  }
-
-  public PrintStream getErr() {
-    return err;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ResultCode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ResultCode.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ResultCode.java
deleted file mode 100644
index 27df88c..0000000
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ResultCode.java
+++ /dev/null
@@ -1,31 +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.scm.cli;
-
-/**
- * The possible result code of SCM CLI.
- */
-public final class ResultCode {
-  public static final int SUCCESS = 1;
-
-  public static final int UNRECOGNIZED_CMD = 2;
-
-  public static final int EXECUTION_ERROR = 3;
-
-  private ResultCode() {}
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java
index f54322c..59cd0ba 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java
@@ -6,9 +6,9 @@
  * 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
- *
+ * <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.
@@ -17,69 +17,73 @@
  */
 package org.apache.hadoop.hdds.scm.cli;
 
-import org.apache.commons.cli.BasicParser;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
 import org.apache.hadoop.conf.StorageUnit;
+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.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.XceiverClientManager;
-import org.apache.hadoop.hdds.scm.cli.container.ContainerCommandHandler;
-import org.apache.hadoop.hdds.scm.cli.container.CreateContainerHandler;
+import org.apache.hadoop.hdds.scm.cli.container.CloseSubcommand;
+import org.apache.hadoop.hdds.scm.cli.container.CreateSubcommand;
+import org.apache.hadoop.hdds.scm.cli.container.DeleteSubcommand;
+import org.apache.hadoop.hdds.scm.cli.container.InfoSubcommand;
+import org.apache.hadoop.hdds.scm.cli.container.ListSubcommand;
 import org.apache.hadoop.hdds.scm.client.ContainerOperationClient;
 import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.hdds.scm.protocolPB
     .StorageContainerLocationProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.ToolRunner;
-
-import java.io.IOException;
-import java.io.PrintStream;
-import java.net.InetSocketAddress;
-import java.net.URISyntaxException;
-import java.util.Arrays;
+import org.apache.hadoop.util.NativeCodeLoader;
 
+import org.apache.commons.lang3.StringUtils;
+import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
-    .OZONE_SCM_CONTAINER_SIZE_DEFAULT;
+    .OZONE_SCM_CLIENT_ADDRESS_KEY;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
-    .OZONE_SCM_CONTAINER_SIZE;
-import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
-import static org.apache.hadoop.hdds.scm.cli.ResultCode.EXECUTION_ERROR;
-import static org.apache.hadoop.hdds.scm.cli.ResultCode.SUCCESS;
-import static org.apache.hadoop.hdds.scm.cli.ResultCode.UNRECOGNIZED_CMD;
+    .OZONE_SCM_CONTAINER_SIZE_DEFAULT;
+import org.apache.log4j.ConsoleAppender;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.log4j.PatternLayout;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
 
 /**
  * This class is the CLI of SCM.
  */
-public class SCMCLI extends OzoneBaseCLI {
-
-  public static final String HELP_OP = "help";
-  public static final int CMD_WIDTH = 80;
 
-  private final ScmClient scmClient;
-  private final PrintStream out;
-  private final PrintStream err;
-
-  private final Options options;
-
-  public SCMCLI(ScmClient scmClient) {
-    this(scmClient, System.out, System.err);
-  }
-
-  public SCMCLI(ScmClient scmClient, PrintStream out, PrintStream err) {
-    this.scmClient = scmClient;
-    this.out = out;
-    this.err = err;
-    this.options = getOptions();
-  }
+/**
+ * Container subcommand.
+ */
+@Command(name = "ozone scmcli", hidden = true, description =
+    "Developer tools to handle SCM specific "
+        + "operations.",
+    versionProvider = HddsVersionProvider.class,
+    subcommands = {
+        ListSubcommand.class,
+        InfoSubcommand.class,
+        DeleteSubcommand.class,
+        CreateSubcommand.class,
+        CloseSubcommand.class
+    },
+    mixinStandardHelpOptions = true)
+public class SCMCLI extends GenericCli {
+
+  @Option(names = {"--scm"}, description = "The destination scm (host:port)")
+  private String scm = "";
 
   /**
    * Main for the scm shell Command handling.
@@ -88,30 +92,40 @@ public class SCMCLI extends OzoneBaseCLI {
    * @throws Exception
    */
   public static void main(String[] argv) throws Exception {
-    OzoneConfiguration conf = new OzoneConfiguration();
-    ScmClient scmClient = getScmClient(conf);
-    SCMCLI shell = new SCMCLI(scmClient);
-    conf.setQuietMode(false);
-    shell.setConf(conf);
-    int res = 0;
-    try {
-      res = ToolRunner.run(shell, argv);
-    } catch (Exception ex) {
-      System.exit(1);
-    }
-    System.exit(res);
+
+    LogManager.resetConfiguration();
+    Logger.getRootLogger().setLevel(Level.INFO);
+    Logger.getRootLogger()
+        .addAppender(new ConsoleAppender(new PatternLayout("%m%n")));
+    Logger.getLogger(NativeCodeLoader.class).setLevel(Level.ERROR);
+
+    new SCMCLI().run(argv);
   }
 
-  private static ScmClient getScmClient(OzoneConfiguration ozoneConf)
+  public ScmClient createScmClient()
       throws IOException {
+
+    OzoneConfiguration ozoneConf = createOzoneConfiguration();
+    if (StringUtils.isNotEmpty(scm)) {
+      ozoneConf.set(OZONE_SCM_CLIENT_ADDRESS_KEY, scm);
+    }
+    if (!HddsUtils.getHostNameFromConfigKeys(ozoneConf,
+        ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY).isPresent()) {
+
+      throw new IllegalArgumentException(
+          ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY
+              + " should be set in ozone-site.xml or with the --scm option");
+    }
+
     long version = RPC.getProtocolVersion(
         StorageContainerLocationProtocolPB.class);
     InetSocketAddress scmAddress =
         getScmAddressForClients(ozoneConf);
-    int containerSizeGB = (int)ozoneConf.getStorageSize(
+    int containerSizeGB = (int) ozoneConf.getStorageSize(
         OZONE_SCM_CONTAINER_SIZE, OZONE_SCM_CONTAINER_SIZE_DEFAULT,
         StorageUnit.GB);
-    ContainerOperationClient.setContainerSizeB(containerSizeGB*OzoneConsts.GB);
+    ContainerOperationClient
+        .setContainerSizeB(containerSizeGB * OzoneConsts.GB);
 
     RPC.setProtocolEngine(ozoneConf, StorageContainerLocationProtocolPB.class,
         ProtobufRpcEngine.class);
@@ -121,116 +135,16 @@ public class SCMCLI extends OzoneBaseCLI {
                 scmAddress, UserGroupInformation.getCurrentUser(), ozoneConf,
                 NetUtils.getDefaultSocketFactory(ozoneConf),
                 Client.getRpcTimeout(ozoneConf)));
-    ScmClient storageClient = new ContainerOperationClient(
+    return new ContainerOperationClient(
         client, new XceiverClientManager(ozoneConf));
-    return storageClient;
-  }
-
-  /**
-   * Adds ALL the options that hdfs scm command supports. Given the hierarchy
-   * of commands, the options are added in a cascading manner, e.g.:
-   * {@link SCMCLI} asks {@link ContainerCommandHandler} to add it's options,
-   * which then asks it's sub command, such as
-   * {@link CreateContainerHandler}
-   * to add it's own options.
-   *
-   * We need to do this because {@link BasicParser} need to take all the options
-   * when paring args.
-   * @return ALL the options supported by this CLI.
-   */
-  @Override
-  protected Options getOptions() {
-    Options newOptions = new Options();
-    // add the options
-    addTopLevelOptions(newOptions);
-    ContainerCommandHandler.addOptions(newOptions);
-    // TODO : add pool, node and pipeline commands.
-    addHelpOption(newOptions);
-    return newOptions;
-  }
-
-  private static void addTopLevelOptions(Options options) {
-    Option containerOps =
-        new Option(ContainerCommandHandler.CONTAINER_CMD, false,
-            "Container related options");
-    options.addOption(containerOps);
-    // TODO : add pool, node and pipeline commands.
-  }
-
-  private static void addHelpOption(Options options) {
-    Option helpOp = new Option(HELP_OP, false, "display help message");
-    options.addOption(helpOp);
   }
 
-  @Override
-  protected void displayHelp() {
-    HelpFormatter helpFormatter = new HelpFormatter();
-    Options topLevelOptions = new Options();
-    addTopLevelOptions(topLevelOptions);
-    helpFormatter.printHelp(CMD_WIDTH, "hdfs scmcli <commands> [<options>]",
-        "where <commands> can be one of the following",
-        topLevelOptions, "");
-  }
-
-  @Override
-  public int run(String[] args) throws Exception {
-    CommandLine cmd = parseArgs(args, options);
-    if (cmd == null) {
-      err.println("Unrecognized options:" + Arrays.asList(args));
-      displayHelp();
-      return UNRECOGNIZED_CMD;
-    }
-    return dispatch(cmd, options);
-  }
-
-  /**
-   * This function parses all command line arguments
-   * and returns the appropriate values.
-   *
-   * @param argv - Argv from main
-   *
-   * @return CommandLine
-   */
-  @Override
-  protected CommandLine parseArgs(String[] argv, Options opts)
-      throws ParseException {
-    try {
-      BasicParser parser = new BasicParser();
-      return parser.parse(opts, argv);
-    } catch (ParseException ex) {
-      err.println(ex.getMessage());
+  public void checkContainerExists(ScmClient scmClient, long containerId)
+      throws IOException {
+    ContainerInfo container = scmClient.getContainer(containerId);
+    if (container == null) {
+      throw new IllegalArgumentException("No such container " + containerId);
     }
-    return null;
   }
 
-  @Override
-  protected int dispatch(CommandLine cmd, Options opts)
-      throws IOException, URISyntaxException {
-    OzoneCommandHandler handler = null;
-    try {
-      if (cmd.hasOption(ContainerCommandHandler.CONTAINER_CMD)) {
-        handler = new ContainerCommandHandler(scmClient);
-      }
-
-      if (handler == null) {
-        if (cmd.hasOption(HELP_OP)) {
-          displayHelp();
-          return SUCCESS;
-        } else {
-          displayHelp();
-          err.println("Unrecognized command: " + Arrays.asList(cmd.getArgs()));
-          return UNRECOGNIZED_CMD;
-        }
-      } else {
-        // Redirect stdout and stderr if necessary.
-        handler.setOut(this.out);
-        handler.setErr(this.err);
-        handler.execute(cmd);
-        return SUCCESS;
-      }
-    } catch (IOException ioe) {
-      err.println("Error executing command:" + ioe);
-      return EXECUTION_ERROR;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseContainerHandler.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseContainerHandler.java
deleted file mode 100644
index e2267da..0000000
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseContainerHandler.java
+++ /dev/null
@@ -1,85 +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.scm.cli.container;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler;
-import org.apache.hadoop.hdds.scm.cli.SCMCLI;
-import org.apache.hadoop.hdds.scm.client.ScmClient;
-
-import java.io.IOException;
-import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
-
-/**
- * The handler of close container command.
- */
-public class CloseContainerHandler extends OzoneCommandHandler {
-
-  public static final String CONTAINER_CLOSE = "close";
-  public static final String OPT_CONTAINER_ID = "c";
-
-  @Override
-  public void execute(CommandLine cmd) throws IOException {
-    if (!cmd.hasOption(CONTAINER_CLOSE)) {
-      throw new IOException("Expecting container close");
-    }
-    if (!cmd.hasOption(OPT_CONTAINER_ID)) {
-      displayHelp();
-      if (!cmd.hasOption(SCMCLI.HELP_OP)) {
-        throw new IOException("Expecting container id");
-      } else {
-        return;
-      }
-    }
-    String containerID = cmd.getOptionValue(OPT_CONTAINER_ID);
-
-    ContainerWithPipeline container = getScmClient().
-        getContainerWithPipeline(Long.parseLong(containerID));
-    if (container == null) {
-      throw new IOException("Cannot close an non-exist container "
-          + containerID);
-    }
-    logOut("Closing container : %s.", containerID);
-    getScmClient()
-        .closeContainer(container.getContainerInfo().getContainerID());
-    logOut("Container closed.");
-  }
-
-  @Override
-  public void displayHelp() {
-    Options options = new Options();
-    addOptions(options);
-    HelpFormatter helpFormatter = new HelpFormatter();
-    helpFormatter
-        .printHelp(SCMCLI.CMD_WIDTH, "hdfs scm -container -close <option>",
-            "where <option> is", options, "");
-  }
-
-  public static void addOptions(Options options) {
-    Option containerNameOpt = new Option(OPT_CONTAINER_ID,
-        true, "Specify container ID");
-    options.addOption(containerNameOpt);
-  }
-
-  CloseContainerHandler(ScmClient client) {
-    super(client);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseSubcommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseSubcommand.java
new file mode 100644
index 0000000..173d0ce
--- /dev/null
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CloseSubcommand.java
@@ -0,0 +1,54 @@
+/**
+ * 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.scm.cli.container;
+
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.SCMCLI;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Parameters;
+import picocli.CommandLine.ParentCommand;
+
+/**
+ * The handler of close container command.
+ */
+@Command(
+    name = "close",
+    description = "close container",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class CloseSubcommand implements Callable<Void> {
+
+  @ParentCommand
+  private SCMCLI parent;
+
+  @Parameters(description = "Id of the container to close")
+  private long containerId;
+
+  @Override
+  public Void call() throws Exception {
+    try (ScmClient scmClient = parent.createScmClient()) {
+      parent.checkContainerExists(scmClient, containerId);
+      scmClient.closeContainer(containerId);
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommandHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommandHandler.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommandHandler.java
deleted file mode 100644
index 428f179..0000000
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ContainerCommandHandler.java
+++ /dev/null
@@ -1,128 +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.scm.cli.container;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler;
-import org.apache.hadoop.hdds.scm.client.ScmClient;
-
-import java.io.IOException;
-import java.util.Arrays;
-
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.CMD_WIDTH;
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP;
-import static org.apache.hadoop.hdds.scm.cli.container.CloseContainerHandler
-    .CONTAINER_CLOSE;
-import static org.apache.hadoop.hdds.scm.cli.container.CreateContainerHandler
-    .CONTAINER_CREATE;
-import static org.apache.hadoop.hdds.scm.cli.container.DeleteContainerHandler
-    .CONTAINER_DELETE;
-import static org.apache.hadoop.hdds.scm.cli.container.InfoContainerHandler
-    .CONTAINER_INFO;
-import static org.apache.hadoop.hdds.scm.cli.container.ListContainerHandler
-    .CONTAINER_LIST;
-
-/**
- * The handler class of container-specific commands, e.g. addContainer.
- */
-public class ContainerCommandHandler extends OzoneCommandHandler {
-
-  public static final String CONTAINER_CMD = "container";
-
-  public ContainerCommandHandler(ScmClient scmClient) {
-    super(scmClient);
-  }
-
-  @Override
-  public void execute(CommandLine cmd) throws IOException {
-    // all container commands should contain -container option
-    if (!cmd.hasOption(CONTAINER_CMD)) {
-      throw new IOException("Expecting container cmd");
-    }
-    // check which each the sub command it is
-    OzoneCommandHandler handler = null;
-    if (cmd.hasOption(CONTAINER_CREATE)) {
-      handler = new CreateContainerHandler(getScmClient());
-    } else if (cmd.hasOption(CONTAINER_DELETE)) {
-      handler = new DeleteContainerHandler(getScmClient());
-    } else if (cmd.hasOption(CONTAINER_INFO)) {
-      handler = new InfoContainerHandler(getScmClient());
-    } else if (cmd.hasOption(CONTAINER_LIST)) {
-      handler = new ListContainerHandler(getScmClient());
-    } else if (cmd.hasOption(CONTAINER_CLOSE)) {
-      handler = new CloseContainerHandler(getScmClient());
-    }
-
-    // execute the sub command, throw exception if no sub command found
-    // unless -help option is given.
-    if (handler != null) {
-      handler.setOut(this.getOut());
-      handler.setErr(this.getErr());
-      handler.execute(cmd);
-    } else {
-      displayHelp();
-      if (!cmd.hasOption(HELP_OP)) {
-        throw new IOException("Unrecognized command "
-            + Arrays.asList(cmd.getArgs()));
-      }
-    }
-  }
-
-  @Override
-  public void displayHelp() {
-    Options options = new Options();
-    addCommandsOption(options);
-    HelpFormatter helpFormatter = new HelpFormatter();
-    helpFormatter.printHelp(CMD_WIDTH,
-        "hdfs scm -container <commands> <options>",
-        "where <commands> can be one of the following", options, "");
-  }
-
-  private static void addCommandsOption(Options options) {
-    Option createContainer =
-        new Option(CONTAINER_CREATE, false, "Create container");
-    Option infoContainer =
-        new Option(CONTAINER_INFO, false, "Info container");
-    Option deleteContainer =
-        new Option(CONTAINER_DELETE, false, "Delete container");
-    Option listContainer =
-        new Option(CONTAINER_LIST, false, "List container");
-    Option closeContainer =
-        new Option(CONTAINER_CLOSE, false, "Close container");
-
-    options.addOption(createContainer);
-    options.addOption(deleteContainer);
-    options.addOption(infoContainer);
-    options.addOption(listContainer);
-    options.addOption(closeContainer);
-    // Every new option should add it's option here.
-  }
-
-  public static void addOptions(Options options) {
-    addCommandsOption(options);
-    // for create container options.
-    DeleteContainerHandler.addOptions(options);
-    InfoContainerHandler.addOptions(options);
-    ListContainerHandler.addOptions(options);
-    CloseContainerHandler.addOptions(options);
-    // Every new option should add it's option here.
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateContainerHandler.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateContainerHandler.java
deleted file mode 100644
index 278ee30..0000000
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateContainerHandler.java
+++ /dev/null
@@ -1,67 +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.scm.cli.container;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler;
-import org.apache.hadoop.hdds.scm.client.ScmClient;
-
-import java.io.IOException;
-
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.CMD_WIDTH;
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP;
-
-/**
- * This is the handler that process container creation command.
- */
-public class CreateContainerHandler extends OzoneCommandHandler {
-
-  public static final String CONTAINER_CREATE = "create";
-  public static final String CONTAINER_OWNER = "OZONE";
-  // TODO Support an optional -p <pipelineID> option to create
-  // container on given datanodes.
-
-  public CreateContainerHandler(ScmClient scmClient) {
-    super(scmClient);
-  }
-
-  @Override
-  public void execute(CommandLine cmd) throws IOException {
-    if (cmd.hasOption(HELP_OP)) {
-      displayHelp();
-    }
-
-    if (!cmd.hasOption(CONTAINER_CREATE)) {
-      throw new IOException("Expecting container create");
-    }
-
-    logOut("Creating container...");
-    getScmClient().createContainer(CONTAINER_OWNER);
-    logOut("Container created.");
-  }
-
-  @Override
-  public void displayHelp() {
-    Options options = new Options();
-    HelpFormatter helpFormatter = new HelpFormatter();
-    helpFormatter.printHelp(CMD_WIDTH, "hdfs scm -container -create",
-        null, options, null);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateSubcommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateSubcommand.java
new file mode 100644
index 0000000..1dda9c4
--- /dev/null
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/CreateSubcommand.java
@@ -0,0 +1,65 @@
+/**
+ * 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.scm.cli.container;
+
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.SCMCLI;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .ContainerWithPipeline;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.ParentCommand;
+
+/**
+ * This is the handler that process container creation command.
+ */
+@Command(
+    name = "create",
+    description = "Create container",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class CreateSubcommand implements Callable<Void> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CreateSubcommand.class);
+
+  @ParentCommand
+  private SCMCLI parent;
+
+  @Option(description = "Owner of the new container", defaultValue = "OZONE",
+      required = false, names = {
+      "-o", "--owner"})
+
+  private String owner;
+
+  @Override
+  public Void call() throws Exception {
+    try (ScmClient scmClient = parent.createScmClient()) {
+      ContainerWithPipeline container = scmClient.createContainer(owner);
+      LOG.info("Container {} is created.",
+          container.getContainerInfo().getContainerID());
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteContainerHandler.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteContainerHandler.java
deleted file mode 100644
index 1b26665..0000000
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteContainerHandler.java
+++ /dev/null
@@ -1,95 +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.scm.cli.container;
-
-import com.google.common.base.Preconditions;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler;
-import org.apache.hadoop.hdds.scm.client.ScmClient;
-
-import java.io.IOException;
-import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
-
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.CMD_WIDTH;
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP;
-
-/**
- * This is the handler that process delete container command.
- */
-public class DeleteContainerHandler extends OzoneCommandHandler {
-
-  protected static final String CONTAINER_DELETE = "delete";
-  protected static final String OPT_FORCE = "f";
-  protected static final String OPT_CONTAINER_ID = "c";
-
-  public DeleteContainerHandler(ScmClient scmClient) {
-    super(scmClient);
-  }
-
-  @Override
-  public void execute(CommandLine cmd) throws IOException {
-    Preconditions.checkArgument(cmd.hasOption(CONTAINER_DELETE),
-        "Expecting command delete");
-    if (!cmd.hasOption(OPT_CONTAINER_ID)) {
-      displayHelp();
-      if (!cmd.hasOption(HELP_OP)) {
-        throw new IOException("Expecting container name");
-      } else {
-        return;
-      }
-    }
-
-    String containerID = cmd.getOptionValue(OPT_CONTAINER_ID);
-
-    ContainerWithPipeline container = getScmClient().getContainerWithPipeline(
-        Long.parseLong(containerID));
-    if (container == null) {
-      throw new IOException("Cannot delete an non-exist container "
-          + containerID);
-    }
-
-    logOut("Deleting container : %s.", containerID);
-    getScmClient()
-        .deleteContainer(container.getContainerInfo().getContainerID(),
-            container.getPipeline(), cmd.hasOption(OPT_FORCE));
-    logOut("Container %s deleted.", containerID);
-  }
-
-  @Override
-  public void displayHelp() {
-    Options options = new Options();
-    addOptions(options);
-    HelpFormatter helpFormatter = new HelpFormatter();
-    helpFormatter.printHelp(CMD_WIDTH, "hdfs scm -container -delete <option>",
-        "where <option> is", options, "");
-  }
-
-  public static void addOptions(Options options) {
-    Option forceOpt = new Option(OPT_FORCE,
-        false,
-        "forcibly delete a container");
-    options.addOption(forceOpt);
-    Option containerNameOpt = new Option(OPT_CONTAINER_ID,
-        true, "Specify container id");
-    options.addOption(containerNameOpt);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteSubcommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteSubcommand.java
new file mode 100644
index 0000000..c163a3a
--- /dev/null
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/DeleteSubcommand.java
@@ -0,0 +1,60 @@
+/**
+ * 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.scm.cli.container;
+
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.SCMCLI;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.Parameters;
+import picocli.CommandLine.ParentCommand;
+
+/**
+ * This is the handler that process delete container command.
+ */
+@Command(
+    name = "delete",
+    description = "Delete container",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class DeleteSubcommand implements Callable<Void> {
+
+  @Parameters(description = "Id of the container to close")
+  private long containerId;
+
+  @Option(names = {"-f",
+      "--force"}, description = "forcibly delete the container")
+  private boolean force;
+
+  @ParentCommand
+  private SCMCLI parent;
+
+  @Override
+  public Void call() throws Exception {
+    try (ScmClient scmClient = parent.createScmClient()) {
+      parent.checkContainerExists(scmClient, containerId);
+      scmClient.deleteContainer(containerId, force);
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java
deleted file mode 100644
index 89215fa..0000000
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoContainerHandler.java
+++ /dev/null
@@ -1,114 +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.scm.cli.container;
-
-import com.google.common.base.Preconditions;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler;
-import org.apache.hadoop.hdds.scm.client.ScmClient;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .ContainerData;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
-    .ContainerLifeCycleState;
-
-import java.io.IOException;
-import java.util.stream.Collectors;
-import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
-
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.CMD_WIDTH;
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP;
-
-/**
- * This is the handler that process container info command.
- */
-public class InfoContainerHandler extends OzoneCommandHandler {
-
-  public static final String CONTAINER_INFO = "info";
-  protected static final String OPT_CONTAINER_ID = "c";
-
-  /**
-   * Constructs a handler object.
-   *
-   * @param scmClient scm client.
-   */
-  public InfoContainerHandler(ScmClient scmClient) {
-    super(scmClient);
-  }
-
-  @Override
-  public void execute(CommandLine cmd) throws IOException {
-    if (!cmd.hasOption(CONTAINER_INFO)) {
-      throw new IOException("Expecting container info");
-    }
-    if (!cmd.hasOption(OPT_CONTAINER_ID)) {
-      displayHelp();
-      if (!cmd.hasOption(HELP_OP)) {
-        throw new IOException("Expecting container name");
-      } else {
-        return;
-      }
-    }
-    String containerID = cmd.getOptionValue(OPT_CONTAINER_ID);
-    ContainerWithPipeline container = getScmClient().
-        getContainerWithPipeline(Long.parseLong(containerID));
-    Preconditions.checkNotNull(container, "Container cannot be null");
-
-    ContainerData containerData = getScmClient().readContainer(container
-        .getContainerInfo().getContainerID(), container.getPipeline());
-
-    // Print container report info.
-    logOut("Container id: %s", containerID);
-    String openStatus =
-        containerData.getState() == ContainerLifeCycleState.OPEN ? "OPEN" :
-            "CLOSED";
-    logOut("Container State: %s", openStatus);
-    logOut("Container Path: %s", containerData.getContainerPath());
-
-    // Output meta data.
-    String metadataStr = containerData.getMetadataList().stream().map(
-        p -> p.getKey() + ":" + p.getValue()).collect(Collectors.joining(", "));
-    logOut("Container Metadata: {%s}", metadataStr);
-
-    // Print pipeline of an existing container.
-    logOut("LeaderID: %s", container.getPipeline()
-        .getLeader().getHostName());
-    String machinesStr = container.getPipeline()
-        .getMachines().stream().map(
-        DatanodeDetails::getHostName).collect(Collectors.joining(","));
-    logOut("Datanodes: [%s]", machinesStr);
-  }
-
-  @Override
-  public void displayHelp() {
-    Options options = new Options();
-    addOptions(options);
-    HelpFormatter helpFormatter = new HelpFormatter();
-    helpFormatter.printHelp(CMD_WIDTH, "hdfs scm -container -info <option>",
-        "where <option> is", options, "");
-  }
-
-  public static void addOptions(Options options) {
-    Option containerIdOpt = new Option(OPT_CONTAINER_ID,
-        true, "Specify container id");
-    options.addOption(containerIdOpt);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java
new file mode 100644
index 0000000..0135df3
--- /dev/null
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/InfoSubcommand.java
@@ -0,0 +1,94 @@
+/**
+ * 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.scm.cli.container;
+
+import java.util.concurrent.Callable;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
+    .ContainerLifeCycleState;
+import org.apache.hadoop.hdds.scm.cli.SCMCLI;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.hdds.scm.container.common.helpers
+    .ContainerWithPipeline;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Parameters;
+import picocli.CommandLine.ParentCommand;
+
+/**
+ * This is the handler that process container info command.
+ */
+@Command(
+    name = "info",
+    description = "Show information about a specific container",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class InfoSubcommand implements Callable<Void> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(InfoSubcommand.class);
+
+  @ParentCommand
+  private SCMCLI parent;
+
+  @Parameters(description = "Decimal id of the container.")
+  private long containerID;
+
+  @Override
+  public Void call() throws Exception {
+    try (ScmClient scmClient = parent.createScmClient()) {
+      ContainerWithPipeline container = scmClient.
+          getContainerWithPipeline(containerID);
+      Preconditions.checkNotNull(container, "Container cannot be null");
+
+      ContainerData containerData = scmClient.readContainer(container
+          .getContainerInfo().getContainerID(), container.getPipeline());
+
+      // Print container report info.
+      LOG.info("Container id: {}", containerID);
+      String openStatus =
+          containerData.getState() == ContainerLifeCycleState.OPEN ? "OPEN" :
+              "CLOSED";
+      LOG.info("Container State: {}", openStatus);
+      LOG.info("Container Path: {}", containerData.getContainerPath());
+
+      // Output meta data.
+      String metadataStr = containerData.getMetadataList().stream().map(
+          p -> p.getKey() + ":" + p.getValue())
+          .collect(Collectors.joining(", "));
+      LOG.info("Container Metadata: {}", metadataStr);
+
+      // Print pipeline of an existing container.
+      LOG.info("LeaderID: {}", container.getPipeline()
+          .getLeader().getHostName());
+      String machinesStr = container.getPipeline()
+          .getMachines().stream().map(
+              DatanodeDetails::getHostName).collect(Collectors.joining(","));
+      LOG.info("Datanodes: [{}]", machinesStr);
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListContainerHandler.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListContainerHandler.java
deleted file mode 100644
index 3483b3e..0000000
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListContainerHandler.java
+++ /dev/null
@@ -1,117 +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.scm.cli.container;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.hdds.scm.cli.OzoneCommandHandler;
-import org.apache.hadoop.hdds.scm.client.ScmClient;
-import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
-import org.apache.hadoop.ozone.web.utils.JsonUtils;
-
-import java.io.IOException;
-import java.util.List;
-
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.CMD_WIDTH;
-import static org.apache.hadoop.hdds.scm.cli.SCMCLI.HELP_OP;
-
-/**
- * This is the handler that process container list command.
- */
-public class ListContainerHandler extends OzoneCommandHandler {
-
-  public static final String CONTAINER_LIST = "list";
-  public static final String OPT_START_CONTAINER = "start";
-  public static final String OPT_COUNT = "count";
-
-  /**
-   * Constructs a handler object.
-   *
-   * @param scmClient scm client
-   */
-  public ListContainerHandler(ScmClient scmClient) {
-    super(scmClient);
-  }
-
-  @Override
-  public void execute(CommandLine cmd) throws IOException {
-    if (!cmd.hasOption(CONTAINER_LIST)) {
-      throw new IOException("Expecting container list");
-    }
-    if (cmd.hasOption(HELP_OP)) {
-      displayHelp();
-      return;
-    }
-
-    if (!cmd.hasOption(OPT_COUNT)) {
-      displayHelp();
-      if (!cmd.hasOption(HELP_OP)) {
-        throw new IOException("Expecting container count");
-      } else {
-        return;
-      }
-    }
-
-    String startID = cmd.getOptionValue(OPT_START_CONTAINER);
-    int count = 0;
-
-    if (cmd.hasOption(OPT_COUNT)) {
-      count = Integer.parseInt(cmd.getOptionValue(OPT_COUNT));
-      if (count < 0) {
-        displayHelp();
-        throw new IOException("-count should not be negative");
-      }
-    }
-
-    List<ContainerInfo> containerList =
-        getScmClient().listContainer(
-            Long.parseLong(startID), count);
-
-    // Output data list
-    for (ContainerInfo container : containerList) {
-      outputContainerInfo(container);
-    }
-  }
-
-  private void outputContainerInfo(ContainerInfo containerInfo)
-      throws IOException {
-    // Print container report info.
-    logOut("%s", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
-        containerInfo.toJsonString()));
-  }
-
-  @Override
-  public void displayHelp() {
-    Options options = new Options();
-    addOptions(options);
-    HelpFormatter helpFormatter = new HelpFormatter();
-    helpFormatter.printHelp(CMD_WIDTH, "hdfs scm -container -list <option>",
-        "where <option> can be the following", options, "");
-  }
-
-  public static void addOptions(Options options) {
-    Option startContainerOpt = new Option(OPT_START_CONTAINER,
-        true, "Specify start container id");
-    Option countOpt = new Option(OPT_COUNT, true,
-        "Specify count number, required");
-    options.addOption(countOpt);
-    options.addOption(startContainerOpt);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java
new file mode 100644
index 0000000..0f520fd
--- /dev/null
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/ListSubcommand.java
@@ -0,0 +1,83 @@
+/**
+ * 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.scm.cli.container;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.SCMCLI;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
+import org.apache.hadoop.ozone.web.utils.JsonUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Help.Visibility;
+import picocli.CommandLine.Option;
+import picocli.CommandLine.ParentCommand;
+
+/**
+ * This is the handler that process container list command.
+ */
+@Command(
+    name = "list",
+    description = "List containers",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class ListSubcommand implements Callable<Void> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ListSubcommand.class);
+
+  @ParentCommand
+  private SCMCLI parent;
+
+  @Option(names = {"-s", "--start"},
+      description = "Container id to start the iteration", required = true)
+  private long startId;
+
+  @Option(names = {"-c", "--count"},
+      description = "Maximum number of containers to list",
+      defaultValue = "20", showDefaultValue = Visibility.ALWAYS)
+  private int count = 20;
+
+  private void outputContainerInfo(ContainerInfo containerInfo)
+      throws IOException {
+    // Print container report info.
+    LOG.info("{}", JsonUtils.toJsonStringWithDefaultPrettyPrinter(
+        containerInfo.toJsonString()));
+  }
+
+  @Override
+  public Void call() throws Exception {
+    try (ScmClient scmClient = parent.createScmClient()) {
+
+      List<ContainerInfo> containerList =
+          scmClient.listContainer(startId, count);
+
+      // Output data list
+      for (ContainerInfo container : containerList) {
+        outputContainerInfo(container);
+      }
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/package-info.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/package-info.java
index 0630df2..ff8adbc 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/package-info.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/container/package-info.java
@@ -16,4 +16,7 @@
  * limitations under the License.
  */
 
+/**
+ * Contains all of the container related scm commands.
+ */
 package org.apache.hadoop.hdds.scm.cli.container;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/package-info.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/package-info.java
index 4762d55..d358b3c 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/package-info.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/package-info.java
@@ -6,14 +6,18 @@
  * 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
- *
+ * <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.
+ * <p>
+ * SCM related cli tools.
+ */
+/**
+ * SCM related cli tools.
  */
-
 package org.apache.hadoop.hdds.scm.cli;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bae4cc/hadoop-ozone/common/src/main/bin/ozone
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/bin/ozone b/hadoop-ozone/common/src/main/bin/ozone
index 6a30433..17b47a9 100755
--- a/hadoop-ozone/common/src/main/bin/ozone
+++ b/hadoop-ozone/common/src/main/bin/ozone
@@ -116,7 +116,7 @@ function ozonecmd_case
       HADOOP_CLASSNAME=org.apache.hadoop.fs.FsShell
     ;;
     scmcli)
-      HADOOP_CLASSNAME=org.apache.hadoop.ozone.scm.cli.SCMCLI
+      HADOOP_CLASSNAME=org.apache.hadoop.hdds.scm.cli.SCMCLI
     ;;
     version)
       HADOOP_CLASSNAME=org.apache.hadoop.ozone.util.OzoneVersionInfo


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org