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 bh...@apache.org on 2018/07/09 20:20:00 UTC

[24/50] [abbrv] 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/ksm/helpers/KsmVolumeArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmVolumeArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmVolumeArgs.java
deleted file mode 100644
index 6b42c27..0000000
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/KsmVolumeArgs.java
+++ /dev/null
@@ -1,223 +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.helpers;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.VolumeInfo;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.KeyValue;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-
-/**
- * A class that encapsulates the KsmVolumeArgs Args.
- */
-public final class KsmVolumeArgs {
-  private final String adminName;
-  private final String ownerName;
-  private final String volume;
-  private final long creationTime;
-  private final long quotaInBytes;
-  private final Map<String, String> keyValueMap;
-  private final KsmOzoneAclMap aclMap;
-
-  /**
-   * Private constructor, constructed via builder.
-   * @param adminName  - Administrator's name.
-   * @param ownerName  - Volume owner's name
-   * @param volume - volume name
-   * @param quotaInBytes - Volume Quota in bytes.
-   * @param keyValueMap - keyValue map.
-   * @param aclMap - User to access rights map.
-   * @param creationTime - Volume creation time.
-   */
-  private KsmVolumeArgs(String adminName, String ownerName, String volume,
-                        long quotaInBytes, Map<String, String> keyValueMap,
-                        KsmOzoneAclMap aclMap, long creationTime) {
-    this.adminName = adminName;
-    this.ownerName = ownerName;
-    this.volume = volume;
-    this.quotaInBytes = quotaInBytes;
-    this.keyValueMap = keyValueMap;
-    this.aclMap = aclMap;
-    this.creationTime = creationTime;
-  }
-
-  /**
-   * Returns the Admin Name.
-   * @return String.
-   */
-  public String getAdminName() {
-    return adminName;
-  }
-
-  /**
-   * Returns the owner Name.
-   * @return String
-   */
-  public String getOwnerName() {
-    return ownerName;
-  }
-
-  /**
-   * Returns the volume Name.
-   * @return String
-   */
-  public String getVolume() {
-    return volume;
-  }
-
-  /**
-   * Returns creation time.
-   * @return long
-   */
-  public long getCreationTime() {
-    return creationTime;
-  }
-
-  /**
-   * Returns Quota in Bytes.
-   * @return long, Quota in bytes.
-   */
-  public long getQuotaInBytes() {
-    return quotaInBytes;
-  }
-
-  public Map<String, String> getKeyValueMap() {
-    return keyValueMap;
-  }
-
-  public KsmOzoneAclMap getAclMap() {
-    return aclMap;
-  }
-  /**
-   * Returns new builder class that builds a KsmVolumeArgs.
-   *
-   * @return Builder
-   */
-  public static Builder newBuilder() {
-    return new Builder();
-  }
-
-  /**
-   * Builder for KsmVolumeArgs.
-   */
-  public static class Builder {
-    private String adminName;
-    private String ownerName;
-    private String volume;
-    private long creationTime;
-    private long quotaInBytes;
-    private Map<String, String> keyValueMap;
-    private KsmOzoneAclMap aclMap;
-
-    /**
-     * Constructs a builder.
-     */
-    Builder() {
-      keyValueMap = new HashMap<>();
-      aclMap = new KsmOzoneAclMap();
-    }
-
-    public Builder setAdminName(String admin) {
-      this.adminName = admin;
-      return this;
-    }
-
-    public Builder setOwnerName(String owner) {
-      this.ownerName = owner;
-      return this;
-    }
-
-    public Builder setVolume(String volumeName) {
-      this.volume = volumeName;
-      return this;
-    }
-
-    public Builder setCreationTime(long createdOn) {
-      this.creationTime = createdOn;
-      return this;
-    }
-
-    public Builder setQuotaInBytes(long quota) {
-      this.quotaInBytes = quota;
-      return this;
-    }
-
-    public Builder addMetadata(String key, String value) {
-      keyValueMap.put(key, value); // overwrite if present.
-      return this;
-    }
-
-    public Builder addOzoneAcls(OzoneAclInfo acl) throws IOException {
-      aclMap.addAcl(acl);
-      return this;
-    }
-
-    /**
-     * Constructs a CreateVolumeArgument.
-     * @return CreateVolumeArgs.
-     */
-    public KsmVolumeArgs build() {
-      Preconditions.checkNotNull(adminName);
-      Preconditions.checkNotNull(ownerName);
-      Preconditions.checkNotNull(volume);
-      return new KsmVolumeArgs(adminName, ownerName, volume, quotaInBytes,
-          keyValueMap, aclMap, creationTime);
-    }
-  }
-
-  public VolumeInfo getProtobuf() {
-    List<KeyValue> metadataList = new LinkedList<>();
-    for (Map.Entry<String, String> entry : keyValueMap.entrySet()) {
-      metadataList.add(KeyValue.newBuilder().setKey(entry.getKey()).
-          setValue(entry.getValue()).build());
-    }
-    List<OzoneAclInfo> aclList = aclMap.ozoneAclGetProtobuf();
-
-    return VolumeInfo.newBuilder()
-        .setAdminName(adminName)
-        .setOwnerName(ownerName)
-        .setVolume(volume)
-        .setQuotaInBytes(quotaInBytes)
-        .addAllMetadata(metadataList)
-        .addAllVolumeAcls(aclList)
-        .setCreationTime(creationTime)
-        .build();
-  }
-
-  public static KsmVolumeArgs getFromProtobuf(VolumeInfo volInfo) {
-    Map<String, String> kvMap = volInfo.getMetadataList().stream()
-        .collect(Collectors.toMap(KeyValue::getKey,
-            KeyValue::getValue));
-    KsmOzoneAclMap aclMap =
-        KsmOzoneAclMap.ozoneAclGetFromProtobuf(volInfo.getVolumeAclsList());
-
-    return new KsmVolumeArgs(volInfo.getAdminName(), volInfo.getOwnerName(),
-        volInfo.getVolume(), volInfo.getQuotaInBytes(), kvMap, aclMap,
-        volInfo.getCreationTime());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/OpenKeySession.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/OpenKeySession.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/OpenKeySession.java
deleted file mode 100644
index c19c04b..0000000
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/OpenKeySession.java
+++ /dev/null
@@ -1,50 +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.ksm.helpers;
-
-/**
- * This class represents a open key "session". A session here means a key is
- * opened by a specific client, the client sends the handler to server, such
- * that servers can recognize this client, and thus know how to close the key.
- */
-public class OpenKeySession {
-  private final int id;
-  private final KsmKeyInfo keyInfo;
-  // the version of the key when it is being opened in this session.
-  // a block that has a create version equals to open version means it will
-  // be committed only when this open session is closed.
-  private long openVersion;
-
-  public OpenKeySession(int id, KsmKeyInfo info, long version) {
-    this.id = id;
-    this.keyInfo = info;
-    this.openVersion = version;
-  }
-
-  public long getOpenVersion() {
-    return this.openVersion;
-  }
-
-  public KsmKeyInfo getKeyInfo() {
-    return keyInfo;
-  }
-
-  public int getId() {
-    return id;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/ServiceInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/ServiceInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/ServiceInfo.java
deleted file mode 100644
index e07232d..0000000
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/ServiceInfo.java
+++ /dev/null
@@ -1,237 +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.helpers;
-
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.ObjectReader;
-import com.fasterxml.jackson.databind.ObjectWriter;
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.ozone.client.rest.response.BucketInfo;
-import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos;
-import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos
-    .ServicePort;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-/**
- * ServiceInfo holds the config details of Ozone services.
- */
-public final class ServiceInfo {
-
-  private static final ObjectReader READER =
-      new ObjectMapper().readerFor(ServiceInfo.class);
-  private static final ObjectWriter WRITER =
-      new ObjectMapper().writerWithDefaultPrettyPrinter();
-
-  /**
-   * Type of node/service.
-   */
-  private NodeType nodeType;
-  /**
-   * Hostname of the node in which the service is running.
-   */
-  private String hostname;
-
-  /**
-   * List of ports the service listens to.
-   */
-  private Map<ServicePort.Type, Integer> ports;
-
-  /**
-   * Default constructor for JSON deserialization.
-   */
-  public ServiceInfo() {}
-
-  /**
-   * Constructs the ServiceInfo for the {@code nodeType}.
-   * @param nodeType type of node/service
-   * @param hostname hostname of the service
-   * @param portList list of ports the service listens to
-   */
-  private ServiceInfo(
-      NodeType nodeType, String hostname, List<ServicePort> portList) {
-    Preconditions.checkNotNull(nodeType);
-    Preconditions.checkNotNull(hostname);
-    this.nodeType = nodeType;
-    this.hostname = hostname;
-    this.ports = new HashMap<>();
-    for (ServicePort port : portList) {
-      ports.put(port.getType(), port.getValue());
-    }
-  }
-
-  /**
-   * Returns the type of node/service.
-   * @return node type
-   */
-  public NodeType getNodeType() {
-    return nodeType;
-  }
-
-  /**
-   * Returns the hostname of the service.
-   * @return hostname
-   */
-  public String getHostname() {
-    return hostname;
-  }
-
-  /**
-   * Returns ServicePort.Type to port mappings.
-   * @return ports
-   */
-  public Map<ServicePort.Type, Integer> getPorts() {
-    return ports;
-  }
-
-  /**
-   * Returns the port for given type, null if the service doesn't support
-   * the type.
-   *
-   * @param type the type of port.
-   *             ex: RPC, HTTP, HTTPS, etc..
-   */
-  @JsonIgnore
-  public int getPort(ServicePort.Type type) {
-    return ports.get(type);
-  }
-
-  /**
-   * Converts {@link ServiceInfo} to KeySpaceManagerProtocolProtos.ServiceInfo.
-   *
-   * @return KeySpaceManagerProtocolProtos.ServiceInfo
-   */
-  @JsonIgnore
-  public KeySpaceManagerProtocolProtos.ServiceInfo getProtobuf() {
-    KeySpaceManagerProtocolProtos.ServiceInfo.Builder builder =
-        KeySpaceManagerProtocolProtos.ServiceInfo.newBuilder();
-    builder.setNodeType(nodeType)
-        .setHostname(hostname)
-        .addAllServicePorts(
-            ports.entrySet().stream()
-                .map(
-                    entry ->
-                        ServicePort.newBuilder()
-                            .setType(entry.getKey())
-                            .setValue(entry.getValue()).build())
-                .collect(Collectors.toList()));
-    return builder.build();
-  }
-
-  /**
-   * Converts KeySpaceManagerProtocolProtos.ServiceInfo to {@link ServiceInfo}.
-   *
-   * @return {@link ServiceInfo}
-   */
-  @JsonIgnore
-  public static ServiceInfo getFromProtobuf(
-      KeySpaceManagerProtocolProtos.ServiceInfo serviceInfo) {
-    return new ServiceInfo(serviceInfo.getNodeType(),
-        serviceInfo.getHostname(),
-        serviceInfo.getServicePortsList());
-  }
-
-  /**
-   * Returns a JSON string of this object.
-   *
-   * @return String - json string
-   * @throws IOException
-   */
-  public String toJsonString() throws IOException {
-    return WRITER.writeValueAsString(this);
-  }
-
-  /**
-   * Parse a JSON string into ServiceInfo Object.
-   *
-   * @param jsonString Json String
-   * @return BucketInfo
-   * @throws IOException
-   */
-  public static BucketInfo parse(String jsonString) throws IOException {
-    return READER.readValue(jsonString);
-  }
-
-  /**
-   * Creates a new builder to build {@link ServiceInfo}.
-   * @return {@link ServiceInfo.Builder}
-   */
-  public static Builder newBuilder() {
-    return new Builder();
-  }
-
-  /**
-   * Builder used to build/construct {@link ServiceInfo}.
-   */
-  public static class Builder {
-
-    private NodeType node;
-    private String host;
-    private List<ServicePort> portList = new ArrayList<>();
-
-
-    /**
-     * Sets the node/service type.
-     * @param nodeType type of node
-     * @return the builder
-     */
-    public Builder setNodeType(NodeType nodeType) {
-      node = nodeType;
-      return this;
-    }
-
-    /**
-     * Sets the hostname of the service.
-     * @param hostname service hostname
-     * @return the builder
-     */
-    public Builder setHostname(String hostname) {
-      host = hostname;
-      return this;
-    }
-
-    /**
-     * Adds the service port to the service port list.
-     * @param servicePort RPC port
-     * @return the builder
-     */
-    public Builder addServicePort(ServicePort servicePort) {
-      portList.add(servicePort);
-      return this;
-    }
-
-
-    /**
-     * Builds and returns {@link ServiceInfo} with the set values.
-     * @return {@link ServiceInfo}
-     */
-    public ServiceInfo build() {
-      return new ServiceInfo(node, host, portList);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/VolumeArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/VolumeArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/VolumeArgs.java
deleted file mode 100644
index 1a3d486..0000000
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/VolumeArgs.java
+++ /dev/null
@@ -1,140 +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.helpers;
-
-import com.google.common.base.Preconditions;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * A class that encapsulates the createVolume Args.
- */
-public final class VolumeArgs {
-  private final String adminName;
-  private final String ownerName;
-  private final String volume;
-  private final long quotaInBytes;
-  private final Map<String, String> extendedAttributes;
-
-  /**
-   * Private constructor, constructed via builder.
-   *
-   * @param adminName - Administrator name.
-   * @param ownerName - Volume owner's name
-   * @param volume - volume name
-   * @param quotaInBytes - Volume Quota in bytes.
-   * @param keyValueMap - keyValue map.
-   */
-  private VolumeArgs(String adminName, String ownerName, String volume,
-      long quotaInBytes, Map<String, String> keyValueMap) {
-    this.adminName = adminName;
-    this.ownerName = ownerName;
-    this.volume = volume;
-    this.quotaInBytes = quotaInBytes;
-    this.extendedAttributes = keyValueMap;
-  }
-
-  /**
-   * Returns the Admin Name.
-   *
-   * @return String.
-   */
-  public String getAdminName() {
-    return adminName;
-  }
-
-  /**
-   * Returns the owner Name.
-   *
-   * @return String
-   */
-  public String getOwnerName() {
-    return ownerName;
-  }
-
-  /**
-   * Returns the volume Name.
-   *
-   * @return String
-   */
-  public String getVolume() {
-    return volume;
-  }
-
-  /**
-   * Returns Quota in Bytes.
-   *
-   * @return long, Quota in bytes.
-   */
-  public long getQuotaInBytes() {
-    return quotaInBytes;
-  }
-
-  public Map<String, String> getExtendedAttributes() {
-    return extendedAttributes;
-  }
-
-  static class Builder {
-    private String adminName;
-    private String ownerName;
-    private String volume;
-    private long quotaInBytes;
-    private Map<String, String> extendedAttributes;
-
-    /**
-     * Constructs a builder.
-     */
-    Builder() {
-      extendedAttributes = new HashMap<>();
-    }
-
-    public void setAdminName(String adminName) {
-      this.adminName = adminName;
-    }
-
-    public void setOwnerName(String ownerName) {
-      this.ownerName = ownerName;
-    }
-
-    public void setVolume(String volume) {
-      this.volume = volume;
-    }
-
-    public void setQuotaInBytes(long quotaInBytes) {
-      this.quotaInBytes = quotaInBytes;
-    }
-
-    public void addMetadata(String key, String value) {
-      extendedAttributes.put(key, value); // overwrite if present.
-    }
-
-    /**
-     * Constructs a CreateVolumeArgument.
-     *
-     * @return CreateVolumeArgs.
-     */
-    public VolumeArgs build() {
-      Preconditions.checkNotNull(adminName);
-      Preconditions.checkNotNull(ownerName);
-      Preconditions.checkNotNull(volume);
-      return new VolumeArgs(adminName, ownerName, volume, quotaInBytes,
-          extendedAttributes);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/package-info.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/package-info.java
deleted file mode 100644
index ce627a5..0000000
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/helpers/package-info.java
+++ /dev/null
@@ -1,18 +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.ksm.helpers;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/package-info.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/package-info.java
deleted file mode 100644
index 7698ee1..0000000
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/package-info.java
+++ /dev/null
@@ -1,21 +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.ksm;
-/**
- This package contains client side protocol library to communicate with KSM.
- */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocol/KeySpaceManagerProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocol/KeySpaceManagerProtocol.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocol/KeySpaceManagerProtocol.java
deleted file mode 100644
index 54862d3..0000000
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocol/KeySpaceManagerProtocol.java
+++ /dev/null
@@ -1,252 +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.protocol;
-
-import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
-import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
-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.ksm.helpers.KsmVolumeArgs;
-import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession;
-import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
-import java.io.IOException;
-import java.util.List;
-
-/**
- * Protocol to talk to KSM.
- */
-public interface KeySpaceManagerProtocol {
-
-  /**
-   * Creates a volume.
-   * @param args - Arguments to create Volume.
-   * @throws IOException
-   */
-  void createVolume(KsmVolumeArgs args) throws IOException;
-
-  /**
-   * Changes the owner of a volume.
-   * @param volume  - Name of the volume.
-   * @param owner - Name of the owner.
-   * @throws IOException
-   */
-  void setOwner(String volume, String owner) throws IOException;
-
-  /**
-   * Changes the Quota on a volume.
-   * @param volume - Name of the volume.
-   * @param quota - Quota in bytes.
-   * @throws IOException
-   */
-  void setQuota(String volume, long quota) throws IOException;
-
-  /**
-   * Checks if the specified user can access this volume.
-   * @param volume - volume
-   * @param userAcl - user acls which needs to be checked for access
-   * @return true if the user has required access for the volume,
-   *         false otherwise
-   * @throws IOException
-   */
-  boolean checkVolumeAccess(String volume, OzoneAclInfo userAcl)
-      throws IOException;
-
-  /**
-   * Gets the volume information.
-   * @param volume - Volume name.
-   * @return VolumeArgs or exception is thrown.
-   * @throws IOException
-   */
-  KsmVolumeArgs getVolumeInfo(String volume) throws IOException;
-
-  /**
-   * Deletes an existing empty volume.
-   * @param volume - Name of the volume.
-   * @throws IOException
-   */
-  void deleteVolume(String volume) throws IOException;
-
-  /**
-   * Lists volume owned by a specific user.
-   * @param userName - user name
-   * @param prefix  - Filter prefix -- Return only entries that match this.
-   * @param prevKey - Previous key -- List starts from the next from the prevkey
-   * @param maxKeys - Max number of keys to return.
-   * @return List of Volumes.
-   * @throws IOException
-   */
-  List<KsmVolumeArgs> listVolumeByUser(String userName, String prefix, String
-      prevKey, int maxKeys) throws IOException;
-
-  /**
-   * Lists volume all volumes in the cluster.
-   * @param prefix  - Filter prefix -- Return only entries that match this.
-   * @param prevKey - Previous key -- List starts from the next from the prevkey
-   * @param maxKeys - Max number of keys to return.
-   * @return List of Volumes.
-   * @throws IOException
-   */
-  List<KsmVolumeArgs> listAllVolumes(String prefix, String
-      prevKey, int maxKeys) throws IOException;
-
-  /**
-   * Creates a bucket.
-   * @param bucketInfo - BucketInfo to create Bucket.
-   * @throws IOException
-   */
-  void createBucket(KsmBucketInfo bucketInfo) throws IOException;
-
-  /**
-   * Gets the bucket information.
-   * @param volumeName - Volume name.
-   * @param bucketName - Bucket name.
-   * @return KsmBucketInfo or exception is thrown.
-   * @throws IOException
-   */
-  KsmBucketInfo getBucketInfo(String volumeName, String bucketName)
-      throws IOException;
-
-  /**
-   * Sets bucket property from args.
-   * @param args - BucketArgs.
-   * @throws IOException
-   */
-  void setBucketProperty(KsmBucketArgs args) throws IOException;
-
-  /**
-   * Open the given key and return an open key session.
-   *
-   * @param args the args of the key.
-   * @return OpenKeySession instance that client uses to talk to container.
-   * @throws IOException
-   */
-  OpenKeySession openKey(KsmKeyArgs args) throws IOException;
-
-  /**
-   * Commit a key. This will make the change from the client visible. The client
-   * is identified by the clientID.
-   *
-   * @param args the key to commit
-   * @param clientID the client identification
-   * @throws IOException
-   */
-  void commitKey(KsmKeyArgs args, int clientID) throws IOException;
-
-  /**
-   * Allocate a new block, it is assumed that the client is having an open key
-   * session going on. This block will be appended to this open key session.
-   *
-   * @param args the key to append
-   * @param clientID the client identification
-   * @return an allocated block
-   * @throws IOException
-   */
-  KsmKeyLocationInfo allocateBlock(KsmKeyArgs args, int clientID)
-      throws IOException;
-
-  /**
-   * Look up for the container of an existing key.
-   *
-   * @param args the args of the key.
-   * @return KsmKeyInfo instance that client uses to talk to container.
-   * @throws IOException
-   */
-  KsmKeyInfo lookupKey(KsmKeyArgs args) throws IOException;
-
-  /**
-   * Rename an existing key within a bucket
-   * @param args the args of the key.
-   * @param toKeyName New name to be used for the Key
-   */
-  void renameKey(KsmKeyArgs args, String toKeyName) throws IOException;
-
-  /**
-   * Deletes an existing key.
-   *
-   * @param args the args of the key.
-   * @throws IOException
-   */
-  void deleteKey(KsmKeyArgs args) throws IOException;
-
-  /**
-   * Deletes an existing empty bucket from volume.
-   * @param volume - Name of the volume.
-   * @param bucket - Name of the bucket.
-   * @throws IOException
-   */
-  void deleteBucket(String volume, String bucket) throws IOException;
-
-  /**
-   * Returns a list of buckets represented by {@link KsmBucketInfo}
-   * in the given volume. Argument volumeName is required, others
-   * are optional.
-   *
-   * @param volumeName
-   *   the name of the volume.
-   * @param startBucketName
-   *   the start bucket name, only the buckets whose name is
-   *   after this value will be included in the result.
-   * @param bucketPrefix
-   *   bucket name prefix, only the buckets whose name has
-   *   this prefix will be included in the result.
-   * @param maxNumOfBuckets
-   *   the maximum number of buckets to return. It ensures
-   *   the size of the result will not exceed this limit.
-   * @return a list of buckets.
-   * @throws IOException
-   */
-  List<KsmBucketInfo> listBuckets(String volumeName,
-      String startBucketName, String bucketPrefix, int maxNumOfBuckets)
-      throws IOException;
-
-  /**
-   * Returns a list of keys represented by {@link KsmKeyInfo}
-   * in the given bucket. Argument volumeName, bucketName is required,
-   * others are optional.
-   *
-   * @param volumeName
-   *   the name of the volume.
-   * @param bucketName
-   *   the name of the bucket.
-   * @param startKeyName
-   *   the start key name, only the keys whose name is
-   *   after this value will be included in the result.
-   * @param keyPrefix
-   *   key name prefix, only the keys whose name has
-   *   this prefix will be included in the result.
-   * @param maxKeys
-   *   the maximum number of keys to return. It ensures
-   *   the size of the result will not exceed this limit.
-   * @return a list of keys.
-   * @throws IOException
-   */
-  List<KsmKeyInfo> listKeys(String volumeName,
-      String bucketName, String startKeyName, String keyPrefix, int maxKeys)
-      throws IOException;
-
-  /**
-   * Returns list of Ozone services with its configuration details.
-   *
-   * @return list of Ozone services
-   * @throws IOException
-   */
-  List<ServiceInfo> getServiceList() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocol/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocol/package-info.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocol/package-info.java
deleted file mode 100644
index f77e5fd..0000000
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocol/package-info.java
+++ /dev/null
@@ -1,19 +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.ksm.protocol;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java
deleted file mode 100644
index 0f38169..0000000
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolClientSideTranslatorPB.java
+++ /dev/null
@@ -1,769 +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.protocolPB;
-
-import com.google.common.base.Strings;
-import com.google.common.collect.Lists;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.ipc.ProtobufHelper;
-import org.apache.hadoop.ipc.ProtocolTranslator;
-import org.apache.hadoop.ozone.ksm.helpers.KsmBucketArgs;
-import org.apache.hadoop.ozone.ksm.helpers.KsmBucketInfo;
-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.ksm.helpers.KsmVolumeArgs;
-import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession;
-import org.apache.hadoop.ozone.ksm.helpers.ServiceInfo;
-import org.apache.hadoop.ozone.ksm.protocol.KeySpaceManagerProtocol;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.AllocateBlockRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.AllocateBlockResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.CommitKeyRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.CommitKeyResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.BucketArgs;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.BucketInfo;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.CreateBucketRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.CreateBucketResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.InfoBucketRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.InfoBucketResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.SetBucketPropertyRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.SetBucketPropertyResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.DeleteBucketRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.DeleteBucketResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.CreateVolumeRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.CreateVolumeResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.LocateKeyRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.LocateKeyResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.RenameKeyRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.RenameKeyResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.KeyArgs;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.SetVolumePropertyRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.SetVolumePropertyResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.DeleteVolumeRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.DeleteVolumeResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.InfoVolumeRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.InfoVolumeResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.CheckVolumeAccessRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.CheckVolumeAccessResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.ListBucketsRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.ListBucketsResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.ListKeysRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.ListKeysResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.VolumeInfo;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.Status;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.OzoneAclInfo;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.ListVolumeRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.ListVolumeResponse;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.ServiceListRequest;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.ServiceListResponse;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.stream.Collectors;
-
-/**
- *  The client side implementation of KeySpaceManagerProtocol.
- */
-
-@InterfaceAudience.Private
-public final class KeySpaceManagerProtocolClientSideTranslatorPB
-    implements KeySpaceManagerProtocol, ProtocolTranslator, Closeable {
-
-  /**
-   * RpcController is not used and hence is set to null.
-   */
-  private static final RpcController NULL_RPC_CONTROLLER = null;
-
-  private final KeySpaceManagerProtocolPB rpcProxy;
-
-  /**
-   * Constructor for KeySpaceManger Client.
-   * @param rpcProxy
-   */
-  public KeySpaceManagerProtocolClientSideTranslatorPB(
-      KeySpaceManagerProtocolPB rpcProxy) {
-    this.rpcProxy = rpcProxy;
-  }
-
-  /**
-   * Closes this stream and releases any system resources associated
-   * with it. If the stream is already closed then invoking this
-   * method has no effect.
-   * <p>
-   * <p> As noted in {@link AutoCloseable#close()}, cases where the
-   * close may fail require careful attention. It is strongly advised
-   * to relinquish the underlying resources and to internally
-   * <em>mark</em> the {@code Closeable} as closed, prior to throwing
-   * the {@code IOException}.
-   *
-   * @throws IOException if an I/O error occurs
-   */
-  @Override
-  public void close() throws IOException {
-
-  }
-
-  /**
-   * Creates a volume.
-   *
-   * @param args - Arguments to create Volume.
-   * @throws IOException
-   */
-  @Override
-  public void createVolume(KsmVolumeArgs args) throws IOException {
-    CreateVolumeRequest.Builder req =
-        CreateVolumeRequest.newBuilder();
-    VolumeInfo volumeInfo = args.getProtobuf();
-    req.setVolumeInfo(volumeInfo);
-
-    final CreateVolumeResponse resp;
-    try {
-      resp = rpcProxy.createVolume(NULL_RPC_CONTROLLER,
-          req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-
-    if (resp.getStatus() != Status.OK) {
-      throw new
-          IOException("Volume creation failed, error:" + resp.getStatus());
-    }
-  }
-
-  /**
-   * Changes the owner of a volume.
-   *
-   * @param volume - Name of the volume.
-   * @param owner - Name of the owner.
-   * @throws IOException
-   */
-  @Override
-  public void setOwner(String volume, String owner) throws IOException {
-    SetVolumePropertyRequest.Builder req =
-        SetVolumePropertyRequest.newBuilder();
-    req.setVolumeName(volume).setOwnerName(owner);
-    final SetVolumePropertyResponse resp;
-    try {
-      resp = rpcProxy.setVolumeProperty(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new
-          IOException("Volume owner change failed, error:" + resp.getStatus());
-    }
-  }
-
-  /**
-   * Changes the Quota on a volume.
-   *
-   * @param volume - Name of the volume.
-   * @param quota - Quota in bytes.
-   * @throws IOException
-   */
-  @Override
-  public void setQuota(String volume, long quota) throws IOException {
-    SetVolumePropertyRequest.Builder req =
-        SetVolumePropertyRequest.newBuilder();
-    req.setVolumeName(volume).setQuotaInBytes(quota);
-    final SetVolumePropertyResponse resp;
-    try {
-      resp = rpcProxy.setVolumeProperty(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new
-          IOException("Volume quota change failed, error:" + resp.getStatus());
-    }
-  }
-
-  /**
-   * Checks if the specified user can access this volume.
-   *
-   * @param volume - volume
-   * @param userAcl - user acls which needs to be checked for access
-   * @return true if the user has required access for the volume,
-   *         false otherwise
-   * @throws IOException
-   */
-  @Override
-  public boolean checkVolumeAccess(String volume, OzoneAclInfo userAcl) throws
-      IOException {
-    CheckVolumeAccessRequest.Builder req =
-        CheckVolumeAccessRequest.newBuilder();
-    req.setVolumeName(volume).setUserAcl(userAcl);
-    final CheckVolumeAccessResponse resp;
-    try {
-      resp = rpcProxy.checkVolumeAccess(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-
-    if (resp.getStatus() == Status.ACCESS_DENIED) {
-      return false;
-    } else if (resp.getStatus() == Status.OK) {
-      return true;
-    } else {
-      throw new
-          IOException("Check Volume Access failed, error:" + resp.getStatus());
-    }
-  }
-
-  /**
-   * Gets the volume information.
-   *
-   * @param volume - Volume name.
-   * @return KsmVolumeArgs or exception is thrown.
-   * @throws IOException
-   */
-  @Override
-  public KsmVolumeArgs getVolumeInfo(String volume) throws IOException {
-    InfoVolumeRequest.Builder req = InfoVolumeRequest.newBuilder();
-    req.setVolumeName(volume);
-    final InfoVolumeResponse resp;
-    try {
-      resp = rpcProxy.infoVolume(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new
-          IOException("Info Volume failed, error:" + resp.getStatus());
-    }
-    return KsmVolumeArgs.getFromProtobuf(resp.getVolumeInfo());
-  }
-
-  /**
-   * Deletes an existing empty volume.
-   *
-   * @param volume - Name of the volume.
-   * @throws IOException
-   */
-  @Override
-  public void deleteVolume(String volume) throws IOException {
-    DeleteVolumeRequest.Builder req = DeleteVolumeRequest.newBuilder();
-    req.setVolumeName(volume);
-    final DeleteVolumeResponse resp;
-    try {
-      resp = rpcProxy.deleteVolume(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new
-          IOException("Delete Volume failed, error:" + resp.getStatus());
-    }
-  }
-
-  /**
-   * Lists volume owned by a specific user.
-   *
-   * @param userName - user name
-   * @param prefix - Filter prefix -- Return only entries that match this.
-   * @param prevKey - Previous key -- List starts from the next from the
-   * prevkey
-   * @param maxKeys - Max number of keys to return.
-   * @return List of Volumes.
-   * @throws IOException
-   */
-  @Override
-  public List<KsmVolumeArgs> listVolumeByUser(String userName, String prefix,
-                                              String prevKey, int maxKeys)
-      throws IOException {
-    ListVolumeRequest.Builder builder = ListVolumeRequest.newBuilder();
-    if (!Strings.isNullOrEmpty(prefix)) {
-      builder.setPrefix(prefix);
-    }
-    if (!Strings.isNullOrEmpty(prevKey)) {
-      builder.setPrevKey(prevKey);
-    }
-    builder.setMaxKeys(maxKeys);
-    builder.setUserName(userName);
-    builder.setScope(ListVolumeRequest.Scope.VOLUMES_BY_USER);
-    return listVolume(builder.build());
-  }
-
-  /**
-   * Lists volume all volumes in the cluster.
-   *
-   * @param prefix - Filter prefix -- Return only entries that match this.
-   * @param prevKey - Previous key -- List starts from the next from the
-   * prevkey
-   * @param maxKeys - Max number of keys to return.
-   * @return List of Volumes.
-   * @throws IOException
-   */
-  @Override
-  public List<KsmVolumeArgs> listAllVolumes(String prefix, String prevKey,
-      int maxKeys) throws IOException {
-    ListVolumeRequest.Builder builder = ListVolumeRequest.newBuilder();
-    if (!Strings.isNullOrEmpty(prefix)) {
-      builder.setPrefix(prefix);
-    }
-    if (!Strings.isNullOrEmpty(prevKey)) {
-      builder.setPrevKey(prevKey);
-    }
-    builder.setMaxKeys(maxKeys);
-    builder.setScope(ListVolumeRequest.Scope.VOLUMES_BY_CLUSTER);
-    return listVolume(builder.build());
-  }
-
-  private List<KsmVolumeArgs> listVolume(ListVolumeRequest request)
-      throws IOException {
-    final ListVolumeResponse resp;
-    try {
-      resp = rpcProxy.listVolumes(NULL_RPC_CONTROLLER, request);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-
-    if (resp.getStatus() != Status.OK) {
-      throw new IOException("List volume failed, error: "
-          + resp.getStatus());
-    }
-
-    List<KsmVolumeArgs> result = Lists.newArrayList();
-    for (VolumeInfo volInfo : resp.getVolumeInfoList()) {
-      KsmVolumeArgs volArgs = KsmVolumeArgs.getFromProtobuf(volInfo);
-      result.add(volArgs);
-    }
-
-    return resp.getVolumeInfoList().stream()
-        .map(item -> KsmVolumeArgs.getFromProtobuf(item))
-        .collect(Collectors.toList());
-  }
-
-  /**
-   * Creates a bucket.
-   *
-   * @param bucketInfo - BucketInfo to create bucket.
-   * @throws IOException
-   */
-  @Override
-  public void createBucket(KsmBucketInfo bucketInfo) throws IOException {
-    CreateBucketRequest.Builder req =
-        CreateBucketRequest.newBuilder();
-    BucketInfo bucketInfoProtobuf = bucketInfo.getProtobuf();
-    req.setBucketInfo(bucketInfoProtobuf);
-
-    final CreateBucketResponse resp;
-    try {
-      resp = rpcProxy.createBucket(NULL_RPC_CONTROLLER,
-          req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new IOException("Bucket creation failed, error: "
-          + resp.getStatus());
-    }
-  }
-
-  /**
-   * Gets the bucket information.
-   *
-   * @param volume - Volume name.
-   * @param bucket - Bucket name.
-   * @return KsmBucketInfo or exception is thrown.
-   * @throws IOException
-   */
-  @Override
-  public KsmBucketInfo getBucketInfo(String volume, String bucket)
-      throws IOException {
-    InfoBucketRequest.Builder req =
-        InfoBucketRequest.newBuilder();
-    req.setVolumeName(volume);
-    req.setBucketName(bucket);
-
-    final InfoBucketResponse resp;
-    try {
-      resp = rpcProxy.infoBucket(NULL_RPC_CONTROLLER,
-          req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() == Status.OK) {
-      return KsmBucketInfo.getFromProtobuf(resp.getBucketInfo());
-    } else {
-      throw new IOException("Info Bucket failed, error: "
-          + resp.getStatus());
-    }
-  }
-
-  /**
-   * Sets bucket property from args.
-   * @param args - BucketArgs.
-   * @throws IOException
-   */
-  @Override
-  public void setBucketProperty(KsmBucketArgs args)
-      throws IOException {
-    SetBucketPropertyRequest.Builder req =
-        SetBucketPropertyRequest.newBuilder();
-    BucketArgs bucketArgs = args.getProtobuf();
-    req.setBucketArgs(bucketArgs);
-    final SetBucketPropertyResponse resp;
-    try {
-      resp = rpcProxy.setBucketProperty(NULL_RPC_CONTROLLER,
-          req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new IOException("Setting bucket property failed, error: "
-          + resp.getStatus());
-    }
-  }
-
-  /**
-   * List buckets in a volume.
-   *
-   * @param volumeName
-   * @param startKey
-   * @param prefix
-   * @param count
-   * @return
-   * @throws IOException
-   */
-  @Override
-  public List<KsmBucketInfo> listBuckets(String volumeName,
-      String startKey, String prefix, int count) throws IOException {
-    List<KsmBucketInfo> buckets = new ArrayList<>();
-    ListBucketsRequest.Builder reqBuilder = ListBucketsRequest.newBuilder();
-    reqBuilder.setVolumeName(volumeName);
-    reqBuilder.setCount(count);
-    if (startKey != null) {
-      reqBuilder.setStartKey(startKey);
-    }
-    if (prefix != null) {
-      reqBuilder.setPrefix(prefix);
-    }
-    ListBucketsRequest request = reqBuilder.build();
-    final ListBucketsResponse resp;
-    try {
-      resp = rpcProxy.listBuckets(NULL_RPC_CONTROLLER, request);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-
-    if (resp.getStatus() == Status.OK) {
-      buckets.addAll(
-          resp.getBucketInfoList().stream()
-              .map(KsmBucketInfo::getFromProtobuf)
-              .collect(Collectors.toList()));
-      return buckets;
-    } else {
-      throw new IOException("List Buckets failed, error: "
-          + resp.getStatus());
-    }
-  }
-
-  /**
-   * Create a new open session of the key, then use the returned meta info to
-   * talk to data node to actually write the key.
-   * @param args the args for the key to be allocated
-   * @return a handler to the key, returned client
-   * @throws IOException
-   */
-  @Override
-  public OpenKeySession openKey(KsmKeyArgs args) throws IOException {
-    LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder();
-    KeyArgs.Builder keyArgs = KeyArgs.newBuilder()
-        .setVolumeName(args.getVolumeName())
-        .setBucketName(args.getBucketName())
-        .setFactor(args.getFactor())
-        .setType(args.getType())
-        .setKeyName(args.getKeyName());
-    if (args.getDataSize() > 0) {
-      keyArgs.setDataSize(args.getDataSize());
-    }
-    req.setKeyArgs(keyArgs.build());
-
-    final LocateKeyResponse resp;
-    try {
-      resp = rpcProxy.createKey(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new IOException("Create key failed, error:" + resp.getStatus());
-    }
-    return new OpenKeySession(resp.getID(),
-        KsmKeyInfo.getFromProtobuf(resp.getKeyInfo()), resp.getOpenVersion());
-  }
-
-  @Override
-  public KsmKeyLocationInfo allocateBlock(KsmKeyArgs args, int clientID)
-      throws IOException {
-    AllocateBlockRequest.Builder req = AllocateBlockRequest.newBuilder();
-    KeyArgs keyArgs = KeyArgs.newBuilder()
-        .setVolumeName(args.getVolumeName())
-        .setBucketName(args.getBucketName())
-        .setKeyName(args.getKeyName())
-        .setDataSize(args.getDataSize()).build();
-    req.setKeyArgs(keyArgs);
-    req.setClientID(clientID);
-
-    final AllocateBlockResponse resp;
-    try {
-      resp = rpcProxy.allocateBlock(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new IOException("Allocate block failed, error:" +
-          resp.getStatus());
-    }
-    return KsmKeyLocationInfo.getFromProtobuf(resp.getKeyLocation());
-  }
-
-  @Override
-  public void commitKey(KsmKeyArgs args, int clientID)
-      throws IOException {
-    CommitKeyRequest.Builder req = CommitKeyRequest.newBuilder();
-    KeyArgs keyArgs = KeyArgs.newBuilder()
-        .setVolumeName(args.getVolumeName())
-        .setBucketName(args.getBucketName())
-        .setKeyName(args.getKeyName())
-        .setDataSize(args.getDataSize()).build();
-    req.setKeyArgs(keyArgs);
-    req.setClientID(clientID);
-
-    final CommitKeyResponse resp;
-    try {
-      resp = rpcProxy.commitKey(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new IOException("Commit key failed, error:" +
-          resp.getStatus());
-    }
-  }
-
-
-  @Override
-  public KsmKeyInfo lookupKey(KsmKeyArgs args) throws IOException {
-    LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder();
-    KeyArgs keyArgs = KeyArgs.newBuilder()
-        .setVolumeName(args.getVolumeName())
-        .setBucketName(args.getBucketName())
-        .setKeyName(args.getKeyName())
-        .setDataSize(args.getDataSize()).build();
-    req.setKeyArgs(keyArgs);
-
-    final LocateKeyResponse resp;
-    try {
-      resp = rpcProxy.lookupKey(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new IOException("Lookup key failed, error:" +
-          resp.getStatus());
-    }
-    return KsmKeyInfo.getFromProtobuf(resp.getKeyInfo());
-  }
-
-  @Override
-  public void renameKey(KsmKeyArgs args, String toKeyName) throws IOException {
-    RenameKeyRequest.Builder req = RenameKeyRequest.newBuilder();
-    KeyArgs keyArgs = KeyArgs.newBuilder()
-        .setVolumeName(args.getVolumeName())
-        .setBucketName(args.getBucketName())
-        .setKeyName(args.getKeyName())
-        .setDataSize(args.getDataSize()).build();
-    req.setKeyArgs(keyArgs);
-    req.setToKeyName(toKeyName);
-
-    final RenameKeyResponse resp;
-    try {
-      resp = rpcProxy.renameKey(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new IOException("Rename key failed, error:" +
-          resp.getStatus());
-    }
-  }
-
-  /**
-   * Deletes an existing key.
-   *
-   * @param args the args of the key.
-   * @throws IOException
-   */
-  @Override
-  public void deleteKey(KsmKeyArgs args) throws IOException {
-    LocateKeyRequest.Builder req = LocateKeyRequest.newBuilder();
-    KeyArgs keyArgs = KeyArgs.newBuilder()
-        .setVolumeName(args.getVolumeName())
-        .setBucketName(args.getBucketName())
-        .setKeyName(args.getKeyName()).build();
-    req.setKeyArgs(keyArgs);
-
-    final LocateKeyResponse resp;
-    try {
-      resp = rpcProxy.deleteKey(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new IOException("Delete key failed, error:" +
-          resp.getStatus());
-    }
-  }
-
-  /**
-   * Deletes an existing empty bucket from volume.
-   * @param volume - Name of the volume.
-   * @param bucket - Name of the bucket.
-   * @throws IOException
-   */
-  public void deleteBucket(String volume, String bucket) throws IOException {
-    DeleteBucketRequest.Builder req = DeleteBucketRequest.newBuilder();
-    req.setVolumeName(volume);
-    req.setBucketName(bucket);
-    final DeleteBucketResponse resp;
-    try {
-      resp = rpcProxy.deleteBucket(NULL_RPC_CONTROLLER, req.build());
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    if (resp.getStatus() != Status.OK) {
-      throw new
-          IOException("Delete Bucket failed, error:" + resp.getStatus());
-    }
-  }
-
-  /**
-   * List keys in a bucket.
-   */
-  @Override
-  public List<KsmKeyInfo> listKeys(String volumeName, String bucketName,
-      String startKey, String prefix, int maxKeys) throws IOException {
-    List<KsmKeyInfo> keys = new ArrayList<>();
-    ListKeysRequest.Builder reqBuilder = ListKeysRequest.newBuilder();
-    reqBuilder.setVolumeName(volumeName);
-    reqBuilder.setBucketName(bucketName);
-    reqBuilder.setCount(maxKeys);
-
-    if (startKey != null) {
-      reqBuilder.setStartKey(startKey);
-    }
-
-    if (prefix != null) {
-      reqBuilder.setPrefix(prefix);
-    }
-
-    ListKeysRequest request = reqBuilder.build();
-    final ListKeysResponse resp;
-    try {
-      resp = rpcProxy.listKeys(NULL_RPC_CONTROLLER, request);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-
-    if (resp.getStatus() == Status.OK) {
-      keys.addAll(
-          resp.getKeyInfoList().stream()
-              .map(KsmKeyInfo::getFromProtobuf)
-              .collect(Collectors.toList()));
-      return keys;
-    } else {
-      throw new IOException("List Keys failed, error: "
-          + resp.getStatus());
-    }
-  }
-
-  @Override
-  public List<ServiceInfo> getServiceList() throws IOException {
-    ServiceListRequest request = ServiceListRequest.newBuilder().build();
-    final ServiceListResponse resp;
-    try {
-      resp = rpcProxy.getServiceList(NULL_RPC_CONTROLLER, request);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-
-    if (resp.getStatus() == Status.OK) {
-      return resp.getServiceInfoList().stream()
-              .map(ServiceInfo::getFromProtobuf)
-              .collect(Collectors.toList());
-    } else {
-      throw new IOException("Getting service list failed, error: "
-          + resp.getStatus());
-    }
-  }
-
-  /**
-   * Return the proxy object underlying this protocol translator.
-   *
-   * @return the proxy object underlying this protocol translator.
-   */
-  @Override
-  public Object getUnderlyingProxyObject() {
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolPB.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolPB.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolPB.java
deleted file mode 100644
index 8acca8a..0000000
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/KeySpaceManagerProtocolPB.java
+++ /dev/null
@@ -1,34 +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.ksm.protocolPB;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.ozone.protocol.proto
-    .KeySpaceManagerProtocolProtos.KeySpaceManagerService;
-
-/**
- * Protocol used to communicate with KSM.
- */
-@ProtocolInfo(protocolName =
-    "org.apache.hadoop.ozone.protocol.KeySpaceManagerProtocol",
-    protocolVersion = 1)
-@InterfaceAudience.Private
-public interface KeySpaceManagerProtocolPB
-    extends KeySpaceManagerService.BlockingInterface {
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/package-info.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/package-info.java
deleted file mode 100644
index 67f9f7b..0000000
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/ksm/protocolPB/package-info.java
+++ /dev/null
@@ -1,19 +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.ksm.protocolPB;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
new file mode 100644
index 0000000..b9ca296
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
@@ -0,0 +1,81 @@
+/**
+ * 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.om;
+
+import org.apache.hadoop.ozone.OzoneAcl;
+/**
+ * Ozone Manager Constants.
+ */
+public final class OMConfigKeys {
+  /**
+   * Never constructed.
+   */
+  private OMConfigKeys() {
+  }
+
+
+  public static final String OZONE_OM_HANDLER_COUNT_KEY =
+      "ozone.om.handler.count.key";
+  public static final int OZONE_OM_HANDLER_COUNT_DEFAULT = 20;
+
+  public static final String OZONE_OM_ADDRESS_KEY =
+      "ozone.om.address";
+  public static final String OZONE_OM_BIND_HOST_DEFAULT =
+      "0.0.0.0";
+  public static final int OZONE_OM_PORT_DEFAULT = 9862;
+
+  public static final String OZONE_OM_HTTP_ENABLED_KEY =
+      "ozone.om.http.enabled";
+  public static final String OZONE_OM_HTTP_BIND_HOST_KEY =
+      "ozone.om.http-bind-host";
+  public static final String OZONE_OM_HTTPS_BIND_HOST_KEY =
+      "ozone.om.https-bind-host";
+  public static final String OZONE_OM_HTTP_ADDRESS_KEY =
+      "ozone.om.http-address";
+  public static final String OZONE_OM_HTTPS_ADDRESS_KEY =
+      "ozone.om.https-address";
+  public static final String OZONE_OM_KEYTAB_FILE =
+      "ozone.om.keytab.file";
+  public static final String OZONE_OM_HTTP_BIND_HOST_DEFAULT = "0.0.0.0";
+  public static final int OZONE_OM_HTTP_BIND_PORT_DEFAULT = 9874;
+  public static final int OZONE_OM_HTTPS_BIND_PORT_DEFAULT = 9875;
+
+  // LevelDB cache file uses an off-heap cache in LevelDB of 128 MB.
+  public static final String OZONE_OM_DB_CACHE_SIZE_MB =
+      "ozone.om.db.cache.size.mb";
+  public static final int OZONE_OM_DB_CACHE_SIZE_DEFAULT = 128;
+
+  public static final String OZONE_OM_USER_MAX_VOLUME =
+      "ozone.om.user.max.volume";
+  public static final int OZONE_OM_USER_MAX_VOLUME_DEFAULT = 1024;
+
+  // OM Default user/group permissions
+  public static final String OZONE_OM_USER_RIGHTS =
+      "ozone.om.user.rights";
+  public static final OzoneAcl.OzoneACLRights OZONE_OM_USER_RIGHTS_DEFAULT =
+      OzoneAcl.OzoneACLRights.READ_WRITE;
+
+  public static final String OZONE_OM_GROUP_RIGHTS =
+      "ozone.om.group.rights";
+  public static final OzoneAcl.OzoneACLRights OZONE_OM_GROUP_RIGHTS_DEFAULT =
+      OzoneAcl.OzoneACLRights.READ_WRITE;
+
+  public static final String OZONE_KEY_DELETING_LIMIT_PER_TASK =
+      "ozone.key.deleting.limit.per.task";
+  public static final int OZONE_KEY_DELETING_LIMIT_PER_TASK_DEFAULT = 1000;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
new file mode 100644
index 0000000..6aabfef
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketArgs.java
@@ -0,0 +1,233 @@
+/**
+ * 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.om.helpers;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.protocol.proto
+    .OzoneManagerProtocolProtos.BucketArgs;
+import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
+
+/**
+ * A class that encapsulates Bucket Arguments.
+ */
+public final class OmBucketArgs {
+  /**
+   * Name of the volume in which the bucket belongs to.
+   */
+  private final String volumeName;
+  /**
+   * Name of the bucket.
+   */
+  private final String bucketName;
+  /**
+   * ACL's that are to be added for the bucket.
+   */
+  private List<OzoneAcl> addAcls;
+  /**
+   * ACL's that are to be removed from the bucket.
+   */
+  private List<OzoneAcl> removeAcls;
+  /**
+   * Bucket Version flag.
+   */
+  private Boolean isVersionEnabled;
+  /**
+   * Type of storage to be used for this bucket.
+   * [RAM_DISK, SSD, DISK, ARCHIVE]
+   */
+  private StorageType storageType;
+
+  /**
+   * Private constructor, constructed via builder.
+   * @param volumeName - Volume name.
+   * @param bucketName - Bucket name.
+   * @param addAcls - ACL's to be added.
+   * @param removeAcls - ACL's to be removed.
+   * @param isVersionEnabled - Bucket version flag.
+   * @param storageType - Storage type to be used.
+   */
+  private OmBucketArgs(String volumeName, String bucketName,
+                       List<OzoneAcl> addAcls, List<OzoneAcl> removeAcls,
+                       Boolean isVersionEnabled, StorageType storageType) {
+    this.volumeName = volumeName;
+    this.bucketName = bucketName;
+    this.addAcls = addAcls;
+    this.removeAcls = removeAcls;
+    this.isVersionEnabled = isVersionEnabled;
+    this.storageType = storageType;
+  }
+
+  /**
+   * Returns the Volume Name.
+   * @return String.
+   */
+  public String getVolumeName() {
+    return volumeName;
+  }
+
+  /**
+   * Returns the Bucket Name.
+   * @return String
+   */
+  public String getBucketName() {
+    return bucketName;
+  }
+
+  /**
+   * Returns the ACL's that are to be added.
+   * @return List<OzoneAclInfo>
+   */
+  public List<OzoneAcl> getAddAcls() {
+    return addAcls;
+  }
+
+  /**
+   * Returns the ACL's that are to be removed.
+   * @return List<OzoneAclInfo>
+   */
+  public List<OzoneAcl> getRemoveAcls() {
+    return removeAcls;
+  }
+
+  /**
+   * Returns true if bucket version is enabled, else false.
+   * @return isVersionEnabled
+   */
+  public Boolean getIsVersionEnabled() {
+    return isVersionEnabled;
+  }
+
+  /**
+   * Returns the type of storage to be used.
+   * @return StorageType
+   */
+  public StorageType getStorageType() {
+    return storageType;
+  }
+
+  /**
+   * Returns new builder class that builds a OmBucketArgs.
+   *
+   * @return Builder
+   */
+  public static Builder newBuilder() {
+    return new Builder();
+  }
+
+  /**
+   * Builder for OmBucketArgs.
+   */
+  public static class Builder {
+    private String volumeName;
+    private String bucketName;
+    private List<OzoneAcl> addAcls;
+    private List<OzoneAcl> removeAcls;
+    private Boolean isVersionEnabled;
+    private StorageType storageType;
+
+    public Builder setVolumeName(String volume) {
+      this.volumeName = volume;
+      return this;
+    }
+
+    public Builder setBucketName(String bucket) {
+      this.bucketName = bucket;
+      return this;
+    }
+
+    public Builder setAddAcls(List<OzoneAcl> acls) {
+      this.addAcls = acls;
+      return this;
+    }
+
+    public Builder setRemoveAcls(List<OzoneAcl> acls) {
+      this.removeAcls = acls;
+      return this;
+    }
+
+    public Builder setIsVersionEnabled(Boolean versionFlag) {
+      this.isVersionEnabled = versionFlag;
+      return this;
+    }
+
+    public Builder setStorageType(StorageType storage) {
+      this.storageType = storage;
+      return this;
+    }
+
+    /**
+     * Constructs the OmBucketArgs.
+     * @return instance of OmBucketArgs.
+     */
+    public OmBucketArgs build() {
+      Preconditions.checkNotNull(volumeName);
+      Preconditions.checkNotNull(bucketName);
+      return new OmBucketArgs(volumeName, bucketName, addAcls,
+          removeAcls, isVersionEnabled, storageType);
+    }
+  }
+
+  /**
+   * Creates BucketArgs protobuf from OmBucketArgs.
+   */
+  public BucketArgs getProtobuf() {
+    BucketArgs.Builder builder = BucketArgs.newBuilder();
+    builder.setVolumeName(volumeName)
+        .setBucketName(bucketName);
+    if(addAcls != null && !addAcls.isEmpty()) {
+      builder.addAllAddAcls(addAcls.stream().map(
+          OMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
+    }
+    if(removeAcls != null && !removeAcls.isEmpty()) {
+      builder.addAllRemoveAcls(removeAcls.stream().map(
+          OMPBHelper::convertOzoneAcl).collect(Collectors.toList()));
+    }
+    if(isVersionEnabled != null) {
+      builder.setIsVersionEnabled(isVersionEnabled);
+    }
+    if(storageType != null) {
+      builder.setStorageType(
+          PBHelperClient.convertStorageType(storageType));
+    }
+    return builder.build();
+  }
+
+  /**
+   * Parses BucketInfo protobuf and creates OmBucketArgs.
+   * @param bucketArgs
+   * @return instance of OmBucketArgs
+   */
+  public static OmBucketArgs getFromProtobuf(BucketArgs bucketArgs) {
+    return new OmBucketArgs(bucketArgs.getVolumeName(),
+        bucketArgs.getBucketName(),
+        bucketArgs.getAddAclsList().stream().map(
+            OMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
+        bucketArgs.getRemoveAclsList().stream().map(
+            OMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
+        bucketArgs.hasIsVersionEnabled() ?
+            bucketArgs.getIsVersionEnabled() : null,
+        bucketArgs.hasStorageType() ? PBHelperClient.convertStorageType(
+            bucketArgs.getStorageType()) : null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
new file mode 100644
index 0000000..bf5abdd
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
@@ -0,0 +1,235 @@
+/**
+ * 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.om.helpers;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.protocol.proto
+    .OzoneManagerProtocolProtos.BucketInfo;
+import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * A class that encapsulates Bucket Info.
+ */
+public final class OmBucketInfo {
+  /**
+   * Name of the volume in which the bucket belongs to.
+   */
+  private final String volumeName;
+  /**
+   * Name of the bucket.
+   */
+  private final String bucketName;
+  /**
+   * ACL Information.
+   */
+  private List<OzoneAcl> acls;
+  /**
+   * Bucket Version flag.
+   */
+  private Boolean isVersionEnabled;
+  /**
+   * Type of storage to be used for this bucket.
+   * [RAM_DISK, SSD, DISK, ARCHIVE]
+   */
+  private StorageType storageType;
+  /**
+   * Creation time of bucket.
+   */
+  private final long creationTime;
+
+  /**
+   * Private constructor, constructed via builder.
+   * @param volumeName - Volume name.
+   * @param bucketName - Bucket name.
+   * @param acls - list of ACLs.
+   * @param isVersionEnabled - Bucket version flag.
+   * @param storageType - Storage type to be used.
+   * @param creationTime - Bucket creation time.
+   */
+  private OmBucketInfo(String volumeName, String bucketName,
+                       List<OzoneAcl> acls, boolean isVersionEnabled,
+                       StorageType storageType, long creationTime) {
+    this.volumeName = volumeName;
+    this.bucketName = bucketName;
+    this.acls = acls;
+    this.isVersionEnabled = isVersionEnabled;
+    this.storageType = storageType;
+    this.creationTime = creationTime;
+  }
+
+  /**
+   * Returns the Volume Name.
+   * @return String.
+   */
+  public String getVolumeName() {
+    return volumeName;
+  }
+
+  /**
+   * Returns the Bucket Name.
+   * @return String
+   */
+  public String getBucketName() {
+    return bucketName;
+  }
+
+  /**
+   * Returns the ACL's associated with this bucket.
+   * @return List<OzoneAcl>
+   */
+  public List<OzoneAcl> getAcls() {
+    return acls;
+  }
+
+  /**
+   * Returns true if bucket version is enabled, else false.
+   * @return isVersionEnabled
+   */
+  public boolean getIsVersionEnabled() {
+    return isVersionEnabled;
+  }
+
+  /**
+   * Returns the type of storage to be used.
+   * @return StorageType
+   */
+  public StorageType getStorageType() {
+    return storageType;
+  }
+
+  /**
+   * Returns creation time.
+   *
+   * @return long
+   */
+  public long getCreationTime() {
+    return creationTime;
+  }
+
+  /**
+   * Returns new builder class that builds a OmBucketInfo.
+   *
+   * @return Builder
+   */
+  public static Builder newBuilder() {
+    return new Builder();
+  }
+
+  /**
+   * Builder for OmBucketInfo.
+   */
+  public static class Builder {
+    private String volumeName;
+    private String bucketName;
+    private List<OzoneAcl> acls;
+    private Boolean isVersionEnabled;
+    private StorageType storageType;
+    private long creationTime;
+
+    Builder() {
+      //Default values
+      this.acls = new LinkedList<>();
+      this.isVersionEnabled = false;
+      this.storageType = StorageType.DISK;
+    }
+
+    public Builder setVolumeName(String volume) {
+      this.volumeName = volume;
+      return this;
+    }
+
+    public Builder setBucketName(String bucket) {
+      this.bucketName = bucket;
+      return this;
+    }
+
+    public Builder setAcls(List<OzoneAcl> listOfAcls) {
+      this.acls = listOfAcls;
+      return this;
+    }
+
+    public Builder setIsVersionEnabled(Boolean versionFlag) {
+      this.isVersionEnabled = versionFlag;
+      return this;
+    }
+
+    public Builder setStorageType(StorageType storage) {
+      this.storageType = storage;
+      return this;
+    }
+
+    public Builder setCreationTime(long createdOn) {
+      this.creationTime = createdOn;
+      return this;
+    }
+
+    /**
+     * Constructs the OmBucketInfo.
+     * @return instance of OmBucketInfo.
+     */
+    public OmBucketInfo build() {
+      Preconditions.checkNotNull(volumeName);
+      Preconditions.checkNotNull(bucketName);
+      Preconditions.checkNotNull(acls);
+      Preconditions.checkNotNull(isVersionEnabled);
+      Preconditions.checkNotNull(storageType);
+
+      return new OmBucketInfo(volumeName, bucketName, acls,
+          isVersionEnabled, storageType, creationTime);
+    }
+  }
+
+  /**
+   * Creates BucketInfo protobuf from OmBucketInfo.
+   */
+  public BucketInfo getProtobuf() {
+    return BucketInfo.newBuilder()
+        .setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .addAllAcls(acls.stream().map(
+            OMPBHelper::convertOzoneAcl).collect(Collectors.toList()))
+        .setIsVersionEnabled(isVersionEnabled)
+        .setStorageType(PBHelperClient.convertStorageType(
+            storageType))
+        .setCreationTime(creationTime)
+        .build();
+  }
+
+  /**
+   * Parses BucketInfo protobuf and creates OmBucketInfo.
+   * @param bucketInfo
+   * @return instance of OmBucketInfo
+   */
+  public static OmBucketInfo getFromProtobuf(BucketInfo bucketInfo) {
+    return new OmBucketInfo(
+        bucketInfo.getVolumeName(),
+        bucketInfo.getBucketName(),
+        bucketInfo.getAclsList().stream().map(
+            OMPBHelper::convertOzoneAcl).collect(Collectors.toList()),
+        bucketInfo.getIsVersionEnabled(),
+        PBHelperClient.convertStorageType(
+            bucketInfo.getStorageType()), bucketInfo.getCreationTime());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/061b1685/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java
new file mode 100644
index 0000000..1f8ed5f
--- /dev/null
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyArgs.java
@@ -0,0 +1,119 @@
+/**
+ * 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.om.helpers;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
+
+/**
+ * Args for key. Client use this to specify key's attributes on  key creation
+ * (putKey()).
+ */
+public final class OmKeyArgs {
+  private final String volumeName;
+  private final String bucketName;
+  private final String keyName;
+  private long dataSize;
+  private final ReplicationType type;
+  private final ReplicationFactor factor;
+
+  private OmKeyArgs(String volumeName, String bucketName, String keyName,
+                    long dataSize, ReplicationType type, ReplicationFactor factor) {
+    this.volumeName = volumeName;
+    this.bucketName = bucketName;
+    this.keyName = keyName;
+    this.dataSize = dataSize;
+    this.type = type;
+    this.factor = factor;
+  }
+
+  public ReplicationType getType() {
+    return type;
+  }
+
+  public ReplicationFactor getFactor() {
+    return factor;
+  }
+
+  public String getVolumeName() {
+    return volumeName;
+  }
+
+  public String getBucketName() {
+    return bucketName;
+  }
+
+  public String getKeyName() {
+    return keyName;
+  }
+
+  public long getDataSize() {
+    return dataSize;
+  }
+
+  public void setDataSize(long size) {
+    dataSize = size;
+  }
+
+  /**
+   * Builder class of OmKeyArgs.
+   */
+  public static class Builder {
+    private String volumeName;
+    private String bucketName;
+    private String keyName;
+    private long dataSize;
+    private ReplicationType type;
+    private ReplicationFactor factor;
+
+
+    public Builder setVolumeName(String volume) {
+      this.volumeName = volume;
+      return this;
+    }
+
+    public Builder setBucketName(String bucket) {
+      this.bucketName = bucket;
+      return this;
+    }
+
+    public Builder setKeyName(String key) {
+      this.keyName = key;
+      return this;
+    }
+
+    public Builder setDataSize(long size) {
+      this.dataSize = size;
+      return this;
+    }
+
+    public Builder setType(ReplicationType replicationType) {
+      this.type = replicationType;
+      return this;
+    }
+
+    public Builder setFactor(ReplicationFactor replicationFactor) {
+      this.factor = replicationFactor;
+      return this;
+    }
+
+    public OmKeyArgs build() {
+      return new OmKeyArgs(volumeName, bucketName, keyName, dataSize,
+          type, factor);
+    }
+  }
+}


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