You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by pr...@apache.org on 2015/06/09 08:38:16 UTC

[3/3] incubator-lens git commit: LENS-513: Add jar CLI command can now take regex of paths

LENS-513: Add jar CLI command can now take regex of paths


Project: http://git-wip-us.apache.org/repos/asf/incubator-lens/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-lens/commit/84666df0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-lens/tree/84666df0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-lens/diff/84666df0

Branch: refs/heads/master
Commit: 84666df01ded1e944b418abde08625a00789305d
Parents: 10d647f
Author: Yash Sharma <ya...@gmail.com>
Authored: Tue Jun 9 12:06:57 2015 +0530
Committer: Rajat Khandelwal <ra...@gmail.com>
Committed: Tue Jun 9 12:06:57 2015 +0530

----------------------------------------------------------------------
 .../lens/cli/TestLensConnectionCliCommands.java | 132 ++++++++++++-
 .../lens/server/session/SessionResource.java    |  56 ++++--
 .../apache/lens/server/util/ScannedPaths.java   | 197 +++++++++++++++++++
 .../apache/lens/server/TestServerRestart.java   |   2 +-
 .../lens/server/util/TestScannedPaths.java      | 144 ++++++++++++++
 5 files changed, 516 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/84666df0/lens-cli/src/test/java/org/apache/lens/cli/TestLensConnectionCliCommands.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/test/java/org/apache/lens/cli/TestLensConnectionCliCommands.java b/lens-cli/src/test/java/org/apache/lens/cli/TestLensConnectionCliCommands.java
index 3238edb..7503221 100644
--- a/lens-cli/src/test/java/org/apache/lens/cli/TestLensConnectionCliCommands.java
+++ b/lens-cli/src/test/java/org/apache/lens/cli/TestLensConnectionCliCommands.java
@@ -20,6 +20,7 @@ package org.apache.lens.cli;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.URI;
 
 import javax.ws.rs.BadRequestException;
 
@@ -79,13 +80,53 @@ public class TestLensConnectionCliCommands extends LensCliApplicationTest {
   private File createNewPath(String fileName) {
     File f = new File(fileName);
     try {
-      f.createNewFile();
+      if (!f.exists()) {
+        f.createNewFile();
+      }
     } catch (IOException e) {
       Assert.fail("Unable to create test file, so bailing out.");
     }
     return f;
   }
 
+  private String getFilePathFromUri(String uripath){
+    try {
+      return new URI(uripath).getPath();
+    } catch (Exception e){
+      return null;
+    }
+  }
+
+  private File createNewFile(String fileName) {
+    File f = null;
+    try {
+      String filepath = getFilePathFromUri(fileName);
+      Assert.assertNotNull(fileName, "Unable to get filepath from uri pattern.");
+      f = new File(filepath);
+      if (!f.exists()) {
+        f.createNewFile();
+      }
+    } catch (Exception e) {
+      Assert.fail("Unable to create test file, so bailing out.");
+    }
+    return f;
+  }
+
+  private File deleteFile(String fileName) {
+    File f = null;
+    try {
+      String filepath = getFilePathFromUri(fileName);
+      Assert.assertNotNull(fileName, "Unable to get filepath from uri pattern.");
+      f = new File(filepath);
+      if (f.exists()) {
+        f.delete();
+      }
+    } catch (Exception e) {
+      Assert.fail("Unable to delete test file, so bailing out.");
+    }
+    return f;
+  }
+
   /**
    * Test file commands.
    */
@@ -106,6 +147,7 @@ public class TestLensConnectionCliCommands extends LensCliApplicationTest {
 
       result = commands.removeFile(filename);
       Assert.assertEquals("Delete resource succeeded", result);
+
       LOG.debug("Testing set/remove file operation done");
     } finally {
       if (f != null) {
@@ -116,6 +158,44 @@ public class TestLensConnectionCliCommands extends LensCliApplicationTest {
   }
 
   /**
+   * Test file commands with URI in regex.
+   */
+  @Test
+  public void testFileCommandsWithURIRegex() {
+    LensClient client = new LensClient();
+    LensConnectionCommands commands = new LensConnectionCommands();
+    commands.setClient(client);
+    LOG.debug("Testing set/remove file operations");
+
+    java.io.File file = new java.io.File("");
+    String projectdir = file.getAbsolutePath();
+
+    /* Tests input file pattern file: and file://  */
+    String filenameA = "file:" + projectdir + "/target/tempdata_a.txt";
+    String filenameB = "file://" + projectdir +"/target/tempdata_b.txt";
+
+    String fileRegex = "file:" + projectdir + "/target/tempdata_*.txt";
+
+    try {
+      createNewFile(filenameA);
+      createNewFile(filenameB);
+
+      String result = commands.addFile(fileRegex);
+      Assert.assertEquals("Add resource succeeded", result);
+
+      result = commands.removeFile(fileRegex);
+      Assert.assertEquals("Delete resource succeeded", result);
+
+      LOG.debug("Testing set/remove file operation done");
+    } finally {
+      deleteFile(filenameA);
+      deleteFile(filenameB);
+      commands.quitShell();
+    }
+  }
+
+
+  /**
    * Test jar commands.
    */
   @Test
@@ -145,6 +225,56 @@ public class TestLensConnectionCliCommands extends LensCliApplicationTest {
   }
 
   /**
+   * Test jar commands with regex specified.
+   */
+  @Test
+  public void testResourceCommandsWithRegex() {
+    LensClient client = new LensClient();
+    LensConnectionCommands commands = new LensConnectionCommands();
+    commands.setClient(client);
+    LOG.debug("Testing set/remove file operations");
+
+    File fileA = null, fileB = null;
+    String filenameA, filenameB, fileRegex, result;
+    try {
+      filenameA = "target/tempdata_a";
+      filenameB = "target/tempdata_b";
+      fileRegex = "target/tempdata_*";
+
+      fileA = createNewPath(filenameA);
+      fileB = createNewPath(filenameB);
+      result = commands.addFile(fileRegex);
+      Assert.assertEquals("Add resource succeeded", result);
+
+      result = commands.removeFile(fileRegex);
+      Assert.assertEquals("Delete resource succeeded", result);
+
+      filenameA = "target/tempdata_a.jar";
+      filenameB = "target/tempdata_b.jar";
+      fileRegex = "target/tempdata_*.jar";
+
+      fileA = createNewPath(filenameA);
+      fileB = createNewPath(filenameB);
+
+      result = commands.addJar(fileRegex);
+      Assert.assertEquals("Add resource succeeded", result);
+
+      result = commands.removeJar(fileRegex);
+      Assert.assertEquals("Delete resource succeeded", result);
+
+      LOG.debug("Testing set/remove resource operation done");
+    } finally {
+      if (fileA != null) {
+        fileA.delete();
+      }
+      if (fileB != null) {
+        fileB.delete();
+      }
+      commands.quitShell();
+    }
+  }
+
+  /**
    * Test list resources commands.
    */
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/84666df0/lens-server/src/main/java/org/apache/lens/server/session/SessionResource.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/session/SessionResource.java b/lens-server/src/main/java/org/apache/lens/server/session/SessionResource.java
index 5ffc086..15a8e06 100644
--- a/lens-server/src/main/java/org/apache/lens/server/session/SessionResource.java
+++ b/lens-server/src/main/java/org/apache/lens/server/session/SessionResource.java
@@ -19,6 +19,7 @@
 package org.apache.lens.server.session;
 
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
@@ -34,6 +35,7 @@ import org.apache.lens.server.LensService;
 import org.apache.lens.server.LensServices;
 import org.apache.lens.server.api.error.LensException;
 import org.apache.lens.server.api.session.SessionService;
+import org.apache.lens.server.util.ScannedPaths;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -141,10 +143,24 @@ public class SessionResource {
   @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML, MediaType.TEXT_PLAIN})
   public APIResult addResource(@FormDataParam("sessionid") LensSessionHandle sessionid,
     @FormDataParam("type") String type, @FormDataParam("path") String path) {
-    int numAdded = sessionService.addResourceToAllServices(sessionid, type, path);
+    Iterator<String> foundFiles = new ScannedPaths(path, type).iterator();
+    String matchedPath = null;
+    int matchedPathsCount = 0;
+
+    if (foundFiles == null) {
+      return new APIResult(Status.FAILED, "No matching resources found for provided path.");
+    }
+
+    int numAdded = 0;
+    while (foundFiles.hasNext()) {
+      matchedPath = foundFiles.next();
+      numAdded += sessionService.addResourceToAllServices(sessionid, type, matchedPath);
+      matchedPathsCount++;
+    }
+
     if (numAdded == 0) {
-      return new APIResult(Status.FAILED, "Add resource has failed ");
-    } else if (numAdded != LensServices.get().getLensServices().size()) {
+      return new APIResult(Status.FAILED, "Add resource has failed");
+    } else if ((numAdded / matchedPathsCount) != LensServices.get().getLensServices().size()) {
       return new APIResult(Status.PARTIAL, "Add resource is partial");
     }
     return new APIResult(Status.SUCCEEDED, "Add resource succeeded");
@@ -182,21 +198,34 @@ public class SessionResource {
    */
   @PUT
   @Path("resources/delete")
+
   @Consumes({MediaType.MULTIPART_FORM_DATA})
   @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML, MediaType.TEXT_PLAIN})
   public APIResult deleteResource(@FormDataParam("sessionid") LensSessionHandle sessionid,
     @FormDataParam("type") String type, @FormDataParam("path") String path) {
+    Iterator<String> foundFiles = new ScannedPaths(path, type).iterator();
+    String matchedPath = null;
+
+    if (foundFiles == null) {
+      return new APIResult(Status.PARTIAL, "No matching resources found for provided path.");
+    }
+
     int numDeleted = 0;
-    for (LensService service : LensServices.get().getLensServices()) {
-      try {
-        service.deleteResource(sessionid, type, path);
-        numDeleted++;
-      } catch (LensException e) {
-        LOG.error("Failed to delete resource in service:" + service, e);
-        if (numDeleted != 0) {
-          return new APIResult(Status.PARTIAL, "Delete resource is partial, failed for service:" + service.getName());
-        } else {
-          return new APIResult(Status.PARTIAL, "Delete resource has failed");
+
+    while(foundFiles.hasNext()) {
+      matchedPath = foundFiles.next();
+
+      for (LensService service : LensServices.get().getLensServices()) {
+        try {
+          service.deleteResource(sessionid, type, matchedPath);
+          numDeleted++;
+        } catch (LensException e) {
+          LOG.error("Failed to delete resource in service:" + service, e);
+          if (numDeleted != 0) {
+            return new APIResult(Status.PARTIAL, "Delete resource is partial, failed for service:" + service.getName());
+          } else {
+            return new APIResult(Status.FAILED, "Delete resource has failed");
+          }
         }
       }
     }
@@ -245,4 +274,5 @@ public class SessionResource {
     sessionService.setSessionParameter(sessionid, key, value);
     return new APIResult(Status.SUCCEEDED, "Set param succeeded");
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/84666df0/lens-server/src/main/java/org/apache/lens/server/util/ScannedPaths.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/util/ScannedPaths.java b/lens-server/src/main/java/org/apache/lens/server/util/ScannedPaths.java
new file mode 100644
index 0000000..ac773fe
--- /dev/null
+++ b/lens-server/src/main/java/org/apache/lens/server/util/ScannedPaths.java
@@ -0,0 +1,197 @@
+/**
+ * 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.lens.server.util;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+
+import org.apache.hadoop.fs.Path;
+
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class ScannedPaths implements Iterable<String> {
+  private String path = null;
+  private String type = null;
+
+  private Path fsPath;
+
+  /** Is the provided expression parsed **/
+  private boolean isScanned;
+
+  /* Keep all matched paths so we don't have to query filesystem multiple times */
+  private Map<String, String> matchedPaths = null;
+
+  /* The Chosen Ones */
+  @Getter(lazy=true) private final List<String> finalPaths = getMatchedPaths();
+
+  public ScannedPaths(String path, String type) {
+    this.path = path;
+    this.type = type;
+  }
+
+  @Override
+  public Iterator<String> iterator() {
+    /** Does all the pattern matching and returns the iterator to finalPaths collection **/
+    return (getFinalPaths() == null) ? null : getFinalPaths().iterator();
+  }
+
+  /**
+   * Method that computes path of resources matching the input path or path regex pattern.
+   * If provided path is a directory it additionally checks for the jar_order or glob_order file
+   * that imposes ordering of resources and filters out other resources.
+   *
+   * Updates finalPaths List with matched paths and returns an iterator for matched paths.
+   */
+  private List<String> getMatchedPaths() {
+    List<String> finalPaths = null;
+    try {
+      FileSystem fs = FileSystem.get(new URI(path), new Configuration());
+      fsPath = new Path(new URI(path).getPath());
+
+      if (fs.isDirectory(fsPath)) {
+        findAllMatchedPaths(true);
+        filterByJarType();
+      /* Updates finalPaths List with restrictions imposed
+         by jar_order/glob_order file */
+        finalPaths = getMatchedPathsFilteredByOrder();
+      } else {
+        findAllMatchedPaths(false);
+        filterByJarType();
+        if (matchedPaths != null) {
+          finalPaths = new ArrayList<String>(matchedPaths.values());
+        }
+      }
+    } catch (FileNotFoundException fex) {
+      log.error("File not found while scanning path.", fex);
+    } catch (URISyntaxException | IOException ex) {
+      log.error("Exception while initializing PathScanner.", ex);
+    }
+    return finalPaths;
+  }
+
+  /**
+   * Populates the matchedPaths[] with all paths matching the pattern.
+   */
+  private void findAllMatchedPaths(boolean isDir) {
+    try {
+      Path path = isDir ? new Path(fsPath, "*") : fsPath;
+      FileStatus[] statuses = path.getFileSystem(new Configuration()).globStatus(path);
+
+      if (statuses == null || statuses.length == 0) {
+        log.info("No matched paths found for expression " + path);
+        return;
+      }
+      matchedPaths = new HashMap<String, String>();
+      for (int count = 0; count < statuses.length; count++) {
+        matchedPaths.put(statuses[count].getPath().getName(), statuses[count].getPath().toString());
+      }
+    } catch (FileNotFoundException fex) {
+      log.error("File not found while scanning path.", fex);
+      return;
+    } catch (IOException ioex) {
+      log.error("IOException while scanning path.", ioex);
+      return;
+    }
+  }
+
+  /**
+   * Filters the matchedPaths by "jar" type.
+   * Removes non-jar resources if type is specified as "jar".
+   */
+  private void filterByJarType() {
+    if (matchedPaths == null) {
+      return;
+    } else if (type.equalsIgnoreCase("jar")) {
+      Iterator<Map.Entry<String, String>> iter = matchedPaths.entrySet().iterator();
+
+      while (iter.hasNext()) {
+        Map.Entry<String, String> entry = iter.next();
+        if (!entry.getKey().endsWith(".jar")) {
+          iter.remove();
+        }
+      }
+    }
+  }
+
+  /**
+   * Filters the matchedPath[] to remove unwanted resources
+   * and apply ordering to the resources as specified in jar_order or glob_order file.
+   * Bypasses filtering if none of the files is present in the directory.
+   */
+  private List<String> getMatchedPathsFilteredByOrder() {
+    if (matchedPaths == null) {
+      return  null;
+    }
+
+    List<String> finalPaths = null;
+    InputStream resourceOrderIStream = null;
+    List<String> resources;
+    try {
+      FileSystem fs = fsPath.getFileSystem(new Configuration());
+      Path resourceOrderFile = new Path(fsPath.toUri().getPath(), "jar_order");
+
+      if (!fs.exists(resourceOrderFile)) {
+        resourceOrderFile = new Path(fsPath.toUri().getPath(), "glob_order");
+        if (!fs.exists(resourceOrderFile)) {
+          /* No order file present. Bypass filtering and Add all resource matching pattern */
+          return new ArrayList<>(matchedPaths.values());
+        }
+      }
+
+      resourceOrderIStream = fs.open(resourceOrderFile);
+      resources = IOUtils.readLines(resourceOrderIStream, Charset.forName("UTF-8"));
+      finalPaths = new ArrayList<>();
+      for(String resource : resources) {
+        if (resource == null || resource.isEmpty()) {
+          continue;
+        }
+
+        if (matchedPaths.containsKey(resource)) {
+          finalPaths.add(matchedPaths.get(resource));
+        }
+      }
+    } catch (FileNotFoundException fex) {
+      log.error("File not found while scanning path.", fex);
+      finalPaths = null;
+    } catch (IOException ioex) {
+      log.error("IOException while scanning path.", ioex);
+      finalPaths = null;
+    } finally {
+      IOUtils.closeQuietly(resourceOrderIStream);
+    }
+    return finalPaths;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/84666df0/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java b/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java
index 01dcf54..09f8dc7 100644
--- a/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java
+++ b/lens-server/src/test/java/org/apache/lens/server/TestServerRestart.java
@@ -409,7 +409,7 @@ public class TestServerRestart extends LensAllApplicationJerseyTest {
     Assert.assertEquals(session.getLensSessionPersistInfo().getResources().size(), 1);
     LensSessionImpl.ResourceEntry resourceEntry = session.getLensSessionPersistInfo().getResources().get(0);
     Assert.assertEquals(resourceEntry.getType(), "file");
-    Assert.assertEquals(resourceEntry.getLocation(), "target/test-classes/lens-site.xml");
+    Assert.assertTrue(resourceEntry.getLocation().contains("target/test-classes/lens-site.xml"));
 
     // close session
     result = sessionTarget.queryParam("sessionid", restartTestSession).request().delete(APIResult.class);

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/84666df0/lens-server/src/test/java/org/apache/lens/server/util/TestScannedPaths.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/util/TestScannedPaths.java b/lens-server/src/test/java/org/apache/lens/server/util/TestScannedPaths.java
new file mode 100644
index 0000000..5073634
--- /dev/null
+++ b/lens-server/src/test/java/org/apache/lens/server/util/TestScannedPaths.java
@@ -0,0 +1,144 @@
+/**
+ * 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.lens.server.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.Iterator;
+
+import org.apache.commons.io.FileUtils;
+
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.Iterators;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+@Test(groups = "unit-test")
+public class TestScannedPaths {
+
+  public void testScannedPaths() throws Exception {
+    File fileA = null, fileB = null;
+    String filenameA, filenameB, fileRegex;
+    String tempPath = "target/tempfiles/";
+    ScannedPaths sc;
+    Iterator<String> iter = null;
+
+    try {
+      filenameA = "tempdata_a";
+      filenameB = "tempdata_b";
+      fileRegex = tempPath + "tempdata_*";
+
+      fileA = createNewPath(tempPath + filenameA);
+      fileB = createNewPath(tempPath + filenameB);
+      sc = new ScannedPaths(fileRegex, "file");
+
+      Assert.assertEquals(Iterators.size(sc.iterator()), 2, "Incorrect number of matches found");
+      iter = sc.iterator();
+      Assert.assertTrue(iter.next().contains(filenameA));
+      Assert.assertTrue(iter.next().contains(filenameB));
+
+      /**
+       * Testing negative Scenario
+       **/
+      fileRegex = tempPath + "tempdata_unknown_*";
+      sc = new ScannedPaths(fileRegex, "file");
+
+      Assert.assertNull(sc.iterator(), "Iterator should be null for unmatched path patterns.");
+    } catch (Exception e) {
+      Assert.fail("Exception while testing ScannedPaths : " + e.getMessage());
+    } finally {
+      FileUtils.deleteQuietly(new File(tempPath));
+    }
+  }
+
+  public void testScannedPathsJarGlobOrder() throws Exception {
+    File fileA = null, fileB = null, fileC = null;
+    File jarFile = null, globFile = null;
+    String filenameA, filenameB, filenameC, fileRegex;
+    String tempPath = "target/tempfiles/";
+    ScannedPaths sc;
+    Iterator<String> iter = null;
+    PrintWriter writer;
+
+    try {
+      filenameA = "tempdata_a.jar";
+      filenameB = "tempdata_b.jar";
+      filenameC = "tempdata_c.data";
+      fileRegex = tempPath;
+
+      fileA = createNewPath(tempPath + filenameA);
+      fileB = createNewPath(tempPath + filenameB);
+      fileC = createNewPath(tempPath + filenameC);
+
+      /** Test jar_order **/
+      jarFile = createNewPath(tempPath + "jar_order");
+      writer = new PrintWriter(jarFile, "UTF-8");
+      writer.println(filenameB);
+      writer.println(filenameA);
+      writer.close();
+      sc = new ScannedPaths(fileRegex, "jar");
+      Assert.assertEquals(Iterators.size(sc.iterator()), 2, "Incorrect number of matches found");
+      iter = sc.iterator();
+      Assert.assertTrue(iter.next().contains(filenameB));
+      Assert.assertTrue(iter.next().contains(filenameA));
+
+      /** Test glob_order **/
+      if (jarFile != null) {
+        jarFile.delete();
+      }
+      globFile = createNewPath(tempPath + "glob_order");
+      writer = new PrintWriter(globFile, "UTF-8");
+      writer.println(filenameB);
+      writer.println(filenameA);
+      writer.println(filenameC);
+      writer.close();
+      sc = new ScannedPaths(fileRegex, "file");
+      Assert.assertEquals(Iterators.size(sc.iterator()), 3, "Incorrect number of matches found");
+      iter = sc.iterator();
+      Assert.assertTrue(iter.next().contains(filenameB));
+      Assert.assertTrue(iter.next().contains(filenameA));
+      Assert.assertTrue(iter.next().contains(filenameC));
+    } catch (Exception e) {
+      Assert.fail("Exception while testing ScannedPaths : " + e.getMessage());
+    } finally {
+      FileUtils.deleteQuietly(new File(tempPath));
+    }
+  }
+
+  private File createNewPath(String fileName) {
+    File f = new File(fileName);
+    try {
+      if (!f.getParentFile().exists()) {
+        f.getParentFile().mkdirs();
+      }
+      if (!f.exists()) {
+        f.createNewFile();
+      }
+    } catch (IOException e) {
+      Assert.fail("Unable to create test file, so bailing out.");
+    }
+    return f;
+  }
+
+}