You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@twill.apache.org by ch...@apache.org on 2014/04/22 08:06:36 UTC

[39/50] [abbrv] git commit: (TWILL-62) Enhance Location API to add support for listing directory content and also check if a location is directory.

(TWILL-62) Enhance Location API to add support for listing directory content and also check if a location is directory.

Signed-off-by: Terence Yim <te...@continuuity.com>


Project: http://git-wip-us.apache.org/repos/asf/incubator-twill/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-twill/commit/6149ea21
Tree: http://git-wip-us.apache.org/repos/asf/incubator-twill/tree/6149ea21
Diff: http://git-wip-us.apache.org/repos/asf/incubator-twill/diff/6149ea21

Branch: refs/heads/site
Commit: 6149ea21178b332b8e28e6c4ed9ffc4b42334ade
Parents: d4a352d
Author: Terence Yim <te...@continuuity.com>
Authored: Fri Mar 21 16:33:53 2014 -0700
Committer: Terence Yim <te...@continuuity.com>
Committed: Fri Mar 21 18:05:31 2014 -0700

----------------------------------------------------------------------
 .../apache/twill/filesystem/LocalLocation.java  |  41 +++++++
 .../org/apache/twill/filesystem/Location.java   |  16 +++
 .../twill/filesystem/LocalLocationTest.java     |  64 -----------
 .../apache/twill/filesystem/HDFSLocation.java   |  41 +++++++
 .../twill/filesystem/HDFSLocationTest.java      |  52 +++++++++
 .../twill/filesystem/LocalLocationTest.java     |  40 +++++++
 .../twill/filesystem/LocationTestBase.java      | 111 +++++++++++++++++++
 7 files changed, 301 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6149ea21/twill-common/src/main/java/org/apache/twill/filesystem/LocalLocation.java
----------------------------------------------------------------------
diff --git a/twill-common/src/main/java/org/apache/twill/filesystem/LocalLocation.java b/twill-common/src/main/java/org/apache/twill/filesystem/LocalLocation.java
index d107eac..c4e3dbf 100644
--- a/twill-common/src/main/java/org/apache/twill/filesystem/LocalLocation.java
+++ b/twill-common/src/main/java/org/apache/twill/filesystem/LocalLocation.java
@@ -17,10 +17,13 @@
  */
 package org.apache.twill.filesystem;
 
+import com.google.common.base.Objects;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 import java.io.File;
 import java.io.FileInputStream;
+import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -28,6 +31,7 @@ import java.io.OutputStream;
 import java.net.URI;
 import java.util.Collections;
 import java.util.Deque;
+import java.util.List;
 import java.util.UUID;
 
 /**
@@ -202,4 +206,41 @@ final class LocalLocation implements Location {
   public long lastModified() {
     return file.lastModified();
   }
+
+  @Override
+  public boolean isDirectory() throws IOException {
+    return file.isDirectory();
+  }
+
+  @Override
+  public List<Location> list() throws IOException {
+    File[] files = file.listFiles();
+    ImmutableList.Builder<Location> result = ImmutableList.builder();
+    if (files != null) {
+      for (File file : files) {
+        result.add(new LocalLocation(file));
+      }
+    } else if (!file.exists()) {
+      throw new FileNotFoundException("File " + file + " does not exist.");
+    }
+    return result.build();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    LocalLocation other = (LocalLocation) o;
+    return Objects.equal(file, other.file);
+  }
+
+  @Override
+  public int hashCode() {
+    return file.hashCode();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6149ea21/twill-common/src/main/java/org/apache/twill/filesystem/Location.java
----------------------------------------------------------------------
diff --git a/twill-common/src/main/java/org/apache/twill/filesystem/Location.java b/twill-common/src/main/java/org/apache/twill/filesystem/Location.java
index a2d1f9a..db3a8e8 100644
--- a/twill-common/src/main/java/org/apache/twill/filesystem/Location.java
+++ b/twill-common/src/main/java/org/apache/twill/filesystem/Location.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.URI;
+import java.util.List;
 import javax.annotation.Nullable;
 
 /**
@@ -151,4 +152,19 @@ public interface Location {
    * @return Last modified time of file.
    */
   long lastModified() throws IOException;
+
+  /**
+   * Checks if this location represents a directory.
+   *
+   * @return {@code true} if it is a directory, {@code false} otherwise.
+   */
+  boolean isDirectory() throws IOException;
+
+  /**
+   * List the locations under this location.
+   *
+   * @return Immutable List of locations under this location.
+   *         An empty list is returned if this location is not a directory.
+   */
+  List<Location> list() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6149ea21/twill-common/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java
----------------------------------------------------------------------
diff --git a/twill-common/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java b/twill-common/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java
deleted file mode 100644
index 198f77f..0000000
--- a/twill-common/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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.twill.filesystem;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-
-/**
- *
- */
-public class LocalLocationTest {
-
-  @Test
-  public void testDelete() throws IOException {
-    LocationFactory factory = new LocalLocationFactory(new File(System.getProperty("java.io.tmpdir")));
-
-    Location base = factory.create("test").getTempFile(".tmp");
-    Assert.assertTrue(base.mkdirs());
-
-    Assert.assertTrue(base.append("test1").getTempFile(".tmp").createNew());
-    Assert.assertTrue(base.append("test2").getTempFile(".tmp").createNew());
-
-    Location subDir = base.append("test3");
-    Assert.assertTrue(subDir.mkdirs());
-
-    Assert.assertTrue(subDir.append("test4").getTempFile(".tmp").createNew());
-    Assert.assertTrue(subDir.append("test5").getTempFile(".tmp").createNew());
-
-    Assert.assertTrue(base.delete(true));
-    Assert.assertFalse(base.exists());
-  }
-
-  @Test
-  public void testHelper() {
-    LocationFactory factory = LocationFactories.namespace(
-                                new LocalLocationFactory(new File(System.getProperty("java.io.tmpdir"))),
-                                "testhelper");
-
-    Location location = factory.create("test");
-    Assert.assertTrue(location.toURI().getPath().endsWith("testhelper/test"));
-
-    location = factory.create(URI.create("test2"));
-    Assert.assertTrue(location.toURI().getPath().endsWith("testhelper/test2"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6149ea21/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocation.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocation.java b/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocation.java
index b0eeb43..59a371c 100644
--- a/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocation.java
+++ b/twill-yarn/src/main/java/org/apache/twill/filesystem/HDFSLocation.java
@@ -17,7 +17,10 @@
  */
 package org.apache.twill.filesystem;
 
+import com.google.common.base.Objects;
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
@@ -28,6 +31,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.URI;
+import java.util.List;
 import java.util.UUID;
 
 /**
@@ -190,4 +194,41 @@ final class HDFSLocation implements Location {
   public long lastModified() throws IOException {
     return fs.getFileStatus(path).getModificationTime();
   }
+
+  @Override
+  public boolean isDirectory() throws IOException {
+    return fs.isDirectory(path);
+  }
+
+  @Override
+  public List<Location> list() throws IOException {
+    FileStatus[] statuses = fs.listStatus(path);
+    ImmutableList.Builder<Location> result = ImmutableList.builder();
+    if (statuses != null) {
+      for (FileStatus status : statuses) {
+        if (!Objects.equal(path, status.getPath())) {
+          result.add(new HDFSLocation(fs, status.getPath()));
+        }
+      }
+    }
+    return result.build();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    HDFSLocation other = (HDFSLocation) o;
+    return Objects.equal(path, other.path);
+  }
+
+  @Override
+  public int hashCode() {
+    return path.hashCode();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6149ea21/twill-yarn/src/test/java/org/apache/twill/filesystem/HDFSLocationTest.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/filesystem/HDFSLocationTest.java b/twill-yarn/src/test/java/org/apache/twill/filesystem/HDFSLocationTest.java
new file mode 100644
index 0000000..20f7403
--- /dev/null
+++ b/twill-yarn/src/test/java/org/apache/twill/filesystem/HDFSLocationTest.java
@@ -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.twill.filesystem;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ *
+ */
+public class HDFSLocationTest extends LocationTestBase {
+
+  private static MiniDFSCluster dfsCluster;
+  private static LocationFactory locationFactory;
+
+  @BeforeClass
+  public static void init() throws IOException {
+    Configuration conf = new Configuration();
+    conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, tmpFolder.newFolder().getAbsolutePath());
+    dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    locationFactory = new HDFSLocationFactory(dfsCluster.getFileSystem());
+  }
+
+  @AfterClass
+  public static void finish() {
+    dfsCluster.shutdown();
+  }
+
+  @Override
+  protected LocationFactory getLocationFactory() {
+    return locationFactory;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6149ea21/twill-yarn/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java b/twill-yarn/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java
new file mode 100644
index 0000000..3f6d931
--- /dev/null
+++ b/twill-yarn/src/test/java/org/apache/twill/filesystem/LocalLocationTest.java
@@ -0,0 +1,40 @@
+/*
+ * 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.twill.filesystem;
+
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+/**
+ *
+ */
+public class LocalLocationTest extends LocationTestBase {
+
+  private static LocationFactory locationFactory;
+
+  @BeforeClass
+  public static void init() throws IOException {
+    locationFactory = new LocalLocationFactory(tmpFolder.newFolder());
+  }
+
+  @Override
+  protected LocationFactory getLocationFactory() {
+    return locationFactory;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-twill/blob/6149ea21/twill-yarn/src/test/java/org/apache/twill/filesystem/LocationTestBase.java
----------------------------------------------------------------------
diff --git a/twill-yarn/src/test/java/org/apache/twill/filesystem/LocationTestBase.java b/twill-yarn/src/test/java/org/apache/twill/filesystem/LocationTestBase.java
new file mode 100644
index 0000000..ee591e7
--- /dev/null
+++ b/twill-yarn/src/test/java/org/apache/twill/filesystem/LocationTestBase.java
@@ -0,0 +1,111 @@
+/*
+ * 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.twill.filesystem;
+
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+
+/**
+ * Base class for defining {@link Location} and {@link LocationFactory} tests.
+ */
+public abstract class LocationTestBase {
+
+  @ClassRule
+  public static TemporaryFolder tmpFolder = new TemporaryFolder();
+
+
+  @Test
+  public void testDelete() throws IOException {
+    LocationFactory factory = getLocationFactory();
+
+    Location base = factory.create("test").getTempFile(".tmp");
+    Assert.assertTrue(base.mkdirs());
+
+    Assert.assertTrue(base.append("test1").getTempFile(".tmp").createNew());
+    Assert.assertTrue(base.append("test2").getTempFile(".tmp").createNew());
+
+    Location subDir = base.append("test3");
+    Assert.assertTrue(subDir.mkdirs());
+
+    Assert.assertTrue(subDir.append("test4").getTempFile(".tmp").createNew());
+    Assert.assertTrue(subDir.append("test5").getTempFile(".tmp").createNew());
+
+    Assert.assertTrue(base.delete(true));
+    Assert.assertFalse(base.exists());
+  }
+
+  @Test
+  public void testHelper() {
+    LocationFactory factory = LocationFactories.namespace(getLocationFactory(), "testhelper");
+
+    Location location = factory.create("test");
+    Assert.assertTrue(location.toURI().getPath().endsWith("testhelper/test"));
+
+    location = factory.create(URI.create("test2"));
+    Assert.assertTrue(location.toURI().getPath().endsWith("testhelper/test2"));
+  }
+
+  @Test
+  public void testList() throws IOException {
+    LocationFactory factory = getLocationFactory();
+
+    Location dir = factory.create("dir");
+
+    // Check and create the directory
+    Assert.assertFalse(dir.isDirectory());
+    Assert.assertTrue(dir.mkdirs());
+    Assert.assertTrue(dir.isDirectory());
+
+    // Should have nothing inside
+    Assert.assertTrue(dir.list().isEmpty());
+
+    // Check and create a file inside the directory
+    Location file = dir.append("file");
+    Assert.assertFalse(file.isDirectory());
+    Assert.assertTrue(file.createNew());
+    Assert.assertFalse(file.isDirectory());
+
+    // List on file should gives empty list
+    Assert.assertTrue(file.list().isEmpty());
+
+    // List on directory should gives the file inside
+    List<Location> listing = dir.list();
+    Assert.assertEquals(1, listing.size());
+    Assert.assertEquals(file, listing.get(0));
+
+    // After deleting the file inside the directory, list on directory should be empty again.
+    file.delete();
+    Assert.assertTrue(dir.list().isEmpty());
+
+    // List on a non-exist location would throw exception
+    try {
+      file.list();
+      Assert.fail("List should fail on non-exist location.");
+    } catch (IOException e) {
+      // Expected
+    }
+  }
+
+  protected abstract LocationFactory getLocationFactory();
+}