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 [4/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/ContractTestUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,759 @@
+/*
+ * 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.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Assert;
+import org.junit.internal.AssumptionViolatedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Properties;
+
+/**
+ * Utilities used across test cases
+ */
+public class ContractTestUtils extends Assert {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContractTestUtils.class);
+
+  public static final String IO_FILE_BUFFER_SIZE = "io.file.buffer.size";
+
+  /**
+   * Assert that a property in the property set matches the expected value
+   * @param props property set
+   * @param key property name
+   * @param expected expected value. If null, the property must not be in the set
+   */
+  public static void assertPropertyEquals(Properties props,
+                                          String key,
+                                          String expected) {
+    String val = props.getProperty(key);
+    if (expected == null) {
+      assertNull("Non null property " + key + " = " + val, val);
+    } else {
+      assertEquals("property " + key + " = " + val,
+                          expected,
+                          val);
+    }
+  }
+
+  /**
+   *
+   * Write a file and read it in, validating the result. Optional flags control
+   * whether file overwrite operations should be enabled, and whether the
+   * file should be deleted afterwards.
+   *
+   * If there is a mismatch between what was written and what was expected,
+   * a small range of bytes either side of the first error are logged to aid
+   * diagnosing what problem occurred -whether it was a previous file
+   * or a corrupting of the current file. This assumes that two
+   * sequential runs to the same path use datasets with different character
+   * moduli.
+   *
+   * @param fs filesystem
+   * @param path path to write to
+   * @param len length of data
+   * @param overwrite should the create option allow overwrites?
+   * @param delete should the file be deleted afterwards? -with a verification
+   * that it worked. Deletion is not attempted if an assertion has failed
+   * earlier -it is not in a <code>finally{}</code> block.
+   * @throws IOException IO problems
+   */
+  public static void writeAndRead(FileSystem fs,
+                                  Path path,
+                                  byte[] src,
+                                  int len,
+                                  int blocksize,
+                                  boolean overwrite,
+                                  boolean delete) throws IOException {
+    fs.mkdirs(path.getParent());
+
+    writeDataset(fs, path, src, len, blocksize, overwrite);
+
+    byte[] dest = readDataset(fs, path, len);
+
+    compareByteArrays(src, dest, len);
+
+    if (delete) {
+      rejectRootOperation(path);
+      boolean deleted = fs.delete(path, false);
+      assertTrue("Deleted", deleted);
+      assertPathDoesNotExist(fs, "Cleanup failed", path);
+    }
+  }
+
+  /**
+   * Write a file.
+   * Optional flags control
+   * whether file overwrite operations should be enabled
+   * @param fs filesystem
+   * @param path path to write to
+   * @param len length of data
+   * @param overwrite should the create option allow overwrites?
+   * @throws IOException IO problems
+   */
+  public static void writeDataset(FileSystem fs,
+                                   Path path,
+                                   byte[] src,
+                                   int len,
+                                   int buffersize,
+                                   boolean overwrite) throws IOException {
+    assertTrue(
+      "Not enough data in source array to write " + len + " bytes",
+      src.length >= len);
+    FSDataOutputStream out = fs.create(path,
+                                       overwrite,
+                                       fs.getConf()
+                                         .getInt(IO_FILE_BUFFER_SIZE,
+                                                 4096),
+                                       (short) 1,
+                                       buffersize);
+    out.write(src, 0, len);
+    out.close();
+    assertFileHasLength(fs, path, len);
+  }
+
+  /**
+   * Read the file and convert to a byte dataset.
+   * This implements readfully internally, so that it will read
+   * in the file without ever having to seek()
+   * @param fs filesystem
+   * @param path path to read from
+   * @param len length of data to read
+   * @return the bytes
+   * @throws IOException IO problems
+   */
+  public static byte[] readDataset(FileSystem fs, Path path, int len)
+      throws IOException {
+    FSDataInputStream in = fs.open(path);
+    byte[] dest = new byte[len];
+    int offset =0;
+    int nread = 0;
+    try {
+      while (nread < len) {
+        int nbytes = in.read(dest, offset + nread, len - nread);
+        if (nbytes < 0) {
+          throw new EOFException("End of file reached before reading fully.");
+        }
+        nread += nbytes;
+      }
+    } finally {
+      in.close();
+    }
+    return dest;
+  }
+
+  /**
+   * Read a file, verify its length and contents match the expected array
+   * @param fs filesystem
+   * @param path path to file
+   * @param original original dataset
+   * @throws IOException IO Problems
+   */
+  public static void verifyFileContents(FileSystem fs,
+                                        Path path,
+                                        byte[] original) throws IOException {
+    FileStatus stat = fs.getFileStatus(path);
+    String statText = stat.toString();
+    assertTrue("not a file " + statText, stat.isFile());
+    assertEquals("wrong length " + statText, original.length, stat.getLen());
+    byte[] bytes = readDataset(fs, path, original.length);
+    compareByteArrays(original,bytes,original.length);
+  }
+
+  /**
+   * Verify that the read at a specific offset in a stream
+   * matches that expected
+   * @param stm stream
+   * @param fileContents original file contents
+   * @param seekOff seek offset
+   * @param toRead number of bytes to read
+   * @throws IOException IO problems
+   */
+  public static void verifyRead(FSDataInputStream stm, byte[] fileContents,
+                                int seekOff, int toRead) throws IOException {
+    byte[] out = new byte[toRead];
+    stm.seek(seekOff);
+    stm.readFully(out);
+    byte[] expected = Arrays.copyOfRange(fileContents, seekOff,
+                                         seekOff + toRead);
+    compareByteArrays(expected, out,toRead);
+  }
+
+  /**
+   * Assert that tthe array original[0..len] and received[] are equal.
+   * A failure triggers the logging of the bytes near where the first
+   * difference surfaces.
+   * @param original source data
+   * @param received actual
+   * @param len length of bytes to compare
+   */
+  public static void compareByteArrays(byte[] original,
+                                       byte[] received,
+                                       int len) {
+    assertEquals("Number of bytes read != number written",
+                        len, received.length);
+    int errors = 0;
+    int first_error_byte = -1;
+    for (int i = 0; i < len; i++) {
+      if (original[i] != received[i]) {
+        if (errors == 0) {
+          first_error_byte = i;
+        }
+        errors++;
+      }
+    }
+
+    if (errors > 0) {
+      String message = String.format(" %d errors in file of length %d",
+                                     errors, len);
+      LOG.warn(message);
+      // the range either side of the first error to print
+      // this is a purely arbitrary number, to aid user debugging
+      final int overlap = 10;
+      for (int i = Math.max(0, first_error_byte - overlap);
+           i < Math.min(first_error_byte + overlap, len);
+           i++) {
+        byte actual = received[i];
+        byte expected = original[i];
+        String letter = toChar(actual);
+        String line = String.format("[%04d] %2x %s\n", i, actual, letter);
+        if (expected != actual) {
+          line = String.format("[%04d] %2x %s -expected %2x %s\n",
+                               i,
+                               actual,
+                               letter,
+                               expected,
+                               toChar(expected));
+        }
+        LOG.warn(line);
+      }
+      fail(message);
+    }
+  }
+
+  /**
+   * Convert a byte to a character for printing. If the
+   * byte value is < 32 -and hence unprintable- the byte is
+   * returned as a two digit hex value
+   * @param b byte
+   * @return the printable character string
+   */
+  public static String toChar(byte b) {
+    if (b >= 0x20) {
+      return Character.toString((char) b);
+    } else {
+      return String.format("%02x", b);
+    }
+  }
+
+  /**
+   * Convert a buffer to a string, character by character
+   * @param buffer input bytes
+   * @return a string conversion
+   */
+  public static String toChar(byte[] buffer) {
+    StringBuilder builder = new StringBuilder(buffer.length);
+    for (byte b : buffer) {
+      builder.append(toChar(b));
+    }
+    return builder.toString();
+  }
+
+  public static byte[] toAsciiByteArray(String s) {
+    char[] chars = s.toCharArray();
+    int len = chars.length;
+    byte[] buffer = new byte[len];
+    for (int i = 0; i < len; i++) {
+      buffer[i] = (byte) (chars[i] & 0xff);
+    }
+    return buffer;
+  }
+
+  /**
+   * Cleanup at the end of a test run
+   * @param action action triggering the operation (for use in logging)
+   * @param fileSystem filesystem to work with. May be null
+   * @param cleanupPath path to delete as a string
+   */
+  public static void cleanup(String action,
+                             FileSystem fileSystem,
+                             String cleanupPath) {
+    if (fileSystem == null) {
+      return;
+    }
+    Path path = new Path(cleanupPath).makeQualified(fileSystem.getUri(),
+        fileSystem.getWorkingDirectory());
+    cleanup(action, fileSystem, path);
+  }
+
+  /**
+   * Cleanup at the end of a test run
+   * @param action action triggering the operation (for use in logging)
+   * @param fileSystem filesystem to work with. May be null
+   * @param path path to delete
+   */
+  public static void cleanup(String action, FileSystem fileSystem, Path path) {
+    noteAction(action);
+    try {
+      rm(fileSystem, path, true, false);
+    } catch (Exception e) {
+      LOG.error("Error deleting in "+ action + " - "  + path + ": " + e, e);
+    }
+  }
+
+  /**
+   * Delete a directory. There's a safety check for operations against the
+   * root directory -these are intercepted and rejected with an IOException
+   * unless the allowRootDelete flag is true
+   * @param fileSystem filesystem to work with. May be null
+   * @param path path to delete
+   * @param recursive flag to enable recursive delete
+   * @param allowRootDelete can the root directory be deleted?
+   * @throws IOException on any problem.
+   */
+  public static boolean rm(FileSystem fileSystem,
+      Path path,
+      boolean recursive,
+      boolean allowRootDelete) throws
+      IOException {
+    if (fileSystem != null) {
+      rejectRootOperation(path, allowRootDelete);
+      if (fileSystem.exists(path)) {
+        return fileSystem.delete(path, recursive);
+      }
+    }
+    return false;
+
+  }
+
+  /**
+   * Block any operation on the root path. This is a safety check
+   * @param path path in the filesystem
+   * @param allowRootOperation can the root directory be manipulated?
+   * @throws IOException if the operation was rejected
+   */
+  public static void rejectRootOperation(Path path,
+      boolean allowRootOperation) throws IOException {
+    if (path.isRoot() && !allowRootOperation) {
+      throw new IOException("Root directory operation rejected: " + path);
+    }
+  }
+
+  /**
+   * Block any operation on the root path. This is a safety check
+   * @param path path in the filesystem
+   * @throws IOException if the operation was rejected
+   */
+  public static void rejectRootOperation(Path path) throws IOException {
+    rejectRootOperation(path, false);
+  }
+
+
+  public static void noteAction(String action) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("==============  "+ action +" =============");
+    }
+  }
+
+  /**
+   * downgrade a failure to a message and a warning, then an
+   * exception for the Junit test runner to mark as failed
+   * @param message text message
+   * @param failure what failed
+   * @throws AssumptionViolatedException always
+   */
+  public static void downgrade(String message, Throwable failure) {
+    LOG.warn("Downgrading test " + message, failure);
+    AssumptionViolatedException ave =
+      new AssumptionViolatedException(failure, null);
+    throw ave;
+  }
+
+  /**
+   * report an overridden test as unsupported
+   * @param message message to use in the text
+   * @throws AssumptionViolatedException always
+   */
+  public static void unsupported(String message) {
+    skip(message);
+  }
+
+  /**
+   * report a test has been skipped for some reason
+   * @param message message to use in the text
+   * @throws AssumptionViolatedException always
+   */
+  public static void skip(String message) {
+    LOG.info("Skipping: {}", message);
+    throw new AssumptionViolatedException(message);
+  }
+
+  /**
+   * Fail with an exception that was received
+   * @param text text to use in the exception
+   * @param thrown a (possibly null) throwable to init the cause with
+   * @throws AssertionError with the text and throwable -always
+   */
+  public static void fail(String text, Throwable thrown) {
+    AssertionError e = new AssertionError(text);
+    e.initCause(thrown);
+    throw e;
+  }
+
+  /**
+   * Make an assertion about the length of a file
+   * @param fs filesystem
+   * @param path path of the file
+   * @param expected expected length
+   * @throws IOException on File IO problems
+   */
+  public static void assertFileHasLength(FileSystem fs, Path path,
+                                         int expected) throws IOException {
+    FileStatus status = fs.getFileStatus(path);
+    assertEquals(
+      "Wrong file length of file " + path + " status: " + status,
+      expected,
+      status.getLen());
+  }
+
+  /**
+   * Assert that a path refers to a directory
+   * @param fs filesystem
+   * @param path path of the directory
+   * @throws IOException on File IO problems
+   */
+  public static void assertIsDirectory(FileSystem fs,
+                                       Path path) throws IOException {
+    FileStatus fileStatus = fs.getFileStatus(path);
+    assertIsDirectory(fileStatus);
+  }
+
+  /**
+   * Assert that a path refers to a directory
+   * @param fileStatus stats to check
+   */
+  public static void assertIsDirectory(FileStatus fileStatus) {
+    assertTrue("Should be a directory -but isn't: " + fileStatus,
+               fileStatus.isDirectory());
+  }
+
+  /**
+   * Write the text to a file, returning the converted byte array
+   * for use in validating the round trip
+   * @param fs filesystem
+   * @param path path of file
+   * @param text text to write
+   * @param overwrite should the operation overwrite any existing file?
+   * @return the read bytes
+   * @throws IOException on IO problems
+   */
+  public static byte[] writeTextFile(FileSystem fs,
+                                   Path path,
+                                   String text,
+                                   boolean overwrite) throws IOException {
+    byte[] bytes = new byte[0];
+    if (text != null) {
+      bytes = toAsciiByteArray(text);
+    }
+    createFile(fs, path, overwrite, bytes);
+    return bytes;
+  }
+
+  /**
+   * Create a file
+   * @param fs filesystem
+   * @param path       path to write
+   * @param overwrite overwrite flag
+   * @param data source dataset. Can be null
+   * @throws IOException on any problem
+   */
+  public static void createFile(FileSystem fs,
+                                 Path path,
+                                 boolean overwrite,
+                                 byte[] data) throws IOException {
+    FSDataOutputStream stream = fs.create(path, overwrite);
+    if (data != null && data.length > 0) {
+      stream.write(data);
+    }
+    stream.close();
+  }
+
+  /**
+   * Touch a file
+   * @param fs filesystem
+   * @param path path
+   * @throws IOException IO problems
+   */
+  public static void touch(FileSystem fs,
+                           Path path) throws IOException {
+    createFile(fs, path, true, null);
+  }
+
+  /**
+   * Delete a file/dir and assert that delete() returned true
+   * <i>and</i> that the path no longer exists. This variant rejects
+   * all operations on root directories
+   * @param fs filesystem
+   * @param file path to delete
+   * @param recursive flag to enable recursive delete
+   * @throws IOException IO problems
+   */
+  public static void assertDeleted(FileSystem fs,
+                                   Path file,
+                                   boolean recursive) throws IOException {
+    assertDeleted(fs, file, recursive, false);
+  }
+
+  /**
+   * Delete a file/dir and assert that delete() returned true
+   * <i>and</i> that the path no longer exists. This variant rejects
+   * all operations on root directories
+   * @param fs filesystem
+   * @param file path to delete
+   * @param recursive flag to enable recursive delete
+   * @param allowRootOperations can the root dir be deleted?
+   * @throws IOException IO problems
+   */
+  public static void assertDeleted(FileSystem fs,
+      Path file,
+      boolean recursive,
+      boolean allowRootOperations) throws IOException {
+    rejectRootOperation(file, allowRootOperations);
+    assertPathExists(fs, "about to be deleted file", file);
+    boolean deleted = fs.delete(file, recursive);
+    String dir = ls(fs, file.getParent());
+    assertTrue("Delete failed on " + file + ": " + dir, deleted);
+    assertPathDoesNotExist(fs, "Deleted file", file);
+  }
+
+  /**
+   * Read in "length" bytes, convert to an ascii string
+   * @param fs filesystem
+   * @param path path to read
+   * @param length #of bytes to read.
+   * @return the bytes read and converted to a string
+   * @throws IOException IO problems
+   */
+  public static String readBytesToString(FileSystem fs,
+                                  Path path,
+                                  int length) throws IOException {
+    FSDataInputStream in = fs.open(path);
+    try {
+      byte[] buf = new byte[length];
+      in.readFully(0, buf);
+      return toChar(buf);
+    } finally {
+      in.close();
+    }
+  }
+
+  /**
+   * Take an array of filestats and convert to a string (prefixed w/ a [01] counter
+   * @param stats array of stats
+   * @param separator separator after every entry
+   * @return a stringified set
+   */
+  public static String fileStatsToString(FileStatus[] stats, String separator) {
+    StringBuilder buf = new StringBuilder(stats.length * 128);
+    for (int i = 0; i < stats.length; i++) {
+      buf.append(String.format("[%02d] %s", i, stats[i])).append(separator);
+    }
+    return buf.toString();
+  }
+
+  /**
+   * List a directory
+   * @param fileSystem FS
+   * @param path path
+   * @return a directory listing or failure message
+   * @throws IOException
+   */
+  public static String ls(FileSystem fileSystem, Path path) throws IOException {
+    if (path == null) {
+      //surfaces when someone calls getParent() on something at the top of the path
+      return "/";
+    }
+    FileStatus[] stats;
+    String pathtext = "ls " + path;
+    try {
+      stats = fileSystem.listStatus(path);
+    } catch (FileNotFoundException e) {
+      return pathtext + " -file not found";
+    } catch (IOException e) {
+      return pathtext + " -failed: " + e;
+    }
+    return dumpStats(pathtext, stats);
+  }
+
+  public static String dumpStats(String pathname, FileStatus[] stats) {
+    return pathname + fileStatsToString(stats, "\n");
+  }
+
+   /**
+   * Assert that a file exists and whose {@link FileStatus} entry
+   * declares that this is a file and not a symlink or directory.
+   * @param fileSystem filesystem to resolve path against
+   * @param filename name of the file
+   * @throws IOException IO problems during file operations
+   */
+  public static void assertIsFile(FileSystem fileSystem, Path filename) throws
+                                                                 IOException {
+    assertPathExists(fileSystem, "Expected file", filename);
+    FileStatus status = fileSystem.getFileStatus(filename);
+    assertIsFile(filename, status);
+  }
+
+  /**
+   * 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
+   * @param status file status
+   */
+  public static void assertIsFile(Path filename, FileStatus status) {
+    String fileInfo = filename + "  " + status;
+    assertFalse("File claims to be a directory " + fileInfo,
+                status.isDirectory());
+    assertFalse("File claims to be a symlink " + fileInfo,
+                       status.isSymlink());
+  }
+
+  /**
+   * Create a dataset for use in the tests; all data is in the range
+   * base to (base+modulo-1) inclusive
+   * @param len length of data
+   * @param base base of the data
+   * @param modulo the modulo
+   * @return the newly generated dataset
+   */
+  public static byte[] dataset(int len, int base, int modulo) {
+    byte[] dataset = new byte[len];
+    for (int i = 0; i < len; i++) {
+      dataset[i] = (byte) (base + (i % modulo));
+    }
+    return dataset;
+  }
+
+  /**
+   * Assert that a path exists -but make no assertions as to the
+   * type of that entry
+   *
+   * @param fileSystem filesystem to examine
+   * @param message message to include in the assertion failure message
+   * @param path path in the filesystem
+   * @throws FileNotFoundException raised if the path is missing
+   * @throws IOException IO problems
+   */
+  public static void assertPathExists(FileSystem fileSystem, String message,
+                               Path path) throws IOException {
+    if (!fileSystem.exists(path)) {
+      //failure, report it
+      ls(fileSystem, path.getParent());
+      throw new FileNotFoundException(message + ": not found " + path
+                                      + " in " + path.getParent());
+    }
+  }
+
+  /**
+   * Assert that a path does not exist
+   *
+   * @param fileSystem filesystem to examine
+   * @param message message to include in the assertion failure message
+   * @param path path in the filesystem
+   * @throws IOException IO problems
+   */
+  public static void assertPathDoesNotExist(FileSystem fileSystem,
+                                            String message,
+                                            Path path) throws IOException {
+    try {
+      FileStatus status = fileSystem.getFileStatus(path);
+      fail(message + ": unexpectedly found " + path + " as  " + status);
+    } catch (FileNotFoundException expected) {
+      //this is expected
+
+    }
+  }
+
+  /**
+   * Assert that a FileSystem.listStatus on a dir finds the subdir/child entry
+   * @param fs filesystem
+   * @param dir directory to scan
+   * @param subdir full path to look for
+   * @throws IOException IO probles
+   */
+  public static void assertListStatusFinds(FileSystem fs,
+                                           Path dir,
+                                           Path subdir) throws IOException {
+    FileStatus[] stats = fs.listStatus(dir);
+    boolean found = false;
+    StringBuilder builder = new StringBuilder();
+    for (FileStatus stat : stats) {
+      builder.append(stat.toString()).append('\n');
+      if (stat.getPath().equals(subdir)) {
+        found = true;
+      }
+    }
+    assertTrue("Path " + subdir
+                      + " not found in directory " + dir + ":" + builder,
+                      found);
+  }
+
+  /**
+   * Test for the host being an OSX machine
+   * @return true if the JVM thinks that is running on OSX
+   */
+  public static boolean isOSX() {
+    return System.getProperty("os.name").contains("OS X");
+  }
+
+  /**
+   * compare content of file operations using a double byte array
+   * @param concat concatenated files
+   * @param bytes bytes
+   */
+  public static void validateFileContent(byte[] concat, byte[][] bytes) {
+    int idx = 0;
+    boolean mismatch = false;
+
+    for (byte[] bb : bytes) {
+      for (byte b : bb) {
+        if (b != concat[idx++]) {
+          mismatch = true;
+          break;
+        }
+      }
+      if (mismatch)
+        break;
+    }
+    assertFalse("File content of file is not as expected at offset " + idx,
+                mismatch);
+  }
+
+
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/FTPContract.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/FTPContract.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/FTPContract.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/FTPContract.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,63 @@
+/*
+ * 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.ftp;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
+import org.junit.Assert;
+
+import java.net.URI;
+
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * The contract of FTP; requires the option "test.testdir" to be set
+ */
+public class FTPContract extends AbstractBondedFSContract {
+
+  public static final String CONTRACT_XML = "contract/ftp.xml";
+  /**
+   *
+   */
+  public static final String TEST_FS_TESTDIR = "test.ftp.testdir";
+  private String fsName;
+  private URI fsURI;
+  private FileSystem fs;
+
+  public FTPContract(Configuration conf) {
+    super(conf);
+    //insert the base features
+    addConfResource(CONTRACT_XML);
+  }
+
+  @Override
+  public String getScheme() {
+    return "ftp";
+  }
+
+  @Override
+  public Path getTestPath() {
+    String pathString = getOption(TEST_FS_TESTDIR, null);
+    assertNotNull("Undefined test option " + TEST_FS_TESTDIR, pathString);
+    Path path = new Path(pathString);
+    return path;
+  }
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractCreate.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractCreate.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractCreate.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractCreate.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,32 @@
+/*
+ * 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.ftp;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestFTPContractCreate extends AbstractContractCreateTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new FTPContract(conf);
+  }
+
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractDelete.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractDelete.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractDelete.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractDelete.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,32 @@
+/*
+ * 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.ftp;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestFTPContractDelete extends AbstractContractDeleteTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new FTPContract(conf);
+  }
+
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractMkdir.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractMkdir.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractMkdir.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractMkdir.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,34 @@
+/*
+ * 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.ftp;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Test dir operations on a the local FS.
+ */
+public class TestFTPContractMkdir extends AbstractContractMkdirTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new FTPContract(conf);
+  }
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractOpen.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractOpen.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractOpen.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractOpen.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,32 @@
+/*
+ * 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.ftp;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestFTPContractOpen extends AbstractContractOpenTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new FTPContract(conf);
+  }
+
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractRename.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractRename.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractRename.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/TestFTPContractRename.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,66 @@
+/*
+ * 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.ftp;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.ftp.FTPFileSystem;
+
+import java.io.IOException;
+
+public class TestFTPContractRename extends AbstractContractRenameTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new FTPContract(conf);
+  }
+
+  /**
+   * Check the exception was about cross-directory renames
+   * -if not, rethrow it.
+   * @param e exception raised
+   * @throws IOException
+   */
+  private void verifyUnsupportedDirRenameException(IOException e) throws IOException {
+    if (!e.toString().contains(FTPFileSystem.E_SAME_DIRECTORY_ONLY)) {
+      throw e;
+    }
+  }
+
+  @Override
+  public void testRenameDirIntoExistingDir() throws Throwable {
+    try {
+      super.testRenameDirIntoExistingDir();
+      fail("Expected a failure");
+    } catch (IOException e) {
+      verifyUnsupportedDirRenameException(e);
+    }
+  }
+
+  @Override
+  public void testRenameFileNonexistentDir() throws Throwable {
+    try {
+      super.testRenameFileNonexistentDir();
+      fail("Expected a failure");
+    } catch (IOException e) {
+      verifyUnsupportedDirRenameException(e);
+    }
+  }
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/package.html
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/package.html?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/package.html (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/package.html Thu Jul  3 12:04:50 2014
@@ -0,0 +1,56 @@
+<!DOCTYPE html>
+<!--
+  ~ 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.
+  -->
+
+<html>
+<head>
+  <title>FTP Contract Tests</title>
+</head>
+<body>
+<h1>FTP Contract</h1>
+
+This package contains tests that verify the FTP filesystem works close to what
+a Hadoop application expects.
+<p></p>
+All these tests are skipped unless a test filesystem is provided
+in <code>hadoop-common/src/test/resources/core-site.xml</code>
+<pre>
+
+  &lt;property&gt;
+    &lt;name&gt;fs.ftp.contract.test.fs.name&lt;/name&gt;
+    &lt;value&gt;ftp://ftpserver/&lt;/value&gt;
+  &lt;/property&gt;
+
+  &lt;property&gt;
+    &lt;name&gt;fs.ftp.contract.test.testdir&lt;/name&gt;
+    &lt;value&gt;/home/testuser/test&lt;/value&gt;
+  &lt;/property&gt;
+
+  &lt;property&gt;
+    &lt;name&gt;fs.ftp.user.ftpserver&lt;/name&gt;
+    &lt;value&gt;testuser&lt;/value&gt;
+  &lt;/property&gt;
+
+  &lt;property&gt;
+    &lt;name&gt;fs.ftp.password.ftpserver&lt;/name&gt;
+    &lt;value&gt;remember-not-to-check-this-file-in&lt;/value&gt;
+  &lt;/property&gt;
+</pre>
+
+</body>
+</html>
\ No newline at end of file

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/LocalFSContract.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/LocalFSContract.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/LocalFSContract.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/LocalFSContract.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,116 @@
+/*
+ * 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.localfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.ContractOptions;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.util.Shell;
+
+import java.io.IOException;
+
+/**
+ * The contract of the Local filesystem.
+ * This changes its feature set from platform for platform -the default
+ * set is updated during initialization.
+ *
+ * This contract contains some override points, to permit
+ * the raw local filesystem and other filesystems to subclass it.
+ */
+public class LocalFSContract extends AbstractFSContract {
+
+  public static final String CONTRACT_XML = "contract/localfs.xml";
+  public static final String SYSPROP_TEST_BUILD_DATA = "test.build.data";
+  public static final String DEFAULT_TEST_BUILD_DATA_DIR = "test/build/data";
+  private FileSystem fs;
+
+  public LocalFSContract(Configuration conf) {
+    super(conf);
+    //insert the base features
+    addConfResource(getContractXml());
+  }
+
+  /**
+   * Return the contract file for this filesystem
+   * @return the XML
+   */
+  protected String getContractXml() {
+    return CONTRACT_XML;
+  }
+
+  @Override
+  public void init() throws IOException {
+    super.init();
+    fs = getLocalFS();
+    adjustContractToLocalEnvironment();
+  }
+
+  /**
+   *  tweak some of the contract parameters based on the local system
+   *  state
+   */
+  protected void adjustContractToLocalEnvironment() {
+    if (Shell.WINDOWS) {
+      //NTFS doesn't do case sensitivity, and its permissions are ACL-based
+      getConf().setBoolean(getConfKey(ContractOptions.IS_CASE_SENSITIVE), false);
+      getConf().setBoolean(getConfKey(ContractOptions.SUPPORTS_UNIX_PERMISSIONS), false);
+    } else if (ContractTestUtils.isOSX()) {
+      //OSX HFS+ is not case sensitive
+      getConf().setBoolean(getConfKey(ContractOptions.IS_CASE_SENSITIVE),
+                           false);
+    }
+  }
+
+  /**
+   * Get the local filesystem. This may be overridden
+   * @return the filesystem
+   * @throws IOException
+   */
+  protected FileSystem getLocalFS() throws IOException {
+    return FileSystem.getLocal(getConf());
+  }
+
+  @Override
+  public FileSystem getTestFileSystem() throws IOException {
+    return fs;
+  }
+
+  @Override
+  public String getScheme() {
+    return "file";
+  }
+
+  @Override
+  public Path getTestPath() {
+    Path path = fs.makeQualified(new Path(
+      getTestDataDir()));
+    return path;
+  }
+
+  /**
+   * Get the test data directory
+   * @return the directory for test data
+   */
+  protected String getTestDataDir() {
+    return System.getProperty(SYSPROP_TEST_BUILD_DATA, DEFAULT_TEST_BUILD_DATA_DIR);
+  }
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractAppend.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractAppend.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractAppend.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractAppend.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,32 @@
+/*
+ * 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.localfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractAppendTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestLocalFSContractAppend extends AbstractContractAppendTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new LocalFSContract(conf);
+  }
+
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractCreate.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractCreate.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractCreate.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractCreate.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,32 @@
+/*
+ * 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.localfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestLocalFSContractCreate extends AbstractContractCreateTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new LocalFSContract(conf);
+  }
+
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractDelete.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractDelete.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractDelete.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractDelete.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,32 @@
+/*
+ * 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.localfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestLocalFSContractDelete extends AbstractContractDeleteTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new LocalFSContract(conf);
+  }
+
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractLoaded.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractLoaded.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractLoaded.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractLoaded.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,53 @@
+/*
+ * 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.localfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
+import org.junit.Test;
+
+import java.net.URL;
+
+/**
+ * just here to make sure that the local.xml resource is actually loading
+ */
+public class TestLocalFSContractLoaded extends AbstractFSContractTestBase {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new LocalFSContract(conf);
+  }
+
+  @Test
+  public void testContractWorks() throws Throwable {
+    String key = getContract().getConfKey(SUPPORTS_ATOMIC_RENAME);
+    assertNotNull("not set: " + key, getContract().getConf().get(key));
+    assertTrue("not true: " + key,
+               getContract().isSupported(SUPPORTS_ATOMIC_RENAME, false));
+  }
+
+  @Test
+  public void testContractResourceOnClasspath() throws Throwable {
+    URL url = this.getClass()
+                       .getClassLoader()
+                       .getResource(LocalFSContract.CONTRACT_XML);
+    assertNotNull("could not find contract resource", url);
+  }
+}
\ No newline at end of file

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractMkdir.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractMkdir.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractMkdir.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractMkdir.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,34 @@
+/*
+ * 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.localfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Test dir operations on a the local FS.
+ */
+public class TestLocalFSContractMkdir extends AbstractContractMkdirTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new LocalFSContract(conf);
+  }
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractOpen.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractOpen.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractOpen.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractOpen.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,31 @@
+/*
+ * 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.localfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestLocalFSContractOpen extends AbstractContractOpenTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new LocalFSContract(conf);
+  }
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractRename.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractRename.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractRename.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractRename.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,32 @@
+/*
+ * 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.localfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestLocalFSContractRename extends AbstractContractRenameTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new LocalFSContract(conf);
+  }
+
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractSeek.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractSeek.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractSeek.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractSeek.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,31 @@
+/*
+ * 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.localfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestLocalFSContractSeek extends AbstractContractSeekTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new LocalFSContract(conf);
+  }
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/RawlocalFSContract.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/RawlocalFSContract.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/RawlocalFSContract.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/RawlocalFSContract.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,52 @@
+/*
+ * 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.rawlocal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.contract.localfs.LocalFSContract;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Raw local filesystem. This is the inner OS-layer FS
+ * before checksumming is added around it.
+ */
+public class RawlocalFSContract extends LocalFSContract {
+  public RawlocalFSContract(Configuration conf) {
+    super(conf);
+  }
+
+  public static final String RAW_CONTRACT_XML = "contract/localfs.xml";
+
+  @Override
+  protected String getContractXml() {
+    return RAW_CONTRACT_XML;
+  }
+
+  @Override
+  protected FileSystem getLocalFS() throws IOException {
+    return FileSystem.getLocal(getConf()).getRawFileSystem();
+  }
+
+  public File getTestDirectory() {
+    return new File(getTestDataDir());
+  }
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractUnderlyingFileBehavior.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractUnderlyingFileBehavior.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractUnderlyingFileBehavior.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractUnderlyingFileBehavior.java Thu Jul  3 12:04:50 2014
@@ -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.contract.rawlocal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+
+public class TestRawLocalContractUnderlyingFileBehavior extends Assert {
+
+  private static File testDirectory;
+
+  @BeforeClass
+  public static void before() {
+    RawlocalFSContract contract =
+      new RawlocalFSContract(new Configuration());
+    testDirectory = contract.getTestDirectory();
+    testDirectory.mkdirs();
+    assertTrue(testDirectory.isDirectory());
+
+  }
+
+  @Test
+  public void testDeleteEmptyPath() throws Throwable {
+    File nonexistent = new File(testDirectory, "testDeleteEmptyPath");
+    assertFalse(nonexistent.exists());
+    assertFalse("nonexistent.delete() returned true", nonexistent.delete());
+  }
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractAppend.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractAppend.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractAppend.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractAppend.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,32 @@
+/*
+ * 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.rawlocal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractAppendTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestRawlocalContractAppend extends AbstractContractAppendTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RawlocalFSContract(conf);
+  }
+
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractCreate.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractCreate.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractCreate.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractCreate.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,32 @@
+/*
+ * 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.rawlocal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestRawlocalContractCreate extends AbstractContractCreateTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RawlocalFSContract(conf);
+  }
+
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractDelete.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractDelete.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractDelete.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractDelete.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,32 @@
+/*
+ * 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.rawlocal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestRawlocalContractDelete extends AbstractContractDeleteTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RawlocalFSContract(conf);
+  }
+
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractMkdir.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractMkdir.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractMkdir.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractMkdir.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,34 @@
+/*
+ * 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.rawlocal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Test dir operations on a the local FS.
+ */
+public class TestRawlocalContractMkdir extends AbstractContractMkdirTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RawlocalFSContract(conf);
+  }
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractOpen.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractOpen.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractOpen.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractOpen.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,31 @@
+/*
+ * 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.rawlocal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestRawlocalContractOpen extends AbstractContractOpenTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RawlocalFSContract(conf);
+  }
+}

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

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractRename.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractRename.java?rev=1607596&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractRename.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractRename.java Thu Jul  3 12:04:50 2014
@@ -0,0 +1,32 @@
+/*
+ * 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.rawlocal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestRawlocalContractRename extends AbstractContractRenameTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RawlocalFSContract(conf);
+  }
+
+}

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