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 st...@apache.org on 2014/07/03 14:04:52 UTC

svn commit: r1607596 [3/5] - in /hadoop/common/trunk/hadoop-common-project/hadoop-common/src: main/java/org/apache/hadoop/fs/ main/java/org/apache/hadoop/fs/ftp/ main/java/org/apache/hadoop/fs/s3/ main/java/org/apache/hadoop/fs/s3native/ site/markdown/...

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractCreateTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractCreateTest.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractCreateTest.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractCreateTest.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,187 @@
+/*
+ * 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.contract;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.Test;
+import org.junit.internal.AssumptionViolatedException;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeTextFile;
+
+/**
+ * Test creating files, overwrite options &c
+ */
+public abstract class AbstractContractCreateTest extends
+                                                 AbstractFSContractTestBase {
+
+  @Test
+  public void testCreateNewFile() throws Throwable {
+    describe("Foundational 'create a file' test");
+    Path path = path("testCreateNewFile");
+    byte[] data = dataset(256, 'a', 'z');
+    writeDataset(getFileSystem(), path, data, data.length, 1024 * 1024, false);
+    ContractTestUtils.verifyFileContents(getFileSystem(), path, data);
+  }
+
+  @Test
+  public void testCreateFileOverExistingFileNoOverwrite() throws Throwable {
+    describe("Verify overwriting an existing file fails");
+    Path path = path("testCreateFileOverExistingFileNoOverwrite");
+    byte[] data = dataset(256, 'a', 'z');
+    writeDataset(getFileSystem(), path, data, data.length, 1024, false);
+    byte[] data2 = dataset(10 * 1024, 'A', 'Z');
+    try {
+      writeDataset(getFileSystem(), path, data2, data2.length, 1024, false);
+      fail("writing without overwrite unexpectedly succeeded");
+    } catch (FileAlreadyExistsException expected) {
+      //expected
+      handleExpectedException(expected);
+    } catch (IOException relaxed) {
+      handleRelaxedException("Creating a file over a file with overwrite==false",
+                             "FileAlreadyExistsException",
+                             relaxed);
+    }
+  }
+
+  /**
+   * This test catches some eventual consistency problems that blobstores exhibit,
+   * as we are implicitly verifying that updates are consistent. This
+   * is why different file lengths and datasets are used
+   * @throws Throwable
+   */
+  @Test
+  public void testOverwriteExistingFile() throws Throwable {
+    describe("Overwrite an existing file and verify the new data is there");
+    Path path = path("testOverwriteExistingFile");
+    byte[] data = dataset(256, 'a', 'z');
+    writeDataset(getFileSystem(), path, data, data.length, 1024, false);
+    ContractTestUtils.verifyFileContents(getFileSystem(), path, data);
+    byte[] data2 = dataset(10 * 1024, 'A', 'Z');
+    writeDataset(getFileSystem(), path, data2, data2.length, 1024, true);
+    ContractTestUtils.verifyFileContents(getFileSystem(), path, data2);
+  }
+
+  @Test
+  public void testOverwriteEmptyDirectory() throws Throwable {
+    describe("verify trying to create a file over an empty dir fails");
+    Path path = path("testOverwriteEmptyDirectory");
+    mkdirs(path);
+    assertIsDirectory(path);
+    byte[] data = dataset(256, 'a', 'z');
+    try {
+      writeDataset(getFileSystem(), path, data, data.length, 1024, true);
+      assertIsDirectory(path);
+      fail("write of file over empty dir succeeded");
+    } catch (FileAlreadyExistsException expected) {
+      //expected
+      handleExpectedException(expected);
+    } catch (FileNotFoundException e) {
+      handleRelaxedException("overwriting a dir with a file ",
+                             "FileAlreadyExistsException",
+                             e);
+    } catch (IOException e) {
+      handleRelaxedException("overwriting a dir with a file ",
+                             "FileAlreadyExistsException",
+                             e);
+    }
+    assertIsDirectory(path);
+  }
+
+  @Test
+  public void testOverwriteNonEmptyDirectory() throws Throwable {
+    describe("verify trying to create a file over a non-empty dir fails");
+    Path path = path("testOverwriteNonEmptyDirectory");
+    mkdirs(path);
+    try {
+      assertIsDirectory(path);
+    } catch (AssertionError failure) {
+      if (isSupported(IS_BLOBSTORE)) {
+        // file/directory hack surfaces here
+        throw new AssumptionViolatedException(failure.toString()).initCause(failure);
+      }
+      // else: rethrow
+      throw failure;
+    }
+    Path child = new Path(path, "child");
+    writeTextFile(getFileSystem(), child, "child file", true);
+    byte[] data = dataset(256, 'a', 'z');
+    try {
+      writeDataset(getFileSystem(), path, data, data.length, 1024,
+                   true);
+      FileStatus status = getFileSystem().getFileStatus(path);
+
+      boolean isDir = status.isDirectory();
+      if (!isDir && isSupported(IS_BLOBSTORE)) {
+        // object store: downgrade to a skip so that the failure is visible
+        // in test results
+        skip("Object store allows a file to overwrite a directory");
+      }
+      fail("write of file over dir succeeded");
+    } catch (FileAlreadyExistsException expected) {
+      //expected
+      handleExpectedException(expected);
+    } catch (FileNotFoundException e) {
+      handleRelaxedException("overwriting a dir with a file ",
+                             "FileAlreadyExistsException",
+                             e);
+    } catch (IOException e) {
+      handleRelaxedException("overwriting a dir with a file ",
+                             "FileAlreadyExistsException",
+                             e);
+    }
+    assertIsDirectory(path);
+    assertIsFile(child);
+  }
+
+  @Test
+  public void testCreatedFileIsImmediatelyVisible() throws Throwable {
+    describe("verify that a newly created file exists as soon as open returns");
+    Path path = path("testCreatedFileIsImmediatelyVisible");
+    FSDataOutputStream out = null;
+    try {
+      out = getFileSystem().create(path,
+                                   false,
+                                   4096,
+                                   (short) 1,
+                                   1024);
+      if (!getFileSystem().exists(path)) {
+
+        if (isSupported(IS_BLOBSTORE)) {
+          // object store: downgrade to a skip so that the failure is visible
+          // in test results
+          skip("Filesystem is an object store and newly created files are not immediately visible");
+        }
+        assertPathExists("expected path to be visible before anything written",
+                         path);
+      }
+    } finally {
+      IOUtils.closeStream(out);
+    }
+  }
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractCreateTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractDeleteTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractDeleteTest.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractDeleteTest.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractDeleteTest.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,97 @@
+/*
+ * 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.contract;
+
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+import java.io.IOException;
+
+/**
+ * Test creating files, overwrite options &c
+ */
+public abstract class AbstractContractDeleteTest extends
+                                                 AbstractFSContractTestBase {
+
+  @Test
+  public void testDeleteEmptyDirNonRecursive() throws Throwable {
+    Path path = path("testDeleteEmptyDirNonRecursive");
+    mkdirs(path);
+    assertDeleted(path, false);
+  }
+
+  @Test
+  public void testDeleteEmptyDirRecursive() throws Throwable {
+    Path path = path("testDeleteEmptyDirRecursive");
+    mkdirs(path);
+    assertDeleted(path, true);
+  }
+
+  @Test
+  public void testDeleteNonexistentPathRecursive() throws Throwable {
+    Path path = path("testDeleteNonexistentPathRecursive");
+    ContractTestUtils.assertPathDoesNotExist(getFileSystem(), "leftover", path);
+    ContractTestUtils.rejectRootOperation(path);
+    assertFalse("Returned true attempting to delete"
+                + " a nonexistent path " + path,
+                getFileSystem().delete(path, false));
+  }
+
+
+  @Test
+  public void testDeleteNonexistentPathNonRecursive() throws Throwable {
+    Path path = path("testDeleteNonexistentPathNonRecursive");
+    ContractTestUtils.assertPathDoesNotExist(getFileSystem(), "leftover", path);
+    ContractTestUtils.rejectRootOperation(path);
+    assertFalse("Returned true attempting to recursively delete"
+                + " a nonexistent path " + path,
+                getFileSystem().delete(path, false));
+  }
+
+  @Test
+  public void testDeleteNonEmptyDirNonRecursive() throws Throwable {
+    Path path = path("testDeleteNonEmptyDirNonRecursive");
+    mkdirs(path);
+    Path file = new Path(path, "childfile");
+    ContractTestUtils.writeTextFile(getFileSystem(), file, "goodbye, world",
+                                    true);
+    try {
+      ContractTestUtils.rejectRootOperation(path);
+      boolean deleted = getFileSystem().delete(path, false);
+      fail("non recursive delete should have raised an exception," +
+           " but completed with exit code " + deleted);
+    } catch (IOException expected) {
+      //expected
+      handleExpectedException(expected);
+    }
+    ContractTestUtils.assertIsDirectory(getFileSystem(), path);
+  }
+
+  @Test
+  public void testDeleteNonEmptyDirRecursive() throws Throwable {
+    Path path = path("testDeleteNonEmptyDirNonRecursive");
+    mkdirs(path);
+    Path file = new Path(path, "childfile");
+    ContractTestUtils.writeTextFile(getFileSystem(), file, "goodbye, world",
+                                    true);
+    assertDeleted(path, true);
+    ContractTestUtils.assertPathDoesNotExist(getFileSystem(), "not deleted", file);
+  }
+
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractDeleteTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMkdirTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMkdirTest.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMkdirTest.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMkdirTest.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,115 @@
+/*
+ * 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.contract;
+
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+
+/**
+ * Test directory operations
+ */
+public abstract class AbstractContractMkdirTest extends AbstractFSContractTestBase {
+
+  @Test
+  public void testMkDirRmDir() throws Throwable {
+    FileSystem fs = getFileSystem();
+
+    Path dir = path("testMkDirRmDir");
+    assertPathDoesNotExist("directory already exists", dir);
+    fs.mkdirs(dir);
+    assertPathExists("mkdir failed", dir);
+    assertDeleted(dir, false);
+  }
+
+  @Test
+  public void testMkDirRmRfDir() throws Throwable {
+    describe("create a directory then recursive delete it");
+    FileSystem fs = getFileSystem();
+    Path dir = path("testMkDirRmRfDir");
+    assertPathDoesNotExist("directory already exists", dir);
+    fs.mkdirs(dir);
+    assertPathExists("mkdir failed", dir);
+    assertDeleted(dir, true);
+  }
+
+  @Test
+  public void testNoMkdirOverFile() throws Throwable {
+    describe("try to mkdir over a file");
+    FileSystem fs = getFileSystem();
+    Path path = path("testNoMkdirOverFile");
+    byte[] dataset = dataset(1024, ' ', 'z');
+    createFile(getFileSystem(), path, false, dataset);
+    try {
+      boolean made = fs.mkdirs(path);
+      fail("mkdirs did not fail over a file but returned " + made
+            + "; " + ls(path));
+    } catch (ParentNotDirectoryException e) {
+      //parent is a directory
+      handleExpectedException(e);
+    } catch (FileAlreadyExistsException e) {
+      //also allowed as an exception (HDFS)
+      handleExpectedException(e);;
+    } catch (IOException e) {
+      //here the FS says "no create"
+      handleRelaxedException("mkdirs", "FileAlreadyExistsException", e);
+    }
+    assertIsFile(path);
+    byte[] bytes = ContractTestUtils.readDataset(getFileSystem(), path,
+                                                 dataset.length);
+    ContractTestUtils.compareByteArrays(dataset, bytes, dataset.length);
+    assertPathExists("mkdir failed", path);
+    assertDeleted(path, true);
+  }
+
+  @Test
+  public void testMkdirOverParentFile() throws Throwable {
+    describe("try to mkdir where a parent is a file");
+    FileSystem fs = getFileSystem();
+    Path path = path("testMkdirOverParentFile");
+    byte[] dataset = dataset(1024, ' ', 'z');
+    createFile(getFileSystem(), path, false, dataset);
+    Path child = new Path(path,"child-to-mkdir");
+    try {
+      boolean made = fs.mkdirs(child);
+      fail("mkdirs did not fail over a file but returned " + made
+           + "; " + ls(path));
+    } catch (ParentNotDirectoryException e) {
+      //parent is a directory
+      handleExpectedException(e);
+    } catch (FileAlreadyExistsException e) {
+      handleExpectedException(e);
+    } catch (IOException e) {
+      handleRelaxedException("mkdirs", "ParentNotDirectoryException", e);
+    }
+    assertIsFile(path);
+    byte[] bytes = ContractTestUtils.readDataset(getFileSystem(), path,
+                                                 dataset.length);
+    ContractTestUtils.compareByteArrays(dataset, bytes, dataset.length);
+    assertPathExists("mkdir failed", path);
+    assertDeleted(path, true);
+  }
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMkdirTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,155 @@
+/*
+ * 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.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+
+/**
+ * Test Seek operations
+ */
+public abstract class AbstractContractOpenTest extends AbstractFSContractTestBase {
+
+  private FSDataInputStream instream;
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    conf.setInt(CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY, 4096);
+    return conf;
+  }
+
+  @Override
+  public void teardown() throws Exception {
+    IOUtils.closeStream(instream);
+    instream = null;
+    super.teardown();
+  }
+
+  @Test
+  public void testOpenReadZeroByteFile() throws Throwable {
+    describe("create & read a 0 byte file");
+    Path path = path("zero.txt");
+    touch(getFileSystem(), path);
+    instream = getFileSystem().open(path);
+    assertEquals(0, instream.getPos());
+    //expect initial read to fail
+    int result = instream.read();
+    assertMinusOne("initial byte read", result);
+  }
+
+  @Test
+  public void testOpenReadDir() throws Throwable {
+    describe("create & read a directory");
+    Path path = path("zero.dir");
+    mkdirs(path);
+    try {
+      instream = getFileSystem().open(path);
+      //at this point we've opened a directory
+      fail("A directory has been opened for reading");
+    } catch (FileNotFoundException e) {
+      handleExpectedException(e);
+    } catch (IOException e) {
+      handleRelaxedException("opening a directory for reading",
+                             "FileNotFoundException",
+                             e);
+    }
+  }
+
+  @Test
+  public void testOpenReadDirWithChild() throws Throwable {
+    describe("create & read a directory which has a child");
+    Path path = path("zero.dir");
+    mkdirs(path);
+    Path path2 = new Path(path, "child");
+    mkdirs(path2);
+
+    try {
+      instream = getFileSystem().open(path);
+      //at this point we've opened a directory
+      fail("A directory has been opened for reading");
+    } catch (FileNotFoundException e) {
+      handleExpectedException(e);
+    } catch (IOException e) {
+      handleRelaxedException("opening a directory for reading",
+                             "FileNotFoundException",
+                             e);
+    }
+  }
+
+  @Test
+  public void testOpenFileTwice() throws Throwable {
+    describe("verify that two opened file streams are independent");
+    Path path = path("testopenfiletwice.txt");
+    byte[] block = dataset(TEST_FILE_LEN, 0, 255);
+    //this file now has a simple rule: offset => value
+    createFile(getFileSystem(), path, false, block);
+    //open first
+    FSDataInputStream instream1 = getFileSystem().open(path);
+    int c = instream1.read();
+    assertEquals(0,c);
+    FSDataInputStream instream2 = null;
+    try {
+      instream2 = getFileSystem().open(path);
+      assertEquals("first read of instream 2", 0, instream2.read());
+      assertEquals("second read of instream 1", 1, instream1.read());
+      instream1.close();
+      assertEquals("second read of instream 2", 1, instream2.read());
+      //close instream1 again
+      instream1.close();
+    } finally {
+      IOUtils.closeStream(instream1);
+      IOUtils.closeStream(instream2);
+    }
+  }
+
+  @Test
+  public void testSequentialRead() throws Throwable {
+    describe("verify that sequential read() operations return values");
+    Path path = path("testsequentialread.txt");
+    int len = 4;
+    int base = 0x40; // 64
+    byte[] block = dataset(len, base, base + len);
+    //this file now has a simple rule: offset => (value | 0x40)
+    createFile(getFileSystem(), path, false, block);
+    //open first
+    instream = getFileSystem().open(path);
+    assertEquals(base, instream.read());
+    assertEquals(base + 1, instream.read());
+    assertEquals(base + 2, instream.read());
+    assertEquals(base + 3, instream.read());
+    // and now, failures
+    assertEquals(-1, instream.read());
+    assertEquals(-1, instream.read());
+    instream.close();
+  }
+
+
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,185 @@
+/*
+ * 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.contract;
+
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+
+/**
+ * Test creating files, overwrite options &c
+ */
+public abstract class AbstractContractRenameTest extends
+                                                 AbstractFSContractTestBase {
+
+  @Test
+  public void testRenameNewFileSameDir() throws Throwable {
+    describe("rename a file into a new file in the same directory");
+    Path renameSrc = path("rename_src");
+    Path renameTarget = path("rename_dest");
+    byte[] data = dataset(256, 'a', 'z');
+    writeDataset(getFileSystem(), renameSrc,
+        data, data.length, 1024 * 1024, false);
+    boolean rename = rename(renameSrc, renameTarget);
+    assertTrue("rename("+renameSrc+", "+ renameTarget+") returned false",
+        rename);
+    ContractTestUtils.assertListStatusFinds(getFileSystem(),
+        renameTarget.getParent(), renameTarget);
+    ContractTestUtils.verifyFileContents(getFileSystem(), renameTarget, data);
+  }
+
+  @Test
+  public void testRenameNonexistentFile() throws Throwable {
+    describe("rename a file into a new file in the same directory");
+    Path missing = path("testRenameNonexistentFileSrc");
+    Path target = path("testRenameNonexistentFileDest");
+    boolean renameReturnsFalseOnFailure =
+        isSupported(ContractOptions.RENAME_RETURNS_FALSE_IF_SOURCE_MISSING);
+    mkdirs(missing.getParent());
+    try {
+      boolean renamed = rename(missing, target);
+      //expected an exception
+      if (!renameReturnsFalseOnFailure) {
+        String destDirLS = generateAndLogErrorListing(missing, target);
+        fail("expected rename(" + missing + ", " + target + " ) to fail," +
+             " got a result of " + renamed
+             + " and a destination directory of " + destDirLS);
+      } else {
+        // at least one FS only returns false here, if that is the case
+        // warn but continue
+        getLog().warn("Rename returned {} renaming a nonexistent file", renamed);
+        assertFalse("Renaming a missing file returned true", renamed);
+      }
+    } catch (FileNotFoundException e) {
+      if (renameReturnsFalseOnFailure) {
+        ContractTestUtils.fail(
+            "Renaming a missing file unexpectedly threw an exception", e);
+      }
+      handleExpectedException(e);
+    } catch (IOException e) {
+      handleRelaxedException("rename nonexistent file",
+          "FileNotFoundException",
+          e);
+    }
+    assertPathDoesNotExist("rename nonexistent file created a destination file", target);
+  }
+
+  /**
+   * Rename test -handles filesystems that will overwrite the destination
+   * as well as those that do not (i.e. HDFS).
+   * @throws Throwable
+   */
+  @Test
+  public void testRenameFileOverExistingFile() throws Throwable {
+    describe("Verify renaming a file onto an existing file matches expectations");
+    Path srcFile = path("source-256.txt");
+    byte[] srcData = dataset(256, 'a', 'z');
+    writeDataset(getFileSystem(), srcFile, srcData, srcData.length, 1024, false);
+    Path destFile = path("dest-512.txt");
+    byte[] destData = dataset(512, 'A', 'Z');
+    writeDataset(getFileSystem(), destFile, destData, destData.length, 1024, false);
+    assertIsFile(destFile);
+    boolean renameOverwritesDest = isSupported(RENAME_OVERWRITES_DEST);
+    boolean renameReturnsFalseOnRenameDestExists =
+        !isSupported(RENAME_RETURNS_FALSE_IF_DEST_EXISTS);
+    boolean destUnchanged = true;
+    try {
+      boolean renamed = rename(srcFile, destFile);
+
+      if (renameOverwritesDest) {
+      // the filesystem supports rename(file, file2) by overwriting file2
+
+      assertTrue("Rename returned false", renamed);
+        destUnchanged = false;
+      } else {
+        // rename is rejected by returning 'false' or throwing an exception
+        if (renamed && !renameReturnsFalseOnRenameDestExists) {
+          //expected an exception
+          String destDirLS = generateAndLogErrorListing(srcFile, destFile);
+          getLog().error("dest dir {}", destDirLS);
+          fail("expected rename(" + srcFile + ", " + destFile + " ) to fail," +
+               " but got success and destination of " + destDirLS);
+        }
+      }
+    } catch (FileAlreadyExistsException e) {
+      handleExpectedException(e);
+    }
+    // verify that the destination file is as expected based on the expected
+    // outcome
+    ContractTestUtils.verifyFileContents(getFileSystem(), destFile,
+        destUnchanged? destData: srcData);
+  }
+
+  @Test
+  public void testRenameDirIntoExistingDir() throws Throwable {
+    describe("Verify renaming a dir into an existing dir puts it underneath"
+             +" and leaves existing files alone");
+    FileSystem fs = getFileSystem();
+    String sourceSubdir = "source";
+    Path srcDir = path(sourceSubdir);
+    Path srcFilePath = new Path(srcDir, "source-256.txt");
+    byte[] srcDataset = dataset(256, 'a', 'z');
+    writeDataset(fs, srcFilePath, srcDataset, srcDataset.length, 1024, false);
+    Path destDir = path("dest");
+
+    Path destFilePath = new Path(destDir, "dest-512.txt");
+    byte[] destDateset = dataset(512, 'A', 'Z');
+    writeDataset(fs, destFilePath, destDateset, destDateset.length, 1024, false);
+    assertIsFile(destFilePath);
+
+    boolean rename = rename(srcDir, destDir);
+    Path renamedSrc = new Path(destDir, sourceSubdir);
+    assertIsFile(destFilePath);
+    assertIsDirectory(renamedSrc);
+    ContractTestUtils.verifyFileContents(fs, destFilePath, destDateset);
+    assertTrue("rename returned false though the contents were copied", rename);
+  }
+
+  @Test
+  public void testRenameFileNonexistentDir() throws Throwable {
+    describe("rename a file into a new file in the same directory");
+    Path renameSrc = path("testRenameSrc");
+    Path renameTarget = path("subdir/testRenameTarget");
+    byte[] data = dataset(256, 'a', 'z');
+    writeDataset(getFileSystem(), renameSrc, data, data.length, 1024 * 1024,
+        false);
+    boolean renameCreatesDestDirs = isSupported(RENAME_CREATES_DEST_DIRS);
+
+    try {
+      boolean rename = rename(renameSrc, renameTarget);
+      if (renameCreatesDestDirs) {
+        assertTrue(rename);
+        ContractTestUtils.verifyFileContents(getFileSystem(), renameTarget, data);
+      } else {
+        assertFalse(rename);
+        ContractTestUtils.verifyFileContents(getFileSystem(), renameSrc, data);
+      }
+    } catch (FileNotFoundException e) {
+       // allowed unless that rename flag is set
+      assertFalse(renameCreatesDestDirs);
+    }
+  }
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRenameTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,123 @@
+/*
+ * 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.contract;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+
+/**
+ * This class does things to the root directory.
+ * Only subclass this for tests against transient filesystems where
+ * you don't care about the data.
+ */
+public abstract class AbstractContractRootDirectoryTest extends AbstractFSContractTestBase {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractContractRootDirectoryTest.class);
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    skipIfUnsupported(TEST_ROOT_TESTS_ENABLED);
+  }
+
+  @Test
+  public void testMkDirDepth1() throws Throwable {
+    FileSystem fs = getFileSystem();
+    Path dir = new Path("/testmkdirdepth1");
+    assertPathDoesNotExist("directory already exists", dir);
+    fs.mkdirs(dir);
+    ContractTestUtils.assertIsDirectory(getFileSystem(), dir);
+    assertPathExists("directory already exists", dir);
+    assertDeleted(dir, true);
+  }
+
+  @Test
+  public void testRmEmptyRootDirNonRecursive() throws Throwable {
+    //extra sanity checks here to avoid support calls about complete loss of data
+    skipIfUnsupported(TEST_ROOT_TESTS_ENABLED);
+    Path root = new Path("/");
+    ContractTestUtils.assertIsDirectory(getFileSystem(), root);
+    boolean deleted = getFileSystem().delete(root, true);
+    LOG.info("rm / of empty dir result is {}", deleted);
+    ContractTestUtils.assertIsDirectory(getFileSystem(), root);
+  }
+
+  @Test
+  public void testRmNonEmptyRootDirNonRecursive() throws Throwable {
+    //extra sanity checks here to avoid support calls about complete loss of data
+    skipIfUnsupported(TEST_ROOT_TESTS_ENABLED);
+    Path root = new Path("/");
+    String touchfile = "/testRmNonEmptyRootDirNonRecursive";
+    Path file = new Path(touchfile);
+    ContractTestUtils.touch(getFileSystem(), file);
+    ContractTestUtils.assertIsDirectory(getFileSystem(), root);
+    try {
+      boolean deleted = getFileSystem().delete(root, false);
+      fail("non recursive delete should have raised an exception," +
+           " but completed with exit code " + deleted);
+    } catch (IOException e) {
+      //expected
+      handleExpectedException(e);
+    } finally {
+      getFileSystem().delete(file, false);
+    }
+    ContractTestUtils.assertIsDirectory(getFileSystem(), root);
+  }
+
+  @Test
+  public void testRmRootRecursive() throws Throwable {
+    //extra sanity checks here to avoid support calls about complete loss of data
+    skipIfUnsupported(TEST_ROOT_TESTS_ENABLED);
+    Path root = new Path("/");
+    ContractTestUtils.assertIsDirectory(getFileSystem(), root);
+    Path file = new Path("/testRmRootRecursive");
+    ContractTestUtils.touch(getFileSystem(), file);
+    boolean deleted = getFileSystem().delete(root, true);
+    ContractTestUtils.assertIsDirectory(getFileSystem(), root);
+    LOG.info("rm -rf / result is {}", deleted);
+    if (deleted) {
+      assertPathDoesNotExist("expected file to be deleted", file);
+    } else {
+      assertPathExists("expected file to be preserved", file);;
+    }
+  }
+
+  @Test
+  public void testCreateFileOverRoot() throws Throwable {
+    Path root = new Path("/");
+    byte[] dataset = dataset(1024, ' ', 'z');
+    try {
+      createFile(getFileSystem(), root, false, dataset);
+      fail("expected an exception, got a file created over root: " + ls(root));
+    } catch (IOException e) {
+      //expected
+      handleExpectedException(e);
+    }
+    assertIsDirectory(root);
+  }
+
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractRootDirectoryTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractSeekTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractSeekTest.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractSeekTest.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractSeekTest.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,348 @@
+/*
+ * 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.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.Random;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.cleanup;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyRead;
+
+/**
+ * Test Seek operations
+ */
+public abstract class AbstractContractSeekTest extends AbstractFSContractTestBase {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractContractSeekTest.class);
+
+  public static final int DEFAULT_RANDOM_SEEK_COUNT = 100;
+
+  private Path testPath;
+  private Path smallSeekFile;
+  private Path zeroByteFile;
+  private FSDataInputStream instream;
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    skipIfUnsupported(SUPPORTS_SEEK);
+    //delete the test directory
+    testPath = getContract().getTestPath();
+    smallSeekFile = path("seekfile.txt");
+    zeroByteFile = path("zero.txt");
+    byte[] block = dataset(TEST_FILE_LEN, 0, 255);
+    //this file now has a simple rule: offset => value
+    createFile(getFileSystem(), smallSeekFile, false, block);
+    touch(getFileSystem(), zeroByteFile);
+  }
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    conf.setInt(CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY, 4096);
+    return conf;
+  }
+
+  @Override
+  public void teardown() throws Exception {
+    IOUtils.closeStream(instream);
+    instream = null;
+    super.teardown();
+  }
+
+  @Test
+  public void testSeekZeroByteFile() throws Throwable {
+    describe("seek and read a 0 byte file");
+    instream = getFileSystem().open(zeroByteFile);
+    assertEquals(0, instream.getPos());
+    //expect initial read to fai;
+    int result = instream.read();
+    assertMinusOne("initial byte read", result);
+    byte[] buffer = new byte[1];
+    //expect that seek to 0 works
+    instream.seek(0);
+    //reread, expect same exception
+    result = instream.read();
+    assertMinusOne("post-seek byte read", result);
+    result = instream.read(buffer, 0, 1);
+    assertMinusOne("post-seek buffer read", result);
+  }
+
+  @Test
+  public void testBlockReadZeroByteFile() throws Throwable {
+    describe("do a block read on a 0 byte file");
+    instream = getFileSystem().open(zeroByteFile);
+    assertEquals(0, instream.getPos());
+    //expect that seek to 0 works
+    byte[] buffer = new byte[1];
+    int result = instream.read(buffer, 0, 1);
+    assertMinusOne("block read zero byte file", result);
+  }
+
+  /**
+   * Seek and read on a closed file.
+   * Some filesystems let callers seek on a closed file -these must
+   * still fail on the subsequent reads.
+   * @throws Throwable
+   */
+  @Test
+  public void testSeekReadClosedFile() throws Throwable {
+    boolean supportsSeekOnClosedFiles = isSupported(SUPPORTS_SEEK_ON_CLOSED_FILE);
+
+    instream = getFileSystem().open(smallSeekFile);
+    getLog().debug(
+      "Stream is of type " + instream.getClass().getCanonicalName());
+    instream.close();
+    try {
+      instream.seek(0);
+      if (!supportsSeekOnClosedFiles) {
+        fail("seek succeeded on a closed stream");
+      }
+    } catch (IOException e) {
+      //expected a closed file
+    }
+    try {
+      int data = instream.available();
+      fail("read() succeeded on a closed stream, got " + data);
+    } catch (IOException e) {
+      //expected a closed file
+    }
+    try {
+      int data = instream.read();
+      fail("read() succeeded on a closed stream, got " + data);
+    } catch (IOException e) {
+      //expected a closed file
+    }
+    try {
+      byte[] buffer = new byte[1];
+      int result = instream.read(buffer, 0, 1);
+      fail("read(buffer, 0, 1) succeeded on a closed stream, got " + result);
+    } catch (IOException e) {
+      //expected a closed file
+    }
+    //what position does a closed file have?
+    try {
+      long offset = instream.getPos();
+    } catch (IOException e) {
+      // its valid to raise error here; but the test is applied to make
+      // sure there's no other exception like an NPE.
+
+    }
+    //and close again
+    instream.close();
+  }
+
+  @Test
+  public void testNegativeSeek() throws Throwable {
+    instream = getFileSystem().open(smallSeekFile);
+    assertEquals(0, instream.getPos());
+    try {
+      instream.seek(-1);
+      long p = instream.getPos();
+      LOG.warn("Seek to -1 returned a position of " + p);
+      int result = instream.read();
+      fail(
+        "expected an exception, got data " + result + " at a position of " + p);
+    } catch (EOFException e) {
+      //bad seek -expected
+      handleExpectedException(e);
+    } catch (IOException e) {
+      //bad seek -expected, but not as preferred as an EOFException
+      handleRelaxedException("a negative seek", "EOFException", e);
+    }
+    assertEquals(0, instream.getPos());
+  }
+
+  @Test
+  public void testSeekFile() throws Throwable {
+    describe("basic seek operations");
+    instream = getFileSystem().open(smallSeekFile);
+    assertEquals(0, instream.getPos());
+    //expect that seek to 0 works
+    instream.seek(0);
+    int result = instream.read();
+    assertEquals(0, result);
+    assertEquals(1, instream.read());
+    assertEquals(2, instream.getPos());
+    assertEquals(2, instream.read());
+    assertEquals(3, instream.getPos());
+    instream.seek(128);
+    assertEquals(128, instream.getPos());
+    assertEquals(128, instream.read());
+    instream.seek(63);
+    assertEquals(63, instream.read());
+  }
+
+  @Test
+  public void testSeekAndReadPastEndOfFile() throws Throwable {
+    describe("verify that reading past the last bytes in the file returns -1");
+    instream = getFileSystem().open(smallSeekFile);
+    assertEquals(0, instream.getPos());
+    //expect that seek to 0 works
+    //go just before the end
+    instream.seek(TEST_FILE_LEN - 2);
+    assertTrue("Premature EOF", instream.read() != -1);
+    assertTrue("Premature EOF", instream.read() != -1);
+    assertMinusOne("read past end of file", instream.read());
+  }
+
+  @Test
+  public void testSeekPastEndOfFileThenReseekAndRead() throws Throwable {
+    describe("do a seek past the EOF, then verify the stream recovers");
+    instream = getFileSystem().open(smallSeekFile);
+    //go just before the end. This may or may not fail; it may be delayed until the
+    //read
+    boolean canSeekPastEOF =
+        !getContract().isSupported(ContractOptions.REJECTS_SEEK_PAST_EOF, true);
+    try {
+      instream.seek(TEST_FILE_LEN + 1);
+      //if this doesn't trigger, then read() is expected to fail
+      assertMinusOne("read after seeking past EOF", instream.read());
+    } catch (EOFException e) {
+      //This is an error iff the FS claims to be able to seek past the EOF
+      if (canSeekPastEOF) {
+        //a failure wasn't expected
+        throw e;
+      }
+      handleExpectedException(e);
+    } catch (IOException e) {
+      //This is an error iff the FS claims to be able to seek past the EOF
+      if (canSeekPastEOF) {
+        //a failure wasn't expected
+        throw e;
+      }
+      handleRelaxedException("a seek past the end of the file",
+          "EOFException", e);
+    }
+    //now go back and try to read from a valid point in the file
+    instream.seek(1);
+    assertTrue("Premature EOF", instream.read() != -1);
+  }
+
+  /**
+   * Seek round a file bigger than IO buffers
+   * @throws Throwable
+   */
+  @Test
+  public void testSeekBigFile() throws Throwable {
+    describe("Seek round a large file and verify the bytes are what is expected");
+    Path testSeekFile = path("bigseekfile.txt");
+    byte[] block = dataset(65536, 0, 255);
+    createFile(getFileSystem(), testSeekFile, false, block);
+    instream = getFileSystem().open(testSeekFile);
+    assertEquals(0, instream.getPos());
+    //expect that seek to 0 works
+    instream.seek(0);
+    int result = instream.read();
+    assertEquals(0, result);
+    assertEquals(1, instream.read());
+    assertEquals(2, instream.read());
+
+    //do seek 32KB ahead
+    instream.seek(32768);
+    assertEquals("@32768", block[32768], (byte) instream.read());
+    instream.seek(40000);
+    assertEquals("@40000", block[40000], (byte) instream.read());
+    instream.seek(8191);
+    assertEquals("@8191", block[8191], (byte) instream.read());
+    instream.seek(0);
+    assertEquals("@0", 0, (byte) instream.read());
+  }
+
+  @Test
+  public void testPositionedBulkReadDoesntChangePosition() throws Throwable {
+    describe(
+      "verify that a positioned read does not change the getPos() value");
+    Path testSeekFile = path("bigseekfile.txt");
+    byte[] block = dataset(65536, 0, 255);
+    createFile(getFileSystem(), testSeekFile, false, block);
+    instream = getFileSystem().open(testSeekFile);
+    instream.seek(39999);
+    assertTrue(-1 != instream.read());
+    assertEquals(40000, instream.getPos());
+
+    byte[] readBuffer = new byte[256];
+    instream.read(128, readBuffer, 0, readBuffer.length);
+    //have gone back
+    assertEquals(40000, instream.getPos());
+    //content is the same too
+    assertEquals("@40000", block[40000], (byte) instream.read());
+    //now verify the picked up data
+    for (int i = 0; i < 256; i++) {
+      assertEquals("@" + i, block[i + 128], readBuffer[i]);
+    }
+  }
+
+  /**
+   * Lifted from TestLocalFileSystem:
+   * Regression test for HADOOP-9307: BufferedFSInputStream returning
+   * wrong results after certain sequences of seeks and reads.
+   */
+  @Test
+  public void testRandomSeeks() throws Throwable {
+    int limit = getContract().getLimit(TEST_RANDOM_SEEK_COUNT,
+                                       DEFAULT_RANDOM_SEEK_COUNT);
+    describe("Testing " + limit + " random seeks");
+    int filesize = 10 * 1024;
+    byte[] buf = dataset(filesize, 0, 255);
+    Path randomSeekFile = path("testrandomseeks.bin");
+    createFile(getFileSystem(), randomSeekFile, false, buf);
+    Random r = new Random();
+    FSDataInputStream stm = getFileSystem().open(randomSeekFile);
+
+    // Record the sequence of seeks and reads which trigger a failure.
+    int[] seeks = new int[10];
+    int[] reads = new int[10];
+    try {
+      for (int i = 0; i < limit; i++) {
+        int seekOff = r.nextInt(buf.length);
+        int toRead = r.nextInt(Math.min(buf.length - seekOff, 32000));
+
+        seeks[i % seeks.length] = seekOff;
+        reads[i % reads.length] = toRead;
+        verifyRead(stm, buf, seekOff, toRead);
+      }
+    } catch (AssertionError afe) {
+      StringBuilder sb = new StringBuilder();
+      sb.append("Sequence of actions:\n");
+      for (int j = 0; j < seeks.length; j++) {
+        sb.append("seek @ ").append(seeks[j]).append("  ")
+          .append("read ").append(reads[j]).append("\n");
+      }
+      LOG.error(sb.toString());
+      throw afe;
+    } finally {
+      stm.close();
+    }
+  }
+
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractSeekTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContract.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContract.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContract.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContract.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,201 @@
+/*
+ * 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.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+
+/**
+ * Class representing a filesystem contract that a filesystem
+ * implementation is expected implement.
+ *
+ * Part of this contract class is to allow FS implementations to
+ * provide specific opt outs and limits, so that tests can be
+ * skip unsupported features (e.g. case sensitivity tests),
+ * dangerous operations (e.g. trying to delete the root directory),
+ * and limit filesize and other numeric variables for scale tests
+ */
+public abstract class AbstractFSContract extends Configured {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractFSContract.class);
+
+  private boolean enabled = true;
+
+
+  /**
+   * Constructor: loads the authentication keys if found
+   * @param conf configuration to work with
+   */
+  protected AbstractFSContract(Configuration conf) {
+    super(conf);
+    if (maybeAddConfResource(ContractOptions.CONTRACT_OPTIONS_RESOURCE)) {
+      LOG.debug("Loaded authentication keys from {}", ContractOptions.CONTRACT_OPTIONS_RESOURCE);
+    } else {
+      LOG.debug("Not loaded: {}", ContractOptions.CONTRACT_OPTIONS_RESOURCE);
+    }
+  }
+
+  /**
+   * Any initialisation logic can go here
+   * @throws IOException IO problems
+   */
+  public void init() throws IOException {
+
+  }
+
+  /**
+   * Add a configuration resource to this instance's configuration
+   * @param resource resource reference
+   * @throws AssertionError if the resource was not found.
+   */
+  protected void addConfResource(String resource) {
+    boolean found = maybeAddConfResource(resource);
+    Assert.assertTrue("Resource not found " + resource, found);
+  }
+
+  /**
+   * Add a configuration resource to this instance's configuration,
+   * return true if the resource was found
+   * @param resource resource reference
+   */
+  protected boolean maybeAddConfResource(String resource) {
+    URL url = this.getClass().getClassLoader().getResource(resource);
+    boolean found = url != null;
+    if (found) {
+      getConf().addResource(resource);
+    }
+    return found;
+  }
+
+
+  /**
+   * Get the FS from a URI. The default implementation just retrieves
+   * it from the norrmal FileSystem factory/cache, with the local configuration
+   * @param uri URI of FS
+   * @return the filesystem
+   * @throws IOException IO problems
+   */
+  public FileSystem getFileSystem(URI uri) throws IOException {
+    return FileSystem.get(uri, getConf());
+  }
+
+  /**
+   * Get the filesystem for these tests
+   * @return the test fs
+   * @throws IOException IO problems
+   */
+  public abstract FileSystem getTestFileSystem() throws IOException;
+
+  /**
+   * Get the scheme of this FS
+   * @return the scheme this FS supports
+   */
+  public abstract String getScheme();
+
+  /**
+   * Return the path string for tests, e.g. <code>file:///tmp</code>
+   * @return a path in the test FS
+   */
+  public abstract Path getTestPath();
+
+  /**
+   * Boolean to indicate whether or not the contract test are enabled
+   * for this test run.
+   * @return true if the tests can be run.
+   */
+  public boolean isEnabled() {
+    return enabled;
+  }
+
+  /**
+   * Boolean to indicate whether or not the contract test are enabled
+   * for this test run.
+   * @param enabled flag which must be true if the tests can be run.
+   */
+  public void setEnabled(boolean enabled) {
+    this.enabled = enabled;
+  }
+
+  /**
+   * Query for a feature being supported. This may include a probe for the feature
+   *
+   * @param feature feature to query
+   * @param defval default value
+   * @return true if the feature is supported
+   * @throws IOException IO problems
+   */
+  public boolean isSupported(String feature, boolean defval) {
+    return getConf().getBoolean(getConfKey(feature), defval);
+  }
+
+  /**
+   * Query for a feature's limit. This may include a probe for the feature
+   *
+   * @param feature feature to query
+   * @param defval default value
+   * @return true if the feature is supported
+   * @throws IOException IO problems
+   */
+  public int getLimit(String feature, int defval) {
+    return getConf().getInt(getConfKey(feature), defval);
+  }
+
+  public String getOption(String feature, String defval) {
+    return getConf().get(getConfKey(feature), defval);
+  }
+
+  /**
+   * Build a configuration key
+   * @param feature feature to query
+   * @return the configuration key base with the feature appended
+   */
+  public String getConfKey(String feature) {
+    return ContractOptions.FS_CONTRACT_KEY + feature;
+  }
+
+  /**
+   * Create a URI off the scheme
+   * @param path path of URI
+   * @return a URI
+   * @throws IOException if the URI could not be created
+   */
+  protected URI toURI(String path) throws IOException {
+    try {
+      return new URI(getScheme(),path, null);
+    } catch (URISyntaxException e) {
+      throw new IOException(e.toString() + " with " + path, e);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "FSContract for " + getScheme();
+  }
+
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContract.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,363 @@
+/*
+ * 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.contract;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.internal.AssumptionViolatedException;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.cleanup;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
+
+/**
+ * This is the base class for all the contract tests
+ */
+public abstract class AbstractFSContractTestBase extends Assert
+  implements ContractOptions {
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(AbstractFSContractTestBase.class);
+
+  /**
+   * Length of files to work with: {@value}
+   */
+  public static final int TEST_FILE_LEN = 1024;
+
+  /**
+   * standard test timeout: {@value}
+   */
+  public static final int DEFAULT_TEST_TIMEOUT = 180 * 1000;
+
+  /**
+   * The FS contract used for these tets
+   */
+  private AbstractFSContract contract;
+
+  /**
+   * The test filesystem extracted from it
+   */
+  private FileSystem fileSystem;
+
+  /**
+   * The path for tests
+   */
+  private Path testPath;
+
+  /**
+   * This must be implemented by all instantiated test cases
+   * -provide the FS contract
+   * @return the FS contract
+   */
+  protected abstract AbstractFSContract createContract(Configuration conf);
+
+  /**
+   * Get the contract
+   * @return the contract, which will be non-null once the setup operation has
+   * succeeded
+   */
+  protected AbstractFSContract getContract() {
+    return contract;
+  }
+
+  /**
+   * Get the filesystem created in startup
+   * @return the filesystem to use for tests
+   */
+  public FileSystem getFileSystem() {
+    return fileSystem;
+  }
+
+  /**
+   * Get the log of the base class
+   * @return a logger
+   */
+  public static Logger getLog() {
+    return LOG;
+  }
+
+  /**
+   * Skip a test if a feature is unsupported in this FS
+   * @param feature feature to look for
+   * @throws IOException IO problem
+   */
+  protected void skipIfUnsupported(String feature) throws IOException {
+    if (!isSupported(feature)) {
+      skip("Skipping as unsupported feature: " + feature);
+    }
+  }
+
+  /**
+   * Is a feature supported?
+   * @param feature feature
+   * @return true iff the feature is supported
+   * @throws IOException IO problems
+   */
+  protected boolean isSupported(String feature) throws IOException {
+    return contract.isSupported(feature, false);
+  }
+
+  /**
+   * Include at the start of tests to skip them if the FS is not enabled.
+   */
+  protected void assumeEnabled() {
+    if (!contract.isEnabled())
+      throw new AssumptionViolatedException("test cases disabled for " + contract);
+  }
+
+  /**
+   * Create a configuration. May be overridden by tests/instantiations
+   * @return a configuration
+   */
+  protected Configuration createConfiguration() {
+    return new Configuration();
+  }
+
+  /**
+   * Set the timeout for every test
+   */
+  @Rule
+  public Timeout testTimeout = new Timeout(getTestTimeoutMillis());
+
+  /**
+   * Option for tests to override the default timeout value
+   * @return the current test timeout
+   */
+  protected int getTestTimeoutMillis() {
+    return DEFAULT_TEST_TIMEOUT;
+  }
+
+
+  /**
+   * Setup: create the contract then init it
+   * @throws Exception on any failure
+   */
+  @Before
+  public void setup() throws Exception {
+    contract = createContract(createConfiguration());
+    contract.init();
+    //skip tests if they aren't enabled
+    assumeEnabled();
+    //extract the test FS
+    fileSystem = contract.getTestFileSystem();
+    assertNotNull("null filesystem", fileSystem);
+    URI fsURI = fileSystem.getUri();
+    LOG.info("Test filesystem = {} implemented by {}",
+        fsURI, fileSystem);
+    //sanity check to make sure that the test FS picked up really matches
+    //the scheme chosen. This is to avoid defaulting back to the localFS
+    //which would be drastic for root FS tests
+    assertEquals("wrong filesystem of " + fsURI,
+                 contract.getScheme(), fsURI.getScheme());
+    //create the test path
+    testPath = getContract().getTestPath();
+    mkdirs(testPath);
+  }
+
+  /**
+   * Teardown
+   * @throws Exception on any failure
+   */
+  @After
+  public void teardown() throws Exception {
+    deleteTestDirInTeardown();
+  }
+
+  /**
+   * Delete the test dir in the per-test teardown
+   * @throws IOException
+   */
+  protected void deleteTestDirInTeardown() throws IOException {
+    cleanup("TEARDOWN", getFileSystem(), testPath);
+  }
+
+  /**
+   * Create a path under the test path provided by
+   * the FS contract
+   * @param filepath path string in
+   * @return a path qualified by the test filesystem
+   * @throws IOException IO problems
+   */
+  protected Path path(String filepath) throws IOException {
+    return getFileSystem().makeQualified(
+      new Path(getContract().getTestPath(), filepath));
+  }
+
+  /**
+   * Take a simple path like "/something" and turn it into
+   * a qualified path against the test FS
+   * @param filepath path string in
+   * @return a path qualified by the test filesystem
+   * @throws IOException IO problems
+   */
+  protected Path absolutepath(String filepath) throws IOException {
+    return getFileSystem().makeQualified(new Path(filepath));
+  }
+
+  /**
+   * List a path in the test FS
+   * @param path path to list
+   * @return the contents of the path/dir
+   * @throws IOException IO problems
+   */
+  protected String ls(Path path) throws IOException {
+    return ContractTestUtils.ls(fileSystem, path);
+  }
+
+  /**
+   * Describe a test. This is a replacement for javadocs
+   * where the tests role is printed in the log output
+   * @param text description
+   */
+  protected void describe(String text) {
+    LOG.info(text);
+  }
+
+  /**
+   * Handle the outcome of an operation not being the strictest
+   * exception desired, but one that, while still within the boundary
+   * of the contract, is a bit looser.
+   *
+   * If the FS contract says that they support the strictest exceptions,
+   * that is what they must return, and the exception here is rethrown
+   * @param action Action
+   * @param expectedException what was expected
+   * @param e exception that was received
+   */
+  protected void handleRelaxedException(String action,
+                                        String expectedException,
+                                        Exception e) throws Exception {
+    if (getContract().isSupported(SUPPORTS_STRICT_EXCEPTIONS, false)) {
+      throw e;
+    }
+    LOG.warn("The expected exception {}  was not the exception class" +
+             " raised on {}: {}", action , e.getClass(), expectedException, e);
+  }
+
+  /**
+   * Handle expected exceptions through logging and/or other actions
+   * @param e exception raised.
+   */
+  protected void handleExpectedException(Exception e) {
+    getLog().debug("expected :{}" ,e, e);
+  }
+
+  /**
+   * assert that a path exists
+   * @param message message to use in an assertion
+   * @param path path to probe
+   * @throws IOException IO problems
+   */
+  public void assertPathExists(String message, Path path) throws IOException {
+    ContractTestUtils.assertPathExists(fileSystem, message, path);
+  }
+
+  /**
+   * assert that a path does not
+   * @param message message to use in an assertion
+   * @param path path to probe
+   * @throws IOException IO problems
+   */
+  public void assertPathDoesNotExist(String message, Path path) throws
+                                                                IOException {
+    ContractTestUtils.assertPathDoesNotExist(fileSystem, message, path);
+  }
+
+  /**
+   * Assert that a file exists and whose {@link FileStatus} entry
+   * declares that this is a file and not a symlink or directory.
+   *
+   * @param filename name of the file
+   * @throws IOException IO problems during file operations
+   */
+  protected void assertIsFile(Path filename) throws IOException {
+    ContractTestUtils.assertIsFile(fileSystem, filename);
+  }
+
+  /**
+   * Assert that a file exists and whose {@link FileStatus} entry
+   * declares that this is a file and not a symlink or directory.
+   *
+   * @param path name of the file
+   * @throws IOException IO problems during file operations
+   */
+  protected void assertIsDirectory(Path path) throws IOException {
+    ContractTestUtils.assertIsDirectory(fileSystem, path);
+  }
+
+
+  /**
+   * Assert that a file exists and whose {@link FileStatus} entry
+   * declares that this is a file and not a symlink or directory.
+   *
+   * @throws IOException IO problems during file operations
+   */
+  protected void mkdirs(Path path) throws IOException {
+    assertTrue("Failed to mkdir " + path, fileSystem.mkdirs(path));
+  }
+
+  /**
+   * Assert that a delete succeeded
+   * @param path path to delete
+   * @param recursive recursive flag
+   * @throws IOException IO problems
+   */
+  protected void assertDeleted(Path path, boolean recursive) throws
+                                                             IOException {
+    ContractTestUtils.assertDeleted(fileSystem, path, recursive);
+  }
+
+  /**
+   * Assert that the result value == -1; which implies
+   * that a read was successful
+   * @param text text to include in a message (usually the operation)
+   * @param result read result to validate
+   */
+  protected void assertMinusOne(String text, int result) {
+    assertEquals(text + " wrong read result " + result, -1, result);
+  }
+
+  boolean rename(Path src, Path dst) throws IOException {
+    return getFileSystem().rename(src, dst);
+  }
+
+  protected String generateAndLogErrorListing(Path src, Path dst) throws
+                                                                  IOException {
+    FileSystem fs = getFileSystem();
+    getLog().error(
+      "src dir " + ContractTestUtils.ls(fs, src.getParent()));
+    String destDirLS = ContractTestUtils.ls(fs, dst.getParent());
+    if (fs.isDirectory(dst)) {
+      //include the dir into the listing
+      destDirLS = destDirLS + "\n" + ContractTestUtils.ls(fs, dst);
+    }
+    return destDirLS;
+  }
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractOptions.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractOptions.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractOptions.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractOptions.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,170 @@
+/*
+ * 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.contract;
+
+/**
+ * Options for contract tests: keys for FS-specific values,
+ * defaults.
+ */
+public interface ContractOptions {
+
+
+  /**
+   * name of the (optional) resource containing filesystem binding keys : {@value}
+   * If found, it it will be loaded
+   */
+  String CONTRACT_OPTIONS_RESOURCE = "contract-test-options.xml";
+
+  /**
+   * Prefix for all contract keys in the configuration files
+   */
+  String FS_CONTRACT_KEY = "fs.contract.";
+
+  /**
+   * Is a filesystem case sensitive.
+   * Some of the filesystems that say "no" here may mean
+   * that it varies from platform to platform -the localfs being the key
+   * example.
+   */
+  String IS_CASE_SENSITIVE = "is-case-sensitive";
+
+  /**
+   * Blobstore flag. Implies it's not a real directory tree and
+   * consistency is below that which Hadoop expects
+   */
+  String IS_BLOBSTORE = "is-blobstore";
+
+  /**
+   * Flag to indicate that the FS can rename into directories that
+   * don't exist, creating them as needed.
+   * @{value}
+   */
+  String RENAME_CREATES_DEST_DIRS = "rename-creates-dest-dirs";
+
+  /**
+   * Flag to indicate that the FS does not follow the rename contract -and
+   * instead only returns false on a failure.
+   * @{value}
+   */
+  String RENAME_OVERWRITES_DEST = "rename-overwrites-dest";
+
+  /**
+   * Flag to indicate that the FS returns false if the destination exists
+   * @{value}
+   */
+  String RENAME_RETURNS_FALSE_IF_DEST_EXISTS =
+      "rename-returns-false-if-dest-exists";
+
+  /**
+   * Flag to indicate that the FS returns false on a rename
+   * if the source is missing
+   * @{value}
+   */
+  String RENAME_RETURNS_FALSE_IF_SOURCE_MISSING =
+      "rename-returns-false-if-source-missing";
+
+  /**
+   * Flag to indicate that append is supported
+   * @{value}
+   */
+  String SUPPORTS_APPEND = "supports-append";
+
+  /**
+   * Flag to indicate that renames are atomic
+   * @{value}
+   */
+  String SUPPORTS_ATOMIC_RENAME = "supports-atomic-rename";
+
+  /**
+   * Flag to indicate that directory deletes are atomic
+   * @{value}
+   */
+  String SUPPORTS_ATOMIC_DIRECTORY_DELETE = "supports-atomic-directory-delete";
+
+  /**
+   * Does the FS support multiple block locations?
+   * @{value}
+   */
+  String SUPPORTS_BLOCK_LOCALITY = "supports-block-locality";
+
+  /**
+   * Does the FS support the concat() operation?
+   * @{value}
+   */
+  String SUPPORTS_CONCAT = "supports-concat";
+
+  /**
+   * Is seeking supported at all?
+   * @{value}
+   */
+  String SUPPORTS_SEEK = "supports-seek";
+
+  /**
+   * Is seeking past the EOF allowed?
+   * @{value}
+   */
+  String REJECTS_SEEK_PAST_EOF = "rejects-seek-past-eof";
+
+  /**
+   * Is seeking on a closed file supported? Some filesystems only raise an
+   * exception later, when trying to read.
+   * @{value}
+   */
+  String SUPPORTS_SEEK_ON_CLOSED_FILE = "supports-seek-on-closed-file";
+
+  /**
+   * Flag to indicate that this FS expects to throw the strictest
+   * exceptions it can, not generic IOEs, which, if returned,
+   * must be rejected.
+   * @{value}
+   */
+  String SUPPORTS_STRICT_EXCEPTIONS = "supports-strict-exceptions";
+
+  /**
+   * Are unix permissions
+   * @{value}
+   */
+  String SUPPORTS_UNIX_PERMISSIONS = "supports-unix-permissions";
+
+  /**
+   * Maximum path length
+   * @{value}
+   */
+  String MAX_PATH_ = "max-path";
+
+  /**
+   * Maximum filesize: 0 or -1 for no limit
+   * @{value}
+   */
+  String MAX_FILESIZE = "max-filesize";
+
+  /**
+   * Flag to indicate that tests on the root directories of a filesystem/
+   * object store are permitted
+   * @{value}
+   */
+  String TEST_ROOT_TESTS_ENABLED = "test.root-tests-enabled";
+
+  /**
+   * Limit for #of random seeks to perform.
+   * Keep low for remote filesystems for faster tests
+   */
+  String TEST_RANDOM_SEEK_COUNT = "test.random-seek-count";
+
+}

Propchange: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractOptions.java
------------------------------------------------------------------------------
    svn:eol-style = native