Bläddra i källkod

HDFS-12632. Ozone: OzoneFileSystem: Add contract tests to OzoneFileSystem. Contributed by Mukul Kumar Singh.

Chen Liang 7 år sedan
förälder
incheckning
895a4cc60a
14 ändrade filer med 688 tillägg och 1 borttagningar
  1. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/protocolPB/KeySpaceManagerProtocolServerSideTranslatorPB.java
  2. 5 0
      hadoop-tools/hadoop-ozone/pom.xml
  3. 1 1
      hadoop-tools/hadoop-ozone/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
  4. 48 0
      hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractCreate.java
  5. 48 0
      hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractDelete.java
  6. 50 0
      hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractDistCp.java
  7. 61 0
      hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractGetFileStatus.java
  8. 48 0
      hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractMkdir.java
  9. 47 0
      hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractOpen.java
  10. 49 0
      hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractRename.java
  11. 51 0
      hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractRootDir.java
  12. 47 0
      hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractSeek.java
  13. 119 0
      hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/OzoneContract.java
  14. 113 0
      hadoop-tools/hadoop-ozone/src/test/resources/contract/ozone.xml

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/protocolPB/KeySpaceManagerProtocolServerSideTranslatorPB.java

@@ -468,6 +468,7 @@ public class KeySpaceManagerProtocolServerSideTranslatorPB implements
           .setVolumeName(keyArgs.getVolumeName())
           .setBucketName(keyArgs.getBucketName())
           .setKeyName(keyArgs.getKeyName())
+          .setDataSize(keyArgs.getDataSize())
           .build();
       int id = request.getClientID();
       impl.commitKey(ksmKeyArgs, id);

+ 5 - 0
hadoop-tools/hadoop-ozone/pom.xml

@@ -113,5 +113,10 @@
       <scope>test</scope>
       <type>test-jar</type>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>

+ 1 - 1
hadoop-tools/hadoop-ozone/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java

@@ -114,7 +114,7 @@ public class OzoneFileSystem extends FileSystem {
       uri = new URIBuilder().setScheme(OZONE_URI_SCHEME).setHost(hostStr)
           .setPath(OZONE_URI_DELIMITER + volumeStr + OZONE_URI_DELIMITER
               + bucketStr + OZONE_URI_DELIMITER).build();
-      LOG.info("Ozone URI for ozfs initialization is " + uri);
+      LOG.trace("Ozone URI for ozfs initialization is " + uri);
       this.ozone = new OzoneRestClient(OZONE_HTTP_SCHEME + hostStr);
       try {
         this.userName =

+ 48 - 0
hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractCreate.java

@@ -0,0 +1,48 @@
+/*
+ * 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.fs.ozone.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ * Ozone contract tests creating files.
+ */
+public class ITestOzoneContractCreate extends AbstractContractCreateTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    OzoneContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    OzoneContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new OzoneContract(conf);
+  }
+}

+ 48 - 0
hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractDelete.java

@@ -0,0 +1,48 @@
+/*
+ * 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.fs.ozone.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ * Ozone contract tests covering deletes.
+ */
+public class ITestOzoneContractDelete extends AbstractContractDeleteTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    OzoneContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    OzoneContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new OzoneContract(conf);
+  }
+}

+ 50 - 0
hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractDistCp.java

@@ -0,0 +1,50 @@
+/**
+ * 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.fs.ozone.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+
+/**
+ * Contract test suite covering S3A integration with DistCp.
+ * Uses the block output stream, buffered to disk. This is the
+ * recommended output mechanism for DistCP due to its scalability.
+ */
+public class ITestOzoneContractDistCp extends AbstractContractDistCpTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    OzoneContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    OzoneContract.destroyCluster();
+  }
+
+  @Override
+  protected OzoneContract createContract(Configuration conf) {
+    return new OzoneContract(conf);
+  }
+}

+ 61 - 0
hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractGetFileStatus.java

@@ -0,0 +1,61 @@
+/**
+ * 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.fs.ozone.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ * Ozone contract tests covering getFileStatus.
+ */
+public class ITestOzoneContractGetFileStatus
+    extends AbstractContractGetFileStatusTest {
+
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    OzoneContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    OzoneContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new OzoneContract(conf);
+  }
+
+  @Override
+  public void teardown() throws Exception {
+    getLog().info("FS details {}", getFileSystem());
+    super.teardown();
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    return super.createConfiguration();
+  }
+}

+ 48 - 0
hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractMkdir.java

@@ -0,0 +1,48 @@
+/*
+ * 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.fs.ozone.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ * Test dir operations on Ozone.
+ */
+public class ITestOzoneContractMkdir extends AbstractContractMkdirTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    OzoneContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    OzoneContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new OzoneContract(conf);
+  }
+}

+ 47 - 0
hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractOpen.java

@@ -0,0 +1,47 @@
+/*
+ * 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.fs.ozone.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ * Ozone contract tests opening files.
+ */
+public class ITestOzoneContractOpen extends AbstractContractOpenTest {
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    OzoneContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    OzoneContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new OzoneContract(conf);
+  }
+}

+ 49 - 0
hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractRename.java

@@ -0,0 +1,49 @@
+/*
+ * 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.fs.ozone.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ * Ozone contract tests covering rename.
+ */
+public class ITestOzoneContractRename extends AbstractContractRenameTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    OzoneContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    OzoneContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new OzoneContract(conf);
+  }
+
+}

+ 51 - 0
hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractRootDir.java

@@ -0,0 +1,51 @@
+/*
+ * 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.fs.ozone.contract;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+import java.io.IOException;
+
+/**
+ * Ozone contract test for ROOT directory operations.
+ */
+public class ITestOzoneContractRootDir extends
+    AbstractContractRootDirectoryTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    OzoneContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    OzoneContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new OzoneContract(conf);
+  }
+
+}

+ 47 - 0
hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/ITestOzoneContractSeek.java

@@ -0,0 +1,47 @@
+/*
+ * 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.fs.ozone.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ * Ozone contract tests covering file seek.
+ */
+public class ITestOzoneContractSeek extends AbstractContractSeekTest {
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    OzoneContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    OzoneContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new OzoneContract(conf);
+  }
+}

+ 119 - 0
hadoop-tools/hadoop-ozone/src/test/java/org/apache/hadoop/fs/ozone/contract/OzoneContract.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
+ *
+ *       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.fs.ozone.contract;
+
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.OzoneConfiguration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.ozone.Constants;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.web.exceptions.OzoneException;
+import org.apache.hadoop.ozone.web.handlers.BucketArgs;
+import org.apache.hadoop.ozone.web.handlers.UserArgs;
+import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
+import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
+import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+import org.junit.Assert;
+
+import java.io.IOException;
+
+/**
+ * The contract of Ozone: only enabled if the test bucket is provided.
+ */
+class OzoneContract extends AbstractFSContract {
+
+  private static MiniOzoneCluster cluster;
+  private static StorageHandler storageHandler;
+  private static final String CONTRACT_XML = "contract/ozone.xml";
+
+  public OzoneContract(Configuration conf) {
+    super(conf);
+    //insert the base features
+    addConfResource(CONTRACT_XML);
+  }
+
+  @Override
+  public String getScheme() {
+    return Constants.OZONE_URI_SCHEME;
+  }
+
+  @Override
+  public Path getTestPath() {
+    Path path = new Path("/test");
+    return path;
+  }
+
+  public static void createCluster() throws IOException {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    conf.addResource(CONTRACT_XML);
+
+    cluster =
+        new MiniOzoneCluster.Builder(conf).numDataNodes(5)
+            .setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
+    cluster.waitClusterUp();
+    storageHandler = new ObjectStoreHandler(conf).getStorageHandler();
+  }
+
+  @Override
+  public FileSystem getTestFileSystem() throws IOException {
+    //assumes cluster is not null
+    Assert.assertNotNull("cluster not created", cluster);
+
+    String userName = "user" + RandomStringUtils.randomNumeric(5);
+    String adminName = "admin" + RandomStringUtils.randomNumeric(5);
+    String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
+    String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
+
+
+    UserArgs userArgs = new UserArgs(null, OzoneUtils.getRequestID(),
+        null, null, null, null);
+    VolumeArgs volumeArgs = new VolumeArgs(volumeName, userArgs);
+    volumeArgs.setUserName(userName);
+    volumeArgs.setAdminName(adminName);
+    BucketArgs bucketArgs = new BucketArgs(volumeName, bucketName, userArgs);
+    try {
+      storageHandler.createVolume(volumeArgs);
+
+
+      storageHandler.createBucket(bucketArgs);
+    } catch (OzoneException e) {
+      throw new IOException(e.getMessage());
+    }
+    DataNode dataNode = cluster.getDataNodes().get(0);
+    final int port = dataNode.getInfoPort();
+
+    String uri = String.format("%s://localhost:%d/%s/%s",
+        Constants.OZONE_URI_SCHEME, port, volumeName, bucketName);
+    getConf().set("fs.defaultFS", uri);
+    return FileSystem.get(getConf());
+  }
+
+  public static void destroyCluster() throws IOException {
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }
+  }
+}

+ 113 - 0
hadoop-tools/hadoop-ozone/src/test/resources/contract/ozone.xml

@@ -0,0 +1,113 @@
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~  or more contributor license agreements.  See the NOTICE file
+  ~  distributed with this work for additional information
+  ~  regarding copyright ownership.  The ASF licenses this file
+  ~  to you under the Apache License, Version 2.0 (the
+  ~  "License"); you may not use this file except in compliance
+  ~  with the License.  You may obtain a copy of the License at
+  ~
+  ~       http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~  Unless required by applicable law or agreed to in writing, software
+  ~  distributed under the License is distributed on an "AS IS" BASIS,
+  ~  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~  See the License for the specific language governing permissions and
+  ~  limitations under the License.
+  -->
+
+<configuration>
+  <!--
+  Ozone is a blobstore, with very different behavior than a classic filesystem.
+  -->
+
+    <property>
+        <name>fs.contract.test.root-tests-enabled</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.test.random-seek-count</name>
+        <value>10</value>
+    </property>
+
+    <property>
+        <name>fs.contract.is-blobstore</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.create-visibility-delayed</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.is-case-sensitive</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.rename-returns-false-if-source-missing</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.rename-remove-dest-if-empty-dir</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-append</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-atomic-directory-delete</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-atomic-rename</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-block-locality</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-concat</name>
+        <value>false</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-getfilestatus</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-seek</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-seek-on-closed-file</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.rejects-seek-past-eof</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-strict-exceptions</name>
+        <value>true</value>
+    </property>
+
+    <property>
+        <name>fs.contract.supports-unix-permissions</name>
+        <value>false</value>
+    </property>
+</configuration>