|
@@ -17,10 +17,8 @@
|
|
*/
|
|
*/
|
|
|
|
|
|
package org.apache.hadoop.ozone.container.ozoneimpl;
|
|
package org.apache.hadoop.ozone.container.ozoneimpl;
|
|
-
|
|
|
|
-import org.apache.hadoop.conf.Configuration;
|
|
|
|
-import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
|
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos;
|
|
|
|
+import org.apache.hadoop.ozone.MiniOzoneCluster;
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
import org.apache.hadoop.ozone.OzoneConfiguration;
|
|
import org.apache.hadoop.ozone.OzoneConfiguration;
|
|
import org.apache.hadoop.ozone.container.ContainerTestHelper;
|
|
import org.apache.hadoop.ozone.container.ContainerTestHelper;
|
|
@@ -28,15 +26,23 @@ import org.apache.hadoop.ozone.container.common.helpers.Pipeline;
|
|
import org.apache.hadoop.ozone.container.common.transport.client.XceiverClient;
|
|
import org.apache.hadoop.ozone.container.common.transport.client.XceiverClient;
|
|
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
|
|
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
|
|
import org.junit.Assert;
|
|
import org.junit.Assert;
|
|
|
|
+import org.junit.Rule;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
|
|
+import org.junit.rules.Timeout;
|
|
|
|
|
|
import java.net.URL;
|
|
import java.net.URL;
|
|
|
|
|
|
public class TestOzoneContainer {
|
|
public class TestOzoneContainer {
|
|
|
|
+ /**
|
|
|
|
+ * Set the timeout for every test.
|
|
|
|
+ */
|
|
|
|
+ @Rule
|
|
|
|
+ public Timeout testTimeout = new Timeout(300000);
|
|
|
|
+
|
|
@Test
|
|
@Test
|
|
public void testCreateOzoneContainer() throws Exception {
|
|
public void testCreateOzoneContainer() throws Exception {
|
|
String containerName = OzoneUtils.getRequestID();
|
|
String containerName = OzoneUtils.getRequestID();
|
|
- Configuration conf = new OzoneConfiguration();
|
|
|
|
|
|
+ OzoneConfiguration conf = new OzoneConfiguration();
|
|
URL p = conf.getClass().getResource("");
|
|
URL p = conf.getClass().getResource("");
|
|
String path = p.getPath().concat(
|
|
String path = p.getPath().concat(
|
|
TestOzoneContainer.class.getSimpleName());
|
|
TestOzoneContainer.class.getSimpleName());
|
|
@@ -44,15 +50,11 @@ public class TestOzoneContainer {
|
|
OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
|
|
OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
|
|
conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
|
|
conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
|
|
|
|
|
|
|
|
+ MiniOzoneCluster cluster = new MiniOzoneCluster.Builder(conf)
|
|
|
|
+ .setHandlerType("local").build();
|
|
|
|
+
|
|
// We don't start Ozone Container via data node, we will do it
|
|
// We don't start Ozone Container via data node, we will do it
|
|
// independently in our test path.
|
|
// independently in our test path.
|
|
- conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, false);
|
|
|
|
- conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY, "local");
|
|
|
|
-
|
|
|
|
- MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
|
|
|
|
- cluster.waitActive();
|
|
|
|
-
|
|
|
|
-
|
|
|
|
Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline
|
|
Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline
|
|
(containerName);
|
|
(containerName);
|
|
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
|
|
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
|
|
@@ -78,7 +80,7 @@ public class TestOzoneContainer {
|
|
public void testOzoneContainerViaDataNode() throws Exception {
|
|
public void testOzoneContainerViaDataNode() throws Exception {
|
|
String keyName = OzoneUtils.getRequestID();
|
|
String keyName = OzoneUtils.getRequestID();
|
|
String containerName = OzoneUtils.getRequestID();
|
|
String containerName = OzoneUtils.getRequestID();
|
|
- Configuration conf = new OzoneConfiguration();
|
|
|
|
|
|
+ OzoneConfiguration conf = new OzoneConfiguration();
|
|
URL p = conf.getClass().getResource("");
|
|
URL p = conf.getClass().getResource("");
|
|
String path = p.getPath().concat(
|
|
String path = p.getPath().concat(
|
|
TestOzoneContainer.class.getSimpleName());
|
|
TestOzoneContainer.class.getSimpleName());
|
|
@@ -87,16 +89,14 @@ public class TestOzoneContainer {
|
|
conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
|
|
conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
|
|
|
|
|
|
// Start ozone container Via Datanode create.
|
|
// Start ozone container Via Datanode create.
|
|
- conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true);
|
|
|
|
- conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY, "local");
|
|
|
|
|
|
|
|
Pipeline pipeline =
|
|
Pipeline pipeline =
|
|
ContainerTestHelper.createSingleNodePipeline(containerName);
|
|
ContainerTestHelper.createSingleNodePipeline(containerName);
|
|
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
|
|
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
|
|
pipeline.getLeader().getContainerPort());
|
|
pipeline.getLeader().getContainerPort());
|
|
|
|
|
|
- MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
|
|
|
|
- cluster.waitActive();
|
|
|
|
|
|
+ MiniOzoneCluster cluster = new MiniOzoneCluster.Builder(conf)
|
|
|
|
+ .setHandlerType("local").build();
|
|
|
|
|
|
// This client talks to ozone container via datanode.
|
|
// This client talks to ozone container via datanode.
|
|
XceiverClient client = new XceiverClient(pipeline, conf);
|
|
XceiverClient client = new XceiverClient(pipeline, conf);
|