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 cn...@apache.org on 2016/03/15 00:18:33 UTC

hadoop git commit: HDFS-9944. Ozone : Add container dispatcher. Contributed by Anu Engineer.

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7240 657bc690c -> 4683d68ee


HDFS-9944. Ozone : Add container dispatcher. 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/4683d68e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4683d68e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4683d68e

Branch: refs/heads/HDFS-7240
Commit: 4683d68ee69d192fb08d51450222eb8a2e3eab44
Parents: 657bc69
Author: Chris Nauroth <cn...@apache.org>
Authored: Mon Mar 14 16:16:35 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Mon Mar 14 16:16:35 2016 -0700

----------------------------------------------------------------------
 .../ozone/container/helpers/ContainerData.java  | 170 +++++++++++++++++
 .../ozone/container/helpers/ContainerUtils.java | 110 +++++++++++
 .../ozone/container/helpers/package-info.java   |  21 +++
 .../container/interfaces/ContainerManager.java  |  75 ++++++++
 .../ozone/container/ozoneimpl/Dispatcher.java   | 185 +++++++++++++++++++
 .../ozone/container/ozoneimpl/package-info.java |  22 +++
 .../main/proto/DatanodeContainerProtocol.proto  |  11 ++
 .../ozone/container/ContainerTestHelper.java    |   1 +
 .../transport/server/TestContainerServer.java   |  46 ++++-
 9 files changed, 636 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4683d68e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/helpers/ContainerData.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/helpers/ContainerData.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/helpers/ContainerData.java
new file mode 100644
index 0000000..dd2d173
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/helpers/ContainerData.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.helpers;
+
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * This class maintains the information about a container in the ozone world.
+ * <p>
+ * A container is a name, along with metadata- which is a set of key value
+ * pair.
+ */
+public class ContainerData {
+
+  private final String containerName;
+  private final Map<String, String> metadata;
+
+  private String path;
+
+  /**
+   * Constructs a  ContainerData Object.
+   *
+   * @param containerName - Name
+   */
+  public ContainerData(String containerName) {
+    this.metadata = new TreeMap<>();
+    this.containerName = containerName;
+  }
+
+  /**
+   * Constructs a ContainerData object from ProtoBuf classes.
+   *
+   * @param protoData - ProtoBuf Message
+   * @throws IOException
+   */
+  public static ContainerData getFromProtBuf(
+      ContainerProtos.ContainerData protoData) throws IOException {
+    ContainerData data = new ContainerData(protoData.getName());
+    for (int x = 0; x < protoData.getMetadataCount(); x++) {
+      data.addMetadata(protoData.getMetadata(x).getKey(),
+          protoData.getMetadata(x).getValue());
+    }
+
+    if (protoData.hasContainerPath()) {
+      data.setPath(protoData.getContainerPath());
+    }
+    return data;
+  }
+
+  /**
+   * Returns a ProtoBuf Message from ContainerData.
+   *
+   * @return Protocol Buffer Message
+   */
+  public ContainerProtos.ContainerData getProtoBufMessage() {
+    ContainerProtos.ContainerData.Builder builder = ContainerProtos
+        .ContainerData.newBuilder();
+    builder.setName(this.getContainerName());
+    if (this.getPath() != null) {
+      builder.setContainerPath(this.getPath());
+    }
+    for (Map.Entry<String, String> entry : metadata.entrySet()) {
+      ContainerProtos.KeyValue.Builder keyValBuilder =
+          ContainerProtos.KeyValue.newBuilder();
+      builder.addMetadata(keyValBuilder.setKey(entry.getKey())
+          .setValue(entry.getValue()).build());
+    }
+    return builder.build();
+  }
+
+  /**
+   * Returns the name of the container.
+   *
+   * @return - name
+   */
+  public String getContainerName() {
+    return containerName;
+  }
+
+  /**
+   * Adds metadata.
+   */
+  public void addMetadata(String key, String value) throws IOException {
+    synchronized (this.metadata) {
+      if (this.metadata.containsKey(key)) {
+        throw new IOException("This key already exists. Key " + key);
+      }
+      metadata.put(key, value);
+    }
+  }
+
+  /**
+   * Returns all metadata.
+   */
+  public Map<String, String> getAllMetadata() {
+    synchronized (this.metadata) {
+      return Collections.unmodifiableMap(this.metadata);
+    }
+  }
+
+  /**
+   * Returns value of a key.
+   */
+  public String getValue(String key) {
+    synchronized (this.metadata) {
+      return metadata.get(key);
+    }
+  }
+
+  /**
+   * Deletes a metadata entry from the map.
+   *
+   * @param key - Key
+   */
+  public void deleteKey(String key) {
+    synchronized (this.metadata) {
+      metadata.remove(key);
+    }
+  }
+
+  /**
+   * Returns path.
+   *
+   * @return - path
+   */
+  public String getPath() {
+    return path;
+  }
+
+  /**
+   * Sets path.
+   *
+   * @param path - String.
+   */
+  public void setPath(String path) {
+    this.path = path;
+  }
+
+  /**
+   * This function serves as the generic key for OzoneCache class. Both
+   * ContainerData and ContainerKeyData overrides this function to appropriately
+   * return the right name that can  be used in OzoneCache.
+   *
+   * @return String Name.
+   */
+  public String getName() {
+    return getContainerName();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4683d68e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/helpers/ContainerUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/helpers/ContainerUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/helpers/ContainerUtils.java
new file mode 100644
index 0000000..6aef443
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/helpers/ContainerUtils.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.helpers;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
+
+/**
+ * A set of helper functions to create proper responses.
+ */
+public final class ContainerUtils {
+
+  /**
+   * Returns a CreateContainer Response. This call is used by create and delete
+   * containers which have null success responses.
+   *
+   * @param msg Request
+   * @return Response.
+   */
+  public static ContainerProtos.ContainerCommandResponseProto
+      getContainerResponse(ContainerProtos.ContainerCommandRequestProto msg) {
+    ContainerProtos.ContainerCommandResponseProto.Builder builder =
+        getContainerResponse(msg, ContainerProtos.Result.SUCCESS, "");
+    return builder.build();
+  }
+
+  /**
+   * Returns a ReadContainer Response.
+   *
+   * @param msg Request
+   * @return Response.
+   */
+  public static ContainerProtos.ContainerCommandResponseProto
+      getReadContainerResponse(ContainerProtos.ContainerCommandRequestProto msg,
+                           ContainerData containerData) {
+    Preconditions.checkNotNull(containerData);
+
+    ContainerProtos.ReadContainerResponseProto.Builder response =
+        ContainerProtos.ReadContainerResponseProto.newBuilder();
+    response.setContainerData(containerData.getProtoBufMessage());
+
+    ContainerProtos.ContainerCommandResponseProto.Builder builder =
+        getContainerResponse(msg, ContainerProtos.Result.SUCCESS, "");
+    builder.setReadContainer(response);
+    return builder.build();
+  }
+
+  /**
+   * We found a command type but no associated payload for the command. Hence
+   * return malformed Command as response.
+   *
+   * @param msg - Protobuf message.
+   * @return ContainerCommandResponseProto - MALFORMED_REQUEST.
+   */
+  public static ContainerProtos.ContainerCommandResponseProto.Builder
+      getContainerResponse(ContainerProtos.ContainerCommandRequestProto msg,
+                       ContainerProtos.Result result, String message) {
+    return
+        ContainerProtos.ContainerCommandResponseProto.newBuilder()
+            .setCmdType(msg.getCmdType())
+            .setTraceID(msg.getTraceID())
+            .setResult(result)
+            .setMessage(message);
+  }
+
+  /**
+   * We found a command type but no associated payload for the command. Hence
+   * return malformed Command as response.
+   *
+   * @param msg - Protobuf message.
+   * @return ContainerCommandResponseProto - MALFORMED_REQUEST.
+   */
+  public static ContainerProtos.ContainerCommandResponseProto
+      malformedRequest(ContainerProtos.ContainerCommandRequestProto msg) {
+    return getContainerResponse(msg, ContainerProtos.Result.MALFORMED_REQUEST,
+        "Cmd type does not match the payload.").build();
+  }
+
+  /**
+   * We found a command type that is not supported yet.
+   *
+   * @param msg - Protobuf message.
+   * @return ContainerCommandResponseProto - MALFORMED_REQUEST.
+   */
+  public static ContainerProtos.ContainerCommandResponseProto
+      unsupportedRequest(ContainerProtos.ContainerCommandRequestProto msg) {
+    return getContainerResponse(msg, ContainerProtos.Result.UNSUPPORTED_REQUEST,
+        "Server does not support this command yet.").build();
+  }
+
+  private ContainerUtils() {
+    //never constructed.
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4683d68e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/helpers/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/helpers/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/helpers/package-info.java
new file mode 100644
index 0000000..15a4a28
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/helpers/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.container.helpers;
+/**
+ Contains protocol buffer helper classes.
+ **/
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4683d68e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/interfaces/ContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/interfaces/ContainerManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/interfaces/ContainerManager.java
new file mode 100644
index 0000000..f98544d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/interfaces/ContainerManager.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.interfaces;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ozone.container.helpers.ContainerData;
+import org.apache.hadoop.ozone.container.helpers.Pipeline;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Interface for container operations.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface ContainerManager {
+
+  /**
+   * Creates a container with the given name.
+   *
+   * @param pipeline      -- Nodes which make up this container.
+   * @param containerData - Container Name and metadata.
+   * @throws IOException
+   */
+  void createContainer(Pipeline pipeline, ContainerData containerData)
+      throws IOException;
+
+  /**
+   * Deletes an existing container.
+   *
+   * @param pipeline      - nodes that make this container.
+   * @param containerName - name of the container.
+   * @throws IOException
+   */
+  void deleteContainer(Pipeline pipeline, String containerName)
+      throws IOException;
+
+  /**
+   * As simple interface for container Iterations.
+   *
+   * @param start - Starting index
+   * @param count - how many to return
+   * @param data  - Actual containerData
+   * @throws IOException
+   */
+  void listContainer(long start, long count, List<ContainerData> data)
+      throws IOException;
+
+  /**
+   * Get metadata about a specific container.
+   *
+   * @param containerName - Name of the container
+   * @return ContainerData
+   * @throws IOException
+   */
+  ContainerData readContainer(String containerName) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4683d68e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/Dispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/Dispatcher.java
new file mode 100644
index 0000000..92aa241
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/Dispatcher.java
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.container.ozoneimpl;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandRequestProto;
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandResponseProto;
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.Type;
+import org.apache.hadoop.ozone.container.helpers.ContainerData;
+import org.apache.hadoop.ozone.container.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.helpers.Pipeline;
+import org.apache.hadoop.ozone.container.interfaces.ContainerDispatcher;
+import org.apache.hadoop.ozone.container.interfaces.ContainerManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * Ozone Container dispatcher takes a call from the netty server and routes it
+ * to the right handler function.
+ */
+public class Dispatcher implements ContainerDispatcher {
+  static final Logger LOG = LoggerFactory.getLogger(Dispatcher.class);
+
+  private final ContainerManager containerManager;
+
+  /**
+   * Constructs an OzoneContainer that receives calls from
+   * XceiverServerHandler.
+   *
+   * @param containerManager - A class that manages containers.
+   */
+  public Dispatcher(ContainerManager containerManager) {
+    Preconditions.checkNotNull(containerManager);
+    this.containerManager = containerManager;
+  }
+
+  @Override
+  public ContainerCommandResponseProto dispatch(
+      ContainerCommandRequestProto msg) throws IOException {
+    Preconditions.checkNotNull(msg);
+    Type cmdType = msg.getCmdType();
+    if ((cmdType == Type.CreateContainer) ||
+        (cmdType == Type.DeleteContainer) ||
+        (cmdType == Type.ReadContainer) ||
+        (cmdType == Type.ListContainer)) {
+
+      return containerProcessHandler(msg);
+    }
+
+
+    return ContainerUtils.unsupportedRequest(msg);
+  }
+
+  /**
+   * Handles the all Container related functionality.
+   *
+   * @param msg - command
+   * @return - response
+   * @throws IOException
+   */
+  private ContainerCommandResponseProto containerProcessHandler(
+      ContainerCommandRequestProto msg) throws IOException {
+    try {
+      ContainerData cData = ContainerData.getFromProtBuf(
+          msg.getCreateContainer().getContainerData());
+
+      Pipeline pipeline = Pipeline.getFromProtoBuf(
+          msg.getCreateContainer().getPipeline());
+      Preconditions.checkNotNull(pipeline);
+
+      switch (msg.getCmdType()) {
+      case CreateContainer:
+        return handleCreateContainer(msg, cData, pipeline);
+
+      case DeleteContainer:
+        return handleDeleteContainer(msg, cData, pipeline);
+
+      case ListContainer:
+        return ContainerUtils.unsupportedRequest(msg);
+
+      case ReadContainer:
+        return handleReadContainer(msg, cData);
+
+      default:
+        return ContainerUtils.unsupportedRequest(msg);
+      }
+    } catch (IOException ex) {
+      LOG.warn("Container operation failed. " +
+              "Container: {} Operation: {}  trace ID: {} Error: {}",
+          msg.getCreateContainer().getContainerData().getName(),
+          msg.getCmdType().name(),
+          msg.getTraceID(),
+          ex.toString());
+
+      // TODO : Replace with finer error codes.
+      return ContainerUtils.getContainerResponse(msg,
+          ContainerProtos.Result.CONTAINER_INTERNAL_ERROR,
+          ex.toString()).build();
+    }
+  }
+
+  /**
+   * Calls into container logic and returns appropriate response.
+   *
+   * @param msg   - Request
+   * @param cData - Container Data object
+   * @return ContainerCommandResponseProto
+   * @throws IOException
+   */
+  private ContainerCommandResponseProto handleReadContainer(
+      ContainerCommandRequestProto msg, ContainerData cData)
+      throws IOException {
+
+    if (!msg.hasReadContainer()) {
+      LOG.debug("Malformed read container request. trace ID: {}",
+          msg.getTraceID());
+      return ContainerUtils.malformedRequest(msg);
+    }
+    ContainerData container = this.containerManager.readContainer(
+        cData.getContainerName());
+    return ContainerUtils.getReadContainerResponse(msg, container);
+  }
+
+  /**
+   * Calls into container logic and returns appropriate response.
+   *
+   * @param msg      - Request
+   * @param cData    - ContainerData
+   * @param pipeline - Pipeline is the machines where this container lives.
+   * @return Response.
+   * @throws IOException
+   */
+  private ContainerCommandResponseProto handleDeleteContainer(
+      ContainerCommandRequestProto msg, ContainerData cData,
+      Pipeline pipeline) throws IOException {
+    if (!msg.hasDeleteContainer()) {
+      LOG.debug("Malformed delete container request. trace ID: {}",
+          msg.getTraceID());
+      return ContainerUtils.malformedRequest(msg);
+    }
+    this.containerManager.deleteContainer(pipeline,
+        cData.getContainerName());
+    return ContainerUtils.getContainerResponse(msg);
+  }
+
+  /**
+   * Calls into container logic and returns appropriate response.
+   *
+   * @param msg      - Request
+   * @param cData    - ContainerData
+   * @param pipeline - Pipeline is the machines where this container lives.
+   * @return Response.
+   * @throws IOException
+   */
+  private ContainerCommandResponseProto handleCreateContainer(
+      ContainerCommandRequestProto msg, ContainerData cData,
+      Pipeline pipeline) throws IOException {
+    if (!msg.hasCreateContainer()) {
+      LOG.debug("Malformed create container request. trace ID: {}",
+          msg.getTraceID());
+      return ContainerUtils.malformedRequest(msg);
+    }
+    this.containerManager.createContainer(pipeline, cData);
+    return ContainerUtils.getContainerResponse(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4683d68e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/package-info.java
new file mode 100644
index 0000000..fbdb396
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/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.
+ */
+package org.apache.hadoop.ozone.container.ozoneimpl;
+
+/**
+ This package is contains Ozone container implementation.
+**/
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4683d68e/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeContainerProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeContainerProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeContainerProtocol.proto
index 0fba636..099f93f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeContainerProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeContainerProtocol.proto
@@ -90,6 +90,14 @@ enum Type {
 }
 
 
+enum Result {
+  SUCCESS = 1;
+  UNSUPPORTED_REQUEST = 2;
+  MALFORMED_REQUEST = 3;
+  CONTAINER_INTERNAL_ERROR = 4;
+
+}
+
 message ContainerCommandRequestProto {
   required Type cmdType = 1; // Type of the command
 
@@ -140,6 +148,9 @@ message ContainerCommandResponseProto {
   optional  DeleteChunkResponseProto deleteChunk = 15;
   optional  ListChunkResponseProto listChunk = 16;
 
+  required Result result = 17;
+  optional string message = 18;
+
 }
 
 // A pipeline is composed of one or more datanodes that back a container.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4683d68e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
index 0622c82..28e8afd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java
@@ -96,6 +96,7 @@ public class ContainerTestHelper {
     response.setCmdType(ContainerProtos.Type.CreateContainer);
     response.setTraceID(request.getTraceID());
     response.setCreateContainer(createResponse.build());
+    response.setResult(ContainerProtos.Result.SUCCESS);
     return response.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4683d68e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/transport/server/TestContainerServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/transport/server/TestContainerServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/transport/server/TestContainerServer.java
index f546a12..e49d1c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/transport/server/TestContainerServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/transport/server/TestContainerServer.java
@@ -19,21 +19,24 @@
 package org.apache.hadoop.ozone.container.transport.server;
 
 import io.netty.channel.embedded.EmbeddedChannel;
-import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
-    .ContainerCommandRequestProto;
-import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
-    .ContainerCommandResponseProto;
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandRequestProto;
+import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandResponseProto;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfiguration;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
 import org.apache.hadoop.ozone.container.helpers.Pipeline;
 import org.apache.hadoop.ozone.container.interfaces.ContainerDispatcher;
+import org.apache.hadoop.ozone.container.interfaces.ContainerManager;
+import org.apache.hadoop.ozone.container.ozoneimpl.Dispatcher;
 import org.apache.hadoop.ozone.container.transport.client.XceiverClient;
 import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.IOException;
 
+import static org.mockito.Mockito.mock;
+
 public class TestContainerServer {
 
   @Test
@@ -86,6 +89,39 @@ public class TestContainerServer {
     }
   }
 
+  @Test
+  public void testClientServerWithContainerDispatcher() throws Exception {
+    XceiverServer server = null;
+    XceiverClient client = null;
+
+    try {
+      Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline();
+      OzoneConfiguration conf = new OzoneConfiguration();
+      conf.setInt(OzoneConfigKeys.DFS_OZONE_CONTAINER_IPC_PORT,
+          pipeline.getLeader().getContainerPort());
+
+      server = new XceiverServer(conf, new Dispatcher(
+          mock(ContainerManager.class)));
+      client = new XceiverClient(pipeline, conf);
+
+      server.start();
+      client.connect();
+
+      ContainerCommandRequestProto request =
+          ContainerTestHelper.getCreateContainerRequest();
+      ContainerCommandResponseProto response = client.sendCommand(request);
+      Assert.assertTrue(request.getTraceID().equals(response.getTraceID()));
+      Assert.assertEquals(response.getResult(), ContainerProtos.Result.SUCCESS);
+    } finally {
+      if (client != null) {
+        client.close();
+      }
+      if (server != null) {
+        server.stop();
+      }
+    }
+  }
+
   private class TestContainerDispatcher implements ContainerDispatcher {
     /**
      * Dispatches commands to container layer.
@@ -100,4 +136,4 @@ public class TestContainerServer {
       return ContainerTestHelper.getCreateContainerResponse(msg);
     }
   }
-}
+}
\ No newline at end of file