Browse Source

HDFS-11425. Ozone: add client-facing container APIs and container references. Contributed by Chen Liang.

Anu Engineer 8 years ago
parent
commit
09ad229db9

+ 134 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/client/ContainerOperationClient.java

@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.scm.client;
+
+import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
+import org.apache.hadoop.scm.XceiverClient;
+import org.apache.hadoop.scm.XceiverClientManager;
+import org.apache.hadoop.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.scm.storage.ContainerProtocolCalls;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.UUID;
+
+/**
+ * This class provides the client-facing APIs of container operations.
+ */
+public class ContainerOperationClient implements ScmClient {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContainerOperationClient.class);
+  private static long containerSizeB = -1;
+  private final StorageContainerLocationProtocolClientSideTranslatorPB
+      storageContainerLocationClient;
+  private final XceiverClientManager xceiverClientManager;
+
+  public ContainerOperationClient(
+      StorageContainerLocationProtocolClientSideTranslatorPB
+          storageContainerLocationClient,
+      XceiverClientManager xceiverClientManager) {
+    this.storageContainerLocationClient = storageContainerLocationClient;
+    this.xceiverClientManager = xceiverClientManager;
+  }
+
+  /**
+   * Return the capacity of containers. The current assumption is that all
+   * containers have the same capacity. Therefore one static is sufficient for
+   * any container.
+   * @return The capacity of one container in number of bytes.
+   */
+  public static long getContainerSizeB() {
+    return containerSizeB;
+  }
+
+  /**
+   * Set the capacity of container. Should be exactly once on system start.
+   * @param size Capacity of one container in number of bytes.
+   */
+  public static void setContainerSizeB(long size) {
+    containerSizeB = size;
+  }
+
+  /**
+   * Create a container with the given ID as its name.
+   * @param containerId - String container ID
+   * @return A Pipeline object to actually write/read from the container.
+   * @throws IOException
+   */
+  @Override
+  public Pipeline createContainer(String containerId)
+      throws IOException {
+    XceiverClient client = null;
+    try {
+      Pipeline pipeline =
+          storageContainerLocationClient.allocateContainer(containerId);
+      client = xceiverClientManager.acquireClient(pipeline);
+      String traceID = UUID.randomUUID().toString();
+      ContainerProtocolCalls.createContainer(client, traceID);
+      LOG.info("Created container " + containerId +
+          " leader:" + pipeline.getLeader() +
+          " machines:" + pipeline.getMachines());
+      return pipeline;
+    } finally {
+      if (client != null) {
+        xceiverClientManager.releaseClient(client);
+      }
+    }
+  }
+
+  /**
+   * Delete the container, this will release any resource it uses.
+   * @param pipeline - Pipeline that represents the container.
+   * @throws IOException
+   */
+  @Override
+  public void deleteContainer(Pipeline pipeline) throws IOException {
+    // TODO
+  }
+
+  /**
+   * Given an id, return the pipeline associated with the container.
+   * @param containerId - String Container ID
+   * @return Pipeline of the existing container, corresponding to the given id.
+   * @throws IOException
+   */
+  @Override
+  public Pipeline getContainer(String containerId) throws
+      IOException {
+    // TODO
+    return null;
+  }
+
+  /**
+   * Get the the current usage information.
+   * @param pipeline - Pipeline
+   * @return the size of the given container.
+   * @throws IOException
+   */
+  @Override
+  public long getContainerSize(Pipeline pipeline) throws IOException {
+    // TODO : Pipeline can be null, handle it correctly.
+    long size = getContainerSizeB();
+    if (size == -1) {
+      throw new IOException("Container size unknown!");
+    }
+    return size;
+  }
+}

+ 67 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/client/ScmClient.java

@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.scm.client;
+
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.scm.container.common.helpers.Pipeline;
+
+import java.io.IOException;
+
+/**
+ * The interface to call into underlying container layer.
+ *
+ * Written as interface to allow easy testing: implement a mock container layer
+ * for standalone testing of CBlock API without actually calling into remote
+ * containers. Actual container layer can simply re-implement this.
+ *
+ * NOTE this is temporarily needed class. When SCM containers are full-fledged,
+ * this interface will likely be removed.
+ */
+@InterfaceStability.Unstable
+public interface ScmClient {
+  /**
+   * Creates a Container on SCM and returns the pipeline.
+   * @param containerId - String container ID
+   * @return Pipeline
+   * @throws IOException
+   */
+  Pipeline createContainer(String containerId) throws IOException;
+
+  /**
+   * Gets a container by Name -- Throws if the container does not exist.
+   * @param containerId - String Container ID
+   * @return Pipeline
+   * @throws IOException
+   */
+  Pipeline getContainer(String containerId) throws IOException;
+
+  /**
+   * Delets an existing container.
+   * @param pipeline - Pipeline that represents the container.
+   * @throws IOException
+   */
+  void deleteContainer(Pipeline pipeline) throws IOException;
+
+  /**
+   * Gets the container size -- Computed by SCM from Container Reports.
+   * @param pipeline - Pipeline
+   * @return number of bytes used by this container.
+   * @throws IOException
+   */
+  long getContainerSize(Pipeline pipeline) throws IOException;
+}

+ 24 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/scm/client/package-info.java

@@ -0,0 +1,24 @@
+/**
+ * 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.scm.client;
+
+/**
+ * This package contains classes for the client of the storage container
+ * protocol.
+ */

+ 77 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/TestContainerOperations.java

@@ -0,0 +1,77 @@
+/**
+ * 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;
+
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.scm.XceiverClientManager;
+import org.apache.hadoop.scm.client.ContainerOperationClient;
+import org.apache.hadoop.scm.client.ScmClient;
+import org.apache.hadoop.scm.container.common.helpers.Pipeline;
+import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
+import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolPB;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * This class tests container operations (TODO currently only supports create)
+ * from cblock clients.
+ */
+public class TestContainerOperations {
+
+  private static ScmClient storageClient;
+  private static MiniOzoneCluster cluster;;
+  private static OzoneConfiguration ozoneConf;
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    int containerSizeGB = 5;
+    ContainerOperationClient.setContainerSizeB(containerSizeGB*1024*1024*1024L);
+    ozoneConf = new OzoneConfiguration();
+    cluster = new MiniOzoneCluster.Builder(ozoneConf).numDataNodes(1)
+        .setHandlerType("distributed").build();
+    StorageContainerLocationProtocolClientSideTranslatorPB client =
+        cluster.createStorageContainerLocationClient();
+    RPC.setProtocolEngine(ozoneConf, StorageContainerLocationProtocolPB.class,
+        ProtobufRpcEngine.class);
+    storageClient = new ContainerOperationClient(
+        client, new XceiverClientManager(ozoneConf));
+  }
+
+  @AfterClass
+  public static void cleanup() throws Exception {
+    if(cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * A simple test to create a container with {@link ContainerOperationClient}.
+   * @throws Exception
+   */
+  @Test
+  public void testCreate() throws Exception {
+    Pipeline pipeline0 = storageClient.createContainer("container0");
+    assertEquals("container0", pipeline0.getContainerName());
+
+  }
+
+}