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 ar...@apache.org on 2018/07/06 19:09:49 UTC

[10/14] hadoop git commit: HDDS-167. Rename KeySpaceManager to OzoneManager. Contributed by Arpit Agarwal.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/KSMPBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/KSMPBHelper.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/KSMPBHelper.java
deleted file mode 100644
index fdc3ce7..0000000
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/KSMPBHelper.java
+++ /dev/null
@@ -1,113 +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.protocolPB;
-
-import org.apache.hadoop.ozone.OzoneAcl;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclType;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo.OzoneAclRights;
-
-/**
- * Utilities for converting protobuf classes.
- */
-public final class KSMPBHelper {
-
-  private KSMPBHelper() {
-    /** Hidden constructor */
-  }
-
-  /**
-   * Converts OzoneAcl into protobuf's OzoneAclInfo.
-   * @return OzoneAclInfo
-   */
-  public static OzoneAclInfo convertOzoneAcl(OzoneAcl acl) {
-    OzoneAclInfo.OzoneAclType aclType;
-    switch(acl.getType()) {
-    case USER:
-      aclType = OzoneAclType.USER;
-      break;
-    case GROUP:
-      aclType = OzoneAclType.GROUP;
-      break;
-    case WORLD:
-      aclType = OzoneAclType.WORLD;
-      break;
-    default:
-      throw new IllegalArgumentException("ACL type is not recognized");
-    }
-    OzoneAclInfo.OzoneAclRights aclRights;
-    switch(acl.getRights()) {
-    case READ:
-      aclRights = OzoneAclRights.READ;
-      break;
-    case WRITE:
-      aclRights = OzoneAclRights.WRITE;
-      break;
-    case READ_WRITE:
-      aclRights = OzoneAclRights.READ_WRITE;
-      break;
-    default:
-      throw new IllegalArgumentException("ACL right is not recognized");
-    }
-
-    return OzoneAclInfo.newBuilder().setType(aclType)
-        .setName(acl.getName())
-        .setRights(aclRights)
-        .build();
-  }
-
-  /**
-   * Converts protobuf's OzoneAclInfo into OzoneAcl.
-   * @return OzoneAcl
-   */
-  public static OzoneAcl convertOzoneAcl(OzoneAclInfo aclInfo) {
-    OzoneAcl.OzoneACLType aclType;
-    switch(aclInfo.getType()) {
-    case USER:
-      aclType = OzoneAcl.OzoneACLType.USER;
-      break;
-    case GROUP:
-      aclType = OzoneAcl.OzoneACLType.GROUP;
-      break;
-    case WORLD:
-      aclType = OzoneAcl.OzoneACLType.WORLD;
-      break;
-    default:
-      throw new IllegalArgumentException("ACL type is not recognized");
-    }
-    OzoneAcl.OzoneACLRights aclRights;
-    switch(aclInfo.getRights()) {
-    case READ:
-      aclRights = OzoneAcl.OzoneACLRights.READ;
-      break;
-    case WRITE:
-      aclRights = OzoneAcl.OzoneACLRights.WRITE;
-      break;
-    case READ_WRITE:
-      aclRights = OzoneAcl.OzoneACLRights.READ_WRITE;
-      break;
-    default:
-      throw new IllegalArgumentException("ACL right is not recognized");
-    }
-
-    return new OzoneAcl(aclType, aclInfo.getName(), aclRights);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OMPBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OMPBHelper.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OMPBHelper.java
new file mode 100644
index 0000000..d57d32e
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/protocolPB/OMPBHelper.java
@@ -0,0 +1,113 @@
+/**
+ * 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.protocolPB;
+
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.protocol.proto
+    .OzoneManagerProtocolProtos.OzoneAclInfo;
+import org.apache.hadoop.ozone.protocol.proto
+    .OzoneManagerProtocolProtos.OzoneAclInfo.OzoneAclType;
+import org.apache.hadoop.ozone.protocol.proto
+    .OzoneManagerProtocolProtos.OzoneAclInfo.OzoneAclRights;
+
+/**
+ * Utilities for converting protobuf classes.
+ */
+public final class OMPBHelper {
+
+  private OMPBHelper() {
+    /** Hidden constructor */
+  }
+
+  /**
+   * Converts OzoneAcl into protobuf's OzoneAclInfo.
+   * @return OzoneAclInfo
+   */
+  public static OzoneAclInfo convertOzoneAcl(OzoneAcl acl) {
+    OzoneAclInfo.OzoneAclType aclType;
+    switch(acl.getType()) {
+    case USER:
+      aclType = OzoneAclType.USER;
+      break;
+    case GROUP:
+      aclType = OzoneAclType.GROUP;
+      break;
+    case WORLD:
+      aclType = OzoneAclType.WORLD;
+      break;
+    default:
+      throw new IllegalArgumentException("ACL type is not recognized");
+    }
+    OzoneAclInfo.OzoneAclRights aclRights;
+    switch(acl.getRights()) {
+    case READ:
+      aclRights = OzoneAclRights.READ;
+      break;
+    case WRITE:
+      aclRights = OzoneAclRights.WRITE;
+      break;
+    case READ_WRITE:
+      aclRights = OzoneAclRights.READ_WRITE;
+      break;
+    default:
+      throw new IllegalArgumentException("ACL right is not recognized");
+    }
+
+    return OzoneAclInfo.newBuilder().setType(aclType)
+        .setName(acl.getName())
+        .setRights(aclRights)
+        .build();
+  }
+
+  /**
+   * Converts protobuf's OzoneAclInfo into OzoneAcl.
+   * @return OzoneAcl
+   */
+  public static OzoneAcl convertOzoneAcl(OzoneAclInfo aclInfo) {
+    OzoneAcl.OzoneACLType aclType;
+    switch(aclInfo.getType()) {
+    case USER:
+      aclType = OzoneAcl.OzoneACLType.USER;
+      break;
+    case GROUP:
+      aclType = OzoneAcl.OzoneACLType.GROUP;
+      break;
+    case WORLD:
+      aclType = OzoneAcl.OzoneACLType.WORLD;
+      break;
+    default:
+      throw new IllegalArgumentException("ACL type is not recognized");
+    }
+    OzoneAcl.OzoneACLRights aclRights;
+    switch(aclInfo.getRights()) {
+    case READ:
+      aclRights = OzoneAcl.OzoneACLRights.READ;
+      break;
+    case WRITE:
+      aclRights = OzoneAcl.OzoneACLRights.WRITE;
+      break;
+    case READ_WRITE:
+      aclRights = OzoneAcl.OzoneACLRights.READ_WRITE;
+      break;
+    default:
+      throw new IllegalArgumentException("ACL right is not recognized");
+    }
+
+    return new OzoneAcl(aclType, aclInfo.getName(), aclRights);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/common/src/main/proto/KeySpaceManagerProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/proto/KeySpaceManagerProtocol.proto b/hadoop-ozone/common/src/main/proto/KeySpaceManagerProtocol.proto
deleted file mode 100644
index d3d1de6..0000000
--- a/hadoop-ozone/common/src/main/proto/KeySpaceManagerProtocol.proto
+++ /dev/null
@@ -1,474 +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.
- */
-
-/**
- * These .proto interfaces are private and unstable.
- * Please see http://wiki.apache.org/hadoop/Compatibility
- * for what changes are allowed for a *unstable* .proto interface.
- */
-
-option java_package = "org.apache.hadoop.ozone.protocol.proto";
-option java_outer_classname = "KeySpaceManagerProtocolProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-package hadoop.ozone;
-
-/**
-This is file contains the protocol to communicate with
-Ozone key space manager. Ozone KSM manages the namespace for ozone.
-This is similar to Namenode for Ozone.
-*/
-
-import "hdfs.proto";
-import "hdds.proto";
-
-enum Status {
-    OK = 1;
-    VOLUME_NOT_UNIQUE = 2;
-    VOLUME_NOT_FOUND = 3;
-    VOLUME_NOT_EMPTY = 4;
-    VOLUME_ALREADY_EXISTS = 5;
-    USER_NOT_FOUND = 6;
-    USER_TOO_MANY_VOLUMES = 7;
-    BUCKET_NOT_FOUND = 8;
-    BUCKET_NOT_EMPTY = 9;
-    BUCKET_ALREADY_EXISTS = 10;
-    KEY_ALREADY_EXISTS = 11;
-    KEY_NOT_FOUND = 12;
-    INVALID_KEY_NAME = 13;
-    ACCESS_DENIED = 14;
-    INTERNAL_ERROR = 15;
-}
-
-
-message VolumeInfo {
-    required string adminName = 1;
-    required string ownerName = 2;
-    required string volume = 3;
-    optional uint64 quotaInBytes = 4;
-    repeated hadoop.hdds.KeyValue metadata = 5;
-    repeated OzoneAclInfo volumeAcls = 6;
-    required uint64 creationTime = 7;
-}
-
-/**
-    Creates a volume
-*/
-message CreateVolumeRequest {
-    required VolumeInfo volumeInfo = 1;
-}
-
-message CreateVolumeResponse {
-
-    required Status status = 1;
-}
-
-message VolumeList {
-    repeated string volumeNames = 1;
-}
-
-/**
-    Changes the Volume Properties -- like ownership and quota for a volume.
-*/
-message SetVolumePropertyRequest {
-    required string volumeName = 1;
-    optional string ownerName = 2;
-    optional uint64 quotaInBytes = 3;
-}
-
-message SetVolumePropertyResponse {
-    required Status status = 1;
-}
-
-/**
- * Checks if the user has specified permissions for the volume
- */
-message CheckVolumeAccessRequest {
-    required string volumeName = 1;
-    required OzoneAclInfo userAcl = 2;
-}
-
-message CheckVolumeAccessResponse {
-
-    required Status status = 1;
-}
-
-
-/**
-    Returns information about a volume.
-*/
-
-message InfoVolumeRequest {
-    required string volumeName = 1;
-}
-
-message InfoVolumeResponse {
-    required Status status = 1;
-    optional VolumeInfo volumeInfo = 2;
-
-}
-
-/**
-    Deletes an existing volume.
-*/
-message DeleteVolumeRequest {
-    required string volumeName = 1;
-}
-
-message DeleteVolumeResponse {
-    required Status status = 1;
-}
-
-
-/**
-    List Volumes -- List all volumes in the cluster or by user.
-*/
-
-message ListVolumeRequest {
-    enum Scope {
-        USER_VOLUMES = 1;   // User volumes -- called by user
-        VOLUMES_BY_USER = 2; // User volumes - called by Admin
-        VOLUMES_BY_CLUSTER = 3; // All volumes in the cluster
-    }
-    required Scope scope = 1;
-    optional string userName = 2;
-    optional string prefix = 3;
-    optional string prevKey = 4;
-    optional uint32 maxKeys = 5;
-}
-
-message ListVolumeResponse {
-    required Status status = 1;
-    repeated VolumeInfo volumeInfo = 2;
-}
-
-message BucketInfo {
-    required string volumeName = 1;
-    required string bucketName = 2;
-    repeated OzoneAclInfo acls = 3;
-    required bool isVersionEnabled = 4 [default = false];
-    required hadoop.hdfs.StorageTypeProto storageType = 5 [default = DISK];
-    required uint64 creationTime = 6;
-}
-
-message BucketArgs {
-    required string volumeName = 1;
-    required string bucketName = 2;
-    repeated OzoneAclInfo addAcls = 3;
-    repeated OzoneAclInfo removeAcls = 4;
-    optional bool isVersionEnabled = 5;
-    optional hadoop.hdfs.StorageTypeProto storageType = 6;
-}
-
-message OzoneAclInfo {
-    enum OzoneAclType {
-        USER = 1;
-        GROUP = 2;
-        WORLD = 3;
-    }
-    enum OzoneAclRights {
-        READ = 1;
-        WRITE = 2;
-        READ_WRITE = 3;
-    }
-    required OzoneAclType type = 1;
-    required string name = 2;
-    required OzoneAclRights rights = 3;
-}
-
-message CreateBucketRequest {
-    required BucketInfo bucketInfo = 1;
-}
-
-message CreateBucketResponse {
-    required Status status = 1;
-}
-
-message InfoBucketRequest {
-    required string volumeName = 1;
-    required string bucketName = 2;
-}
-
-message InfoBucketResponse {
-    required Status status = 1;
-    optional BucketInfo bucketInfo = 2;
-}
-
-message ListBucketsRequest {
-    required string volumeName = 1;
-    optional string startKey = 2;
-    optional string prefix = 3;
-    optional int32 count = 4;
-}
-
-message ListBucketsResponse {
-    required Status status = 1;
-    repeated BucketInfo bucketInfo = 2;
-}
-
-message KeyArgs {
-    required string volumeName = 1;
-    required string bucketName = 2;
-    required string keyName = 3;
-    optional uint64 dataSize = 4;
-    optional hadoop.hdds.ReplicationType type = 5;
-    optional hadoop.hdds.ReplicationFactor factor = 6;
-}
-
-message KeyLocation {
-    required hadoop.hdds.BlockID blockID = 1;
-    required bool shouldCreateContainer = 2;
-    required uint64 offset = 3;
-    required uint64 length = 4;
-    // indicated at which version this block gets created.
-    optional uint64 createVersion = 5;
-}
-
-message KeyLocationList {
-    optional uint64 version = 1;
-    repeated KeyLocation keyLocations = 2;
-}
-
-message KeyInfo {
-    required string volumeName = 1;
-    required string bucketName = 2;
-    required string keyName = 3;
-    required uint64 dataSize = 4;
-    required hadoop.hdds.ReplicationType type = 5;
-    required hadoop.hdds.ReplicationFactor factor = 6;
-    repeated KeyLocationList keyLocationList = 7;
-    required uint64 creationTime = 8;
-    required uint64 modificationTime = 9;
-    optional uint64 latestVersion = 10;
-}
-
-message LocateKeyRequest {
-    required KeyArgs keyArgs = 1;
-}
-
-message LocateKeyResponse {
-    required Status status = 1;
-    optional KeyInfo keyInfo = 2;
-    // clients' followup request may carry this ID for stateful operations (similar
-    // to a cookie).
-    optional uint32 ID = 3;
-    // TODO : allow specifiying a particular version to read.
-    optional uint64 openVersion = 4;
-}
-
-message SetBucketPropertyRequest {
-    required BucketArgs bucketArgs = 1;
-}
-
-message SetBucketPropertyResponse {
-    required Status status = 1;
-}
-
-message RenameKeyRequest{
-    required KeyArgs keyArgs = 1;
-    required string toKeyName = 2;
-}
-
-message RenameKeyResponse{
-    required Status status = 1;
-}
-
-message DeleteBucketRequest {
-    required string volumeName = 1;
-    required string bucketName = 2;
-}
-
-message DeleteBucketResponse {
-    required Status status = 1;
-}
-
-message ListKeysRequest {
-    required string volumeName = 1;
-    required string bucketName = 2;
-    optional string startKey = 3;
-    optional string prefix = 4;
-    optional int32 count = 5;
-}
-
-message ListKeysResponse {
-    required Status status = 1;
-    repeated KeyInfo keyInfo = 2;
-}
-
-message AllocateBlockRequest {
-    required KeyArgs keyArgs = 1;
-    required uint32 clientID = 2;
-}
-
-message AllocateBlockResponse {
-    required Status status = 1;
-    required KeyLocation keyLocation = 2;
-}
-
-message CommitKeyRequest {
-    required KeyArgs keyArgs = 1;
-    required uint32 clientID = 2;
-}
-
-message CommitKeyResponse {
-    required Status status = 1;
-}
-
-message ServiceListRequest {
-}
-
-message ServiceListResponse {
-    required Status status = 1;
-    repeated ServiceInfo serviceInfo = 2;
-}
-
-message ServicePort {
-    enum Type {
-        RPC = 1;
-        HTTP = 2;
-        HTTPS = 3;
-        RATIS = 4;
-    };
-    required Type type = 1;
-    required uint32 value = 2;
-}
-
-message ServiceInfo {
-    required hadoop.hdds.NodeType nodeType = 1;
-    required string hostname = 2;
-    repeated ServicePort servicePorts = 3;
-}
-
-/**
- The KSM service that takes care of Ozone namespace.
-*/
-service KeySpaceManagerService {
-
-    /**
-        Creates a Volume.
-    */
-    rpc createVolume(CreateVolumeRequest)
-        returns(CreateVolumeResponse);
-
-    /**
-        Allows modificiation of volume properties.
-    */
-    rpc setVolumeProperty(SetVolumePropertyRequest)
-        returns (SetVolumePropertyResponse);
-
-    /**
-        Checks if the specified volume is accesible by the specified user.
-    */
-    rpc checkVolumeAccess(CheckVolumeAccessRequest)
-        returns (CheckVolumeAccessResponse);
-
-    /**
-        Gets Volume information.
-    */
-    rpc infoVolume(InfoVolumeRequest)
-        returns(InfoVolumeResponse);
-    /**
-        Deletes a volume if it is empty.
-    */
-    rpc deleteVolume(DeleteVolumeRequest)
-        returns (DeleteVolumeResponse);
-
-    /**
-        Lists Volumes
-    */
-    rpc listVolumes(ListVolumeRequest)
-        returns (ListVolumeResponse);
-
-    /**
-        Creates a Bucket.
-    */
-    rpc createBucket(CreateBucketRequest)
-        returns(CreateBucketResponse);
-
-    /**
-        Get Bucket information.
-    */
-    rpc infoBucket(InfoBucketRequest)
-        returns(InfoBucketResponse);
-
-    /**
-        Sets bucket properties.
-    */
-    rpc setBucketProperty(SetBucketPropertyRequest)
-        returns(SetBucketPropertyResponse);
-
-    /**
-        Get key.
-    */
-    rpc createKey(LocateKeyRequest)
-        returns(LocateKeyResponse);
-
-    /**
-       Look up for an existing key.
-    */
-    rpc lookupKey(LocateKeyRequest)
-        returns(LocateKeyResponse);
-
-    /**
-       Rename an existing key within a bucket.
-    */
-    rpc renameKey(RenameKeyRequest)
-        returns(RenameKeyResponse);
-
-    /**
-       Delete an existing key.
-    */
-    rpc deleteKey(LocateKeyRequest)
-        returns(LocateKeyResponse);
-
-    /**
-       Deletes a bucket from volume if it is empty.
-    */
-    rpc deleteBucket(DeleteBucketRequest)
-        returns (DeleteBucketResponse);
-
-    /**
-       List Buckets.
-    */
-    rpc listBuckets(ListBucketsRequest)
-    returns(ListBucketsResponse);
-
-    /**
-       List Keys.
-    */
-    rpc listKeys(ListKeysRequest)
-    returns(ListKeysResponse);
-
-    /**
-      Commit a key.
-    */
-    rpc commitKey(CommitKeyRequest)
-    returns(CommitKeyResponse);
-
-    /**
-      Allocate a new block for a key.
-    */
-    rpc allocateBlock(AllocateBlockRequest)
-    returns(AllocateBlockResponse);
-
-    /**
-      Returns list of Ozone services with its configuration details.
-    */
-    rpc getServiceList(ServiceListRequest)
-    returns(ServiceListResponse);
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/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
new file mode 100644
index 0000000..36b1c83
--- /dev/null
+++ b/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
@@ -0,0 +1,480 @@
+/**
+ * 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.
+ */
+
+/**
+ * These .proto interfaces are private and unstable.
+ * Please see http://wiki.apache.org/hadoop/Compatibility
+ * for what changes are allowed for a *unstable* .proto interface.
+ */
+
+option java_package = "org.apache.hadoop.ozone.protocol.proto";
+option java_outer_classname = "OzoneManagerProtocolProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.ozone;
+
+/**
+This is file contains the protocol to communicate with
+Ozone Manager. Ozone Manager manages the namespace for ozone.
+This is similar to Namenode for Ozone.
+*/
+
+import "hdfs.proto";
+import "hdds.proto";
+
+enum Status {
+    OK = 1;
+    VOLUME_NOT_UNIQUE = 2;
+    VOLUME_NOT_FOUND = 3;
+    VOLUME_NOT_EMPTY = 4;
+    VOLUME_ALREADY_EXISTS = 5;
+    USER_NOT_FOUND = 6;
+    USER_TOO_MANY_VOLUMES = 7;
+    BUCKET_NOT_FOUND = 8;
+    BUCKET_NOT_EMPTY = 9;
+    BUCKET_ALREADY_EXISTS = 10;
+    KEY_ALREADY_EXISTS = 11;
+    KEY_NOT_FOUND = 12;
+    INVALID_KEY_NAME = 13;
+    ACCESS_DENIED = 14;
+    INTERNAL_ERROR = 15;
+    KEY_ALLOCATION_ERROR = 16;
+    KEY_DELETION_ERROR = 17;
+    KEY_RENAME_ERROR = 18;
+    METADATA_ERROR = 19;
+    OM_NOT_INITIALIZED = 20;
+    SCM_VERSION_MISMATCH_ERROR = 21;
+}
+
+
+message VolumeInfo {
+    required string adminName = 1;
+    required string ownerName = 2;
+    required string volume = 3;
+    optional uint64 quotaInBytes = 4;
+    repeated hadoop.hdds.KeyValue metadata = 5;
+    repeated OzoneAclInfo volumeAcls = 6;
+    required uint64 creationTime = 7;
+}
+
+/**
+    Creates a volume
+*/
+message CreateVolumeRequest {
+    required VolumeInfo volumeInfo = 1;
+}
+
+message CreateVolumeResponse {
+
+    required Status status = 1;
+}
+
+message VolumeList {
+    repeated string volumeNames = 1;
+}
+
+/**
+    Changes the Volume Properties -- like ownership and quota for a volume.
+*/
+message SetVolumePropertyRequest {
+    required string volumeName = 1;
+    optional string ownerName = 2;
+    optional uint64 quotaInBytes = 3;
+}
+
+message SetVolumePropertyResponse {
+    required Status status = 1;
+}
+
+/**
+ * Checks if the user has specified permissions for the volume
+ */
+message CheckVolumeAccessRequest {
+    required string volumeName = 1;
+    required OzoneAclInfo userAcl = 2;
+}
+
+message CheckVolumeAccessResponse {
+
+    required Status status = 1;
+}
+
+
+/**
+    Returns information about a volume.
+*/
+
+message InfoVolumeRequest {
+    required string volumeName = 1;
+}
+
+message InfoVolumeResponse {
+    required Status status = 1;
+    optional VolumeInfo volumeInfo = 2;
+
+}
+
+/**
+    Deletes an existing volume.
+*/
+message DeleteVolumeRequest {
+    required string volumeName = 1;
+}
+
+message DeleteVolumeResponse {
+    required Status status = 1;
+}
+
+
+/**
+    List Volumes -- List all volumes in the cluster or by user.
+*/
+
+message ListVolumeRequest {
+    enum Scope {
+        USER_VOLUMES = 1;   // User volumes -- called by user
+        VOLUMES_BY_USER = 2; // User volumes - called by Admin
+        VOLUMES_BY_CLUSTER = 3; // All volumes in the cluster
+    }
+    required Scope scope = 1;
+    optional string userName = 2;
+    optional string prefix = 3;
+    optional string prevKey = 4;
+    optional uint32 maxKeys = 5;
+}
+
+message ListVolumeResponse {
+    required Status status = 1;
+    repeated VolumeInfo volumeInfo = 2;
+}
+
+message BucketInfo {
+    required string volumeName = 1;
+    required string bucketName = 2;
+    repeated OzoneAclInfo acls = 3;
+    required bool isVersionEnabled = 4 [default = false];
+    required hadoop.hdfs.StorageTypeProto storageType = 5 [default = DISK];
+    required uint64 creationTime = 6;
+}
+
+message BucketArgs {
+    required string volumeName = 1;
+    required string bucketName = 2;
+    repeated OzoneAclInfo addAcls = 3;
+    repeated OzoneAclInfo removeAcls = 4;
+    optional bool isVersionEnabled = 5;
+    optional hadoop.hdfs.StorageTypeProto storageType = 6;
+}
+
+message OzoneAclInfo {
+    enum OzoneAclType {
+        USER = 1;
+        GROUP = 2;
+        WORLD = 3;
+    }
+    enum OzoneAclRights {
+        READ = 1;
+        WRITE = 2;
+        READ_WRITE = 3;
+    }
+    required OzoneAclType type = 1;
+    required string name = 2;
+    required OzoneAclRights rights = 3;
+}
+
+message CreateBucketRequest {
+    required BucketInfo bucketInfo = 1;
+}
+
+message CreateBucketResponse {
+    required Status status = 1;
+}
+
+message InfoBucketRequest {
+    required string volumeName = 1;
+    required string bucketName = 2;
+}
+
+message InfoBucketResponse {
+    required Status status = 1;
+    optional BucketInfo bucketInfo = 2;
+}
+
+message ListBucketsRequest {
+    required string volumeName = 1;
+    optional string startKey = 2;
+    optional string prefix = 3;
+    optional int32 count = 4;
+}
+
+message ListBucketsResponse {
+    required Status status = 1;
+    repeated BucketInfo bucketInfo = 2;
+}
+
+message KeyArgs {
+    required string volumeName = 1;
+    required string bucketName = 2;
+    required string keyName = 3;
+    optional uint64 dataSize = 4;
+    optional hadoop.hdds.ReplicationType type = 5;
+    optional hadoop.hdds.ReplicationFactor factor = 6;
+}
+
+message KeyLocation {
+    required hadoop.hdds.BlockID blockID = 1;
+    required bool shouldCreateContainer = 2;
+    required uint64 offset = 3;
+    required uint64 length = 4;
+    // indicated at which version this block gets created.
+    optional uint64 createVersion = 5;
+}
+
+message KeyLocationList {
+    optional uint64 version = 1;
+    repeated KeyLocation keyLocations = 2;
+}
+
+message KeyInfo {
+    required string volumeName = 1;
+    required string bucketName = 2;
+    required string keyName = 3;
+    required uint64 dataSize = 4;
+    required hadoop.hdds.ReplicationType type = 5;
+    required hadoop.hdds.ReplicationFactor factor = 6;
+    repeated KeyLocationList keyLocationList = 7;
+    required uint64 creationTime = 8;
+    required uint64 modificationTime = 9;
+    optional uint64 latestVersion = 10;
+}
+
+message LocateKeyRequest {
+    required KeyArgs keyArgs = 1;
+}
+
+message LocateKeyResponse {
+    required Status status = 1;
+    optional KeyInfo keyInfo = 2;
+    // clients' followup request may carry this ID for stateful operations (similar
+    // to a cookie).
+    optional uint32 ID = 3;
+    // TODO : allow specifiying a particular version to read.
+    optional uint64 openVersion = 4;
+}
+
+message SetBucketPropertyRequest {
+    required BucketArgs bucketArgs = 1;
+}
+
+message SetBucketPropertyResponse {
+    required Status status = 1;
+}
+
+message RenameKeyRequest{
+    required KeyArgs keyArgs = 1;
+    required string toKeyName = 2;
+}
+
+message RenameKeyResponse{
+    required Status status = 1;
+}
+
+message DeleteBucketRequest {
+    required string volumeName = 1;
+    required string bucketName = 2;
+}
+
+message DeleteBucketResponse {
+    required Status status = 1;
+}
+
+message ListKeysRequest {
+    required string volumeName = 1;
+    required string bucketName = 2;
+    optional string startKey = 3;
+    optional string prefix = 4;
+    optional int32 count = 5;
+}
+
+message ListKeysResponse {
+    required Status status = 1;
+    repeated KeyInfo keyInfo = 2;
+}
+
+message AllocateBlockRequest {
+    required KeyArgs keyArgs = 1;
+    required uint32 clientID = 2;
+}
+
+message AllocateBlockResponse {
+    required Status status = 1;
+    required KeyLocation keyLocation = 2;
+}
+
+message CommitKeyRequest {
+    required KeyArgs keyArgs = 1;
+    required uint32 clientID = 2;
+}
+
+message CommitKeyResponse {
+    required Status status = 1;
+}
+
+message ServiceListRequest {
+}
+
+message ServiceListResponse {
+    required Status status = 1;
+    repeated ServiceInfo serviceInfo = 2;
+}
+
+message ServicePort {
+    enum Type {
+        RPC = 1;
+        HTTP = 2;
+        HTTPS = 3;
+        RATIS = 4;
+    };
+    required Type type = 1;
+    required uint32 value = 2;
+}
+
+message ServiceInfo {
+    required hadoop.hdds.NodeType nodeType = 1;
+    required string hostname = 2;
+    repeated ServicePort servicePorts = 3;
+}
+
+/**
+ The OM service that takes care of Ozone namespace.
+*/
+service OzoneManagerService {
+
+    /**
+        Creates a Volume.
+    */
+    rpc createVolume(CreateVolumeRequest)
+        returns(CreateVolumeResponse);
+
+    /**
+        Allows modificiation of volume properties.
+    */
+    rpc setVolumeProperty(SetVolumePropertyRequest)
+        returns (SetVolumePropertyResponse);
+
+    /**
+        Checks if the specified volume is accesible by the specified user.
+    */
+    rpc checkVolumeAccess(CheckVolumeAccessRequest)
+        returns (CheckVolumeAccessResponse);
+
+    /**
+        Gets Volume information.
+    */
+    rpc infoVolume(InfoVolumeRequest)
+        returns(InfoVolumeResponse);
+    /**
+        Deletes a volume if it is empty.
+    */
+    rpc deleteVolume(DeleteVolumeRequest)
+        returns (DeleteVolumeResponse);
+
+    /**
+        Lists Volumes
+    */
+    rpc listVolumes(ListVolumeRequest)
+        returns (ListVolumeResponse);
+
+    /**
+        Creates a Bucket.
+    */
+    rpc createBucket(CreateBucketRequest)
+        returns(CreateBucketResponse);
+
+    /**
+        Get Bucket information.
+    */
+    rpc infoBucket(InfoBucketRequest)
+        returns(InfoBucketResponse);
+
+    /**
+        Sets bucket properties.
+    */
+    rpc setBucketProperty(SetBucketPropertyRequest)
+        returns(SetBucketPropertyResponse);
+
+    /**
+        Get key.
+    */
+    rpc createKey(LocateKeyRequest)
+        returns(LocateKeyResponse);
+
+    /**
+       Look up for an existing key.
+    */
+    rpc lookupKey(LocateKeyRequest)
+        returns(LocateKeyResponse);
+
+    /**
+       Rename an existing key within a bucket.
+    */
+    rpc renameKey(RenameKeyRequest)
+        returns(RenameKeyResponse);
+
+    /**
+       Delete an existing key.
+    */
+    rpc deleteKey(LocateKeyRequest)
+        returns(LocateKeyResponse);
+
+    /**
+       Deletes a bucket from volume if it is empty.
+    */
+    rpc deleteBucket(DeleteBucketRequest)
+        returns (DeleteBucketResponse);
+
+    /**
+       List Buckets.
+    */
+    rpc listBuckets(ListBucketsRequest)
+    returns(ListBucketsResponse);
+
+    /**
+       List Keys.
+    */
+    rpc listKeys(ListKeysRequest)
+    returns(ListKeysResponse);
+
+    /**
+      Commit a key.
+    */
+    rpc commitKey(CommitKeyRequest)
+    returns(CommitKeyResponse);
+
+    /**
+      Allocate a new block for a key.
+    */
+    rpc allocateBlock(AllocateBlockRequest)
+    returns(AllocateBlockResponse);
+
+    /**
+      Returns list of Ozone services with its configuration details.
+    */
+    rpc getServiceList(ServiceListRequest)
+    returns(ServiceListResponse);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/docs/content/GettingStarted.md
----------------------------------------------------------------------
diff --git a/hadoop-ozone/docs/content/GettingStarted.md b/hadoop-ozone/docs/content/GettingStarted.md
index 531d192..117a307 100644
--- a/hadoop-ozone/docs/content/GettingStarted.md
+++ b/hadoop-ozone/docs/content/GettingStarted.md
@@ -194,12 +194,12 @@ This path will be created by datanodes if it doesn't exist already. Here is an
     </property>
     ```
 
-1. **ozone.ksm.address** OM server address. This is used by OzoneClient and
+1. **ozone.om.address** OM server address. This is used by OzoneClient and
 Ozone File System.
     ```
     <property>
-       <name>ozone.ksm.address</name>
-       <value>ksm.hadoop.apache.org</value>
+       <name>ozone.om.address</name>
+       <value>om.hadoop.apache.org</value>
     </property>
     ```
 
@@ -210,10 +210,10 @@ Ozone File System.
 | ozone.enabled                  | True                         | This enables SCM and  containers in HDFS cluster.                |
 | ozone.metadata.dirs            | file path                    | The metadata will be stored here.                                |
 | ozone.scm.names                | SCM server name              | Hostname:port or or IP:port address of SCM.                      |
-| ozone.scm.block.client.address | SCM server name and port     | Used by services like OM                                        |
+| ozone.scm.block.client.address | SCM server name and port     | Used by services like OM                                         |
 | ozone.scm.client.address       | SCM server name and port     | Used by client side                                              |
 | ozone.scm.datanode.address     | SCM server name and port     | Used by datanode to talk to SCM                                  |
-| ozone.ksm.address              | OM server name              | Used by Ozone handler and Ozone file system.                     |
+| ozone.om.address               | OM server name               | Used by Ozone handler and Ozone file system.                     |
 
 
 #### Sample ozone-site.xml
@@ -253,7 +253,7 @@ Ozone File System.
      </property>
 
      <property>
-       <name>ozone.ksm.address</name>
+       <name>ozone.om.address</name>
        <value>127.0.0.1:9874</value>
      </property>
 </configuration>
@@ -286,12 +286,12 @@ ozone --daemon start scm
 
 Once SCM gets started, OM must be initialized.
 ```
-ozone ksm -createObjectStore
+ozone om -createObjectStore
 ```
 
 Start OM.
 ```
-ozone --daemon start ksm
+ozone --daemon start om
 ```
 
 If you would like to start HDFS and Ozone together, you can do that by running
@@ -349,7 +349,7 @@ log4j.additivity.org.apache.hadoop.ozone=false
 ```
 
 On the SCM/OM side, you will be able to see
-1. `hadoop-hdfs-ksm-hostname.log`
+1. `hadoop-hdfs-om-hostname.log`
 1. `hadoop-hdfs-scm-hostname.log`
 
 ## Reporting Bugs

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/docs/content/Metrics.md
----------------------------------------------------------------------
diff --git a/hadoop-ozone/docs/content/Metrics.md b/hadoop-ozone/docs/content/Metrics.md
index dc58460..64a481f 100644
--- a/hadoop-ozone/docs/content/Metrics.md
+++ b/hadoop-ozone/docs/content/Metrics.md
@@ -131,10 +131,10 @@ Following are the counters for containers:
 
 ### Key Space Metrics
 
-The metrics for various key space manager operations in HDFS Ozone.
+The metrics for various Ozone Manager operations in HDFS Ozone.
 
-key space manager (KSM) is a service that similar to the Namenode in HDFS.
-In the current design of KSM, it maintains metadata of all volumes, buckets and keys.
+The Ozone Manager (OM) is a service that similar to the Namenode in HDFS.
+In the current design of OM, it maintains metadata of all volumes, buckets and keys.
 These metrics are only available when ozone is enabled.
 
 Following is the set of counters maintained for each key space operation.
@@ -142,12 +142,12 @@ Following is the set of counters maintained for each key space operation.
 *Total number of operation* - We maintain an array which counts how
 many times a specific operation has been performed.
 Eg.`NumVolumeCreate` tells us how many times create volume has been
-invoked in KSM.
+invoked in OM.
 
 *Total number of failed operation* - This type operation is opposite to the above
 operation.
 Eg.`NumVolumeCreateFails` tells us how many times create volume has been invoked
-failed in KSM.
+failed in OM.
 
 Following are the counters for each of key space operations.
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/docs/content/_index.md
----------------------------------------------------------------------
diff --git a/hadoop-ozone/docs/content/_index.md b/hadoop-ozone/docs/content/_index.md
index ab7eabe..383b2e0 100644
--- a/hadoop-ozone/docs/content/_index.md
+++ b/hadoop-ozone/docs/content/_index.md
@@ -56,14 +56,14 @@ This is like DFSClient in HDFS. This acts as the standard client to talk to
 Ozone. All other components that we have discussed so far rely on Ozone client
 (TODO: Add Ozone client documentation).

 
-## Key Space Manager

+## Ozone Manager
 
-Key Space Manager(KSM) takes care of the Ozone's namespace.
-All ozone entities like volumes, buckets and keys are managed by KSM
-(TODO: Add KSM documentation). In Short, KSM is the metadata manager for Ozone.
-KSM talks to blockManager(SCM) to get blocks and passes it on to the Ozone
+Ozone Manager (OM) takes care of the Ozone's namespace.
+All ozone entities like volumes, buckets and keys are managed by OM
+(TODO: Add OM documentation). In short, OM is the metadata manager for Ozone.
+OM talks to blockManager(SCM) to get blocks and passes it on to the Ozone
 client.  Ozone client writes data to these blocks.
-KSM will eventually be replicated via Apache Ratis for High Availability.

+OM will eventually be replicated via Apache Ratis for High Availability.

 
 ## Storage Container Manager
 Storage Container Manager (SCM) is the block and cluster manager for Ozone.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/docs/static/OzoneOverview.svg
----------------------------------------------------------------------
diff --git a/hadoop-ozone/docs/static/OzoneOverview.svg b/hadoop-ozone/docs/static/OzoneOverview.svg
index 2e14d3f..0120a5c 100644
--- a/hadoop-ozone/docs/static/OzoneOverview.svg
+++ b/hadoop-ozone/docs/static/OzoneOverview.svg
@@ -166,7 +166,7 @@
             <path d="M307.5,148.5 L433.5,148.5" id="Line" stroke="#000000" fill="#000000" stroke-linecap="square"></path>
             <path id="Line-decoration-1" d="M433.5,148.5 L422.7,145.5 L422.7,151.5 L433.5,148.5 Z" stroke="#000000" fill="#000000" stroke-linecap="square"></path>
             <path d="M4,232 L699,232" id="Line" stroke="#000000" stroke-width="2" stroke-linecap="square" stroke-dasharray="5,2,5"></path>
-            <g id="KSM" transform="translate(432.000000, 132.000000)">
+            <g id="OM" transform="translate(432.000000, 132.000000)">
                 <g id="Rectangle-3">
                     <use fill="#C6D4F9" fill-rule="evenodd" xlink:href="#path-19"></use>
                     <rect stroke="#000000" stroke-width="1" x="0.5" y="0.5" width="225" height="35" rx="8"></rect>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/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 091d771..b568672 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
@@ -21,7 +21,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
 import org.apache.hadoop.ozone.client.OzoneClient;
-import org.apache.hadoop.ozone.ksm.KeySpaceManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
 import org.apache.hadoop.hdds.scm.protocolPB
     .StorageContainerLocationProtocolClientSideTranslatorPB;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -82,12 +82,12 @@ public interface MiniOzoneCluster {
   StorageContainerManager getStorageContainerManager();
 
   /**
-   * Returns {@link KeySpaceManager} associated with this
+   * Returns {@link OzoneManager} associated with this
    * {@link MiniOzoneCluster} instance.
    *
-   * @return {@link KeySpaceManager} instance
+   * @return {@link OzoneManager} instance
    */
-  KeySpaceManager getKeySpaceManager();
+  OzoneManager getOzoneManager();
 
   /**
    * Returns the list of {@link HddsDatanodeService} which are part of this
@@ -141,11 +141,11 @@ public interface MiniOzoneCluster {
   void restartStorageContainerManager() throws IOException;
 
   /**
-   * Restarts KeySpaceManager instance.
+   * Restarts OzoneManager instance.
    *
    * @throws IOException
    */
-  void restartKeySpaceManager() throws IOException;
+  void restartOzoneManager() throws IOException;
 
   /**
    * Restart a particular HddsDatanode.
@@ -184,13 +184,13 @@ public interface MiniOzoneCluster {
     protected Optional<Integer> hbInterval = Optional.empty();
     protected Optional<Integer> hbProcessorInterval = Optional.empty();
     protected Optional<String> scmId = Optional.empty();
-    protected Optional<String> ksmId = Optional.empty();
+    protected Optional<String> omId = Optional.empty();
 
     protected Boolean ozoneEnabled = true;
     protected Boolean randomContainerPort = true;
 
     // Use relative smaller number of handlers for testing
-    protected int numOfKsmHandlers = 20;
+    protected int numOfOmHandlers = 20;
     protected int numOfScmHandlers = 20;
     protected int numOfDatanodes = 1;
 
@@ -226,14 +226,14 @@ public interface MiniOzoneCluster {
     }
 
     /**
-     * Sets the KSM id.
+     * Sets the OM id.
      *
-     * @param id KSM Id
+     * @param id OM Id
      *
      * @return MiniOzoneCluster.Builder
      */
-    public Builder setKsmId(String id) {
-      ksmId = Optional.of(id);
+    public Builder setOmId(String id) {
+      omId = Optional.of(id);
       return this;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/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 f0bfef1..b3137bf 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
@@ -34,10 +34,10 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientFactory;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
-import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
-import org.apache.hadoop.ozone.ksm.KeySpaceManager;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OzoneManager;
 import org.apache.hadoop.hdds.scm.server.SCMStorage;
-import org.apache.hadoop.ozone.ksm.KSMStorage;
+import org.apache.hadoop.ozone.om.OMStorage;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.protocolPB
     .StorageContainerLocationProtocolClientSideTranslatorPB;
@@ -73,7 +73,7 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys
 
 /**
  * MiniOzoneCluster creates a complete in-process Ozone cluster suitable for
- * running tests.  The cluster consists of a KeySpaceManager,
+ * running tests.  The cluster consists of a OzoneManager,
  * StorageContainerManager and multiple DataNodes.
  */
 @InterfaceAudience.Private
@@ -84,7 +84,7 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
 
   private final OzoneConfiguration conf;
   private final StorageContainerManager scm;
-  private final KeySpaceManager ksm;
+  private final OzoneManager ozoneManager;
   private final List<HddsDatanodeService> hddsDatanodes;
 
   /**
@@ -93,11 +93,11 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
    * @throws IOException if there is an I/O error
    */
   private MiniOzoneClusterImpl(OzoneConfiguration conf,
-                               KeySpaceManager ksm,
+                               OzoneManager ozoneManager,
                                StorageContainerManager scm,
                                List<HddsDatanodeService> hddsDatanodes) {
     this.conf = conf;
-    this.ksm = ksm;
+    this.ozoneManager = ozoneManager;
     this.scm = scm;
     this.hddsDatanodes = hddsDatanodes;
   }
@@ -147,8 +147,8 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
   }
 
   @Override
-  public KeySpaceManager getKeySpaceManager() {
-    return this.ksm;
+  public OzoneManager getOzoneManager() {
+    return this.ozoneManager;
   }
 
   @Override
@@ -209,9 +209,9 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
   }
 
   @Override
-  public void restartKeySpaceManager() throws IOException {
-    ksm.stop();
-    ksm.start();
+  public void restartOzoneManager() throws IOException {
+    ozoneManager.stop();
+    ozoneManager.start();
   }
 
   @Override
@@ -247,10 +247,10 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
               scm.getClientProtocolServer().getScmInfo().getClusterId()));
       FileUtils.deleteDirectory(baseDir);
 
-      if (ksm != null) {
-        LOG.info("Shutting down the keySpaceManager");
-        ksm.stop();
-        ksm.join();
+      if (ozoneManager != null) {
+        LOG.info("Shutting down the OzoneManager");
+        ozoneManager.stop();
+        ozoneManager.join();
       }
 
       if (scm != null) {
@@ -291,11 +291,11 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
       initializeConfiguration();
       StorageContainerManager scm = createSCM();
       scm.start();
-      KeySpaceManager ksm = createKSM();
-      ksm.start();
+      OzoneManager om = createOM();
+      om.start();
       List<HddsDatanodeService> hddsDatanodes = createHddsDatanodes(scm);
       hddsDatanodes.forEach((datanode) -> datanode.start(null));
-      return new MiniOzoneClusterImpl(conf, ksm, scm, hddsDatanodes);
+      return new MiniOzoneClusterImpl(conf, om, scm, hddsDatanodes);
     }
 
     /**
@@ -331,20 +331,20 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
     }
 
     /**
-     * Creates a new KeySpaceManager instance.
+     * Creates a new OzoneManager instance.
      *
-     * @return {@link KeySpaceManager}
+     * @return {@link OzoneManager}
      *
      * @throws IOException
      */
-    private KeySpaceManager createKSM() throws IOException {
-      configureKSM();
-      KSMStorage ksmStore = new KSMStorage(conf);
-      ksmStore.setClusterId(clusterId);
-      ksmStore.setScmId(scmId.get());
-      ksmStore.setKsmId(ksmId.orElse(UUID.randomUUID().toString()));
-      ksmStore.initialize();
-      return KeySpaceManager.createKSM(null, conf);
+    private OzoneManager createOM() throws IOException {
+      configureOM();
+      OMStorage omStore = new OMStorage(conf);
+      omStore.setClusterId(clusterId);
+      omStore.setScmId(scmId.get());
+      omStore.setOmId(omId.orElse(UUID.randomUUID().toString()));
+      omStore.initialize();
+      return OzoneManager.createOm(null, conf);
     }
 
     /**
@@ -415,10 +415,10 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
     }
 
 
-    private void configureKSM() {
-      conf.set(KSMConfigKeys.OZONE_KSM_ADDRESS_KEY, "127.0.0.1:0");
-      conf.set(KSMConfigKeys.OZONE_KSM_HTTP_ADDRESS_KEY, "127.0.0.1:0");
-      conf.setInt(KSMConfigKeys.OZONE_KSM_HANDLER_COUNT_KEY, numOfKsmHandlers);
+    private void configureOM() {
+      conf.set(OMConfigKeys.OZONE_OM_ADDRESS_KEY, "127.0.0.1:0");
+      conf.set(OMConfigKeys.OZONE_OM_HTTP_ADDRESS_KEY, "127.0.0.1:0");
+      conf.setInt(OMConfigKeys.OZONE_OM_HANDLER_COUNT_KEY, numOfOmHandlers);
     }
 
     private void configureHddsDatanodes() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestOzoneConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestOzoneConfigurationFields.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestOzoneConfigurationFields.java
index 4898a1b..717bb68 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestOzoneConfigurationFields.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestOzoneConfigurationFields.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.ozone;
 
 import org.apache.hadoop.conf.TestConfigurationFieldsBase;
-import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 
 /**
@@ -31,7 +31,7 @@ public class TestOzoneConfigurationFields extends TestConfigurationFieldsBase {
     xmlFilename = new String("ozone-default.xml");
     configurationClasses =
         new Class[] {OzoneConfigKeys.class, ScmConfigKeys.class,
-            KSMConfigKeys.class};
+            OMConfigKeys.class};
     errorIfMissingConfigProps = true;
     errorIfMissingXmlProps = true;
     xmlPropsToSkipCompare.add("hadoop.tags.custom");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
index dd1a8de..cc367b3 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hdds.scm.server.SCMClientProtocolServer;
 import org.apache.hadoop.hdds.scm.server.SCMStorage;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.protocol.commands.DeleteBlocksCommand;
 import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@@ -60,8 +61,7 @@ import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
-import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 
 import org.junit.rules.Timeout;
@@ -211,7 +211,7 @@ public class TestStorageContainerManager {
       // Create {numKeys} random names keys.
       TestStorageContainerManagerHelper helper =
           new TestStorageContainerManagerHelper(cluster, conf);
-      Map<String, KsmKeyInfo> keyLocations = helper.createKeys(numKeys, 4096);
+      Map<String, OmKeyInfo> keyLocations = helper.createKeys(numKeys, 4096);
 
       Map<Long, List<Long>> containerBlocks = createDeleteTXLog(delLog,
           keyLocations, helper);
@@ -293,7 +293,7 @@ public class TestStorageContainerManager {
     // Create {numKeys} random names keys.
     TestStorageContainerManagerHelper helper =
         new TestStorageContainerManagerHelper(cluster, conf);
-    Map<String, KsmKeyInfo> keyLocations = helper.createKeys(numKeys, 4096);
+    Map<String, OmKeyInfo> keyLocations = helper.createKeys(numKeys, 4096);
 
     createDeleteTXLog(delLog, keyLocations, helper);
     // Verify a few TX gets created in the TX log.
@@ -320,13 +320,13 @@ public class TestStorageContainerManager {
   }
 
   private Map<Long, List<Long>> createDeleteTXLog(DeletedBlockLog delLog,
-      Map<String, KsmKeyInfo> keyLocations,
+      Map<String, OmKeyInfo> keyLocations,
       TestStorageContainerManagerHelper helper) throws IOException {
     // These keys will be written into a bunch of containers,
     // gets a set of container names, verify container containerBlocks
     // on datanodes.
     Set<Long> containerNames = new HashSet<>();
-    for (Map.Entry<String, KsmKeyInfo> entry : keyLocations.entrySet()) {
+    for (Map.Entry<String, OmKeyInfo> entry : keyLocations.entrySet()) {
       entry.getValue().getLatestVersionLocations().getLocationList()
           .forEach(loc -> containerNames.add(loc.getContainerID()));
     }
@@ -334,7 +334,7 @@ public class TestStorageContainerManager {
     // Total number of containerBlocks of these containers should be equal to
     // total number of containerBlocks via creation call.
     int totalCreatedBlocks = 0;
-    for (KsmKeyInfo info : keyLocations.values()) {
+    for (OmKeyInfo info : keyLocations.values()) {
       totalCreatedBlocks += info.getKeyLocationVersions().size();
     }
     Assert.assertTrue(totalCreatedBlocks > 0);
@@ -343,8 +343,8 @@ public class TestStorageContainerManager {
 
     // Create a deletion TX for each key.
     Map<Long, List<Long>> containerBlocks = Maps.newHashMap();
-    for (KsmKeyInfo info : keyLocations.values()) {
-      List<KsmKeyLocationInfo> list =
+    for (OmKeyInfo info : keyLocations.values()) {
+      List<OmKeyLocationInfo> list =
           info.getLatestVersionLocations().getLocationList();
       list.forEach(location -> {
         if (containerBlocks.containsKey(location.getContainerID())) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/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 4c2a904..a30c6f4 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
@@ -30,8 +30,8 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
-import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
-import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.web.handlers.BucketArgs;
 import org.apache.hadoop.ozone.web.handlers.KeyArgs;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
@@ -67,9 +67,9 @@ public class TestStorageContainerManagerHelper {
     storageHandler = new ObjectStoreHandler(conf).getStorageHandler();
   }
 
-  public Map<String, KsmKeyInfo> createKeys(int numOfKeys, int keySize)
+  public Map<String, OmKeyInfo> createKeys(int numOfKeys, int keySize)
       throws Exception {
-    Map<String, KsmKeyInfo> keyLocationMap = Maps.newHashMap();
+    Map<String, OmKeyInfo> keyLocationMap = Maps.newHashMap();
     String volume = "volume" + RandomStringUtils.randomNumeric(5);
     String bucket = "bucket" + RandomStringUtils.randomNumeric(5);
     String userName = "user" + RandomStringUtils.randomNumeric(5);
@@ -104,12 +104,12 @@ public class TestStorageContainerManagerHelper {
     }
 
     for (String key : keyNames) {
-      KsmKeyArgs arg = new KsmKeyArgs.Builder()
+      OmKeyArgs arg = new OmKeyArgs.Builder()
           .setVolumeName(volume)
           .setBucketName(bucket)
           .setKeyName(key)
           .build();
-      KsmKeyInfo location = cluster.getKeySpaceManager()
+      OmKeyInfo location = cluster.getOzoneManager()
           .lookupKey(arg);
       keyLocationMap.put(key, location);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/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 9918d63..0dc0399 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
@@ -77,10 +77,10 @@ public class TestOzoneRestClient {
         OzoneConsts.OZONE_HANDLER_DISTRIBUTED);
     cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
     cluster.waitForClusterToBeReady();
-    InetSocketAddress ksmHttpAddress = cluster.getKeySpaceManager()
+    InetSocketAddress omHttpAddress = cluster.getOzoneManager()
         .getHttpServer().getHttpAddress();
-    ozClient = OzoneClientFactory.getRestClient(ksmHttpAddress.getHostName(),
-        ksmHttpAddress.getPort(), conf);
+    ozClient = OzoneClientFactory.getRestClient(omHttpAddress.getHostName(),
+        omHttpAddress.getPort(), conf);
     store = ozClient.getObjectStore();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/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 214382e..2fbab36 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
@@ -39,10 +39,10 @@ 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.ksm.KeySpaceManager;
-import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
-import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
-import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.OzoneManager;
+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.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
@@ -73,7 +73,7 @@ public class TestOzoneRpcClient {
   private static MiniOzoneCluster cluster = null;
   private static OzoneClient ozClient = null;
   private static ObjectStore store = null;
-  private static KeySpaceManager keySpaceManager;
+  private static OzoneManager ozoneManager;
   private static StorageContainerLocationProtocolClientSideTranslatorPB
       storageContainerLocationClient;
 
@@ -97,7 +97,7 @@ public class TestOzoneRpcClient {
     store = ozClient.getObjectStore();
     storageContainerLocationClient =
         cluster.getStorageContainerLocationClient();
-    keySpaceManager = cluster.getKeySpaceManager();
+    ozoneManager = cluster.getOzoneManager();
   }
 
   @Test
@@ -376,7 +376,7 @@ public class TestOzoneRpcClient {
   private boolean verifyRatisReplication(String volumeName, String bucketName,
       String keyName, ReplicationType type, ReplicationFactor factor)
       throws IOException {
-    KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
+    OmKeyArgs keyArgs = new OmKeyArgs.Builder()
         .setVolumeName(volumeName)
         .setBucketName(bucketName)
         .setKeyName(keyName)
@@ -385,8 +385,8 @@ public class TestOzoneRpcClient {
         HddsProtos.ReplicationType.valueOf(type.toString());
     HddsProtos.ReplicationFactor replicationFactor =
         HddsProtos.ReplicationFactor.valueOf(factor.getValue());
-    KsmKeyInfo keyInfo = keySpaceManager.lookupKey(keyArgs);
-    for (KsmKeyLocationInfo info:
+    OmKeyInfo keyInfo = ozoneManager.lookupKey(keyArgs);
+    for (OmKeyLocationInfo info:
         keyInfo.getLatestVersionLocations().getLocationList()) {
       ContainerInfo container =
           storageContainerLocationClient.getContainer(info.getContainerID());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/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 43e3f50..62059ec 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
@@ -37,10 +37,10 @@ import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 import org.apache.hadoop.ozone.container.common.helpers.KeyUtils;
 import org.apache.hadoop.ozone.container.common.impl.ContainerManagerImpl;
-import org.apache.hadoop.ozone.ksm.KeySpaceManager;
-import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
-import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
-import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.ozShell.TestOzoneShell;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.utils.MetadataStore;
@@ -61,7 +61,7 @@ public class TestBlockDeletion {
   private static ObjectStore store;
   private static ContainerManagerImpl dnContainerManager = null;
   private static StorageContainerManager scm = null;
-  private static KeySpaceManager ksm = null;
+  private static OzoneManager om = null;
   private static Set<Long> containerIdsWithDeletedBlocks;
 
   @BeforeClass
@@ -88,7 +88,7 @@ public class TestBlockDeletion {
     dnContainerManager =
         (ContainerManagerImpl) cluster.getHddsDatanodes().get(0)
             .getDatanodeStateMachine().getContainer().getContainerManager();
-    ksm = cluster.getKeySpaceManager();
+    om = cluster.getOzoneManager();
     scm = cluster.getStorageContainerManager();
     containerIdsWithDeletedBlocks = new HashSet<>();
   }
@@ -112,23 +112,23 @@ public class TestBlockDeletion {
     out.write(value.getBytes());
     out.close();
 
-    KsmKeyArgs keyArgs = new KsmKeyArgs.Builder().setVolumeName(volumeName)
+    OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
         .setBucketName(bucketName).setKeyName(keyName).setDataSize(0)
         .setType(HddsProtos.ReplicationType.STAND_ALONE)
         .setFactor(HddsProtos.ReplicationFactor.ONE).build();
-    List<KsmKeyLocationInfoGroup> ksmKeyLocationInfoGroupList =
-        ksm.lookupKey(keyArgs).getKeyLocationVersions();
+    List<OmKeyLocationInfoGroup> omKeyLocationInfoGroupList =
+        om.lookupKey(keyArgs).getKeyLocationVersions();
 
     // verify key blocks were created in DN.
-    Assert.assertTrue(verifyBlocksCreated(ksmKeyLocationInfoGroupList));
+    Assert.assertTrue(verifyBlocksCreated(omKeyLocationInfoGroupList));
     // No containers with deleted blocks
     Assert.assertTrue(containerIdsWithDeletedBlocks.isEmpty());
     // Delete transactionIds for the containers should be 0
     matchContainerTransactionIds();
-    ksm.deleteKey(keyArgs);
+    om.deleteKey(keyArgs);
     Thread.sleep(5000);
     // The blocks should be deleted in the DN.
-    Assert.assertTrue(verifyBlocksDeleted(ksmKeyLocationInfoGroupList));
+    Assert.assertTrue(verifyBlocksDeleted(omKeyLocationInfoGroupList));
 
     // Few containers with deleted blocks
     Assert.assertTrue(!containerIdsWithDeletedBlocks.isEmpty());
@@ -155,7 +155,7 @@ public class TestBlockDeletion {
   }
 
   private boolean verifyBlocksCreated(
-      List<KsmKeyLocationInfoGroup> ksmKeyLocationInfoGroups)
+      List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups)
       throws IOException {
     return performOperationOnKeyContainers((blockID) -> {
       try {
@@ -166,11 +166,11 @@ public class TestBlockDeletion {
       } catch (IOException e) {
         e.printStackTrace();
       }
-    }, ksmKeyLocationInfoGroups);
+    }, omKeyLocationInfoGroups);
   }
 
   private boolean verifyBlocksDeleted(
-      List<KsmKeyLocationInfoGroup> ksmKeyLocationInfoGroups)
+      List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups)
       throws IOException {
     return performOperationOnKeyContainers((blockID) -> {
       try {
@@ -186,19 +186,20 @@ public class TestBlockDeletion {
       } catch (IOException e) {
         e.printStackTrace();
       }
-    }, ksmKeyLocationInfoGroups);
+    }, omKeyLocationInfoGroups);
   }
 
   private boolean performOperationOnKeyContainers(Consumer<BlockID> consumer,
-      List<KsmKeyLocationInfoGroup> ksmKeyLocationInfoGroups)
+      List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups)
       throws IOException {
 
     try {
-      for (KsmKeyLocationInfoGroup ksmKeyLocationInfoGroup : ksmKeyLocationInfoGroups) {
-        List<KsmKeyLocationInfo> ksmKeyLocationInfos =
-            ksmKeyLocationInfoGroup.getLocationList();
-        for (KsmKeyLocationInfo ksmKeyLocationInfo : ksmKeyLocationInfos) {
-          BlockID blockID = ksmKeyLocationInfo.getBlockID();
+      for (OmKeyLocationInfoGroup omKeyLocationInfoGroup :
+          omKeyLocationInfoGroups) {
+        List<OmKeyLocationInfo> omKeyLocationInfos =
+            omKeyLocationInfoGroup.getLocationList();
+        for (OmKeyLocationInfo omKeyLocationInfo : omKeyLocationInfos) {
+          BlockID blockID = omKeyLocationInfo.getBlockID();
           consumer.accept(blockID);
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/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 3e514e7..58b831b 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
@@ -34,8 +34,8 @@ import org.apache.hadoop.ozone.client.OzoneClientFactory;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
-import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
-import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.AfterClass;
@@ -45,7 +45,6 @@ import org.junit.Test;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Random;
 import java.util.concurrent.TimeoutException;
 
 public class TestCloseContainerByPipeline {
@@ -98,17 +97,17 @@ public class TestCloseContainerByPipeline {
     key.close();
 
     //get the name of a valid container
-    KsmKeyArgs keyArgs =
-        new KsmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
+    OmKeyArgs keyArgs =
+        new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
             .setType(HddsProtos.ReplicationType.STAND_ALONE)
             .setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
             .setKeyName("testCloseContainer").build();
 
-    KsmKeyLocationInfo ksmKeyLocationInfo =
-        cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions()
+    OmKeyLocationInfo omKeyLocationInfo =
+        cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
             .get(0).getBlocksLatestVersionOnly().get(0);
 
-    long containerID = ksmKeyLocationInfo.getContainerID();
+    long containerID = omKeyLocationInfo.getContainerID();
     List<DatanodeDetails> datanodes = cluster.getStorageContainerManager()
         .getScmContainerManager().getContainerWithPipeline(containerID)
         .getPipeline().getMachines();
@@ -153,17 +152,17 @@ public class TestCloseContainerByPipeline {
     key.close();
 
     //get the name of a valid container
-    KsmKeyArgs keyArgs =
-        new KsmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
+    OmKeyArgs keyArgs =
+        new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
             .setType(HddsProtos.ReplicationType.STAND_ALONE)
             .setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
             .setKeyName("standalone").build();
 
-    KsmKeyLocationInfo ksmKeyLocationInfo =
-        cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions()
+    OmKeyLocationInfo omKeyLocationInfo =
+        cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
             .get(0).getBlocksLatestVersionOnly().get(0);
 
-    long containerID = ksmKeyLocationInfo.getContainerID();
+    long containerID = omKeyLocationInfo.getContainerID();
     List<DatanodeDetails> datanodes = cluster.getStorageContainerManager()
         .getScmContainerManager().getContainerWithPipeline(containerID)
         .getPipeline().getMachines();
@@ -207,16 +206,16 @@ public class TestCloseContainerByPipeline {
     key.close();
 
     //get the name of a valid container
-    KsmKeyArgs keyArgs = new KsmKeyArgs.Builder().setVolumeName("test").
+    OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName("test").
         setBucketName("test").setType(HddsProtos.ReplicationType.RATIS)
         .setFactor(HddsProtos.ReplicationFactor.THREE).setDataSize(1024)
         .setKeyName("ratis").build();
 
-    KsmKeyLocationInfo ksmKeyLocationInfo =
-        cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions()
+    OmKeyLocationInfo omKeyLocationInfo =
+        cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
             .get(0).getBlocksLatestVersionOnly().get(0);
 
-    long containerID = ksmKeyLocationInfo.getContainerID();
+    long containerID = omKeyLocationInfo.getContainerID();
     List<DatanodeDetails> datanodes = cluster.getStorageContainerManager()
         .getScmContainerManager().getContainerWithPipeline(containerID)
         .getPipeline().getMachines();
@@ -232,7 +231,7 @@ public class TestCloseContainerByPipeline {
           .addDatanodeCommand(details.getUuid(),
               new CloseContainerCommand(containerID,
                   HddsProtos.ReplicationType.RATIS));
-  }
+    }
 
     for (DatanodeDetails datanodeDetails : datanodes) {
       GenericTestUtils.waitFor(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
index efb7344..58a5154 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerHandler.java
@@ -28,8 +28,8 @@ import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.client.rest.OzoneException;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
-import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
-import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB;
@@ -69,17 +69,17 @@ public class TestCloseContainerHandler {
     key.close();
 
     //get the name of a valid container
-    KsmKeyArgs keyArgs =
-        new KsmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
+    OmKeyArgs keyArgs =
+        new OmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
             .setType(HddsProtos.ReplicationType.STAND_ALONE)
             .setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
             .setKeyName("test").build();
 
-    KsmKeyLocationInfo ksmKeyLocationInfo =
-        cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions()
+    OmKeyLocationInfo omKeyLocationInfo =
+        cluster.getOzoneManager().lookupKey(keyArgs).getKeyLocationVersions()
             .get(0).getBlocksLatestVersionOnly().get(0);
 
-    long containerID = ksmKeyLocationInfo.getContainerID();
+    long containerID = omKeyLocationInfo.getContainerID();
 
     Assert.assertFalse(isContainerClosed(cluster, containerID));
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java
deleted file mode 100644
index 1cc7ff8..0000000
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ksm/TestContainerReportWithKeys.java
+++ /dev/null
@@ -1,143 +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.ozone.ksm;
-
-import org.apache.commons.lang3.RandomStringUtils;
-
-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.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.helpers.ContainerData;
-import org.apache.hadoop.ozone.container.common.interfaces.ContainerManager;
-import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
-import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
-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;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-/**
- * This class tests container report with DN container state info.
- */
-public class TestContainerReportWithKeys {
-  private static final Logger LOG = LoggerFactory.getLogger(
-      TestContainerReportWithKeys.class);
-  private static MiniOzoneCluster cluster = null;
-  private static OzoneConfiguration conf;
-  private static StorageContainerManager scm;
-
-  @Rule
-  public ExpectedException exception = ExpectedException.none();
-
-  /**
-   * 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);
-    cluster = MiniOzoneCluster.newBuilder(conf).build();
-    cluster.waitForClusterToBeReady();
-    scm = cluster.getStorageContainerManager();
-  }
-
-  /**
-   * Shutdown MiniDFSCluster.
-   */
-  @AfterClass
-  public static void shutdown() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  @Test
-  public void testContainerReportKeyWrite() throws Exception {
-    final String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
-    final String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
-    final String keyName = "key" + RandomStringUtils.randomNumeric(5);
-    final int keySize = 100;
-
-    OzoneClient client = OzoneClientFactory.getClient(conf);
-    ObjectStore objectStore = client.getObjectStore();
-    objectStore.createVolume(volumeName);
-    objectStore.getVolume(volumeName).createBucket(bucketName);
-    OzoneOutputStream key =
-        objectStore.getVolume(volumeName).getBucket(bucketName)
-            .createKey(keyName, keySize, ReplicationType.STAND_ALONE,
-                ReplicationFactor.ONE);
-    String dataString = RandomStringUtils.randomAlphabetic(keySize);
-    key.write(dataString.getBytes());
-    key.close();
-
-    KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
-        .setVolumeName(volumeName)
-        .setBucketName(bucketName)
-        .setKeyName(keyName)
-        .setType(HddsProtos.ReplicationType.STAND_ALONE)
-        .setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(keySize)
-        .build();
-
-
-    KsmKeyLocationInfo keyInfo =
-        cluster.getKeySpaceManager().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());
-
-    LOG.info("SCM Container Info keyCount: {} usedBytes: {}",
-        cinfo.getNumberOfKeys(), cinfo.getUsedBytes());
-  }
-
-
-  private static ContainerData getContainerData(long containerID) {
-    ContainerData containerData;
-    try {
-      ContainerManager containerManager = cluster.getHddsDatanodes().get(0)
-          .getDatanodeStateMachine().getContainer().getContainerManager();
-      containerData = containerManager.readContainer(containerID);
-    } catch (StorageContainerException e) {
-      throw new AssertionError(e);
-    }
-    return containerData;
-  }
-}
\ 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