You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tinkerpop.apache.org by dk...@apache.org on 2016/01/11 17:49:09 UTC

[01/30] incubator-tinkerpop git commit: InputRDD now has readMemoryRDD(). OutputRDD now has writeMemoryRDD(). InputFormatRDD and OutputFormatRDD took the code from SparkExecutor that uses SequenceFiles for output. As such, memory reading/writing has been

Repository: incubator-tinkerpop
Updated Branches:
  refs/heads/TINKERPOP-320 bdfd49023 -> 96388ee83


InputRDD now has readMemoryRDD(). OutputRDD now has writeMemoryRDD(). InputFormatRDD and OutputFormatRDD took the code from SparkExecutor that uses SequenceFiles for output. As such, memory reading/writing has been generalized. Graph system providers that ONLY want to provide Spark support are not required to have HDFS as SparkServer can maintains all persisted data via graphRDD and memoryRDD. There is still more work to do. More tests cases is next.


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

Branch: refs/heads/TINKERPOP-320
Commit: f3ebed0bde6ac889640cb136b50b362c5cd2d2ea
Parents: f8672a4
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Dec 9 10:41:09 2015 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Wed Dec 9 10:41:09 2015 -0700

----------------------------------------------------------------------
 .../process/computer/GraphComputerTest.java     | 54 ++++++++++----------
 .../tinkerpop/gremlin/hadoop/Constants.java     |  8 +++
 .../spark/process/computer/SparkExecutor.java   | 28 ----------
 .../process/computer/SparkGraphComputer.java    | 10 +++-
 .../spark/structure/io/InputFormatRDD.java      | 13 +++++
 .../spark/structure/io/InputOutputHelper.java   |  3 --
 .../gremlin/spark/structure/io/InputRDD.java    | 17 +++++-
 .../spark/structure/io/OutputFormatRDD.java     | 28 ++++++++++
 .../gremlin/spark/structure/io/OutputRDD.java   | 21 ++++++++
 .../spark/structure/io/PersistedInputRDD.java   | 17 ++++--
 .../spark/structure/io/PersistedOutputRDD.java  | 26 ++++++++--
 .../process/computer/LocalPropertyTest.java     |  8 +--
 .../groovy/plugin/SparkGremlinPluginTest.java   | 30 +++++------
 .../gremlin/spark/structure/SparkTest.java      | 10 ++--
 .../spark/structure/io/ExampleInputRDD.java     |  5 ++
 .../spark/structure/io/ExampleOutputRDD.java    |  6 +++
 .../io/PersistedInputOutputRDDTest.java         | 46 ++++++++---------
 .../spark/structure/io/ToyGraphInputRDD.java    |  7 +++
 18 files changed, 220 insertions(+), 117 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/computer/GraphComputerTest.java
----------------------------------------------------------------------
diff --git a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/computer/GraphComputerTest.java b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/computer/GraphComputerTest.java
index 2e603c8..1af0830 100644
--- a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/computer/GraphComputerTest.java
+++ b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/computer/GraphComputerTest.java
@@ -786,14 +786,14 @@ public class GraphComputerTest extends AbstractGremlinProcessTest {
     @LoadGraphWith(MODERN)
     public void shouldSortReduceOutput() throws Exception {
         final ComputerResult results = graph.compute(graphComputerClass.get()).mapReduce(new MapReduceB()).submit().get();
-        final List<Long> ids = results.memory().get("ids");
-        assertEquals(6, ids.size());
-        for (int i = 1; i < ids.size(); i++) {
-            assertTrue(ids.get(i) < ids.get(i - 1));
+        final List<Integer> nameLengths = results.memory().get("nameLengths");
+        assertEquals(6, nameLengths.size());
+        for (int i = 1; i < nameLengths.size(); i++) {
+            assertTrue(nameLengths.get(i) <= nameLengths.get(i - 1));
         }
     }
 
-    public static class MapReduceB extends StaticMapReduce<Long, Long, Long, Long, List<Long>> {
+    public static class MapReduceB extends StaticMapReduce<Integer, Integer, Integer, Integer, List<Integer>> {
 
         @Override
         public boolean doStage(final Stage stage) {
@@ -801,29 +801,29 @@ public class GraphComputerTest extends AbstractGremlinProcessTest {
         }
 
         @Override
-        public void map(final Vertex vertex, final MapEmitter<Long, Long> emitter) {
-            emitter.emit(Long.valueOf(vertex.id().toString()), Long.valueOf(vertex.id().toString()));
+        public void map(final Vertex vertex, final MapEmitter<Integer, Integer> emitter) {
+            emitter.emit(vertex.<String>value("name").length(), vertex.<String>value("name").length());
         }
 
         @Override
-        public void reduce(Long key, Iterator<Long> values, ReduceEmitter<Long, Long> emitter) {
+        public void reduce(Integer key, Iterator<Integer> values, ReduceEmitter<Integer, Integer> emitter) {
             values.forEachRemaining(id -> emitter.emit(id, id));
         }
 
         @Override
-        public Optional<Comparator<Long>> getReduceKeySort() {
-            return Optional.of(Comparator.<Long>reverseOrder());
+        public Optional<Comparator<Integer>> getReduceKeySort() {
+            return Optional.of(Comparator.<Integer>reverseOrder());
         }
 
         @Override
         public String getMemoryKey() {
-            return "ids";
+            return "nameLengths";
         }
 
         @Override
-        public List<Long> generateFinalResult(final Iterator<KeyValue<Long, Long>> keyValues) {
-            final List<Long> list = new ArrayList<>();
-            keyValues.forEachRemaining(id -> list.add(id.getKey()));
+        public List<Integer> generateFinalResult(final Iterator<KeyValue<Integer, Integer>> keyValues) {
+            final List<Integer> list = new ArrayList<>();
+            keyValues.forEachRemaining(nameLength -> list.add(nameLength.getKey()));
             return list;
         }
     }
@@ -833,14 +833,14 @@ public class GraphComputerTest extends AbstractGremlinProcessTest {
     @LoadGraphWith(MODERN)
     public void shouldSortMapOutput() throws Exception {
         final ComputerResult results = graph.compute(graphComputerClass.get()).mapReduce(new MapReduceBB()).submit().get();
-        final List<Long> ids = results.memory().get("ids");
-        assertEquals(6, ids.size());
-        for (int i = 1; i < ids.size(); i++) {
-            assertTrue(ids.get(i) < ids.get(i - 1));
+        final List<Integer> nameLengths = results.memory().get("nameLengths");
+        assertEquals(6, nameLengths.size());
+        for (int i = 1; i < nameLengths.size(); i++) {
+            assertTrue(nameLengths.get(i) <= nameLengths.get(i - 1));
         }
     }
 
-    public static class MapReduceBB extends StaticMapReduce<Long, Long, Long, Long, List<Long>> {
+    public static class MapReduceBB extends StaticMapReduce<Integer, Integer, Integer, Integer, List<Integer>> {
 
         @Override
         public boolean doStage(final Stage stage) {
@@ -848,24 +848,24 @@ public class GraphComputerTest extends AbstractGremlinProcessTest {
         }
 
         @Override
-        public void map(final Vertex vertex, final MapEmitter<Long, Long> emitter) {
-            emitter.emit(Long.valueOf(vertex.id().toString()), Long.valueOf(vertex.id().toString()));
+        public void map(final Vertex vertex, final MapEmitter<Integer, Integer> emitter) {
+            emitter.emit(vertex.<String>value("name").length(), vertex.<String>value("name").length());
         }
 
         @Override
-        public Optional<Comparator<Long>> getMapKeySort() {
-            return Optional.of(Comparator.<Long>reverseOrder());
+        public Optional<Comparator<Integer>> getMapKeySort() {
+            return Optional.of(Comparator.<Integer>reverseOrder());
         }
 
         @Override
         public String getMemoryKey() {
-            return "ids";
+            return "nameLengths";
         }
 
         @Override
-        public List<Long> generateFinalResult(final Iterator<KeyValue<Long, Long>> keyValues) {
-            final List<Long> list = new ArrayList<>();
-            keyValues.forEachRemaining(id -> list.add(id.getKey()));
+        public List<Integer> generateFinalResult(final Iterator<KeyValue<Integer, Integer>> keyValues) {
+            final List<Integer> list = new ArrayList<>();
+            keyValues.forEachRemaining(nameLength -> list.add(nameLength.getKey()));
             return list;
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/Constants.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/Constants.java
index 469e9b0..8678441 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/Constants.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/Constants.java
@@ -52,4 +52,12 @@ public final class Constants {
     public static final String GREMLIN_SPARK_GRAPH_INPUT_RDD = "gremlin.spark.graphInputRDD";
     public static final String GREMLIN_SPARK_GRAPH_OUTPUT_RDD = "gremlin.spark.graphOutputRDD";
     public static final String GREMLIN_SPARK_PERSIST_CONTEXT = "gremlin.spark.persistContext";
+
+    public static String getGraphLocation(final String location) {
+        return location + "/" + Constants.HIDDEN_G;
+    }
+
+    public static String getMemoryLocation(final String location, final String memoryKey) {
+        return location + "/" + memoryKey;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkExecutor.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkExecutor.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkExecutor.java
index 0c6c795..6dd109a 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkExecutor.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkExecutor.java
@@ -20,17 +20,11 @@ package org.apache.tinkerpop.gremlin.spark.process.computer;
 
 import com.google.common.base.Optional;
 import org.apache.commons.configuration.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
 import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.tinkerpop.gremlin.hadoop.Constants;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.HadoopPools;
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable;
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritableIterator;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
 import org.apache.tinkerpop.gremlin.process.computer.MapReduce;
-import org.apache.tinkerpop.gremlin.process.computer.Memory;
 import org.apache.tinkerpop.gremlin.process.computer.MessageCombiner;
 import org.apache.tinkerpop.gremlin.process.computer.VertexProgram;
 import org.apache.tinkerpop.gremlin.process.computer.util.ComputerGraph;
@@ -46,7 +40,6 @@ import org.apache.tinkerpop.gremlin.structure.util.star.StarGraph;
 import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 import scala.Tuple2;
 
-import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
@@ -191,25 +184,4 @@ public final class SparkExecutor {
             reduceRDD = reduceRDD.sortByKey(mapReduce.getReduceKeySort().get(), true, 1);
         return reduceRDD;
     }
-
-    ///////////////////
-    // Input/Output //
-    //////////////////
-
-    public static void saveMapReduceRDD(final JavaPairRDD<Object, Object> mapReduceRDD, final MapReduce mapReduce, final Memory.Admin memory, final org.apache.hadoop.conf.Configuration hadoopConfiguration) {
-        final String outputLocation = hadoopConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, null);
-        if (null != outputLocation) {
-            // map back to a Hadoop stream for output
-            mapReduceRDD.mapToPair(keyValue -> new Tuple2<>(new ObjectWritable<>(keyValue._1()), new ObjectWritable<>(keyValue._2()))).saveAsNewAPIHadoopFile(outputLocation + "/" + mapReduce.getMemoryKey(),
-                    ObjectWritable.class,
-                    ObjectWritable.class,
-                    SequenceFileOutputFormat.class, hadoopConfiguration);
-            // TODO: mapReduce.addResultToMemory(memory, mapReduceRDD.map(tuple -> new KeyValue<>(tuple._1(), tuple._2())).collect().iterator());
-            try {
-                mapReduce.addResultToMemory(memory, new ObjectWritableIterator(hadoopConfiguration, new Path(outputLocation + "/" + mapReduce.getMemoryKey())));
-            } catch (final IOException e) {
-                throw new IllegalStateException(e.getMessage(), e);
-            }
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
index 56364eb..03b3016 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
@@ -224,7 +224,14 @@ public final class SparkGraphComputer extends AbstractHadoopGraphComputer {
                         // reduce
                         final JavaPairRDD reduceRDD = (mapReduce.doStage(MapReduce.Stage.REDUCE)) ? SparkExecutor.executeReduce(mapRDD, mapReduce, newApacheConfiguration) : null;
                         // write the map reduce output back to disk (memory)
-                        SparkExecutor.saveMapReduceRDD(null == reduceRDD ? mapRDD : reduceRDD, mapReduce, finalMemory, hadoopConfiguration);
+                        try {
+                            mapReduce.addResultToMemory(finalMemory,
+                                    hadoopConfiguration.getClass(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, OutputFormatRDD.class, OutputRDD.class)
+                                            .newInstance()
+                                            .writeMemoryRDD(apacheConfiguration, mapReduce.getMemoryKey(), null == reduceRDD ? mapRDD : reduceRDD));
+                        } catch (final InstantiationException | IllegalAccessException e) {
+                            throw new IllegalStateException(e.getMessage(), e);
+                        }
                     }
                     mapReduceGraphRDD.unpersist();
                 }
@@ -232,7 +239,6 @@ public final class SparkGraphComputer extends AbstractHadoopGraphComputer {
                 // unpersist the graphRDD if it will no longer be used
                 if (!PersistedOutputRDD.class.equals(hadoopConfiguration.getClass(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, null)) || this.persist.equals(GraphComputer.Persist.NOTHING)) {
                     graphRDD.unpersist();
-                    Spark.removeRDD(hadoopConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION)); // delete persisted output rdd if it exists
                 }
                 // update runtime and return the newly computed graph
                 finalMemory.setRuntime(System.currentTimeMillis() - startTime);

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputFormatRDD.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputFormatRDD.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputFormatRDD.java
index 72fe57b..12a8268 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputFormatRDD.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputFormatRDD.java
@@ -22,9 +22,11 @@ package org.apache.tinkerpop.gremlin.spark.structure.io;
 import org.apache.commons.configuration.Configuration;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.tinkerpop.gremlin.hadoop.Constants;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
 import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
 import scala.Tuple2;
@@ -43,4 +45,15 @@ public final class InputFormatRDD implements InputRDD {
                 VertexWritable.class)
                 .mapToPair(tuple -> new Tuple2<>(tuple._2().get().id(), new VertexWritable(tuple._2().get())));
     }
+
+    @Override
+    public <K, V> JavaPairRDD<K, V> readMemoryRDD(final Configuration configuration, final String memoryKey, final JavaSparkContext sparkContext) {
+        final org.apache.hadoop.conf.Configuration hadoopConfiguration = ConfUtil.makeHadoopConfiguration(configuration);
+        // use FileInput location
+        return sparkContext.newAPIHadoopRDD(hadoopConfiguration,
+                SequenceFileInputFormat.class,
+                ObjectWritable.class,
+                ObjectWritable.class)
+                .mapToPair(tuple -> new Tuple2<>((K) ((Tuple2<ObjectWritable,ObjectWritable>)tuple)._1().get(), (V) ((Tuple2<ObjectWritable,ObjectWritable>)tuple)._2().get()));
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputOutputHelper.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputOutputHelper.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputOutputHelper.java
index 6736fb0..4753028 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputOutputHelper.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputOutputHelper.java
@@ -71,9 +71,6 @@ public final class InputOutputHelper {
                 newConfiguration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, InputRDDFormat.class.getCanonicalName());
                 //newConfiguration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT, OutputRDDFormat.class.getCanonicalName());
                 newConfiguration.setProperty(Constants.GREMLIN_SPARK_GRAPH_INPUT_RDD, InputOutputHelper.getInputFormat((Class) Class.forName(hadoopConfiguration.getString(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD))).getCanonicalName());
-                if (newConfiguration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION, "").endsWith("/" + Constants.HIDDEN_G)) {  // Spark RDDs are not namespaced the same as Hadoop
-                    newConfiguration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, newConfiguration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION).replace("/" + Constants.HIDDEN_G, ""));
-                }
             }
             return HadoopGraph.open(newConfiguration);
         } catch (final ClassNotFoundException e) {

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputRDD.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputRDD.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputRDD.java
index c84c189..982e8ec 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputRDD.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputRDD.java
@@ -33,9 +33,22 @@ public interface InputRDD {
 
     /**
      * Read the graphRDD from the underlying graph system.
-     * @param configuration the configuration for the {@link org.apache.tinkerpop.gremlin.spark.process.computer.SparkGraphComputer}.
-     * @param sparkContext the Spark context with the requisite methods for generating a {@link JavaPairRDD}.
+     *
+     * @param configuration the configuration for the {@link org.apache.tinkerpop.gremlin.spark.process.computer.SparkGraphComputer}
+     * @param sparkContext  the Spark context with the requisite methods for generating a {@link JavaPairRDD}
      * @return an adjacency list representation of the underlying graph system.
      */
     public JavaPairRDD<Object, VertexWritable> readGraphRDD(final Configuration configuration, final JavaSparkContext sparkContext);
+
+    /**
+     * Read a memoryRDD from the storage location.
+     *
+     * @param configuration the configuration for the {@link org.apache.tinkerpop.gremlin.spark.process.computer.SparkGraphComputer}
+     * @param memoryKey     the memory key of the memoryRDD
+     * @param sparkContext  the Spark context with the requisite methods for generating a {@link JavaPairRDD}
+     * @param <K>           the key class of the memoryRDD
+     * @param <V>           the value class of the memoryRDD
+     * @return the memoryRDD with respective key/value pairs.
+     */
+    public <K, V> JavaPairRDD<K, V> readMemoryRDD(final Configuration configuration, final String memoryKey, final JavaSparkContext sparkContext);
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/OutputFormatRDD.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/OutputFormatRDD.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/OutputFormatRDD.java
index cc6ed61..92158af 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/OutputFormatRDD.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/OutputFormatRDD.java
@@ -20,14 +20,23 @@
 package org.apache.tinkerpop.gremlin.spark.structure.io;
 
 import org.apache.commons.configuration.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.tinkerpop.gremlin.hadoop.Constants;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritableIterator;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
 import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
+import org.apache.tinkerpop.gremlin.process.computer.KeyValue;
 import scala.Tuple2;
 
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
@@ -46,4 +55,23 @@ public final class OutputFormatRDD implements OutputRDD {
                             (Class<OutputFormat<NullWritable, VertexWritable>>) hadoopConfiguration.getClass(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT, OutputFormat.class), hadoopConfiguration);
         }
     }
+
+    @Override
+    public <K, V> Iterator<KeyValue<K, V>> writeMemoryRDD(final Configuration configuration, final String memoryKey, JavaPairRDD<K, V> memoryRDD) {
+        final org.apache.hadoop.conf.Configuration hadoopConfiguration = ConfUtil.makeHadoopConfiguration(configuration);
+        final String outputLocation = hadoopConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION);
+        if (null != outputLocation) {
+            // map back to a Hadoop stream for output
+            memoryRDD.mapToPair(keyValue -> new Tuple2<>(new ObjectWritable<>(keyValue._1()), new ObjectWritable<>(keyValue._2()))).saveAsNewAPIHadoopFile(outputLocation + "/" + memoryKey,
+                    ObjectWritable.class,
+                    ObjectWritable.class,
+                    SequenceFileOutputFormat.class, hadoopConfiguration);
+            try {
+                return (Iterator) new ObjectWritableIterator(hadoopConfiguration, new Path(outputLocation + "/" + memoryKey));
+            } catch (final IOException e) {
+                throw new IllegalStateException(e.getMessage(), e);
+            }
+        }
+        return Collections.emptyIterator();
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/OutputRDD.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/OutputRDD.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/OutputRDD.java
index c2964eb..6512b83 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/OutputRDD.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/OutputRDD.java
@@ -21,11 +21,32 @@ package org.apache.tinkerpop.gremlin.spark.structure.io;
 import org.apache.commons.configuration.Configuration;
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
+import org.apache.tinkerpop.gremlin.process.computer.KeyValue;
+
+import java.util.Iterator;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
 public interface OutputRDD {
 
+    /**
+     * Write the graphRDD to an output location. The {@link Configuration} maintains the specified location via {@link org.apache.tinkerpop.gremlin.hadoop.Constants#GREMLIN_HADOOP_OUTPUT_LOCATION}.
+     *
+     * @param configuration the configuration of the Spark job
+     * @param graphRDD      the graphRDD to output
+     */
     public void writeGraphRDD(final Configuration configuration, final JavaPairRDD<Object, VertexWritable> graphRDD);
+
+    /**
+     * Write the sideEffect memoryRDD to an output location. The {@link Configuration} maintains the specified location via {@link org.apache.tinkerpop.gremlin.hadoop.Constants#GREMLIN_HADOOP_OUTPUT_LOCATION}.
+     *
+     * @param configuration the configuration of the Spark job
+     * @param memoryKey     the memory key of the memoryRDD
+     * @param memoryRDD     the memoryRDD
+     * @param <K>           the key class of the RDD
+     * @param <V>           the value class of the RDD
+     * @return the {@link KeyValue} iterator to store in the final resultant {@link org.apache.tinkerpop.gremlin.process.computer.Memory}.
+     */
+    public <K, V> Iterator<KeyValue<K, V>> writeMemoryRDD(final Configuration configuration, final String memoryKey, final JavaPairRDD<K, V> memoryRDD);
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputRDD.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputRDD.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputRDD.java
index 88e1efa..52d18f1 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputRDD.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputRDD.java
@@ -34,10 +34,19 @@ public final class PersistedInputRDD implements InputRDD {
 
     @Override
     public JavaPairRDD<Object, VertexWritable> readGraphRDD(final Configuration configuration, final JavaSparkContext sparkContext) {
-        final String inputRDDName = configuration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION, null);
-        if (null == inputRDDName)
-            throw new IllegalArgumentException(PersistedInputRDD.class.getSimpleName() + " requires " + Constants.GREMLIN_HADOOP_INPUT_LOCATION + " in order to retrieve the named graphRDD from the SparkContext");
+        if (!configuration.containsKey(Constants.GREMLIN_HADOOP_INPUT_LOCATION))
+            throw new IllegalArgumentException("There is no provided " + Constants.GREMLIN_HADOOP_INPUT_LOCATION + " to read the persisted RDD from");
+        final String graphRDDName = Constants.getGraphLocation(configuration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION));
         Spark.create(sparkContext.sc());
-        return JavaPairRDD.fromJavaRDD((JavaRDD) Spark.getRDD(inputRDDName).toJavaRDD());
+        return JavaPairRDD.fromJavaRDD((JavaRDD) Spark.getRDD(graphRDDName).toJavaRDD());
+    }
+
+    @Override
+    public <K, V> JavaPairRDD<K, V> readMemoryRDD(final Configuration configuration, final String memoryKey, final JavaSparkContext sparkContext) {
+        if (!configuration.containsKey(Constants.GREMLIN_HADOOP_INPUT_LOCATION))
+            throw new IllegalArgumentException("There is no provided " + Constants.GREMLIN_HADOOP_INPUT_LOCATION + " to read the persisted RDD from");
+        final String sideEffectRDDName = Constants.getMemoryLocation(configuration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION), memoryKey);
+        Spark.create(sparkContext.sc());
+        return JavaPairRDD.fromJavaRDD((JavaRDD) Spark.getRDD(sideEffectRDDName).toJavaRDD());
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java
index 5188f9a..5cb9edf 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java
@@ -23,10 +23,14 @@ import org.apache.commons.configuration.Configuration;
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.tinkerpop.gremlin.hadoop.Constants;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
+import org.apache.tinkerpop.gremlin.process.computer.KeyValue;
 import org.apache.tinkerpop.gremlin.spark.structure.Spark;
+import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.Iterator;
+
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
@@ -36,19 +40,31 @@ public final class PersistedOutputRDD implements OutputRDD {
 
     @Override
     public void writeGraphRDD(final Configuration configuration, final JavaPairRDD<Object, VertexWritable> graphRDD) {
+        if (!configuration.getBoolean(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, false))
+            LOGGER.warn("The SparkContext should be persisted in order for the RDD to persist across jobs. To do so, set " + Constants.GREMLIN_SPARK_PERSIST_CONTEXT + " to true");
         if (!configuration.containsKey(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION))
             throw new IllegalArgumentException("There is no provided " + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + " to write the persisted RDD to");
-        Spark.removeRDD(configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));  // this might be bad cause it unpersists the job RDD
+        final String graphRDDName = Constants.getGraphLocation(configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));
+        Spark.removeRDD(graphRDDName);  // this might be bad cause it unpersists the job RDD
         if (!configuration.getBoolean(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT_HAS_EDGES, true))
             graphRDD.mapValues(vertex -> {
                 vertex.get().dropEdges();
                 return vertex;
-            }).setName(configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION)).cache();
+            }).setName(graphRDDName).cache();
         else
-            graphRDD.setName(configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION)).cache();
+            graphRDD.setName(graphRDDName).cache();
+        Spark.refresh();
+    }
+
+    @Override
+    public <K, V> Iterator<KeyValue<K, V>> writeMemoryRDD(final Configuration configuration, final String memoryKey, final JavaPairRDD<K, V> memoryRDD) {
         if (!configuration.getBoolean(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, false))
             LOGGER.warn("The SparkContext should be persisted in order for the RDD to persist across jobs. To do so, set " + Constants.GREMLIN_SPARK_PERSIST_CONTEXT + " to true");
-
-        Spark.refresh();
+        if (!configuration.containsKey(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION))
+            throw new IllegalArgumentException("There is no provided " + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + " to write the persisted RDD to");
+        final String sideEffectRDDName = Constants.getMemoryLocation(configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), memoryKey);
+        Spark.removeRDD(sideEffectRDDName);
+        memoryRDD.setName(sideEffectRDDName).cache();
+        return IteratorUtils.map(memoryRDD.toLocalIterator(), tuple -> new KeyValue<>(tuple._1(), tuple._2()));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/LocalPropertyTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/LocalPropertyTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/LocalPropertyTest.java
index 9aa52d9..1bf5338 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/LocalPropertyTest.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/LocalPropertyTest.java
@@ -51,7 +51,7 @@ public class LocalPropertyTest extends AbstractSparkTest {
     @Test
     public void shouldSetThreadLocalProperties() throws Exception {
         final String testName = "ThreadLocalProperties";
-        final String rddName = "target/test-output/" + UUID.randomUUID();
+        final String rddLocation = "target/test-output/" + UUID.randomUUID();
         final Configuration configuration = new BaseConfiguration();
         configuration.setProperty("spark.master", "local[4]");
         configuration.setProperty("spark.serializer", GryoSerializer.class.getCanonicalName());
@@ -59,7 +59,7 @@ public class LocalPropertyTest extends AbstractSparkTest {
         configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, SparkHadoopGraphProvider.PATHS.get("tinkerpop-modern.kryo"));
         configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, GryoInputFormat.class.getCanonicalName());
         configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
-        configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddName);
+        configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddLocation);
         configuration.setProperty(Constants.GREMLIN_HADOOP_JARS_IN_DISTRIBUTED_CACHE, false);
         configuration.setProperty(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, true);
         configuration.setProperty("spark.jobGroup.id", "22");
@@ -78,10 +78,10 @@ public class LocalPropertyTest extends AbstractSparkTest {
         JavaSparkContext sparkContext = new JavaSparkContext(SparkContext.getOrCreate(sparkConfiguration));
         JavaSparkStatusTracker statusTracker = sparkContext.statusTracker();
         assertTrue(statusTracker.getJobIdsForGroup("22").length >= 1);
-        assertTrue(Spark.hasRDD(rddName));
+        assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
         ///////
         configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_INPUT_RDD, PersistedInputRDD.class.getCanonicalName());
-        configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, rddName);
+        configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, rddLocation);
         configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, null);
         configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, null);
         configuration.setProperty(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, false);

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java
index 3f8ff17..0b60825 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java
@@ -65,7 +65,7 @@ public class SparkGremlinPluginTest extends AbstractSparkTest {
 
     @Test
     public void shouldSupportBasicRDDOperations() throws Exception {
-        String rddName = "target/test-output/graph-1";
+        String rddLocation = "target/test-output/graph-1";
         final Configuration configuration = new BaseConfiguration();
         configuration.setProperty("spark.master", "local[4]");
         configuration.setProperty("spark.serializer", GryoSerializer.class.getCanonicalName());
@@ -74,7 +74,7 @@ public class SparkGremlinPluginTest extends AbstractSparkTest {
         configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, GryoInputFormat.class.getCanonicalName());
         configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
         configuration.setProperty(Constants.GREMLIN_HADOOP_JARS_IN_DISTRIBUTED_CACHE, false);
-        configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddName);
+        configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddLocation);
         configuration.setProperty(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, true);
         Graph graph = GraphFactory.open(configuration);
 
@@ -85,34 +85,34 @@ public class SparkGremlinPluginTest extends AbstractSparkTest {
         this.console.addBinding("graph", graph);
         this.console.eval("graph.compute(SparkGraphComputer).program(PageRankVertexProgram.build().iterations(1).create()).submit().get()");
         assertEquals(1, ((List<String>) this.console.eval("spark.ls()")).size());
-        assertEquals(rddName + " [Memory Deserialized 1x Replicated]", ((List<String>) this.console.eval("spark.ls()")).get(0));
+        assertEquals(Constants.getGraphLocation(rddLocation) + " [Memory Deserialized 1x Replicated]", ((List<String>) this.console.eval("spark.ls()")).get(0));
 
-        rddName = "target/test-output/graph-2";
-        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + rddName + "')");
+        rddLocation = "target/test-output/graph-2";
+        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + rddLocation + "')");
         this.console.eval("graph.compute(SparkGraphComputer).program(PageRankVertexProgram.build().iterations(1).create()).submit().get()");
         assertEquals(2, ((List<String>) this.console.eval("spark.ls()")).size());
-        assertTrue(((List<String>) this.console.eval("spark.ls()")).contains(rddName + " [Memory Deserialized 1x Replicated]"));
+        assertTrue(((List<String>) this.console.eval("spark.ls()")).contains(Constants.getGraphLocation(rddLocation) + " [Memory Deserialized 1x Replicated]"));
 
-        this.console.eval("spark.rm('target/test-output/graph-2')");
+        this.console.eval("spark.rm('target/test-output/graph-2/~g')");
         assertEquals(1, ((List<String>) this.console.eval("spark.ls()")).size());
-        assertTrue(((List<String>) this.console.eval("spark.ls()")).contains("target/test-output/graph-1 [Memory Deserialized 1x Replicated]"));
+        assertTrue(((List<String>) this.console.eval("spark.ls()")).contains("target/test-output/graph-1/~g [Memory Deserialized 1x Replicated]"));
 
-        assertEquals(6, ((List<Object>) this.console.eval("spark.head('target/test-output/graph-1')")).size());
+        assertEquals(6, ((List<Object>) this.console.eval("spark.head('target/test-output/graph-1/~g')")).size());
 
         this.console.eval("spark.rm('target/test-output/graph-*')");
         assertEquals(0, ((List<String>) this.console.eval("spark.ls()")).size());
 
         //////
-        rddName = "target/test-output/graph-1";
-        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + rddName + "')");
+        rddLocation = "target/test-output/graph-1";
+        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + rddLocation + "')");
         this.console.eval("graph.compute(SparkGraphComputer).program(PageRankVertexProgram.build().iterations(1).create()).submit().get()");
 
-        rddName = "target/test-output/graph-2";
-        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + rddName + "')");
+        rddLocation = "target/test-output/graph-2";
+        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + rddLocation + "')");
         this.console.eval("graph.compute(SparkGraphComputer).program(PageRankVertexProgram.build().iterations(1).create()).submit().get()");
 
-        rddName = "target/test-output/x";
-        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + rddName + "')");
+        rddLocation = "target/test-output/x";
+        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + rddLocation + "')");
         this.console.eval("graph.compute(SparkGraphComputer).program(PageRankVertexProgram.build().iterations(1).create()).submit().get()");
 
         assertEquals(3, ((List<String>) this.console.eval("spark.ls()")).size());

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/SparkTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/SparkTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/SparkTest.java
index 9e9882c..af49905 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/SparkTest.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/SparkTest.java
@@ -63,20 +63,22 @@ public class SparkTest extends AbstractSparkTest {
         configuration.setProperty(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, true);
 
         for (int i = 0; i < 10; i++) {
+            final String graphRDDName = Constants.getGraphLocation(prefix + i);
             assertEquals(i, Spark.getRDDs().size());
             configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, prefix + i);
             Graph graph = GraphFactory.open(configuration);
             graph.compute(SparkGraphComputer.class).persist(GraphComputer.Persist.VERTEX_PROPERTIES).program(PageRankVertexProgram.build().iterations(1).create(graph)).submit().get();
-            assertNotNull(Spark.getRDD(prefix + i));
+            assertNotNull(Spark.getRDD(graphRDDName));
             assertEquals(i + 1, Spark.getRDDs().size());
         }
 
         for (int i = 9; i >= 0; i--) {
+            final String graphRDDName = Constants.getGraphLocation(prefix + i);
             assertEquals(i + 1, getPersistedRDDSize());
             assertEquals(i + 1, Spark.getRDDs().size());
-            assertTrue(hasPersistedRDD(prefix + i));
-            Spark.removeRDD(prefix + i);
-            assertFalse(hasPersistedRDD(prefix + i));
+            assertTrue(hasPersistedRDD(graphRDDName));
+            Spark.removeRDD(graphRDDName);
+            assertFalse(hasPersistedRDD(graphRDDName));
         }
 
         assertEquals(0, getPersistedRDDSize());

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ExampleInputRDD.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ExampleInputRDD.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ExampleInputRDD.java
index 86c7610..87c4e2a 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ExampleInputRDD.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ExampleInputRDD.java
@@ -48,4 +48,9 @@ public final class ExampleInputRDD implements InputRDD {
         list.add(StarGraph.open().addVertex(T.id, 6l, T.label, "person", "age", 35));
         return sparkContext.parallelize(list).mapToPair(vertex -> new Tuple2<>(vertex.id(), new VertexWritable(vertex)));
     }
+
+    @Override
+    public <K, V> JavaPairRDD<K, V> readMemoryRDD(Configuration configuration, String memoryKey, JavaSparkContext sparkContext) {
+        return null;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ExampleOutputRDD.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ExampleOutputRDD.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ExampleOutputRDD.java
index 103ec20..321adcc 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ExampleOutputRDD.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ExampleOutputRDD.java
@@ -21,6 +21,7 @@ package org.apache.tinkerpop.gremlin.spark.structure.io;
 import org.apache.commons.configuration.Configuration;
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
+import org.apache.tinkerpop.gremlin.process.computer.KeyValue;
 import org.apache.tinkerpop.gremlin.structure.Vertex;
 
 import java.util.Iterator;
@@ -47,4 +48,9 @@ public final class ExampleOutputRDD implements OutputRDD {
         }
         assertEquals(123, totalAge);
     }
+
+    @Override
+    public <K, V> Iterator<KeyValue<K, V>> writeMemoryRDD(Configuration configuration, String memoryKey, JavaPairRDD<K, V> memoryRDD) {
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputOutputRDDTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputOutputRDDTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputOutputRDDTest.java
index 9ad50a9..954cdfe 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputOutputRDDTest.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputOutputRDDTest.java
@@ -57,7 +57,7 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
     @Test
     public void shouldNotPersistRDDAcrossJobs() throws Exception {
         Spark.create("local[4]");
-        final String rddName = "target/test-output/" + UUID.randomUUID();
+        final String rddLocation = "target/test-output/" + UUID.randomUUID();
         final Configuration configuration = new BaseConfiguration();
         configuration.setProperty("spark.master", "local[4]");
         configuration.setProperty("spark.serializer", GryoSerializer.class.getCanonicalName());
@@ -66,7 +66,7 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
         configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, GryoInputFormat.class.getCanonicalName());
         configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
         configuration.setProperty(Constants.GREMLIN_HADOOP_JARS_IN_DISTRIBUTED_CACHE, false);
-        configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddName);
+        configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddLocation);
         configuration.setProperty(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, false);  // because the spark context is NOT persisted, neither is the RDD
         Graph graph = GraphFactory.open(configuration);
         graph.compute(SparkGraphComputer.class)
@@ -78,13 +78,13 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
                                 "g.V()").create(graph)).submit().get();
         ////////
         Spark.create("local[4]");
-        assertFalse(Spark.hasRDD(rddName));
+        assertFalse(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
         Spark.close();
     }
 
     @Test
     public void shouldPersistRDDAcrossJobs() throws Exception {
-        final String rddName = "target/test-output/" + UUID.randomUUID();
+        final String rddLocation = "target/test-output/" + UUID.randomUUID();
         final Configuration configuration = new BaseConfiguration();
         configuration.setProperty("spark.master", "local[4]");
         configuration.setProperty("spark.serializer", GryoSerializer.class.getCanonicalName());
@@ -92,7 +92,7 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
         configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, SparkHadoopGraphProvider.PATHS.get("tinkerpop-modern.kryo"));
         configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, GryoInputFormat.class.getCanonicalName());
         configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
-        configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddName);
+        configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddLocation);
         configuration.setProperty(Constants.GREMLIN_HADOOP_JARS_IN_DISTRIBUTED_CACHE, false);
         configuration.setProperty(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, true);
         Graph graph = GraphFactory.open(configuration);
@@ -104,10 +104,10 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
                                 "gremlin-groovy",
                                 "g.V()").create(graph)).submit().get();
         ////////
-        assertTrue(Spark.hasRDD(rddName));
+        assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
         ///////
         configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_INPUT_RDD, PersistedInputRDD.class.getCanonicalName());
-        configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, rddName);
+        configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, rddLocation);
         configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, null);
         configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, null);
         graph = GraphFactory.open(configuration);
@@ -125,7 +125,7 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
     public void testBulkLoaderVertexProgramChain() throws Exception {
         Spark.create("local[4]");
 
-        final String rddName = "target/test-output/" + UUID.randomUUID().toString();
+        final String rddLocation = "target/test-output/" + UUID.randomUUID().toString();
         final Configuration readConfiguration = new BaseConfiguration();
         readConfiguration.setProperty("spark.master", "local[4]");
         readConfiguration.setProperty("spark.serializer", GryoSerializer.class.getCanonicalName());
@@ -133,7 +133,7 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
         readConfiguration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, GryoInputFormat.class.getCanonicalName());
         readConfiguration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, SparkHadoopGraphProvider.PATHS.get("tinkerpop-modern.kryo"));
         readConfiguration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
-        readConfiguration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddName);
+        readConfiguration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddLocation);
         readConfiguration.setProperty(Constants.GREMLIN_HADOOP_JARS_IN_DISTRIBUTED_CACHE, false);
         readConfiguration.setProperty(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, true);
         Graph pageRankGraph = GraphFactory.open(readConfiguration);
@@ -147,13 +147,13 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
                 .persist(GraphComputer.Persist.NOTHING)
                 .workers(1)
                 .configure(Constants.GREMLIN_SPARK_GRAPH_INPUT_RDD, PersistedInputRDD.class.getCanonicalName())
-                .configure(Constants.GREMLIN_HADOOP_INPUT_LOCATION, rddName)
+                .configure(Constants.GREMLIN_HADOOP_INPUT_LOCATION, rddLocation)
                 .configure(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, null)
                 .configure(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, null)
                 .program(BulkLoaderVertexProgram.build().userSuppliedIds(true).writeGraph(writeConfiguration).create(bulkLoaderGraph))
                 .submit().get();
         ////
-        assertTrue(Spark.hasRDD(rddName));
+        assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
         ////
         final Graph graph = TinkerGraph.open();
         final GraphTraversalSource g = graph.traversal();
@@ -171,7 +171,7 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
     public void testBulkLoaderVertexProgramChainWithInputOutputHelperMapping() throws Exception {
         Spark.create("local[4]");
 
-        final String rddName = "target/test-output/" + UUID.randomUUID().toString();
+        final String rddLocation = "target/test-output/" + UUID.randomUUID().toString();
         final Configuration readConfiguration = new BaseConfiguration();
         readConfiguration.setProperty("spark.master", "local[4]");
         readConfiguration.setProperty("spark.serializer", GryoSerializer.class.getCanonicalName());
@@ -179,7 +179,7 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
         readConfiguration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, GryoInputFormat.class.getCanonicalName());
         readConfiguration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, SparkHadoopGraphProvider.PATHS.get("tinkerpop-modern.kryo"));
         readConfiguration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
-        readConfiguration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddName);
+        readConfiguration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddLocation);
         readConfiguration.setProperty(Constants.GREMLIN_HADOOP_JARS_IN_DISTRIBUTED_CACHE, false);
         readConfiguration.setProperty(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, true);
         Graph pageRankGraph = GraphFactory.open(readConfiguration);
@@ -196,7 +196,7 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
                 .submit().get();
         ////
         Spark.create(readConfiguration);
-        assertTrue(Spark.hasRDD(rddName));
+        assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
         ////
         final Graph graph = TinkerGraph.open();
         final GraphTraversalSource g = graph.traversal();
@@ -214,7 +214,7 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
     public void testComplexChain() throws Exception {
         Spark.create("local[4]");
 
-        final String rddName = "target/test-output/graphRDD";
+        final String rddLocation = "target/test-output/graphRDD";
         final Configuration configuration = new BaseConfiguration();
         configuration.setProperty("spark.master", "local[4]");
         configuration.setProperty("spark.serializer", GryoSerializer.class.getCanonicalName());
@@ -222,7 +222,7 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
         configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, GryoInputFormat.class.getCanonicalName());
         configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, SparkHadoopGraphProvider.PATHS.get("tinkerpop-modern.kryo"));
         configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
-        configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddName);
+        configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddLocation);
         configuration.setProperty(Constants.GREMLIN_HADOOP_JARS_IN_DISTRIBUTED_CACHE, false);
         configuration.setProperty(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, true);
         Graph graph = GraphFactory.open(configuration);
@@ -233,12 +233,12 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
         assertEquals(6l, g.V().values(PageRankVertexProgram.PAGE_RANK).count().next().longValue());
         assertEquals(6l, g.V().values(PageRankVertexProgram.EDGE_COUNT).count().next().longValue());
         ////
-        assertTrue(Spark.hasRDD(rddName));
+        assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
         ////
         configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_INPUT_RDD, PersistedInputRDD.class.getCanonicalName());
-        configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, rddName);
+        configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, rddLocation);
         configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
-        configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddName);
+        configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddLocation);
         ////
         graph = GraphFactory.open(configuration);
         graph = graph.compute(SparkGraphComputer.class).persist(GraphComputer.Persist.EDGES).program(PageRankVertexProgram.build().iterations(2).create(graph)).submit().get().graph();
@@ -248,7 +248,7 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
         assertEquals(6l, g.V().values(PageRankVertexProgram.PAGE_RANK).count().next().longValue());
         assertEquals(6l, g.V().values(PageRankVertexProgram.EDGE_COUNT).count().next().longValue());
         ////
-        assertTrue(Spark.hasRDD(rddName));
+        assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
         ////
         graph = GraphFactory.open(configuration);
         graph = graph.compute(SparkGraphComputer.class).persist(GraphComputer.Persist.VERTEX_PROPERTIES).program(PageRankVertexProgram.build().iterations(2).create(graph)).submit().get().graph();
@@ -258,18 +258,18 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
         assertEquals(6l, g.V().values(PageRankVertexProgram.PAGE_RANK).count().next().longValue());
         assertEquals(6l, g.V().values(PageRankVertexProgram.EDGE_COUNT).count().next().longValue());
         ////
-        assertTrue(Spark.hasRDD(rddName));
+        assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
         ////
         graph = GraphFactory.open(configuration);
         graph.compute(SparkGraphComputer.class).persist(GraphComputer.Persist.NOTHING).program(PageRankVertexProgram.build().iterations(2).create(graph)).submit().get().graph();
-        assertFalse(Spark.hasRDD(rddName));
+        assertFalse(Spark.hasRDD(rddLocation));
         g = graph.traversal();
         assertEquals(0l, g.V().count().next().longValue());
         assertEquals(0l, g.E().count().next().longValue());
         assertEquals(0l, g.V().values(PageRankVertexProgram.PAGE_RANK).count().next().longValue());
         assertEquals(0l, g.V().values(PageRankVertexProgram.EDGE_COUNT).count().next().longValue());
         ////
-        assertFalse(Spark.hasRDD(rddName));
+        assertFalse(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
         Spark.close();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f3ebed0b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ToyGraphInputRDD.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ToyGraphInputRDD.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ToyGraphInputRDD.java
index ea3636f..5b9ada9 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ToyGraphInputRDD.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ToyGraphInputRDD.java
@@ -70,4 +70,11 @@ public final class ToyGraphInputRDD implements InputRDD {
 
         return sparkContext.parallelize(vertices).mapToPair(vertex -> new Tuple2<>(vertex.get().id(), vertex));
     }
+
+    @Override
+    public <K, V> JavaPairRDD<K, V> readMemoryRDD(Configuration configuration, String memoryKey, JavaSparkContext sparkContext) {
+        return null;
+    }
+
+
 }


[02/30] incubator-tinkerpop git commit: added Storage to gremlin-core. Storage is an interface that OLAP system can implement. It provides ls(), rmr(), rm(), etc. type methods that make it easy for users to interact (via a common interface) with the unde

Posted by dk...@apache.org.
added Storage to gremlin-core. Storage is an interface that OLAP system can implement. It provides ls(), rmr(), rm(), etc. type methods that make it easy for users to interact (via a common interface) with the underlying persitance system. Now both HDFS and Spark provide their own Storage implementations and TADA. Really pretty.


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/58d92407
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/58d92407
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/58d92407

Branch: refs/heads/TINKERPOP-320
Commit: 58d9240764cd6e1f3779097966c53058264e00e6
Parents: f3ebed0
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Dec 9 13:46:43 2015 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Wed Dec 9 13:46:43 2015 -0700

----------------------------------------------------------------------
 .../peerpressure/ClusterCountMapReduce.java     |   7 +-
 .../tinkerpop/gremlin/structure/io/Storage.java |  82 ++++++++
 .../gremlin/structure/util/StringFactory.java   |  18 +-
 .../hadoop/groovy/plugin/HadoopLoader.groovy    | 138 --------------
 .../groovy/plugin/HadoopGremlinPlugin.java      |   7 +-
 .../hadoop/structure/io/FileSystemStorage.java  | 186 +++++++++++++++++++
 .../groovy/plugin/GraphMemoryHDFSCheck.java     |  64 +++++++
 .../hadoop/groovy/plugin/HadoopPluginSuite.java |   2 +-
 .../spark/groovy/plugin/SparkLoader.groovy      |  68 -------
 .../spark/groovy/plugin/SparkGremlinPlugin.java |   5 +-
 .../spark/structure/io/PersistedInputRDD.java   |   8 +-
 .../spark/structure/io/SparkContextStorage.java | 124 +++++++++++++
 .../gremlin/spark/AbstractSparkTest.java        |  30 +++
 .../groovy/plugin/SparkGremlinPluginTest.java   |  10 +-
 .../structure/io/GraphMemorySparkTest.java      |  75 ++++++++
 15 files changed, 593 insertions(+), 231 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/58d92407/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/clustering/peerpressure/ClusterCountMapReduce.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/clustering/peerpressure/ClusterCountMapReduce.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/clustering/peerpressure/ClusterCountMapReduce.java
index 1112a46..d343e8e 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/clustering/peerpressure/ClusterCountMapReduce.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/clustering/peerpressure/ClusterCountMapReduce.java
@@ -64,7 +64,7 @@ public class ClusterCountMapReduce extends StaticMapReduce<MapReduce.NullObject,
 
     @Override
     public boolean doStage(final Stage stage) {
-        return true;
+        return !stage.equals(Stage.COMBINE);
     }
 
     @Override
@@ -76,11 +76,6 @@ public class ClusterCountMapReduce extends StaticMapReduce<MapReduce.NullObject,
     }
 
     @Override
-    public void combine(final NullObject key, final Iterator<Serializable> values, final ReduceEmitter<NullObject, Integer> emitter) {
-        this.reduce(key, values, emitter);
-    }
-
-    @Override
     public void reduce(final NullObject key, final Iterator<Serializable> values, final ReduceEmitter<NullObject, Integer> emitter) {
         final Set<Serializable> set = new HashSet<>();
         values.forEachRemaining(set::add);

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/58d92407/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
new file mode 100644
index 0000000..1f1bcf4
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
@@ -0,0 +1,82 @@
+/*
+ * 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.tinkerpop.gremlin.structure.io;
+
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public interface Storage {
+
+    public List<String> ls();
+
+    public List<String> ls(final String location);
+
+    public boolean mkdir(final String location);
+
+    public boolean cp(final String fromLocation, final String toLocation);
+
+    public boolean exists(final String location);
+
+    public boolean rm(final String location);
+
+    public boolean rmr(final String location);
+
+    public <V> Iterator<V> head(final String location, final int totalLines, final Class<V> objectClass);
+
+    public default Iterator<Object> head(final String location) {
+        return this.head(location, Object.class);
+    }
+
+    public default Iterator<Object> head(final String location, final int totalLines) {
+        return this.head(location, totalLines, Object.class);
+    }
+
+    public default <V> Iterator<V> head(final String location, final Class<V> objectClass) {
+        return this.head(location, Integer.MAX_VALUE, objectClass);
+    }
+
+  /*
+
+        FileSystem.metaClass.copyToLocal = { final String from, final String to ->
+            return ((FileSystem) delegate).copyToLocalFile(new Path(from), new Path(to));
+        }
+
+        FileSystem.metaClass.copyFromLocal = { final String from, final String to ->
+            return ((FileSystem) delegate).copyFromLocalFile(new Path(from), new Path(to));
+        }
+
+        FileSystem.metaClass.mergeToLocal = { final String from, final String to ->
+            final FileSystem fs = (FileSystem) delegate;
+            final FileSystem local = FileSystem.getLocal(new Configuration());
+            final FSDataOutputStream outA = local.create(new Path(to));
+
+            HDFSTools.getAllFilePaths(fs, new Path(from), HiddenFileFilter.instance()).each {
+                final FSDataInputStream inA = fs.open(it);
+                IOUtils.copyBytes(inA, outA, 8192);
+                inA.close();
+            }
+            outA.close();
+        }
+
+     */
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/58d92407/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/StringFactory.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/StringFactory.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/StringFactory.java
index e716a60..9ae8116 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/StringFactory.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/StringFactory.java
@@ -18,18 +18,18 @@
  */
 package org.apache.tinkerpop.gremlin.structure.util;
 
+import org.apache.tinkerpop.gremlin.process.computer.ComputerResult;
+import org.apache.tinkerpop.gremlin.process.computer.GraphComputer;
+import org.apache.tinkerpop.gremlin.process.computer.MapReduce;
+import org.apache.tinkerpop.gremlin.process.computer.Memory;
+import org.apache.tinkerpop.gremlin.process.computer.VertexProgram;
 import org.apache.tinkerpop.gremlin.process.traversal.Step;
 import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
-import org.apache.tinkerpop.gremlin.process.traversal.TraversalSource;
 import org.apache.tinkerpop.gremlin.process.traversal.TraversalEngine;
 import org.apache.tinkerpop.gremlin.process.traversal.TraversalSideEffects;
+import org.apache.tinkerpop.gremlin.process.traversal.TraversalSource;
 import org.apache.tinkerpop.gremlin.process.traversal.TraversalStrategies;
 import org.apache.tinkerpop.gremlin.process.traversal.TraversalStrategy;
-import org.apache.tinkerpop.gremlin.process.computer.ComputerResult;
-import org.apache.tinkerpop.gremlin.process.computer.GraphComputer;
-import org.apache.tinkerpop.gremlin.process.computer.MapReduce;
-import org.apache.tinkerpop.gremlin.process.computer.Memory;
-import org.apache.tinkerpop.gremlin.process.computer.VertexProgram;
 import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalRing;
 import org.apache.tinkerpop.gremlin.structure.Edge;
 import org.apache.tinkerpop.gremlin.structure.Graph;
@@ -75,6 +75,7 @@ public final class StringFactory {
     private static final String EMPTY_PROPERTY = "p[empty]";
     private static final String EMPTY_VERTEX_PROPERTY = "vp[empty]";
     private static final String LINE_SEPARATOR = System.getProperty("line.separator");
+    private static final String STORAGE = "storage";
 
     private static final String featuresStartWith = "supports";
     private static final int prefixLength = featuresStartWith.length();
@@ -237,4 +238,9 @@ public final class StringFactory {
     public static String traversalString(final Traversal.Admin<?, ?> traversal) {
         return traversal.getSteps().toString();
     }
+
+    public static String storageString(final String internalString) {
+        return STORAGE + L_BRACKET + internalString + R_BRACKET;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/58d92407/hadoop-gremlin/src/main/groovy/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopLoader.groovy
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/groovy/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopLoader.groovy b/hadoop-gremlin/src/main/groovy/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopLoader.groovy
deleted file mode 100644
index 616c2f0..0000000
--- a/hadoop-gremlin/src/main/groovy/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopLoader.groovy
+++ /dev/null
@@ -1,138 +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.tinkerpop.gremlin.hadoop.groovy.plugin
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.*
-import org.apache.hadoop.io.IOUtils
-import org.apache.hadoop.io.Text
-import org.apache.tinkerpop.gremlin.hadoop.structure.hdfs.HDFSTools
-import org.apache.tinkerpop.gremlin.hadoop.structure.hdfs.HiddenFileFilter
-import org.apache.tinkerpop.gremlin.hadoop.structure.hdfs.TextIterator
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritableIterator
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritableIterator
-import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-class HadoopLoader {
-
-    private static final String SPACE = " ";
-    private static final String D_SPACE = "(D) ";
-
-    public static void load() {
-
-        FileStatus.metaClass.toString = {
-            StringBuilder s = new StringBuilder();
-            s.append(((FileStatus) delegate).getPermission()).append(SPACE)
-            s.append(((FileStatus) delegate).getOwner()).append(SPACE);
-            s.append(((FileStatus) delegate).getGroup()).append(SPACE);
-            s.append(((FileStatus) delegate).getLen()).append(SPACE);
-            if (((FileStatus) delegate).isDir())
-                s.append(D_SPACE);
-            s.append(((FileStatus) delegate).getPath().getName());
-            return s.toString();
-        }
-
-        FileSystem.metaClass.ls = { String path ->
-            if (null == path || path.equals("/")) path = ((FileSystem) delegate).getHomeDirectory().toString();
-            return ((FileSystem) delegate).globStatus(new Path(path + "/*")).collect {
-                it.toString()
-            };
-        }
-
-        FileSystem.metaClass.mkdir = { String path ->
-            ((FileSystem) delegate).mkdirs(new Path(path));
-        }
-
-        FileSystem.metaClass.cp = { final String from, final String to ->
-            return FileUtil.copy(((FileSystem) delegate), new Path(from), ((FileSystem) delegate), new Path(to), false, new Configuration());
-        }
-
-        FileSystem.metaClass.exists = { final String path ->
-            return ((FileSystem) delegate).exists(new Path(path));
-        }
-
-        FileSystem.metaClass.rm = { final String path ->
-            HDFSTools.globDelete((FileSystem) delegate, path, false);
-        }
-
-        FileSystem.metaClass.rmr = { final String path ->
-            HDFSTools.globDelete((FileSystem) delegate, path, true);
-        }
-
-        FileSystem.metaClass.copyToLocal = { final String from, final String to ->
-            return ((FileSystem) delegate).copyToLocalFile(new Path(from), new Path(to));
-        }
-
-        FileSystem.metaClass.copyFromLocal = { final String from, final String to ->
-            return ((FileSystem) delegate).copyFromLocalFile(new Path(from), new Path(to));
-        }
-
-        FileSystem.metaClass.mergeToLocal = { final String from, final String to ->
-            final FileSystem fs = (FileSystem) delegate;
-            final FileSystem local = FileSystem.getLocal(new Configuration());
-            final FSDataOutputStream outA = local.create(new Path(to));
-
-            HDFSTools.getAllFilePaths(fs, new Path(from), HiddenFileFilter.instance()).each {
-                final FSDataInputStream inA = fs.open(it);
-                IOUtils.copyBytes(inA, outA, 8192);
-                inA.close();
-            }
-            outA.close();
-        }
-
-        FileSystem.metaClass.head = { final String path, final int totalLines ->
-            return headMaker((FileSystem) delegate, path, totalLines, Text.class);
-        }
-
-        FileSystem.metaClass.head = { final String path ->
-            return headMaker((FileSystem) delegate, path, Integer.MAX_VALUE, Text.class);
-        }
-
-        FileSystem.metaClass.head = {
-            final String path, final Class<org.apache.hadoop.io.Writable> writableClass ->
-                return headMaker((FileSystem) delegate, path, Integer.MAX_VALUE, writableClass);
-        }
-
-        FileSystem.metaClass.head = {
-            final String path, final int totalLines, final Class<org.apache.hadoop.io.Writable> writableClass ->
-                return headMaker((FileSystem) delegate, path, totalLines, writableClass);
-        }
-
-        /*FileSystem.metaClass.unzip = { final String from, final String to, final boolean deleteZip ->
-            HDFSTools.decompressPath((FileSystem) delegate, from, to, Tokens.BZ2, deleteZip);
-        }*/
-
-    }
-
-    private static Iterator headMaker(
-            final FileSystem fs,
-            final String path, final int totalLines, final Class<org.apache.hadoop.io.Writable> writableClass) {
-        if (writableClass.equals(ObjectWritable.class))
-            return IteratorUtils.limit(new ObjectWritableIterator(fs.getConf(), new Path(path)), totalLines);
-        else if (writableClass.equals(VertexWritable.class))
-            return IteratorUtils.limit(new VertexWritableIterator(fs.getConf(), new Path(path)), totalLines);
-        else
-            return IteratorUtils.limit(new TextIterator(fs.getConf(), new Path(path)), totalLines);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/58d92407/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java
index d62b4e5..b4f5cd2 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java
@@ -32,6 +32,7 @@ import org.apache.tinkerpop.gremlin.hadoop.process.computer.mapreduce.MapReduceG
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopConfiguration;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
 import org.apache.tinkerpop.gremlin.hadoop.structure.hdfs.HDFSTools;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.FileSystemStorage;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.graphson.GraphSONInputFormat;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.gryo.GryoInputFormat;
@@ -86,10 +87,10 @@ public final class HadoopGremlinPlugin extends AbstractGremlinPlugin {
             pluginAcceptor.eval(String.format("Logger.getLogger(%s).setLevel(Level.INFO)", MapReduceGraphComputer.class.getName()));
             ///
             pluginAcceptor.eval(String.format("Logger.getLogger(%s).setLevel(Level.INFO)", HadoopGraph.class.getName()));
-            pluginAcceptor.eval(HadoopLoader.class.getCanonicalName() + ".load()");
+            //pluginAcceptor.eval(HadoopLoader.class.getCanonicalName() + ".load()");
 
-            pluginAcceptor.addBinding("hdfs", FileSystem.get(new Configuration()));
-            pluginAcceptor.addBinding("local", FileSystem.getLocal(new Configuration()));
+            pluginAcceptor.addBinding("hdfs", new FileSystemStorage(FileSystem.get(new Configuration())));
+            pluginAcceptor.addBinding("local", new FileSystemStorage(FileSystem.getLocal(new Configuration())));
             if (null == System.getenv(Constants.HADOOP_GREMLIN_LIBS))
                 HadoopGraph.LOGGER.warn("Be sure to set the environmental variable: " + Constants.HADOOP_GREMLIN_LIBS);
             else

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/58d92407/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
new file mode 100644
index 0000000..56dfe52
--- /dev/null
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
@@ -0,0 +1,186 @@
+/*
+ * 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.tinkerpop.gremlin.hadoop.structure.io;
+
+import org.apache.hadoop.conf.Configuration;
+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.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.Writable;
+import org.apache.tinkerpop.gremlin.hadoop.structure.hdfs.HDFSTools;
+import org.apache.tinkerpop.gremlin.hadoop.structure.hdfs.HiddenFileFilter;
+import org.apache.tinkerpop.gremlin.hadoop.structure.hdfs.TextIterator;
+import org.apache.tinkerpop.gremlin.structure.io.Storage;
+import org.apache.tinkerpop.gremlin.structure.util.StringFactory;
+import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public final class FileSystemStorage implements Storage {
+
+    private static final String SPACE = " ";
+    private static final String D_SPACE = "(D) ";
+
+    private final FileSystem fs;
+
+    public FileSystemStorage(final FileSystem fileSystem) {
+        this.fs = fileSystem;
+    }
+
+    private static String fileStatusString(final FileStatus status) {
+        StringBuilder s = new StringBuilder();
+        s.append(status.getPermission()).append(" ");
+        s.append(status.getOwner()).append(SPACE);
+        s.append(status.getGroup()).append(SPACE);
+        s.append(status.getLen()).append(SPACE);
+        if (status.isDir())
+            s.append(D_SPACE);
+        s.append(status.getPath().getName());
+        return s.toString();
+    }
+
+    @Override
+    public List<String> ls() {
+        return this.ls("/");
+    }
+
+    @Override
+    public List<String> ls(final String location) {
+        try {
+            final String newLocation;
+            newLocation = location.equals("/") ? this.fs.getHomeDirectory().toString() : location;
+            return Stream.of(this.fs.globStatus(new Path(newLocation + "/*"))).map(FileSystemStorage::fileStatusString).collect(Collectors.toList());
+        } catch (final IOException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public boolean mkdir(final String location) {
+        try {
+            return this.fs.mkdirs(new Path(location));
+        } catch (final IOException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public boolean cp(final String fromLocation, final String toLocation) {
+        try {
+            return FileUtil.copy(this.fs, new Path(fromLocation), this.fs, new Path(toLocation), false, new Configuration());
+        } catch (final IOException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public boolean exists(final String location) {
+        try {
+            return this.fs.exists(new Path(location));
+        } catch (final IOException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public boolean rm(final String location) {
+        try {
+            return HDFSTools.globDelete(this.fs, location, false);
+        } catch (final IOException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public boolean rmr(final String location) {
+        try {
+            return HDFSTools.globDelete(this.fs, location, true);
+        } catch (final IOException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public <V> Iterator<V> head(final String location, final int totalLines, final Class<V> objectClass) {
+        return headMaker(this.fs, location, totalLines, (Class<? extends Writable>) objectClass);
+    }
+
+    @Override
+    public String toString() {
+        return StringFactory.storageString(this.fs.toString());
+    }
+
+    private static Iterator headMaker(final FileSystem fs, final String path, final int totalLines, final Class<? extends Writable> writableClass) {
+        try {
+            if (writableClass.equals(ObjectWritable.class))
+                return IteratorUtils.limit(new ObjectWritableIterator(fs.getConf(), new Path(path)), totalLines);
+            else if (writableClass.equals(VertexWritable.class))
+                return IteratorUtils.limit(new VertexWritableIterator(fs.getConf(), new Path(path)), totalLines);
+            else
+                return IteratorUtils.limit(new TextIterator(fs.getConf(), new Path(path)), totalLines);
+        } catch (final IOException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+
+    /////////
+
+    public void copyToLocal(final String fromLocation, final String toLocation) {
+        try {
+            this.fs.copyToLocalFile(new Path(fromLocation), new Path(toLocation));
+        } catch (final IOException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+
+    public void copyFromLocal(final String fromLocation, final String toLocation) {
+        try {
+            this.fs.copyFromLocalFile(new Path(fromLocation), new Path(toLocation));
+        } catch (final IOException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+
+    public void mergeToLocal(final String fromLocation, final String toLocation) {
+        try {
+            final FileSystem local = FileSystem.getLocal(new Configuration());
+            final FSDataOutputStream outA = local.create(new Path(toLocation));
+            for (final Path path : HDFSTools.getAllFilePaths(fs, new Path(fromLocation), HiddenFileFilter.instance())) {
+                final FSDataInputStream inA = fs.open(path);
+                IOUtils.copyBytes(inA, outA, 8192);
+                inA.close();
+            }
+            outA.close();
+        } catch (final IOException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/58d92407/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/GraphMemoryHDFSCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/GraphMemoryHDFSCheck.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/GraphMemoryHDFSCheck.java
new file mode 100644
index 0000000..d47ce43
--- /dev/null
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/GraphMemoryHDFSCheck.java
@@ -0,0 +1,64 @@
+/*
+ * 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.tinkerpop.gremlin.hadoop.groovy.plugin;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.tinkerpop.gremlin.AbstractGremlinTest;
+import org.apache.tinkerpop.gremlin.LoadGraphWith;
+import org.apache.tinkerpop.gremlin.hadoop.Constants;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.FileSystemStorage;
+import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
+import org.apache.tinkerpop.gremlin.process.computer.ComputerResult;
+import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.ClusterCountMapReduce;
+import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.PeerPressureVertexProgram;
+import org.apache.tinkerpop.gremlin.structure.io.Storage;
+import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public class GraphMemoryHDFSCheck extends AbstractGremlinTest {
+
+    @Test
+    @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+    public void shouldPersistGraphAndMemory() throws Exception {
+        final ComputerResult result = graph.compute(graphComputerClass.get()).program(PeerPressureVertexProgram.build().create(graph)).mapReduce(ClusterCountMapReduce.build().memoryKey("clusterCount").create()).submit().get();
+        /////
+        final Storage storage = new FileSystemStorage(FileSystem.get(ConfUtil.makeHadoopConfiguration(graph.configuration())));
+        // TEST GRAPH PERSISTENCE
+        assertTrue(storage.exists(Constants.getGraphLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION))));
+        assertEquals(6, result.graph().traversal().V().count().next().longValue());
+        assertEquals(0, result.graph().traversal().E().count().next().longValue());
+        assertEquals(6, result.graph().traversal().V().values("name").count().next().longValue());
+        assertEquals(6, result.graph().traversal().V().values(PeerPressureVertexProgram.CLUSTER).count().next().longValue());
+        assertEquals(2, result.graph().traversal().V().values(PeerPressureVertexProgram.CLUSTER).dedup().count().next().longValue());
+        /////
+        // TEST MEMORY PERSISTENCE
+        assertEquals(2, (int) result.memory().get("clusterCount"));
+        assertTrue(storage.exists(Constants.getMemoryLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount")));
+ //       System.out.println(IteratorUtils.list(storage.head(Constants.getMemoryLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount"))));
+//        assertEquals(1, IteratorUtils.count(storage.head(Constants.getMemoryLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount"))));
+        assertEquals(2, storage.head(Constants.getMemoryLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount")).next());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/58d92407/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java
index 6c4cc20..7dc8143 100644
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java
@@ -29,6 +29,6 @@ import org.junit.runners.model.RunnerBuilder;
  */
 public class HadoopPluginSuite extends AbstractGremlinSuite {
     public HadoopPluginSuite(final Class<?> klass, final RunnerBuilder builder) throws InitializationError {
-        super(klass, builder, new Class<?>[]{HadoopGremlinPluginCheck.class}, new Class<?>[]{HadoopGremlinPluginCheck.class}, true, TraversalEngine.Type.COMPUTER);
+        super(klass, builder, new Class<?>[]{HadoopGremlinPluginCheck.class, GraphMemoryHDFSCheck.class}, new Class<?>[]{HadoopGremlinPluginCheck.class, GraphMemoryHDFSCheck.class}, true, TraversalEngine.Type.COMPUTER);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/58d92407/spark-gremlin/src/main/groovy/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkLoader.groovy
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/groovy/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkLoader.groovy b/spark-gremlin/src/main/groovy/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkLoader.groovy
deleted file mode 100644
index 53d385f..0000000
--- a/spark-gremlin/src/main/groovy/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkLoader.groovy
+++ /dev/null
@@ -1,68 +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.tinkerpop.gremlin.spark.groovy.plugin
-
-import org.apache.spark.rdd.RDD
-import org.apache.tinkerpop.gremlin.spark.structure.Spark
-import scala.collection.JavaConversions
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-class SparkLoader {
-
-    public static void load() {
-
-        Spark.metaClass.static.ls = {
-            final List<String> rdds = new ArrayList<>();
-            for (final RDD<?> rdd : Spark.getRDDs()) {
-                rdds.add(rdd.name() + " [" + rdd.getStorageLevel().description() + "]")
-            }
-            return rdds;
-        }
-
-        Spark.metaClass.static.rm = { final String rddName ->
-            for (final RDD<?> rdd : Spark.getRDDs()) {
-                if (rdd.name().matches(rddName.replace(".", "\\.").replace("*", ".*")))
-                    Spark.removeRDD(rdd.name());
-            }
-        }
-
-        Spark.metaClass.static.head = { final String rddName ->
-            return Spark.head(rddName, Integer.MAX_VALUE);
-        }
-
-        Spark.metaClass.static.head = { final String rddName, final int totalLines ->
-            final List<Object> data = new ArrayList<>();
-            final Iterator<?> itty = JavaConversions.asJavaIterator(Spark.getRDD(rddName).toLocalIterator());
-            for (int i = 0; i < totalLines; i++) {
-                if (itty.hasNext())
-                    data.add(itty.next());
-                else
-                    break;
-            }
-            return data;
-        }
-
-        Spark.metaClass.static.describe = { final String rddName ->
-            return Spark.getRDD(rddName).toDebugString();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/58d92407/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java
index 9351a1e..7711435 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java
@@ -27,6 +27,7 @@ import org.apache.tinkerpop.gremlin.groovy.plugin.PluginInitializationException;
 import org.apache.tinkerpop.gremlin.groovy.plugin.RemoteAcceptor;
 import org.apache.tinkerpop.gremlin.spark.process.computer.SparkGraphComputer;
 import org.apache.tinkerpop.gremlin.spark.structure.Spark;
+import org.apache.tinkerpop.gremlin.spark.structure.io.SparkContextStorage;
 
 import java.util.HashSet;
 import java.util.Optional;
@@ -43,6 +44,7 @@ public final class SparkGremlinPlugin extends AbstractGremlinPlugin {
         add("import org.apache.log4j.*");
         add(IMPORT_SPACE + SparkGraphComputer.class.getPackage().getName() + DOT_STAR);
         add(IMPORT_SPACE + Spark.class.getPackage().getName() + DOT_STAR);
+        add(IMPORT_SPACE + SparkContextStorage.class.getPackage().getName() + DOT_STAR);
     }};
 
     @Override
@@ -56,8 +58,7 @@ public final class SparkGremlinPlugin extends AbstractGremlinPlugin {
         try {
             pluginAcceptor.eval(String.format("Logger.getLogger(%s).setLevel(Level.INFO)", SparkGraphComputer.class.getName()));
             pluginAcceptor.eval(String.format("Logger.getLogger(%s).setLevel(Level.ERROR)", MetricsSystem.class.getName()));
-            pluginAcceptor.eval("spark = Spark");
-            pluginAcceptor.eval(SparkLoader.class.getCanonicalName() + ".load()");
+            pluginAcceptor.eval("spark = SparkContextStorage.open()");
         } catch (final Exception e) {
             throw new PluginInitializationException(e.getMessage(), e);
         }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/58d92407/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputRDD.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputRDD.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputRDD.java
index 52d18f1..55bf53b 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputRDD.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputRDD.java
@@ -36,8 +36,9 @@ public final class PersistedInputRDD implements InputRDD {
     public JavaPairRDD<Object, VertexWritable> readGraphRDD(final Configuration configuration, final JavaSparkContext sparkContext) {
         if (!configuration.containsKey(Constants.GREMLIN_HADOOP_INPUT_LOCATION))
             throw new IllegalArgumentException("There is no provided " + Constants.GREMLIN_HADOOP_INPUT_LOCATION + " to read the persisted RDD from");
-        final String graphRDDName = Constants.getGraphLocation(configuration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION));
         Spark.create(sparkContext.sc());
+        final String inputLocation = configuration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION);
+        final String graphRDDName = Spark.hasRDD(inputLocation) ? inputLocation : Constants.getGraphLocation(inputLocation);
         return JavaPairRDD.fromJavaRDD((JavaRDD) Spark.getRDD(graphRDDName).toJavaRDD());
     }
 
@@ -45,8 +46,9 @@ public final class PersistedInputRDD implements InputRDD {
     public <K, V> JavaPairRDD<K, V> readMemoryRDD(final Configuration configuration, final String memoryKey, final JavaSparkContext sparkContext) {
         if (!configuration.containsKey(Constants.GREMLIN_HADOOP_INPUT_LOCATION))
             throw new IllegalArgumentException("There is no provided " + Constants.GREMLIN_HADOOP_INPUT_LOCATION + " to read the persisted RDD from");
-        final String sideEffectRDDName = Constants.getMemoryLocation(configuration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION), memoryKey);
+        final String inputLocation = configuration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION);
+        final String memoryRDDName = Spark.hasRDD(inputLocation) ? inputLocation : Constants.getMemoryLocation(inputLocation, memoryKey);
         Spark.create(sparkContext.sc());
-        return JavaPairRDD.fromJavaRDD((JavaRDD) Spark.getRDD(sideEffectRDDName).toJavaRDD());
+        return JavaPairRDD.fromJavaRDD((JavaRDD) Spark.getRDD(memoryRDDName).toJavaRDD());
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/58d92407/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
new file mode 100644
index 0000000..2db267f
--- /dev/null
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
@@ -0,0 +1,124 @@
+/*
+ * 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.tinkerpop.gremlin.spark.structure.io;
+
+import org.apache.commons.configuration.Configuration;
+import org.apache.spark.SparkContext;
+import org.apache.spark.rdd.RDD;
+import org.apache.tinkerpop.gremlin.spark.structure.Spark;
+import org.apache.tinkerpop.gremlin.structure.io.Storage;
+import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
+import scala.collection.JavaConversions;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public final class SparkContextStorage implements Storage {
+
+    private SparkContextStorage() {
+
+    }
+
+    public static SparkContextStorage open() {
+        return new SparkContextStorage();
+    }
+
+    public static SparkContextStorage open(final String master) {
+        Spark.create(master);
+        return new SparkContextStorage();
+    }
+
+    public static SparkContextStorage open(final Configuration configuration) {
+        Spark.create(configuration);
+        return new SparkContextStorage();
+    }
+
+    public static SparkContextStorage open(final SparkContext sparkContext) {
+        Spark.create(sparkContext);
+        return new SparkContextStorage();
+    }
+
+
+    @Override
+    public List<String> ls() {
+        return ls("*");
+    }
+
+    @Override
+    public List<String> ls(final String location) {
+        final List<String> rdds = new ArrayList<>();
+        final String wildCardLocation = location.replace(".", "\\.").replace("*", ".*");
+        for (final RDD<?> rdd : Spark.getRDDs()) {
+            if (rdd.name().matches(wildCardLocation))
+                rdds.add(rdd.name() + " [" + rdd.getStorageLevel().description() + "]");
+        }
+        return rdds;
+    }
+
+    @Override
+    public boolean mkdir(final String location) {
+        throw new UnsupportedOperationException("This operation does not make sense for a persited SparkContext");
+    }
+
+    @Override
+    public boolean cp(final String fromLocation, final String toLocation) {
+        Spark.getRDD(fromLocation).setName(toLocation).cache();
+        Spark.removeRDD(fromLocation);
+        return true;
+    }
+
+    @Override
+    public boolean exists(final String location) {
+        return Spark.hasRDD(location);
+    }
+
+    @Override
+    public boolean rm(final String location) {
+        if (!Spark.hasRDD(location))
+            return false;
+        Spark.removeRDD(location);
+        return true;
+    }
+
+    @Override
+    public boolean rmr(final String location) {
+        final List<String> rdds = new ArrayList<>();
+        final String wildCardLocation = location.replace(".", "\\.").replace("*", ".*");
+        for (final RDD<?> rdd : Spark.getRDDs()) {
+            if (rdd.name().matches(wildCardLocation))
+                rdds.add(rdd.name());
+        }
+        rdds.forEach(Spark::removeRDD);
+        return rdds.size() > 0;
+    }
+
+    @Override
+    public <V> Iterator<V> head(final String location, final int totalLines, final Class<V> objectClass) {
+        return IteratorUtils.limit((Iterator) JavaConversions.asJavaIterator(Spark.getRDD(location).toLocalIterator()), totalLines);
+    }
+
+    public String describe(final String location) {
+        return Spark.getRDD(location).toDebugString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/58d92407/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/AbstractSparkTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/AbstractSparkTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/AbstractSparkTest.java
index ccff1ab..3fc2a59 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/AbstractSparkTest.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/AbstractSparkTest.java
@@ -19,10 +19,21 @@
 
 package org.apache.tinkerpop.gremlin.spark;
 
+import org.apache.commons.configuration.BaseConfiguration;
+import org.apache.commons.configuration.Configuration;
 import org.apache.spark.SparkConf;
 import org.apache.spark.SparkContext;
 import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.tinkerpop.gremlin.hadoop.Constants;
+import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.graphson.GraphSONInputFormat;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.gryo.GryoInputFormat;
+import org.apache.tinkerpop.gremlin.spark.process.computer.SparkHadoopGraphProvider;
 import org.apache.tinkerpop.gremlin.spark.structure.Spark;
+import org.apache.tinkerpop.gremlin.spark.structure.io.PersistedInputRDD;
+import org.apache.tinkerpop.gremlin.spark.structure.io.PersistedOutputRDD;
+import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.GryoSerializer;
+import org.apache.tinkerpop.gremlin.structure.Graph;
 import org.junit.After;
 import org.junit.Before;
 
@@ -43,4 +54,23 @@ public abstract class AbstractSparkTest {
         Spark.close();
         System.out.println("SparkContext has been closed for " + this.getClass().getCanonicalName() + "-setupTest");
     }
+
+    protected Configuration getBaseConfiguration(final String inputLocation) {
+        final BaseConfiguration configuration = new BaseConfiguration();
+        configuration.setDelimiterParsingDisabled(true);
+        configuration.setProperty("spark.master", "local[4]");
+        configuration.setProperty("spark.serializer", GryoSerializer.class.getCanonicalName());
+        configuration.setProperty(Graph.GRAPH, HadoopGraph.class.getName());
+        configuration.setProperty(Constants.GREMLIN_HADOOP_JARS_IN_DISTRIBUTED_CACHE, false);
+        configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, SparkHadoopGraphProvider.PATHS.get("tinkerpop-modern.kryo"));
+        if (inputLocation.contains(".kryo"))
+            configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, GryoInputFormat.class.getCanonicalName());
+        else if (inputLocation.contains(".json"))
+            configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, GraphSONInputFormat.class.getCanonicalName());
+        else
+            configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_INPUT_RDD, PersistedInputRDD.class.getCanonicalName());
+
+        configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
+        return configuration;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/58d92407/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java
index 0b60825..7574908 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java
@@ -34,9 +34,11 @@ import org.apache.tinkerpop.gremlin.spark.structure.io.PersistedOutputRDD;
 import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.GryoSerializer;
 import org.apache.tinkerpop.gremlin.structure.Graph;
 import org.apache.tinkerpop.gremlin.structure.util.GraphFactory;
+import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.util.Iterator;
 import java.util.List;
 
 import static org.junit.Assert.assertEquals;
@@ -97,9 +99,9 @@ public class SparkGremlinPluginTest extends AbstractSparkTest {
         assertEquals(1, ((List<String>) this.console.eval("spark.ls()")).size());
         assertTrue(((List<String>) this.console.eval("spark.ls()")).contains("target/test-output/graph-1/~g [Memory Deserialized 1x Replicated]"));
 
-        assertEquals(6, ((List<Object>) this.console.eval("spark.head('target/test-output/graph-1/~g')")).size());
+        assertEquals(6, IteratorUtils.count(((Iterator<Object>) this.console.eval("spark.head('target/test-output/graph-1/~g')"))));
 
-        this.console.eval("spark.rm('target/test-output/graph-*')");
+        this.console.eval("spark.rmr('target/test-output/graph-*')");
         assertEquals(0, ((List<String>) this.console.eval("spark.ls()")).size());
 
         //////
@@ -116,9 +118,9 @@ public class SparkGremlinPluginTest extends AbstractSparkTest {
         this.console.eval("graph.compute(SparkGraphComputer).program(PageRankVertexProgram.build().iterations(1).create()).submit().get()");
 
         assertEquals(3, ((List<String>) this.console.eval("spark.ls()")).size());
-        this.console.eval("spark.rm('target/test-output/graph-*')");
+        this.console.eval("spark.rmr('target/test-output/graph-*')");
         assertEquals(1, ((List<String>) this.console.eval("spark.ls()")).size());
-        this.console.eval("spark.rm('*')");
+        this.console.eval("spark.rmr('*')");
         assertEquals(0, ((List<String>) this.console.eval("spark.ls()")).size());
 
         //

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/58d92407/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/GraphMemorySparkTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/GraphMemorySparkTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/GraphMemorySparkTest.java
new file mode 100644
index 0000000..10153b0
--- /dev/null
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/GraphMemorySparkTest.java
@@ -0,0 +1,75 @@
+/*
+ * 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.tinkerpop.gremlin.spark.structure.io;
+
+import org.apache.commons.configuration.Configuration;
+import org.apache.tinkerpop.gremlin.hadoop.Constants;
+import org.apache.tinkerpop.gremlin.process.computer.ComputerResult;
+import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.ClusterCountMapReduce;
+import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.PeerPressureVertexProgram;
+import org.apache.tinkerpop.gremlin.spark.AbstractSparkTest;
+import org.apache.tinkerpop.gremlin.spark.process.computer.SparkGraphComputer;
+import org.apache.tinkerpop.gremlin.spark.process.computer.SparkHadoopGraphProvider;
+import org.apache.tinkerpop.gremlin.structure.Graph;
+import org.apache.tinkerpop.gremlin.structure.io.Storage;
+import org.apache.tinkerpop.gremlin.structure.util.GraphFactory;
+import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
+import org.junit.Test;
+import scala.Tuple2;
+
+import java.util.UUID;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public class GraphMemorySparkTest extends AbstractSparkTest {
+
+    @Test
+    public void shouldPersistGraphAndMemory() throws Exception {
+        final String outputLocation = "target/test-output/" + UUID.randomUUID();
+        final Configuration configuration = getBaseConfiguration(SparkHadoopGraphProvider.PATHS.get("tinkerpop-modern.kryo"));
+        configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
+        configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, outputLocation);
+        configuration.setProperty(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, true);
+        /////
+        Graph graph = GraphFactory.open(configuration);
+        final ComputerResult result = graph.compute(SparkGraphComputer.class).program(PeerPressureVertexProgram.build().create(graph)).mapReduce(ClusterCountMapReduce.build().memoryKey("clusterCount").create()).submit().get();
+        /////
+        final Storage storage = SparkContextStorage.open("local[4]");
+
+        assertEquals(2, storage.ls().size());
+        // TEST GRAPH PERSISTENCE
+        assertTrue(storage.exists(Constants.getGraphLocation(outputLocation)));
+        assertEquals(6, IteratorUtils.count(storage.head(Constants.getGraphLocation(outputLocation), Tuple2.class)));
+        assertEquals(6, result.graph().traversal().V().count().next().longValue());
+        assertEquals(0, result.graph().traversal().E().count().next().longValue());
+        assertEquals(6, result.graph().traversal().V().values("name").count().next().longValue());
+        assertEquals(6, result.graph().traversal().V().values(PeerPressureVertexProgram.CLUSTER).count().next().longValue());
+        /////
+        // TEST MEMORY PERSISTENCE
+        assertEquals(2, (int) result.memory().get("clusterCount"));
+        assertTrue(storage.exists(Constants.getMemoryLocation(outputLocation, "clusterCount")));
+        assertEquals(2, storage.head(Constants.getMemoryLocation(outputLocation, "clusterCount"), Tuple2.class).next()._2());
+    }
+
+}


[22/30] incubator-tinkerpop git commit: Merge remote-tracking branch 'origin/master' into TINKERPOP-320

Posted by dk...@apache.org.
Merge remote-tracking branch 'origin/master' into TINKERPOP-320


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/43eaf8dd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/43eaf8dd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/43eaf8dd

Branch: refs/heads/TINKERPOP-320
Commit: 43eaf8dda431f5deedf0f2725d9c87f0bf58aa47
Parents: bdfd490 114609d
Author: Daniel Kuppitz <da...@hotmail.com>
Authored: Fri Jan 8 20:19:39 2016 +0100
Committer: Daniel Kuppitz <da...@hotmail.com>
Committed: Fri Jan 8 20:19:39 2016 +0100

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |   7 +
 docs/src/reference/implementations.asciidoc     | 104 ++++----
 .../upgrade/release-3.1.x-incubating.asciidoc   |  26 ++
 .../process/computer/GiraphGraphComputer.java   |  43 ++--
 .../GiraphHadoopGremlinIntegrateTest.java       |  33 +++
 .../computer/GiraphHadoopGraphProvider.java     |   6 +-
 .../GiraphHadoopGremlinPluginIntegrateTest.java |  33 ---
 .../peerpressure/ClusterCountMapReduce.java     |   7 +-
 .../tinkerpop/gremlin/structure/io/Storage.java | 139 +++++++++++
 .../gremlin/structure/util/StringFactory.java   |  18 +-
 .../tinkerpop/gremlin/driver/Cluster.java       |   3 +
 .../tinkerpop/gremlin/driver/Settings.java      |   2 +
 .../process/computer/GraphComputerTest.java     |  54 ++--
 .../conf/hadoop-grateful-gryo.properties        |   6 +-
 .../hadoop/groovy/plugin/HadoopLoader.groovy    | 138 -----------
 .../tinkerpop/gremlin/hadoop/Constants.java     |  20 ++
 .../groovy/plugin/HadoopGremlinPlugin.java      |  11 +-
 .../process/computer/util/MapReduceHelper.java  |   8 +-
 .../hadoop/structure/HadoopConfiguration.java   |   5 +
 .../gremlin/hadoop/structure/HadoopGraph.java   |   4 +-
 .../hadoop/structure/hdfs/HDFSTools.java        | 113 ---------
 .../structure/hdfs/HadoopEdgeIterator.java      |  83 -------
 .../structure/hdfs/HadoopElementIterator.java   |  74 ------
 .../structure/hdfs/HadoopVertexIterator.java    |  82 -------
 .../hadoop/structure/hdfs/HiddenFileFilter.java |  44 ----
 .../hadoop/structure/hdfs/TextIterator.java     |  91 -------
 .../hadoop/structure/io/FileSystemStorage.java  | 246 +++++++++++++++++++
 .../hadoop/structure/io/HadoopEdgeIterator.java |  79 ++++++
 .../structure/io/HadoopElementIterator.java     |  75 ++++++
 .../structure/io/HadoopVertexIterator.java      |  78 ++++++
 .../hadoop/structure/io/HiddenFileFilter.java   |  44 ++++
 .../hadoop/structure/io/InputOutputHelper.java  |   2 +-
 .../structure/io/ObjectWritableIterator.java    |  12 +-
 .../hadoop/structure/io/TextIterator.java       |  91 +++++++
 .../structure/io/VertexWritableIterator.java    |  10 +-
 .../gremlin/hadoop/HadoopGremlinSuite.java      |  36 +++
 .../groovy/plugin/HadoopGremlinPluginCheck.java |  71 +-----
 .../hadoop/groovy/plugin/HadoopPluginSuite.java |  34 ---
 .../structure/io/AbstractStorageCheck.java      | 145 +++++++++++
 .../structure/io/FileSystemStorageCheck.java    |  88 +++++++
 pom.xml                                         |   2 +-
 spark-gremlin/pom.xml                           |   2 +-
 .../spark/groovy/plugin/SparkLoader.groovy      |  68 -----
 .../spark/groovy/plugin/SparkGremlinPlugin.java |   5 +-
 .../spark/process/computer/SparkExecutor.java   |  28 ---
 .../process/computer/SparkGraphComputer.java    |  18 +-
 .../gremlin/spark/structure/Spark.java          |   2 +
 .../spark/structure/io/InputFormatRDD.java      |  15 ++
 .../spark/structure/io/InputOutputHelper.java   |   3 -
 .../gremlin/spark/structure/io/InputRDD.java    |  17 +-
 .../spark/structure/io/OutputFormatRDD.java     |  30 ++-
 .../gremlin/spark/structure/io/OutputRDD.java   |  21 ++
 .../spark/structure/io/PersistedInputRDD.java   |  14 +-
 .../spark/structure/io/PersistedOutputRDD.java  |  25 +-
 .../spark/structure/io/SparkContextStorage.java | 164 +++++++++++++
 .../gremlin/spark/AbstractSparkTest.java        |  30 +++
 .../gremlin/spark/SparkGremlinSuite.java        |  35 +++
 .../gremlin/spark/SparkGremlinTest.java         |  33 +++
 .../gremlin/spark/SparkHadoopGremlinTest.java   |  33 +++
 .../process/computer/LocalPropertyTest.java     |   2 +-
 .../computer/SparkHadoopGraphProvider.java      |  12 +-
 .../groovy/plugin/SparkGremlinPluginTest.java   | 126 ----------
 .../plugin/SparkHadoopGremlinPluginTest.java    |  33 ---
 .../gremlin/spark/structure/SparkTest.java      |  10 +-
 .../spark/structure/io/ExampleInputRDD.java     |   5 +
 .../spark/structure/io/ExampleOutputRDD.java    |   6 +
 .../io/PersistedInputOutputRDDTest.java         |  29 +--
 .../structure/io/SparkContextStorageCheck.java  |  74 ++++++
 .../spark/structure/io/ToyGraphInputRDD.java    |   7 +
 .../tinkergraph/structure/TinkerIoRegistry.java |   1 +
 .../tinkergraph/structure/TinkerGraphTest.java  | 129 +++++++++-
 71 files changed, 1939 insertions(+), 1205 deletions(-)
----------------------------------------------------------------------



[06/30] incubator-tinkerpop git commit: lots more clean up, tests, and organization. She is a real beauty.

Posted by dk...@apache.org.
lots more clean up, tests, and organization. She is a real beauty.


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/3fff8f54
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/3fff8f54
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/3fff8f54

Branch: refs/heads/TINKERPOP-320
Commit: 3fff8f546501d10a4c1d34762a626a2493e758be
Parents: b4d8e96
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Dec 9 16:57:28 2015 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Wed Dec 9 16:57:28 2015 -0700

----------------------------------------------------------------------
 .../GiraphHadoopGremlinPluginIntegrateTest.java | 33 --------------------
 .../GiraphHadoopGremlinPluginIntegrateTest.java | 33 ++++++++++++++++++++
 .../tinkerpop/gremlin/structure/io/Storage.java | 12 +++----
 .../hadoop/structure/io/FileSystemStorage.java  |  4 +--
 .../groovy/plugin/FileSystemStorageCheck.java   |  4 +--
 .../groovy/plugin/HadoopGremlinPluginCheck.java |  4 +--
 .../process/computer/SparkGraphComputer.java    |  7 ++---
 .../spark/structure/io/InputFormatRDD.java      |  6 ++--
 .../spark/structure/io/OutputFormatRDD.java     |  6 ++--
 .../spark/structure/io/PersistedInputRDD.java   |  9 ++----
 .../spark/structure/io/PersistedOutputRDD.java  | 16 +++++-----
 .../spark/structure/io/SparkContextStorage.java | 18 +++++------
 .../structure/io/SparkContextStorageTest.java   |  4 +--
 13 files changed, 75 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3fff8f54/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/process/computer/groovy/GiraphHadoopGremlinPluginIntegrateTest.java
----------------------------------------------------------------------
diff --git a/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/process/computer/groovy/GiraphHadoopGremlinPluginIntegrateTest.java b/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/process/computer/groovy/GiraphHadoopGremlinPluginIntegrateTest.java
deleted file mode 100644
index 947b776..0000000
--- a/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/process/computer/groovy/GiraphHadoopGremlinPluginIntegrateTest.java
+++ /dev/null
@@ -1,33 +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.tinkerpop.gremlin.giraph.process.computer.groovy;
-
-import org.apache.tinkerpop.gremlin.GraphProviderClass;
-import org.apache.tinkerpop.gremlin.giraph.process.computer.GiraphHadoopGraphProvider;
-import org.apache.tinkerpop.gremlin.hadoop.groovy.plugin.HadoopPluginSuite;
-import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
-import org.junit.runner.RunWith;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-@RunWith(HadoopPluginSuite.class)
-@GraphProviderClass(provider = GiraphHadoopGraphProvider.class, graph = HadoopGraph.class)
-public class GiraphHadoopGremlinPluginIntegrateTest {
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3fff8f54/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/process/computer/groovy/plugin/GiraphHadoopGremlinPluginIntegrateTest.java
----------------------------------------------------------------------
diff --git a/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/process/computer/groovy/plugin/GiraphHadoopGremlinPluginIntegrateTest.java b/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/process/computer/groovy/plugin/GiraphHadoopGremlinPluginIntegrateTest.java
new file mode 100644
index 0000000..4660c42
--- /dev/null
+++ b/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/process/computer/groovy/plugin/GiraphHadoopGremlinPluginIntegrateTest.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tinkerpop.gremlin.giraph.process.computer.groovy.plugin;
+
+import org.apache.tinkerpop.gremlin.GraphProviderClass;
+import org.apache.tinkerpop.gremlin.giraph.process.computer.GiraphHadoopGraphProvider;
+import org.apache.tinkerpop.gremlin.hadoop.groovy.plugin.HadoopPluginSuite;
+import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
+import org.junit.runner.RunWith;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+@RunWith(HadoopPluginSuite.class)
+@GraphProviderClass(provider = GiraphHadoopGraphProvider.class, graph = HadoopGraph.class)
+public class GiraphHadoopGremlinPluginIntegrateTest {
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3fff8f54/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
index 3b69ff2..b6e6ebe 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
@@ -50,15 +50,15 @@ public interface Storage {
         return this.head(location, Integer.MAX_VALUE);
     }
 
-    public Iterator<Vertex> headGraph(final String location, final int totalLines, final Class parserClass);
+    public Iterator<Vertex> head(final String location, final Class parserClass, final int totalLines);
 
-    public default Iterator<Vertex> headGraph(final String location, final Class parserClass) {
-        return this.headGraph(location, Integer.MAX_VALUE, parserClass);
+    public default Iterator<Vertex> head(final String location, final Class parserClass) {
+        return this.head(location, parserClass, Integer.MAX_VALUE);
     }
 
-    public <K, V> Iterator<KeyValue<K, V>> headMemory(final String location, final String memoryKey, final int totalLines, final Class parserClass);
+    public <K, V> Iterator<KeyValue<K, V>> head(final String location, final String memoryKey, final Class parserClass, final int totalLines);
 
-    public default <K, V> Iterator<KeyValue<K, V>> headMemory(final String location, final String memoryKey, final Class parserClass) {
-        return this.headMemory(location, memoryKey, Integer.MAX_VALUE, parserClass);
+    public default <K, V> Iterator<KeyValue<K, V>> head(final String location, final String memoryKey, final Class parserClass) {
+        return this.head(location, memoryKey, parserClass, Integer.MAX_VALUE);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3fff8f54/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
index 5d3995c..b34f7a3 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
@@ -161,7 +161,7 @@ public final class FileSystemStorage implements Storage {
     }
 
     @Override
-    public Iterator<Vertex> headGraph(final String location, final int totalLines, final Class parserClass) {
+    public Iterator<Vertex> head(final String location, final Class parserClass, final int totalLines) {
         final org.apache.commons.configuration.Configuration configuration = new BaseConfiguration();
         configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, Constants.getSearchGraphLocation(location, this).get());
         configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, parserClass.getCanonicalName());
@@ -176,7 +176,7 @@ public final class FileSystemStorage implements Storage {
     }
 
     @Override
-    public <K, V> Iterator<KeyValue<K, V>> headMemory(final String location, final String memoryKey, final int totalLines, final Class parserClass) {
+    public <K, V> Iterator<KeyValue<K, V>> head(final String location, final String memoryKey, final Class parserClass, final int totalLines) {
         if (!parserClass.equals(SequenceFileInputFormat.class))
             throw new IllegalArgumentException("Only " + SequenceFileInputFormat.class.getCanonicalName() + " memories are supported");
         final Configuration configuration = new Configuration();

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3fff8f54/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java
index a8c5307..b0517ad 100644
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java
@@ -57,7 +57,7 @@ public class FileSystemStorageCheck extends AbstractGremlinTest {
         // TEST MEMORY PERSISTENCE
         assertEquals(2, (int) result.memory().get("clusterCount"));
         assertTrue(storage.exists(Constants.getMemoryLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount")));
-        assertEquals(1, IteratorUtils.count(storage.headMemory(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount", SequenceFileInputFormat.class)));
-        assertEquals(2, storage.headMemory(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount", SequenceFileInputFormat.class).next().getValue());
+        assertEquals(1, IteratorUtils.count(storage.head(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount", SequenceFileInputFormat.class)));
+        assertEquals(2, storage.head(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount", SequenceFileInputFormat.class).next().getValue());
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3fff8f54/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
index b558169..4e4353d 100644
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
@@ -137,7 +137,7 @@ public class HadoopGremlinPluginCheck extends AbstractGremlinTest {
         AbstractGremlinProcessTest.checkResults(Arrays.asList("ripple", "lop"), traversal);
         assertTrue((Boolean) this.console.eval("hdfs.exists('target/test-output/m')"));
         assertTrue((Boolean) this.console.eval("hdfs.exists('target/test-output/" + TraverserMapReduce.TRAVERSERS + "')"));
-        final List<KeyValue<Integer, Collection<String>>> mList = IteratorUtils.asList(this.console.eval("hdfs.headMemory('target/test-output','m',SequenceFileInputFormat)"));
+        final List<KeyValue<Integer, Collection<String>>> mList = IteratorUtils.asList(this.console.eval("hdfs.head('target/test-output','m',SequenceFileInputFormat)"));
         assertEquals(4, mList.size());
         mList.forEach(keyValue -> {
             if (keyValue.getKey().equals(29))
@@ -151,7 +151,7 @@ public class HadoopGremlinPluginCheck extends AbstractGremlinTest {
             else
                 throw new IllegalStateException("The provided key/value is unknown: " + keyValue);
         });
-        final List<KeyValue<MapReduce.NullObject, Traverser<String>>> traversersList = IteratorUtils.asList(this.console.eval("hdfs.headMemory('target/test-output/'," + "'" + TraverserMapReduce.TRAVERSERS + "',SequenceFileInputFormat)"));
+        final List<KeyValue<MapReduce.NullObject, Traverser<String>>> traversersList = IteratorUtils.asList(this.console.eval("hdfs.head('target/test-output/'," + "'" + TraverserMapReduce.TRAVERSERS + "',SequenceFileInputFormat)"));
         assertEquals(2, traversersList.size());
         traversersList.forEach(keyValue -> {
             assertEquals(MapReduce.NullObject.instance(), keyValue.getKey());

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3fff8f54/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
index f96fd15..3f7efaa 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
@@ -53,6 +53,7 @@ import org.apache.tinkerpop.gremlin.spark.structure.io.InputRDD;
 import org.apache.tinkerpop.gremlin.spark.structure.io.OutputFormatRDD;
 import org.apache.tinkerpop.gremlin.spark.structure.io.OutputRDD;
 import org.apache.tinkerpop.gremlin.spark.structure.io.PersistedOutputRDD;
+import org.apache.tinkerpop.gremlin.spark.structure.io.SparkContextStorage;
 
 import java.io.File;
 import java.io.IOException;
@@ -239,10 +240,8 @@ public final class SparkGraphComputer extends AbstractHadoopGraphComputer {
                 // unpersist the graphRDD if it will no longer be used
                 if (!PersistedOutputRDD.class.equals(hadoopConfiguration.getClass(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, null)) || this.persist.equals(GraphComputer.Persist.NOTHING)) {
                     graphRDD.unpersist();
-                    if (apacheConfiguration.containsKey(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION)) {
-                        Spark.removeRDD(apacheConfiguration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));
-                        Spark.removeRDD(Constants.getGraphLocation(apacheConfiguration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION)));
-                    }
+                    if (apacheConfiguration.containsKey(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION))
+                        SparkContextStorage.open().rmr(apacheConfiguration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));
                 }
                 // update runtime and return the newly computed graph
                 finalMemory.setRuntime(System.currentTimeMillis() - startTime);

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3fff8f54/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputFormatRDD.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputFormatRDD.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputFormatRDD.java
index 12a8268..57d7080 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputFormatRDD.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/InputFormatRDD.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.tinkerpop.gremlin.hadoop.Constants;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.FileSystemStorage;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
 import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
@@ -39,6 +40,7 @@ public final class InputFormatRDD implements InputRDD {
     @Override
     public JavaPairRDD<Object, VertexWritable> readGraphRDD(final Configuration configuration, final JavaSparkContext sparkContext) {
         final org.apache.hadoop.conf.Configuration hadoopConfiguration = ConfUtil.makeHadoopConfiguration(configuration);
+        hadoopConfiguration.set(configuration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION), Constants.getSearchGraphLocation(configuration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION), FileSystemStorage.open(hadoopConfiguration)).get());
         return sparkContext.newAPIHadoopRDD(hadoopConfiguration,
                 (Class<InputFormat<NullWritable, VertexWritable>>) hadoopConfiguration.getClass(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, InputFormat.class),
                 NullWritable.class,
@@ -49,11 +51,11 @@ public final class InputFormatRDD implements InputRDD {
     @Override
     public <K, V> JavaPairRDD<K, V> readMemoryRDD(final Configuration configuration, final String memoryKey, final JavaSparkContext sparkContext) {
         final org.apache.hadoop.conf.Configuration hadoopConfiguration = ConfUtil.makeHadoopConfiguration(configuration);
-        // use FileInput location
+        hadoopConfiguration.set(configuration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION), Constants.getMemoryLocation(configuration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION), memoryKey));
         return sparkContext.newAPIHadoopRDD(hadoopConfiguration,
                 SequenceFileInputFormat.class,
                 ObjectWritable.class,
                 ObjectWritable.class)
-                .mapToPair(tuple -> new Tuple2<>((K) ((Tuple2<ObjectWritable,ObjectWritable>)tuple)._1().get(), (V) ((Tuple2<ObjectWritable,ObjectWritable>)tuple)._2().get()));
+                .mapToPair(tuple -> new Tuple2<>((K) ((Tuple2<ObjectWritable, ObjectWritable>) tuple)._1().get(), (V) ((Tuple2<ObjectWritable, ObjectWritable>) tuple)._2().get()));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3fff8f54/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/OutputFormatRDD.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/OutputFormatRDD.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/OutputFormatRDD.java
index 92158af..edb541b 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/OutputFormatRDD.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/OutputFormatRDD.java
@@ -49,7 +49,7 @@ public final class OutputFormatRDD implements OutputRDD {
         if (null != outputLocation) {
             // map back to a <nullwritable,vertexwritable> stream for output
             graphRDD.mapToPair(tuple -> new Tuple2<>(NullWritable.get(), tuple._2()))
-                    .saveAsNewAPIHadoopFile(outputLocation + "/" + Constants.HIDDEN_G,
+                    .saveAsNewAPIHadoopFile(Constants.getGraphLocation(outputLocation),
                             NullWritable.class,
                             VertexWritable.class,
                             (Class<OutputFormat<NullWritable, VertexWritable>>) hadoopConfiguration.getClass(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT, OutputFormat.class), hadoopConfiguration);
@@ -62,12 +62,12 @@ public final class OutputFormatRDD implements OutputRDD {
         final String outputLocation = hadoopConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION);
         if (null != outputLocation) {
             // map back to a Hadoop stream for output
-            memoryRDD.mapToPair(keyValue -> new Tuple2<>(new ObjectWritable<>(keyValue._1()), new ObjectWritable<>(keyValue._2()))).saveAsNewAPIHadoopFile(outputLocation + "/" + memoryKey,
+            memoryRDD.mapToPair(keyValue -> new Tuple2<>(new ObjectWritable<>(keyValue._1()), new ObjectWritable<>(keyValue._2()))).saveAsNewAPIHadoopFile(Constants.getMemoryLocation(outputLocation, memoryKey),
                     ObjectWritable.class,
                     ObjectWritable.class,
                     SequenceFileOutputFormat.class, hadoopConfiguration);
             try {
-                return (Iterator) new ObjectWritableIterator(hadoopConfiguration, new Path(outputLocation + "/" + memoryKey));
+                return (Iterator) new ObjectWritableIterator(hadoopConfiguration, new Path(Constants.getMemoryLocation(outputLocation, memoryKey)));
             } catch (final IOException e) {
                 throw new IllegalStateException(e.getMessage(), e);
             }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3fff8f54/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputRDD.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputRDD.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputRDD.java
index 55bf53b..d926686 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputRDD.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputRDD.java
@@ -37,18 +37,13 @@ public final class PersistedInputRDD implements InputRDD {
         if (!configuration.containsKey(Constants.GREMLIN_HADOOP_INPUT_LOCATION))
             throw new IllegalArgumentException("There is no provided " + Constants.GREMLIN_HADOOP_INPUT_LOCATION + " to read the persisted RDD from");
         Spark.create(sparkContext.sc());
-        final String inputLocation = configuration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION);
-        final String graphRDDName = Spark.hasRDD(inputLocation) ? inputLocation : Constants.getGraphLocation(inputLocation);
-        return JavaPairRDD.fromJavaRDD((JavaRDD) Spark.getRDD(graphRDDName).toJavaRDD());
+        return JavaPairRDD.fromJavaRDD((JavaRDD) Spark.getRDD(Constants.getSearchGraphLocation(configuration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION), SparkContextStorage.open(sparkContext.sc())).get()).toJavaRDD());
     }
 
     @Override
     public <K, V> JavaPairRDD<K, V> readMemoryRDD(final Configuration configuration, final String memoryKey, final JavaSparkContext sparkContext) {
         if (!configuration.containsKey(Constants.GREMLIN_HADOOP_INPUT_LOCATION))
             throw new IllegalArgumentException("There is no provided " + Constants.GREMLIN_HADOOP_INPUT_LOCATION + " to read the persisted RDD from");
-        final String inputLocation = configuration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION);
-        final String memoryRDDName = Spark.hasRDD(inputLocation) ? inputLocation : Constants.getMemoryLocation(inputLocation, memoryKey);
-        Spark.create(sparkContext.sc());
-        return JavaPairRDD.fromJavaRDD((JavaRDD) Spark.getRDD(memoryRDDName).toJavaRDD());
+        return JavaPairRDD.fromJavaRDD((JavaRDD) Spark.getRDD(Constants.getMemoryLocation(configuration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION), memoryKey)).toJavaRDD());
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3fff8f54/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java
index b78caa9..7833701 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java
@@ -44,17 +44,15 @@ public final class PersistedOutputRDD implements OutputRDD {
             LOGGER.warn("The SparkContext should be persisted in order for the RDD to persist across jobs. To do so, set " + Constants.GREMLIN_SPARK_PERSIST_CONTEXT + " to true");
         if (!configuration.containsKey(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION))
             throw new IllegalArgumentException("There is no provided " + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + " to write the persisted RDD to");
-        final String graphRDDName = Constants.getGraphLocation(configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));
-        Spark.removeRDD(graphRDDName);  // this might be bad cause it unpersists the job RDD
-        Constants.getSearchGraphLocation(configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), SparkContextStorage.open(configuration)).ifPresent(Spark::removeRDD);  // this might be bad cause it unpersists the job RDD
+        SparkContextStorage.open(configuration).rmr(configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));  // this might be bad cause it unpersists the job RDD
         if (!configuration.getBoolean(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT_HAS_EDGES, true))
             graphRDD.mapValues(vertex -> {
                 vertex.get().dropEdges();
                 return vertex;
-            }).setName(graphRDDName).cache();
+            }).setName(Constants.getGraphLocation(configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION))).cache();
         else
-            graphRDD.setName(graphRDDName).cache();
-        Spark.refresh();
+            graphRDD.setName(Constants.getGraphLocation(configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION))).cache();
+        Spark.refresh(); // necessary to do really fast so the Spark GC doesn't clear out the RDD
     }
 
     @Override
@@ -63,9 +61,9 @@ public final class PersistedOutputRDD implements OutputRDD {
             LOGGER.warn("The SparkContext should be persisted in order for the RDD to persist across jobs. To do so, set " + Constants.GREMLIN_SPARK_PERSIST_CONTEXT + " to true");
         if (!configuration.containsKey(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION))
             throw new IllegalArgumentException("There is no provided " + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + " to write the persisted RDD to");
-        final String sideEffectRDDName = configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION) + "/" + memoryKey;
-        Spark.removeRDD(sideEffectRDDName);
-        memoryRDD.setName(sideEffectRDDName).cache();
+        final String memoryRDDName = Constants.getMemoryLocation(configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), memoryKey);
+        Spark.removeRDD(memoryRDDName);
+        memoryRDD.setName(memoryRDDName).cache();
         return IteratorUtils.map(memoryRDD.toLocalIterator(), tuple -> new KeyValue<>(tuple._1(), tuple._2()));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3fff8f54/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
index 887e2f9..2ea0fc3 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
@@ -85,12 +85,12 @@ public final class SparkContextStorage implements Storage {
 
     @Override
     public boolean mkdir(final String location) {
-        throw new UnsupportedOperationException("This operation does not make sense for a persited SparkContext");
+        throw new UnsupportedOperationException("This operation does not make sense for a persisted SparkContext");
     }
 
     @Override
     public boolean cp(final String fromLocation, final String toLocation) {
-        Spark.getRDD(fromLocation).setName(toLocation).cache();
+        Spark.getRDD(fromLocation).setName(toLocation).cache().count();
         Spark.removeRDD(fromLocation);
         return true;
     }
@@ -111,7 +111,7 @@ public final class SparkContextStorage implements Storage {
     @Override
     public boolean rmr(final String location) {
         final List<String> rdds = new ArrayList<>();
-        final String wildCardLocation = location.replace(".", "\\.").replace("*", ".*");
+        final String wildCardLocation = (location.endsWith("*") ? location : location + "*").replace(".", "\\.").replace("*", ".*");
         for (final RDD<?> rdd : Spark.getRDDs()) {
             if (rdd.name().matches(wildCardLocation))
                 rdds.add(rdd.name());
@@ -121,9 +121,9 @@ public final class SparkContextStorage implements Storage {
     }
 
     @Override
-    public Iterator<Vertex> headGraph(final String location, int totalLines, final Class parserClass) {
+    public Iterator<Vertex> head(final String location, final Class parserClass, final int totalLines) {
         final Configuration configuration = new BaseConfiguration();
-        configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, Constants.getSearchGraphLocation(location, this).get());
+        configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, location);
         configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_INPUT_RDD, parserClass.getCanonicalName());
         configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, parserClass.getCanonicalName());
         try {
@@ -135,13 +135,13 @@ public final class SparkContextStorage implements Storage {
         } catch (final Exception e) {
             throw new IllegalArgumentException(e.getMessage(), e);
         }
-        throw new IllegalArgumentException("The provided parserClass must be an " + InputFormat.class.getCanonicalName() + " or a " + InputRDD.class.getCanonicalName() + ": " + parserClass.getCanonicalName());
+        throw new IllegalArgumentException("The provided parserClass must be an " + InputFormat.class.getCanonicalName() + " or an " + InputRDD.class.getCanonicalName() + ": " + parserClass.getCanonicalName());
     }
 
     @Override
-    public <K, V> Iterator<KeyValue<K, V>> headMemory(final String location, final String memoryKey, int totalLines, Class parserClass) {
+    public <K, V> Iterator<KeyValue<K, V>> head(final String location, final String memoryKey, final Class parserClass, final int totalLines) {
         final Configuration configuration = new BaseConfiguration();
-        configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, Constants.getMemoryLocation(location, memoryKey));
+        configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, location);
         configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_INPUT_RDD, parserClass.getCanonicalName());
         configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, parserClass.getCanonicalName());
         try {
@@ -153,7 +153,7 @@ public final class SparkContextStorage implements Storage {
         } catch (final Exception e) {
             throw new IllegalArgumentException(e.getMessage(), e);
         }
-        throw new IllegalArgumentException("The provided parserClass must be an " + InputFormat.class.getCanonicalName() + " or a " + InputRDD.class.getCanonicalName() + ": " + parserClass.getCanonicalName());
+        throw new IllegalArgumentException("The provided parserClass must be an " + InputFormat.class.getCanonicalName() + " or an " + InputRDD.class.getCanonicalName() + ": " + parserClass.getCanonicalName());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3fff8f54/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageTest.java
index 43e8508..8c99c9e 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageTest.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageTest.java
@@ -59,7 +59,7 @@ public class SparkContextStorageTest extends AbstractSparkTest {
         assertEquals(2, storage.ls().size());
         // TEST GRAPH PERSISTENCE
         assertTrue(storage.exists(Constants.getGraphLocation(outputLocation)));
-        assertEquals(6, IteratorUtils.count(storage.headGraph(outputLocation, PersistedInputRDD.class)));
+        assertEquals(6, IteratorUtils.count(storage.head(outputLocation, PersistedInputRDD.class)));
         assertEquals(6, result.graph().traversal().V().count().next().longValue());
         assertEquals(0, result.graph().traversal().E().count().next().longValue());
         assertEquals(6, result.graph().traversal().V().values("name").count().next().longValue());
@@ -68,7 +68,7 @@ public class SparkContextStorageTest extends AbstractSparkTest {
         // TEST MEMORY PERSISTENCE
         assertEquals(2, (int) result.memory().get("clusterCount"));
         assertTrue(storage.exists(Constants.getMemoryLocation(outputLocation, "clusterCount")));
-        assertEquals(2, storage.headMemory(outputLocation, "clusterCount", PersistedInputRDD.class).next().getValue());
+        assertEquals(2, storage.head(outputLocation, "clusterCount", PersistedInputRDD.class).next().getValue());
     }
 
 }


[30/30] incubator-tinkerpop git commit: Merge branch 'master' into TINKERPOP-320

Posted by dk...@apache.org.
Merge branch 'master' into TINKERPOP-320

Resolved Conflicts:
	CHANGELOG.asciidoc


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/96388ee8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/96388ee8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/96388ee8

Branch: refs/heads/TINKERPOP-320
Commit: 96388ee83537bc7e2d5a15d3323e15b81b6a50d7
Parents: d7ae923 1e989b7
Author: Daniel Kuppitz <da...@hotmail.com>
Authored: Mon Jan 11 17:48:00 2016 +0100
Committer: Daniel Kuppitz <da...@hotmail.com>
Committed: Mon Jan 11 17:48:00 2016 +0100

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |  1 +
 .../traversal/util/DependantMutableMetrics.java | 34 +++++++++++---------
 .../process/traversal/util/MutableMetrics.java  |  3 +-
 .../customizer/SimpleSandboxExtension.groovy    |  5 +--
 .../server/GremlinServerIntegrateTest.java      | 30 +++++++++++++++++
 .../hadoop/structure/io/ObjectWritable.java     |  7 +++-
 6 files changed, 61 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/96388ee8/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --cc CHANGELOG.asciidoc
index 27003b6,fdbfb90..85280af
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@@ -26,7 -26,7 +26,8 @@@ image::https://raw.githubusercontent.co
  TinkerPop 3.1.1 (NOT OFFICIALLY RELEASED YET)
  ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
  
 +* Added `BulkDumperVertex` that allows to dump a whole graph in any of the supported IO formats (GraphSON, Gryo, Script).
+ * Fixed a bug around duration calculations of `cap()`-step during profiling.
  * It is possible to completely avoid using HDFS with Spark if `PersistedInputRDD` and `PersistedOutpuRDD` are leveraged.
  * `InputRDD` and `OutputRDD` can now process both graphs and memory (i.e. sideEffects).
  * Removed Groovy specific meta-programming overloads for handling Hadoop `FileSystem` (instead, its all accessible via `FileSystemStorage`).


[03/30] incubator-tinkerpop git commit: merged master and merged conflicts from @spmallettes changes to SparkGremlinPlugin and HadoopGremlinPlugin.

Posted by dk...@apache.org.
merged master and merged conflicts from @spmallettes changes to SparkGremlinPlugin and HadoopGremlinPlugin.


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/2c0d327c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/2c0d327c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/2c0d327c

Branch: refs/heads/TINKERPOP-320
Commit: 2c0d327c04219de9fdf20444a100d3cb3dd1d221
Parents: 58d9240 5a9ba5f
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Dec 9 13:48:49 2015 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Wed Dec 9 13:48:49 2015 -0700

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |  1 +
 .../src/reference/gremlin-applications.asciidoc | 29 +++++++++
 .../upgrade/release-3.1.x-incubating.asciidoc   | 24 ++++++++
 .../groovy/plugin/GiraphGremlinPlugin.java      |  1 -
 gremlin-console/conf/log4j-console.properties   | 15 +++--
 .../gremlin/groovy/engine/GremlinExecutor.java  |  1 -
 gremlin-server/conf/log4j-server.properties     | 17 ++++--
 .../server/op/AbstractEvalOpProcessor.java      | 63 +++++++++++++++-----
 .../gremlin/server/op/session/Session.java      |  1 -
 .../server/GremlinDriverIntegrateTest.java      |  4 +-
 .../groovy/plugin/HadoopGremlinPlugin.java      | 10 ----
 .../spark/groovy/plugin/SparkGremlinPlugin.java |  4 --
 12 files changed, 125 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/2c0d327c/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java
----------------------------------------------------------------------
diff --cc hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java
index b4f5cd2,e8ed4c2..55ba020
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java
@@@ -81,16 -78,10 +79,8 @@@ public final class HadoopGremlinPlugin 
      public void afterPluginTo(final PluginAcceptor pluginAcceptor) throws PluginInitializationException, IllegalEnvironmentException {
          pluginAcceptor.addImports(IMPORTS);
          try {
-             pluginAcceptor.eval(String.format("Logger.getLogger(%s).setLevel(Level.INFO)", JobClient.class.getName()));
-             pluginAcceptor.eval(String.format("Logger.getLogger(%s).setLevel(Level.INFO)", Job.class.getName()));
-             ///
-             pluginAcceptor.eval(String.format("Logger.getLogger(%s).setLevel(Level.INFO)", MapReduceGraphComputer.class.getName()));
-             ///
-             pluginAcceptor.eval(String.format("Logger.getLogger(%s).setLevel(Level.INFO)", HadoopGraph.class.getName()));
-             //pluginAcceptor.eval(HadoopLoader.class.getCanonicalName() + ".load()");
 -            pluginAcceptor.eval(HadoopLoader.class.getCanonicalName() + ".load()");
--
 -            pluginAcceptor.addBinding("hdfs", FileSystem.get(new Configuration()));
 -            pluginAcceptor.addBinding("local", FileSystem.getLocal(new Configuration()));
 +            pluginAcceptor.addBinding("hdfs", new FileSystemStorage(FileSystem.get(new Configuration())));
 +            pluginAcceptor.addBinding("local", new FileSystemStorage(FileSystem.getLocal(new Configuration())));
              if (null == System.getenv(Constants.HADOOP_GREMLIN_LIBS))
                  HadoopGraph.LOGGER.warn("Be sure to set the environmental variable: " + Constants.HADOOP_GREMLIN_LIBS);
              else

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/2c0d327c/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java
----------------------------------------------------------------------
diff --cc spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java
index 7711435,616676c..fcb234c
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java
@@@ -41,10 -39,8 +40,9 @@@ public final class SparkGremlinPlugin e
      protected static String NAME = "tinkerpop.spark";
  
      protected static final Set<String> IMPORTS = new HashSet<String>() {{
-         add("import org.apache.log4j.*");
          add(IMPORT_SPACE + SparkGraphComputer.class.getPackage().getName() + DOT_STAR);
          add(IMPORT_SPACE + Spark.class.getPackage().getName() + DOT_STAR);
 +        add(IMPORT_SPACE + SparkContextStorage.class.getPackage().getName() + DOT_STAR);
      }};
  
      @Override
@@@ -56,9 -52,8 +54,7 @@@
      public void afterPluginTo(final PluginAcceptor pluginAcceptor) throws PluginInitializationException, IllegalEnvironmentException {
          pluginAcceptor.addImports(IMPORTS);
          try {
-             pluginAcceptor.eval(String.format("Logger.getLogger(%s).setLevel(Level.INFO)", SparkGraphComputer.class.getName()));
-             pluginAcceptor.eval(String.format("Logger.getLogger(%s).setLevel(Level.ERROR)", MetricsSystem.class.getName()));
 -            pluginAcceptor.eval("spark = Spark");
 -            pluginAcceptor.eval(SparkLoader.class.getCanonicalName() + ".load()");
 +            pluginAcceptor.eval("spark = SparkContextStorage.open()");
          } catch (final Exception e) {
              throw new PluginInitializationException(e.getMessage(), e);
          }


[23/30] incubator-tinkerpop git commit: added import for BulkDumperVertexProgram to console standard imports

Posted by dk...@apache.org.
added import for BulkDumperVertexProgram to console standard imports


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/53f28d4b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/53f28d4b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/53f28d4b

Branch: refs/heads/TINKERPOP-320
Commit: 53f28d4b8cf2dc89e492e82a7836e70dcfa9e559
Parents: 43eaf8d
Author: Daniel Kuppitz <da...@hotmail.com>
Authored: Sat Jan 9 15:03:36 2016 +0100
Committer: Daniel Kuppitz <da...@hotmail.com>
Committed: Sat Jan 9 15:03:36 2016 +0100

----------------------------------------------------------------------
 .../tinkerpop/gremlin/groovy/AbstractImportCustomizerProvider.java | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/53f28d4b/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/AbstractImportCustomizerProvider.java
----------------------------------------------------------------------
diff --git a/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/AbstractImportCustomizerProvider.java b/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/AbstractImportCustomizerProvider.java
index 5df8cfe..f5bcde8 100644
--- a/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/AbstractImportCustomizerProvider.java
+++ b/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/AbstractImportCustomizerProvider.java
@@ -24,6 +24,7 @@ import org.apache.commons.configuration.Configuration;
 import org.apache.tinkerpop.gremlin.groovy.function.GFunction;
 import org.apache.tinkerpop.gremlin.groovy.loaders.GremlinLoader;
 import org.apache.tinkerpop.gremlin.process.computer.GraphComputer;
+import org.apache.tinkerpop.gremlin.process.computer.bulkdumping.BulkDumperVertexProgram;
 import org.apache.tinkerpop.gremlin.process.computer.bulkloading.BulkLoaderVertexProgram;
 import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.PeerPressureVertexProgram;
 import org.apache.tinkerpop.gremlin.process.computer.ranking.pagerank.PageRankVertexProgram;
@@ -133,6 +134,7 @@ public abstract class AbstractImportCustomizerProvider implements ImportCustomiz
         imports.add(PageRankVertexProgram.class.getPackage().getName() + DOT_STAR);
         imports.add(TraversalVertexProgram.class.getPackage().getName() + DOT_STAR);
         imports.add(BulkLoaderVertexProgram.class.getPackage().getName() + DOT_STAR);
+        imports.add(BulkDumperVertexProgram.class.getPackage().getName() + DOT_STAR);
 
         // groovy extras
         imports.add(Grape.class.getCanonicalName());


[11/30] incubator-tinkerpop git commit: Storage is complete and has a really cool TestSuite. There are two types of Storage. FileSystemStorage (HDFS) and SparkContextStorage (persited RDDs). You can ls(), cp(), rm(), rmr(), head(), etc. There is a single

Posted by dk...@apache.org.
Storage is complete and has a really cool TestSuite. There are two types of Storage. FileSystemStorage (HDFS) and SparkContextStorage (persited RDDs). You can ls(), cp(), rm(), rmr(), head(), etc. There is a single abstract test suite called AbstractStorageCheck that confirms that both Spark and HDFS behave the same. Moved around and organized Hadoop test cases given the new developments.


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

Branch: refs/heads/TINKERPOP-320
Commit: b0f3e4a96ced7f45f5e823b9060eac9dd0be1f7e
Parents: 53e57a7
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Jan 6 10:26:46 2016 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Wed Jan 6 10:26:46 2016 -0700

----------------------------------------------------------------------
 docs/src/reference/implementations.asciidoc     |   7 +-
 .../GiraphHadoopGremlinIntegrateTest.java       |  33 +++++
 .../GiraphHadoopGremlinPluginIntegrateTest.java |  33 -----
 .../tinkerpop/gremlin/structure/io/Storage.java |   5 +
 .../groovy/plugin/HadoopGremlinPlugin.java      |   2 +
 .../hadoop/structure/io/FileSystemStorage.java  |   4 +-
 .../gremlin/hadoop/HadoopGremlinSuite.java      |  36 ++++++
 .../groovy/plugin/FileSystemStorageCheck.java   |  84 ------------
 .../groovy/plugin/HadoopGremlinPluginCheck.java |  61 +--------
 .../hadoop/groovy/plugin/HadoopPluginSuite.java |  34 -----
 .../structure/io/AbstractStorageCheck.java      | 117 +++++++++++++++++
 .../structure/io/FileSystemStorageCheck.java    |  82 ++++++++++++
 .../spark/structure/io/SparkContextStorage.java |  22 ++--
 .../gremlin/spark/SparkGremlinSuite.java        |  35 +++++
 .../gremlin/spark/SparkGremlinTest.java         |  33 +++++
 .../gremlin/spark/SparkHadoopGremlinTest.java   |  33 +++++
 .../computer/SparkHadoopGraphProvider.java      |  11 +-
 .../groovy/plugin/SparkGremlinPluginTest.java   | 129 -------------------
 .../plugin/SparkHadoopGremlinPluginTest.java    |  33 -----
 .../structure/io/SparkContextStorageCheck.java  |  71 ++++++++++
 .../structure/io/SparkContextStorageTest.java   |  74 -----------
 21 files changed, 478 insertions(+), 461 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/docs/src/reference/implementations.asciidoc
----------------------------------------------------------------------
diff --git a/docs/src/reference/implementations.asciidoc b/docs/src/reference/implementations.asciidoc
index 323f481..c3a1df6 100644
--- a/docs/src/reference/implementations.asciidoc
+++ b/docs/src/reference/implementations.asciidoc
@@ -1591,7 +1591,7 @@ g = graph.traversal(computer(SparkGraphComputer))
 hdfs.ls()
 hdfs.ls('output')
 hdfs.ls('output/~reducing')
-hdfs.head('output/~reducing', ObjectWritable)
+hdfs.head('output/~reducing', SequenceFileInputFormat)
 ----
 
 A list of the HDFS methods available are itemized below. Note that these methods are also available for the 'local' variable:
@@ -1609,7 +1609,10 @@ A list of the HDFS methods available are itemized below. Note that these methods
 |hdfs.mergeToLocal(String from, String to)| Merge the files in path to the specified local path.
 |hdfs.head(String path)| Display the data in the path as text.
 |hdfs.head(String path, int lineCount)| Text display only the first `lineCount`-number of lines in the path.
-|hdfs.head(String path, int totalKeyValues, Class<Writable> writableClass)| Display the path interpreting the key values as respective writable.
+|hdfs.head(String path, int totalKeyValues, Class<InputFormat> inputFormatClass)| Parse and display the data using the InputFormat.
+|hdfs.head(String path, Class<InputFormat> inputFormatClass)| Parse and display the data using the InputFormat.
+|hdfs.head(String path, String memoryKey, Class<InputFormat> inputFormatClass, int totalKeyValues) | Parse and display memory data using the InputFormat.
+|hdfs.head(String path, String memoryKey, Class<InputFormat> inputFormatClass) | Parse and display memory data using the InputFormat.
 |=========================================================
 
 A Command Line Example

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/GiraphHadoopGremlinIntegrateTest.java
----------------------------------------------------------------------
diff --git a/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/GiraphHadoopGremlinIntegrateTest.java b/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/GiraphHadoopGremlinIntegrateTest.java
new file mode 100644
index 0000000..ba9e12d
--- /dev/null
+++ b/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/GiraphHadoopGremlinIntegrateTest.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tinkerpop.gremlin.giraph;
+
+import org.apache.tinkerpop.gremlin.GraphProviderClass;
+import org.apache.tinkerpop.gremlin.giraph.process.computer.GiraphHadoopGraphProvider;
+import org.apache.tinkerpop.gremlin.hadoop.HadoopGremlinSuite;
+import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
+import org.junit.runner.RunWith;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+@RunWith(HadoopGremlinSuite.class)
+@GraphProviderClass(provider = GiraphHadoopGraphProvider.class, graph = HadoopGraph.class)
+public class GiraphHadoopGremlinIntegrateTest {
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/process/computer/groovy/plugin/GiraphHadoopGremlinPluginIntegrateTest.java
----------------------------------------------------------------------
diff --git a/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/process/computer/groovy/plugin/GiraphHadoopGremlinPluginIntegrateTest.java b/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/process/computer/groovy/plugin/GiraphHadoopGremlinPluginIntegrateTest.java
deleted file mode 100644
index 4660c42..0000000
--- a/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/process/computer/groovy/plugin/GiraphHadoopGremlinPluginIntegrateTest.java
+++ /dev/null
@@ -1,33 +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.tinkerpop.gremlin.giraph.process.computer.groovy.plugin;
-
-import org.apache.tinkerpop.gremlin.GraphProviderClass;
-import org.apache.tinkerpop.gremlin.giraph.process.computer.GiraphHadoopGraphProvider;
-import org.apache.tinkerpop.gremlin.hadoop.groovy.plugin.HadoopPluginSuite;
-import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
-import org.junit.runner.RunWith;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-@RunWith(HadoopPluginSuite.class)
-@GraphProviderClass(provider = GiraphHadoopGraphProvider.class, graph = HadoopGraph.class)
-public class GiraphHadoopGremlinPluginIntegrateTest {
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
index b6e6ebe..d9c6927 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
@@ -52,6 +52,11 @@ public interface Storage {
 
     public Iterator<Vertex> head(final String location, final Class parserClass, final int totalLines);
 
+    @Deprecated
+    public default Iterator<Vertex> head(final String location, final int totalLines, final Class parserClass) {
+       return this.head(location,parserClass,totalLines);
+    }
+
     public default Iterator<Vertex> head(final String location, final Class parserClass) {
         return this.head(location, parserClass, Integer.MAX_VALUE);
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java
index c50d226..a090e5b 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java
@@ -20,6 +20,7 @@ package org.apache.tinkerpop.gremlin.hadoop.groovy.plugin;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.tinkerpop.gremlin.groovy.plugin.AbstractGremlinPlugin;
 import org.apache.tinkerpop.gremlin.groovy.plugin.IllegalEnvironmentException;
 import org.apache.tinkerpop.gremlin.groovy.plugin.PluginAcceptor;
@@ -64,6 +65,7 @@ public final class HadoopGremlinPlugin extends AbstractGremlinPlugin {
         add(IMPORT_SPACE + GryoInputFormat.class.getPackage().getName() + DOT_STAR);
         add(IMPORT_SPACE + GraphSONInputFormat.class.getPackage().getName() + DOT_STAR);
         add(IMPORT_SPACE + ScriptInputFormat.class.getPackage().getName() + DOT_STAR);
+        add(IMPORT_SPACE + SequenceFileInputFormat.class.getCanonicalName());
         ////
         add(IMPORT_SPACE + MapReduceGraphComputer.class.getPackage().getName() + DOT_STAR);
     }};

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
index 68a6cf8..4f648ee 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
@@ -177,8 +177,8 @@ public final class FileSystemStorage implements Storage {
 
     @Override
     public <K, V> Iterator<KeyValue<K, V>> head(final String location, final String memoryKey, final Class parserClass, final int totalLines) {
-        if (!parserClass.equals(SequenceFileInputFormat.class) && !parserClass.equals(ObjectWritable.class))
-            throw new IllegalArgumentException("Only " + SequenceFileInputFormat.class.getCanonicalName() + " and " + ObjectWritable.class.getCanonicalName() + " memories are supported");
+        if (!parserClass.equals(SequenceFileInputFormat.class) && !parserClass.equals(ObjectWritable.class)) // object writable support for backwards compatibility
+            throw new IllegalArgumentException("Only " + SequenceFileInputFormat.class.getCanonicalName() + " memories are supported");
         final Configuration configuration = new Configuration();
         try {
             return IteratorUtils.limit((Iterator) new ObjectWritableIterator(configuration, new Path(Constants.getMemoryLocation(location, memoryKey))), totalLines);

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGremlinSuite.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGremlinSuite.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGremlinSuite.java
new file mode 100644
index 0000000..89aef80
--- /dev/null
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGremlinSuite.java
@@ -0,0 +1,36 @@
+/*
+ * 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.tinkerpop.gremlin.hadoop;
+
+import org.apache.tinkerpop.gremlin.AbstractGremlinSuite;
+import org.apache.tinkerpop.gremlin.hadoop.groovy.plugin.HadoopGremlinPluginCheck;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.FileSystemStorageCheck;
+import org.apache.tinkerpop.gremlin.process.traversal.TraversalEngine;
+import org.junit.runners.model.InitializationError;
+import org.junit.runners.model.RunnerBuilder;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public class HadoopGremlinSuite extends AbstractGremlinSuite {
+    public HadoopGremlinSuite(final Class<?> klass, final RunnerBuilder builder) throws InitializationError {
+        super(klass, builder, new Class<?>[]{HadoopGremlinPluginCheck.class, FileSystemStorageCheck.class}, new Class<?>[]{HadoopGremlinPluginCheck.class, FileSystemStorageCheck.class}, true, TraversalEngine.Type.COMPUTER);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java
deleted file mode 100644
index aea7db7..0000000
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java
+++ /dev/null
@@ -1,84 +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.tinkerpop.gremlin.hadoop.groovy.plugin;
-
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.apache.tinkerpop.gremlin.AbstractGremlinTest;
-import org.apache.tinkerpop.gremlin.LoadGraphWith;
-import org.apache.tinkerpop.gremlin.hadoop.Constants;
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.FileSystemStorage;
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable;
-import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
-import org.apache.tinkerpop.gremlin.process.computer.ComputerResult;
-import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.ClusterCountMapReduce;
-import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.PeerPressureVertexProgram;
-import org.apache.tinkerpop.gremlin.structure.io.Storage;
-import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-public class FileSystemStorageCheck extends AbstractGremlinTest {
-
-    @Test
-    @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
-    public void shouldPersistGraphAndMemory() throws Exception {
-        final Storage storage = FileSystemStorage.open(ConfUtil.makeHadoopConfiguration(graph.configuration()));
-        final String inputLocation = Constants.getSearchGraphLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION), storage).get();
-        final String outputLocation = graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION);
-
-        // TEST INPUT GRAPH
-        assertTrue(storage.exists(inputLocation));
-        // assertFalse(storage.exists(outputLocation)); AbstractGremlinTest will create this automatically.
-        if (inputLocation.endsWith(".json")) { // gryo is not text readable
-            assertEquals(6, IteratorUtils.count(storage.head(inputLocation)));
-            for (int i = 0; i < 7; i++) {
-                assertEquals(i, IteratorUtils.count(storage.head(inputLocation, i)));
-            }
-            assertEquals(6, IteratorUtils.count(storage.head(inputLocation, 10)));
-        }
-
-        ////////////////////
-
-        final ComputerResult result = graph.compute(graphComputerClass.get()).program(PeerPressureVertexProgram.build().create(graph)).mapReduce(ClusterCountMapReduce.build().memoryKey("clusterCount").create()).submit().get();
-        // TEST OUTPUT GRAPH
-        assertTrue(storage.exists(outputLocation));
-        assertTrue(storage.exists(Constants.getGraphLocation(outputLocation)));
-        assertEquals(6, result.graph().traversal().V().count().next().longValue());
-        assertEquals(0, result.graph().traversal().E().count().next().longValue());
-        assertEquals(6, result.graph().traversal().V().values("name").count().next().longValue());
-        assertEquals(6, result.graph().traversal().V().values(PeerPressureVertexProgram.CLUSTER).count().next().longValue());
-        assertEquals(2, result.graph().traversal().V().values(PeerPressureVertexProgram.CLUSTER).dedup().count().next().longValue());
-        /////
-        // TEST MEMORY PERSISTENCE
-        assertEquals(2, (int) result.memory().get("clusterCount"));
-        assertTrue(storage.exists(Constants.getMemoryLocation(outputLocation, "clusterCount")));
-        assertEquals(1, IteratorUtils.count(storage.head(outputLocation, "clusterCount", SequenceFileInputFormat.class)));
-        assertEquals(2, storage.head(outputLocation, "clusterCount", SequenceFileInputFormat.class).next().getValue());
-        //// backwards compatibility
-        assertEquals(1, IteratorUtils.count(storage.head(outputLocation, "clusterCount", ObjectWritable.class)));
-        assertEquals(2, storage.head(outputLocation, "clusterCount", ObjectWritable.class).next().getValue());
-
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
index 26acee9..711f63e 100644
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
@@ -25,16 +25,12 @@ import org.apache.tinkerpop.gremlin.TestHelper;
 import org.apache.tinkerpop.gremlin.groovy.plugin.RemoteAcceptor;
 import org.apache.tinkerpop.gremlin.groovy.util.TestableConsolePluginAcceptor;
 import org.apache.tinkerpop.gremlin.hadoop.Constants;
-import org.apache.tinkerpop.gremlin.hadoop.HadoopGraphProvider;
-import org.apache.tinkerpop.gremlin.process.AbstractGremlinProcessTest;
-import org.apache.tinkerpop.gremlin.process.computer.traversal.step.sideEffect.mapreduce.TraverserMapReduce;
+import org.apache.tinkerpop.gremlin.hadoop.HadoopGremlinSuite;
 import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
-import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.File;
 import java.util.Arrays;
 import java.util.List;
 
@@ -44,7 +40,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 /**
- * This is an test that is mean to be used in the context of the {@link HadoopPluginSuite} and shouldn't be
+ * This is an test that is mean to be used in the context of the {@link HadoopGremlinSuite} and shouldn't be
  * executed on its own.
  *
  * @author Marko A. Rodriguez (http://markorodriguez.com)
@@ -99,10 +95,6 @@ public class HadoopGremlinPluginCheck extends AbstractGremlinTest {
     @Test
     @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
     public void shouldSupportHDFSMethods() throws Exception {
-        final String hadoopGraphProviderDataDir = graphProvider.getWorkingDirectory() + File.separator;
-        final File testDir = TestHelper.makeTestDataPath(HadoopGremlinPluginCheck.class, "shouldSupportHDFSMethods");
-        final String prefix = TestHelper.convertToRelative(HadoopGremlinPluginCheck.class, testDir);
-        ////////////////
         List<String> ls = (List<String>) this.console.eval("hdfs.ls()");
         for (final String line : ls) {
             assertTrue(line.startsWith("-") || line.startsWith("r") || line.startsWith("w") || line.startsWith("x"));
@@ -113,55 +105,6 @@ public class HadoopGremlinPluginCheck extends AbstractGremlinTest {
             assertTrue(line.startsWith("-") || line.startsWith("r") || line.startsWith("w") || line.startsWith("x"));
             assertEquals(" ", line.substring(9, 10));
         }
-        ////////////////
-        this.console.eval("hdfs.copyFromLocal('" + HadoopGraphProvider.PATHS.get("tinkerpop-classic.txt") + "', '" + prefix + "tinkerpop-classic.txt')");
-        assertTrue((Boolean) this.console.eval("hdfs.exists('" + prefix + "tinkerpop-classic.txt')"));
-        ////////////////
-        List<String> head = IteratorUtils.asList(this.console.eval("hdfs.head('" + prefix + "tinkerpop-classic.txt')"));
-        assertEquals(6, head.size());
-        for (final String line : head) {
-            assertEquals(":", line.substring(1, 2));
-            assertTrue(Integer.valueOf(line.substring(0, 1)) <= 6);
-        }
-        head = IteratorUtils.asList(this.console.eval("hdfs.head('" + prefix + "tinkerpop-classic.txt',3)"));
-        assertEquals(3, head.size());
-        for (final String line : head) {
-            assertEquals(":", line.substring(1, 2));
-            assertTrue(Integer.valueOf(line.substring(0, 1)) <= 3);
-        }
-        ////////////////
-        this.console.eval("hdfs.rm('" + prefix + "tinkerpop-classic.txt')");
-        assertFalse((Boolean) this.console.eval("hdfs.exists('" + prefix + "tinkerpop-classic.txt')"));
-        ////////////////
-        this.console.addBinding("graph", this.graph);
-        this.console.addBinding("g", this.g);
-        this.remote.connect(Arrays.asList("graph", "g"));
-        Traversal<Vertex, String> traversal = (Traversal<Vertex, String>) this.remote.submit(Arrays.asList("g.V().hasLabel('person').group('m').by('age').by('name').out('knows').out('created').values('name')"));
-        AbstractGremlinProcessTest.checkResults(Arrays.asList("ripple", "lop"), traversal);
-//        assertFalse((Boolean) this.console.eval("hdfs.exists('" + hadoopGraphProviderDataDir + "m')"));
-//        assertFalse((Boolean) this.console.eval("hdfs.exists('" + hadoopGraphProviderDataDir + TraverserMapReduce.TRAVERSERS + "')"));
-        /*final List<KeyValue<Integer, Collection<String>>> mList = IteratorUtils.asList(this.console.eval("hdfs.head('" + hadoopGraphProviderDataDir + "m',ObjectWritable)"));
-        assertEquals(4, mList.size());
-        mList.forEach(keyValue -> {
-            if (keyValue.getKey().equals(29))
-                assertTrue(keyValue.getValue().contains("marko"));
-            else if (keyValue.getKey().equals(35))
-                assertTrue(keyValue.getValue().contains("peter"));
-            else if (keyValue.getKey().equals(32))
-                assertTrue(keyValue.getValue().contains("josh"));
-            else if (keyValue.getKey().equals(27))
-                assertTrue(keyValue.getValue().contains("vadas"));
-            else
-                throw new IllegalStateException("The provided key/value is unknown: " + keyValue);
-        });
-        final List<KeyValue<MapReduce.NullObject, Traverser<String>>> traversersList = IteratorUtils.asList(this.console.eval("hdfs.head('" + hadoopGraphProviderDataDir + TraverserMapReduce.TRAVERSERS + "',ObjectWritable)"));
-        assertEquals(2, traversersList.size());
-        traversersList.forEach(keyValue -> {
-            assertEquals(MapReduce.NullObject.instance(), keyValue.getKey());
-            final String name = keyValue.getValue().get();
-            assertTrue(name.equals("ripple") || name.equals("lop"));
-        });*/
-        ////////////////
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java
deleted file mode 100644
index f062156..0000000
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java
+++ /dev/null
@@ -1,34 +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.tinkerpop.gremlin.hadoop.groovy.plugin;
-
-import org.apache.tinkerpop.gremlin.AbstractGremlinSuite;
-import org.apache.tinkerpop.gremlin.process.traversal.TraversalEngine;
-import org.junit.runners.model.InitializationError;
-import org.junit.runners.model.RunnerBuilder;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-public class HadoopPluginSuite extends AbstractGremlinSuite {
-    public HadoopPluginSuite(final Class<?> klass, final RunnerBuilder builder) throws InitializationError {
-        super(klass, builder, new Class<?>[]{FileSystemStorageCheck.class}, new Class<?>[]{FileSystemStorageCheck.class}, true, TraversalEngine.Type.COMPUTER);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/AbstractStorageCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/AbstractStorageCheck.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/AbstractStorageCheck.java
new file mode 100644
index 0000000..195f50d
--- /dev/null
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/AbstractStorageCheck.java
@@ -0,0 +1,117 @@
+/*
+ * 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.tinkerpop.gremlin.hadoop.structure.io;
+
+import org.apache.tinkerpop.gremlin.AbstractGremlinTest;
+import org.apache.tinkerpop.gremlin.hadoop.Constants;
+import org.apache.tinkerpop.gremlin.process.computer.ComputerResult;
+import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.ClusterCountMapReduce;
+import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.PeerPressureVertexProgram;
+import org.apache.tinkerpop.gremlin.structure.io.Storage;
+import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public abstract class AbstractStorageCheck extends AbstractGremlinTest {
+
+    public void checkHeadMethods(final Storage storage, final String inputLocation, final String outputLocation, final Class outputGraphParserClass, final Class outputMemoryParserClass) throws Exception {
+        // TEST INPUT GRAPH
+        assertFalse(storage.exists(outputLocation));
+        if (inputLocation.endsWith(".json") && storage.exists(inputLocation)) { // gryo is not text readable
+            assertEquals(6, IteratorUtils.count(storage.head(inputLocation)));
+            for (int i = 0; i < 7; i++) {
+                assertEquals(i, IteratorUtils.count(storage.head(inputLocation, i)));
+            }
+            assertEquals(6, IteratorUtils.count(storage.head(inputLocation, 10)));
+        }
+
+        ////////////////////
+
+        final ComputerResult result = graph.compute(graphComputerClass.get()).program(PeerPressureVertexProgram.build().create(graph)).mapReduce(ClusterCountMapReduce.build().memoryKey("clusterCount").create()).submit().get();
+        // TEST OUTPUT GRAPH
+        assertTrue(storage.exists(outputLocation));
+        assertTrue(storage.exists(Constants.getGraphLocation(outputLocation)));
+        assertEquals(6, result.graph().traversal().V().count().next().longValue());
+        assertEquals(0, result.graph().traversal().E().count().next().longValue());
+        assertEquals(6, result.graph().traversal().V().values("name").count().next().longValue());
+        assertEquals(6, result.graph().traversal().V().values(PeerPressureVertexProgram.CLUSTER).count().next().longValue());
+        assertEquals(2, result.graph().traversal().V().values(PeerPressureVertexProgram.CLUSTER).dedup().count().next().longValue());
+        assertEquals(6, IteratorUtils.count(storage.head(Constants.getGraphLocation(outputLocation), outputGraphParserClass)));
+        for (int i = 0; i < 7; i++) {
+            assertEquals(i, IteratorUtils.count(storage.head(Constants.getGraphLocation(outputLocation), outputGraphParserClass, i)));
+        }
+        assertEquals(6, IteratorUtils.count(storage.head(Constants.getGraphLocation(outputLocation), outputGraphParserClass, 346)));
+        /////
+        // TEST MEMORY PERSISTENCE
+        assertEquals(2, (int) result.memory().get("clusterCount"));
+        assertTrue(storage.exists(Constants.getMemoryLocation(outputLocation, "clusterCount")));
+        assertEquals(1, IteratorUtils.count(storage.head(outputLocation, "clusterCount", outputMemoryParserClass)));
+        assertEquals(2, storage.head(outputLocation, "clusterCount", outputMemoryParserClass).next().getValue());
+    }
+
+    public void checkRemoveAndListMethods(final Storage storage, final String outputLocation) throws Exception {
+        graph.compute(graphComputerClass.get()).program(PeerPressureVertexProgram.build().create(graph)).mapReduce(ClusterCountMapReduce.build().memoryKey("clusterCount").create()).submit().get();
+        assertTrue(storage.exists(outputLocation));
+        assertTrue(storage.exists(Constants.getGraphLocation(outputLocation)));
+        assertTrue(storage.exists(Constants.getMemoryLocation(outputLocation, "clusterCount")));
+        assertEquals(2, storage.ls(outputLocation).size());
+        assertTrue(storage.rmr(Constants.getGraphLocation(outputLocation)));
+        assertEquals(1, storage.ls(outputLocation).size());
+        assertTrue(storage.rmr(Constants.getMemoryLocation(outputLocation, "clusterCount")));
+        assertEquals(0, storage.ls(outputLocation).size());
+        assertFalse(storage.exists(Constants.getGraphLocation(outputLocation)));
+        assertFalse(storage.exists(Constants.getMemoryLocation(outputLocation, "clusterCount")));
+        if (storage.exists(outputLocation))
+            assertTrue(storage.rmr(outputLocation));
+        assertFalse(storage.exists(outputLocation));
+
+        ////////////////
+
+        graph.compute(graphComputerClass.get()).program(PeerPressureVertexProgram.build().create(graph)).mapReduce(ClusterCountMapReduce.build().memoryKey("clusterCount").create()).submit().get();
+        assertTrue(storage.exists(outputLocation));
+        assertTrue(storage.exists(Constants.getGraphLocation(outputLocation)));
+        assertTrue(storage.exists(Constants.getMemoryLocation(outputLocation, "clusterCount")));
+        assertEquals(2, storage.ls(outputLocation).size());
+        assertTrue(storage.rmr(outputLocation));
+        assertFalse(storage.exists(outputLocation));
+        assertEquals(0, storage.ls(outputLocation).size());
+    }
+
+    public void checkCopyMethods(final Storage storage, final String outputLocation, final String newOutputLocation) throws Exception {
+        graph.compute(graphComputerClass.get()).program(PeerPressureVertexProgram.build().create(graph)).mapReduce(ClusterCountMapReduce.build().memoryKey("clusterCount").create()).submit().get();
+        assertTrue(storage.exists(outputLocation));
+        assertTrue(storage.exists(Constants.getGraphLocation(outputLocation)));
+        assertTrue(storage.exists(Constants.getMemoryLocation(outputLocation, "clusterCount")));
+        assertFalse(storage.exists(newOutputLocation));
+
+        assertTrue(storage.cp(outputLocation, newOutputLocation));
+        assertTrue(storage.exists(outputLocation));
+        assertTrue(storage.exists(Constants.getGraphLocation(outputLocation)));
+        assertTrue(storage.exists(Constants.getMemoryLocation(outputLocation, "clusterCount")));
+        assertTrue(storage.exists(newOutputLocation));
+        assertTrue(storage.exists(Constants.getGraphLocation(newOutputLocation)));
+        assertTrue(storage.exists(Constants.getMemoryLocation(newOutputLocation, "clusterCount")));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorageCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorageCheck.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorageCheck.java
new file mode 100644
index 0000000..f528df9
--- /dev/null
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorageCheck.java
@@ -0,0 +1,82 @@
+/*
+ * 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.tinkerpop.gremlin.hadoop.structure.io;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.tinkerpop.gremlin.LoadGraphWith;
+import org.apache.tinkerpop.gremlin.TestHelper;
+import org.apache.tinkerpop.gremlin.hadoop.Constants;
+import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
+import org.apache.tinkerpop.gremlin.structure.io.Storage;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public class FileSystemStorageCheck extends AbstractStorageCheck {
+
+    @Test
+    @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+    public void shouldSupportHeadMethods() throws Exception {
+        final Storage storage = FileSystemStorage.open(ConfUtil.makeHadoopConfiguration(graph.configuration()));
+        final String inputLocation = Constants.getSearchGraphLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION), storage).get();
+        final String outputLocation = graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION);
+        // TestHelper creates the directory and we need it not to exist
+        deleteDirectory(outputLocation);
+        super.checkHeadMethods(storage, inputLocation, outputLocation, InputOutputHelper.getInputFormat((Class) Class.forName(graph.configuration().getString(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT))), ObjectWritable.class);
+        deleteDirectory(outputLocation);
+        super.checkHeadMethods(storage, inputLocation, outputLocation, InputOutputHelper.getInputFormat((Class) Class.forName(graph.configuration().getString(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT))), SequenceFileInputFormat.class);
+    }
+
+    @Test
+    @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+    public void shouldSupportRemoveAndListMethods() throws Exception {
+        final Storage storage = FileSystemStorage.open(ConfUtil.makeHadoopConfiguration(graph.configuration()));
+        final String outputLocation = graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION);
+        super.checkRemoveAndListMethods(storage, outputLocation);
+    }
+
+    @Test
+    @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+    public void shouldSupportCopyMethods() throws Exception {
+        final Storage storage = FileSystemStorage.open(ConfUtil.makeHadoopConfiguration(graph.configuration()));
+        final String outputLocation = graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION);
+        final String newOutputLocation = TestHelper.makeTestDataDirectory(FileSystemStorageCheck.class, "new-location-for-copy");
+        // TestHelper creates the directory and we need it not to exist
+        deleteDirectory(newOutputLocation);
+        super.checkCopyMethods(storage, outputLocation, newOutputLocation);
+
+    }
+
+    private static void deleteDirectory(final String location) throws IOException {
+        // TestHelper creates the directory and we need it not to exist
+        assertTrue(new File(location).isDirectory());
+        assertTrue(new File(location).exists());
+        FileUtils.deleteDirectory(new File(location));
+        assertFalse(new File(location).exists());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
index 97a26f1..d0fc984 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
@@ -32,11 +32,11 @@ import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.structure.io.Storage;
 import org.apache.tinkerpop.gremlin.structure.util.StringFactory;
 import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
-import scala.collection.JavaConversions;
 
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.stream.Collectors;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
@@ -90,14 +90,18 @@ public final class SparkContextStorage implements Storage {
 
     @Override
     public boolean cp(final String fromLocation, final String toLocation) {
-        Spark.getRDD(fromLocation).setName(toLocation).cache().count();
-        Spark.removeRDD(fromLocation);
+        final List<String> rdds = Spark.getRDDs().stream().filter(r -> r.name().startsWith(fromLocation)).map(RDD::name).collect(Collectors.toList());
+        if (rdds.size() == 0)
+            return false;
+        for (final String rdd : rdds) {
+            Spark.getRDD(rdd).toJavaRDD().filter(a -> false).setName(rdd.equals(fromLocation) ? toLocation : rdd.replace(fromLocation, toLocation)).cache().count();
+        }
         return true;
     }
 
     @Override
     public boolean exists(final String location) {
-        return Spark.hasRDD(location);
+        return this.ls(location).size() > 0;
     }
 
     @Override
@@ -128,9 +132,9 @@ public final class SparkContextStorage implements Storage {
         configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, parserClass.getCanonicalName());
         try {
             if (InputRDD.class.isAssignableFrom(parserClass)) {
-                return IteratorUtils.limit(IteratorUtils.map(((InputRDD) parserClass.getConstructor().newInstance()).readGraphRDD(configuration, new JavaSparkContext(Spark.getContext())).toLocalIterator(), tuple -> tuple._2().get()), totalLines);
+                return IteratorUtils.map(((InputRDD) parserClass.getConstructor().newInstance()).readGraphRDD(configuration, new JavaSparkContext(Spark.getContext())).take(totalLines).iterator(), tuple -> tuple._2().get());
             } else if (InputFormat.class.isAssignableFrom(parserClass)) {
-                return IteratorUtils.limit(IteratorUtils.map(new InputFormatRDD().readGraphRDD(configuration, new JavaSparkContext(Spark.getContext())).toLocalIterator(), tuple -> tuple._2().get()), totalLines);
+                return IteratorUtils.map(new InputFormatRDD().readGraphRDD(configuration, new JavaSparkContext(Spark.getContext())).take(totalLines).iterator(), tuple -> tuple._2().get());
             }
         } catch (final Exception e) {
             throw new IllegalArgumentException(e.getMessage(), e);
@@ -146,9 +150,9 @@ public final class SparkContextStorage implements Storage {
         configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, parserClass.getCanonicalName());
         try {
             if (InputRDD.class.isAssignableFrom(parserClass)) {
-                return IteratorUtils.limit(IteratorUtils.map(((InputRDD) parserClass.getConstructor().newInstance()).readMemoryRDD(configuration, memoryKey, new JavaSparkContext(Spark.getContext())).toLocalIterator(), tuple -> new KeyValue(tuple._1(), tuple._2())), totalLines);
+                return IteratorUtils.map(((InputRDD) parserClass.getConstructor().newInstance()).readMemoryRDD(configuration, memoryKey, new JavaSparkContext(Spark.getContext())).take(totalLines).iterator(), tuple -> new KeyValue(tuple._1(), tuple._2()));
             } else if (InputFormat.class.isAssignableFrom(parserClass)) {
-                return IteratorUtils.limit(IteratorUtils.map(new InputFormatRDD().readMemoryRDD(configuration, memoryKey, new JavaSparkContext(Spark.getContext())).toLocalIterator(), tuple -> new KeyValue(tuple._1(), tuple._2())), totalLines);
+                return IteratorUtils.map(new InputFormatRDD().readMemoryRDD(configuration, memoryKey, new JavaSparkContext(Spark.getContext())).take(totalLines).iterator(), tuple -> new KeyValue(tuple._1(), tuple._2()));
             }
         } catch (final Exception e) {
             throw new IllegalArgumentException(e.getMessage(), e);
@@ -158,7 +162,7 @@ public final class SparkContextStorage implements Storage {
 
     @Override
     public Iterator<String> head(final String location, final int totalLines) {
-        return IteratorUtils.limit(IteratorUtils.map(JavaConversions.asJavaIterator(Spark.getRDD(location).toLocalIterator()), Object::toString), totalLines);
+        return IteratorUtils.map(Spark.getRDD(location).toJavaRDD().take(totalLines).iterator(), Object::toString);
     }
 
     // TODO: @Override

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/SparkGremlinSuite.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/SparkGremlinSuite.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/SparkGremlinSuite.java
new file mode 100644
index 0000000..60af63a
--- /dev/null
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/SparkGremlinSuite.java
@@ -0,0 +1,35 @@
+/*
+ * 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.tinkerpop.gremlin.spark;
+
+import org.apache.tinkerpop.gremlin.AbstractGremlinSuite;
+import org.apache.tinkerpop.gremlin.process.traversal.TraversalEngine;
+import org.apache.tinkerpop.gremlin.spark.structure.io.SparkContextStorageCheck;
+import org.junit.runners.model.InitializationError;
+import org.junit.runners.model.RunnerBuilder;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public class SparkGremlinSuite extends AbstractGremlinSuite {
+    public SparkGremlinSuite(final Class<?> klass, final RunnerBuilder builder) throws InitializationError {
+        super(klass, builder, new Class<?>[]{SparkContextStorageCheck.class}, new Class<?>[]{SparkContextStorageCheck.class}, true, TraversalEngine.Type.COMPUTER);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/SparkGremlinTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/SparkGremlinTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/SparkGremlinTest.java
new file mode 100644
index 0000000..cce9784
--- /dev/null
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/SparkGremlinTest.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tinkerpop.gremlin.spark;
+
+import org.apache.tinkerpop.gremlin.GraphProviderClass;
+import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
+import org.apache.tinkerpop.gremlin.spark.process.computer.SparkHadoopGraphProvider;
+import org.junit.runner.RunWith;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+@RunWith(SparkGremlinSuite.class)
+@GraphProviderClass(provider = SparkHadoopGraphProvider.class, graph = HadoopGraph.class)
+public class SparkGremlinTest {
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/SparkHadoopGremlinTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/SparkHadoopGremlinTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/SparkHadoopGremlinTest.java
new file mode 100644
index 0000000..bd4a68e
--- /dev/null
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/SparkHadoopGremlinTest.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tinkerpop.gremlin.spark;
+
+import org.apache.tinkerpop.gremlin.GraphProviderClass;
+import org.apache.tinkerpop.gremlin.hadoop.HadoopGremlinSuite;
+import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
+import org.apache.tinkerpop.gremlin.spark.process.computer.SparkHadoopGraphProvider;
+import org.junit.runner.RunWith;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+@RunWith(HadoopGremlinSuite.class)
+@GraphProviderClass(provider = SparkHadoopGraphProvider.class, graph = HadoopGraph.class)
+public class SparkHadoopGremlinTest {
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphProvider.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphProvider.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphProvider.java
index 484c55a..108d0ed 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphProvider.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphProvider.java
@@ -22,9 +22,11 @@ import org.apache.tinkerpop.gremlin.GraphProvider;
 import org.apache.tinkerpop.gremlin.LoadGraphWith;
 import org.apache.tinkerpop.gremlin.hadoop.Constants;
 import org.apache.tinkerpop.gremlin.hadoop.HadoopGraphProvider;
-import org.apache.tinkerpop.gremlin.hadoop.groovy.plugin.FileSystemStorageCheck;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.FileSystemStorageCheck;
 import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversalSource;
 import org.apache.tinkerpop.gremlin.process.traversal.engine.ComputerTraversalEngine;
+import org.apache.tinkerpop.gremlin.spark.structure.io.SparkContextStorageCheck;
+import org.apache.tinkerpop.gremlin.spark.structure.io.PersistedOutputRDD;
 import org.apache.tinkerpop.gremlin.spark.structure.io.ToyGraphInputRDD;
 import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.GryoSerializer;
 import org.apache.tinkerpop.gremlin.structure.Graph;
@@ -46,7 +48,12 @@ public final class SparkHadoopGraphProvider extends HadoopGraphProvider {
         config.put(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, true);  // this makes the test suite go really fast
         if (!test.equals(FileSystemStorageCheck.class) && null != loadGraphWith && RANDOM.nextBoolean()) {
             config.put(Constants.GREMLIN_SPARK_GRAPH_INPUT_RDD, ToyGraphInputRDD.class.getCanonicalName());
-            // config.put(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, InputRDDFormat.class.getCanonicalName());
+        }
+
+        // tests persisted RDDs
+        if (test.equals(SparkContextStorageCheck.class)) {
+            config.put(Constants.GREMLIN_SPARK_GRAPH_INPUT_RDD, ToyGraphInputRDD.class.getCanonicalName());
+            config.put(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
         }
         /// spark configuration
         config.put("spark.master", "local[4]");

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java
deleted file mode 100644
index 4eadd97..0000000
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java
+++ /dev/null
@@ -1,129 +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.tinkerpop.gremlin.spark.process.computer.groovy.plugin;
-
-import org.apache.commons.configuration.BaseConfiguration;
-import org.apache.commons.configuration.Configuration;
-import org.apache.tinkerpop.gremlin.TestHelper;
-import org.apache.tinkerpop.gremlin.groovy.util.TestableConsolePluginAcceptor;
-import org.apache.tinkerpop.gremlin.hadoop.Constants;
-import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.gryo.GryoInputFormat;
-import org.apache.tinkerpop.gremlin.process.computer.ranking.pagerank.PageRankVertexProgram;
-import org.apache.tinkerpop.gremlin.spark.AbstractSparkTest;
-import org.apache.tinkerpop.gremlin.spark.groovy.plugin.SparkGremlinPlugin;
-import org.apache.tinkerpop.gremlin.spark.process.computer.SparkHadoopGraphProvider;
-import org.apache.tinkerpop.gremlin.spark.structure.Spark;
-import org.apache.tinkerpop.gremlin.spark.structure.io.PersistedOutputRDD;
-import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.GryoSerializer;
-import org.apache.tinkerpop.gremlin.structure.Graph;
-import org.apache.tinkerpop.gremlin.structure.util.GraphFactory;
-import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.Iterator;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-public class SparkGremlinPluginTest extends AbstractSparkTest {
-
-    @Before
-    public void setupTest() {
-        try {
-            this.console = new TestableConsolePluginAcceptor();
-            final SparkGremlinPlugin plugin = new SparkGremlinPlugin();
-            plugin.pluginTo(this.console);
-            this.console.eval("import " + PageRankVertexProgram.class.getPackage().getName() + ".*");
-        } catch (final Exception e) {
-            throw new IllegalStateException(e.getMessage(), e);
-        }
-    }
-
-    ///////////////////
-
-    private TestableConsolePluginAcceptor console;
-
-    @Test
-    public void shouldSupportBasicRDDOperations() throws Exception {
-
-        final String root = TestHelper.makeTestDataDirectory(SparkGremlinPluginTest.class, "shouldSupportBasicRDDOperations");
-        final String rddName1 = TestHelper.makeTestDataDirectory(SparkGremlinPluginTest.class, "shouldSupportBasicRDDOperations", "graph-1");
-        final Configuration configuration = new BaseConfiguration();
-        configuration.setProperty("spark.master", "local[4]");
-        configuration.setProperty("spark.serializer", GryoSerializer.class.getCanonicalName());
-        configuration.setProperty(Graph.GRAPH, HadoopGraph.class.getName());
-        configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, SparkHadoopGraphProvider.PATHS.get("tinkerpop-modern.kryo"));
-        configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, GryoInputFormat.class.getCanonicalName());
-        configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
-        configuration.setProperty(Constants.GREMLIN_HADOOP_JARS_IN_DISTRIBUTED_CACHE, false);
-        configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddName1);
-        configuration.setProperty(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, true);
-        Graph graph = GraphFactory.open(configuration);
-
-        Spark.create("local[4]");
-
-        assertEquals(0, ((List<String>) this.console.eval("spark.ls()")).size());
-
-        this.console.addBinding("graph", graph);
-        this.console.eval("graph.compute(SparkGraphComputer).program(PageRankVertexProgram.build().iterations(1).create()).submit().get()");
-        assertEquals(1, ((List<String>) this.console.eval("spark.ls()")).size());
-        assertEquals(Constants.getGraphLocation(rddName1) + " [Memory Deserialized 1x Replicated]", ((List<String>) this.console.eval("spark.ls()")).get(0));
-
-        final String rddName2 = TestHelper.makeTestDataDirectory(SparkGremlinPluginTest.class, "shouldSupportBasicRDDOperations", "graph-2");
-        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + rddName2 + "')");
-        this.console.eval("graph.compute(SparkGraphComputer).program(PageRankVertexProgram.build().iterations(1).create()).submit().get()");
-        assertEquals(2, ((List<String>) this.console.eval("spark.ls()")).size());
-        assertTrue(((List<String>) this.console.eval("spark.ls()")).contains(Constants.getGraphLocation(rddName2) + " [Memory Deserialized 1x Replicated]"));
-
-        this.console.eval("spark.rm('" + Constants.getGraphLocation(rddName2) + "')");
-        assertEquals(1, ((List<String>) this.console.eval("spark.ls()")).size());
-        assertTrue(((List<String>) this.console.eval("spark.ls()")).contains(Constants.getGraphLocation(rddName1) + " [Memory Deserialized 1x Replicated]"));
-
-        assertEquals(6, IteratorUtils.count(((Iterator<Object>) this.console.eval("spark.head('" + Constants.getGraphLocation(rddName1) + "')"))));
-
-        this.console.eval("spark.rmr('" + root + "graph-*')");
-        assertEquals(0, ((List<String>) this.console.eval("spark.ls()")).size());
-
-        //////
-        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + Constants.getGraphLocation(rddName1) + "')");
-        this.console.eval("graph.compute(SparkGraphComputer).program(PageRankVertexProgram.build().iterations(1).create()).submit().get()");
-
-        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + Constants.getGraphLocation(rddName2) + "')");
-        this.console.eval("graph.compute(SparkGraphComputer).program(PageRankVertexProgram.build().iterations(1).create()).submit().get()");
-
-        final String rddName3 = TestHelper.makeTestDataDirectory(SparkGremlinPluginTest.class, "shouldSupportBasicRDDOperations", "x");
-        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + Constants.getGraphLocation(rddName3) + "')");
-        this.console.eval("graph.compute(SparkGraphComputer).program(PageRankVertexProgram.build().iterations(1).create()).submit().get()");
-
-        assertEquals(3, ((List<String>) this.console.eval("spark.ls()")).size());
-        this.console.eval("spark.rmr('" + root + "graph-*')");
-        assertEquals(1, ((List<String>) this.console.eval("spark.ls()")).size());
-        this.console.eval("spark.rmr('*')");
-        assertEquals(0, ((List<String>) this.console.eval("spark.ls()")).size());
-
-        //
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkHadoopGremlinPluginTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkHadoopGremlinPluginTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkHadoopGremlinPluginTest.java
deleted file mode 100644
index 63e7b7e..0000000
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkHadoopGremlinPluginTest.java
+++ /dev/null
@@ -1,33 +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.tinkerpop.gremlin.spark.process.computer.groovy.plugin;
-
-import org.apache.tinkerpop.gremlin.GraphProviderClass;
-import org.apache.tinkerpop.gremlin.hadoop.groovy.plugin.HadoopPluginSuite;
-import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
-import org.apache.tinkerpop.gremlin.spark.process.computer.SparkHadoopGraphProvider;
-import org.junit.runner.RunWith;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-@RunWith(HadoopPluginSuite.class)
-@GraphProviderClass(provider = SparkHadoopGraphProvider.class, graph = HadoopGraph.class)
-public class SparkHadoopGremlinPluginTest {
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageCheck.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageCheck.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageCheck.java
new file mode 100644
index 0000000..60b44ef
--- /dev/null
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageCheck.java
@@ -0,0 +1,71 @@
+/*
+ * 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.tinkerpop.gremlin.spark.structure.io;
+
+import org.apache.tinkerpop.gremlin.LoadGraphWith;
+import org.apache.tinkerpop.gremlin.hadoop.Constants;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.AbstractStorageCheck;
+import org.apache.tinkerpop.gremlin.spark.structure.Spark;
+import org.apache.tinkerpop.gremlin.spark.structure.io.PersistedInputRDD;
+import org.apache.tinkerpop.gremlin.spark.structure.io.SparkContextStorage;
+import org.apache.tinkerpop.gremlin.structure.io.Storage;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertFalse;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public class SparkContextStorageCheck extends AbstractStorageCheck {
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+        SparkContextStorage.open("local[4]");
+        Spark.close();
+    }
+
+    @Test
+    @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+    public void shouldSupportHeadMethods() throws Exception {
+        final Storage storage = SparkContextStorage.open("local[4]");
+        final String outputLocation = graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION);
+        assertFalse(storage.exists(outputLocation));
+        super.checkHeadMethods(storage, graph.configuration().getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION), outputLocation, PersistedInputRDD.class, PersistedInputRDD.class);
+    }
+
+    @Test
+    @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+    public void shouldSupportRemoveAndListMethods() throws Exception {
+        final Storage storage = SparkContextStorage.open("local[4]");
+        final String outputLocation = graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION);
+        super.checkRemoveAndListMethods(storage, outputLocation);
+    }
+
+    @Test
+    @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+    public void shouldSupportCopyMethods() throws Exception {
+        final Storage storage = SparkContextStorage.open("local[4]");
+        final String outputLocation = graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION);
+        final String newOutputLocation = "new-location-for-copy";
+        super.checkCopyMethods(storage, outputLocation, newOutputLocation);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b0f3e4a9/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageTest.java
deleted file mode 100644
index 8c99c9e..0000000
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageTest.java
+++ /dev/null
@@ -1,74 +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.tinkerpop.gremlin.spark.structure.io;
-
-import org.apache.commons.configuration.Configuration;
-import org.apache.tinkerpop.gremlin.hadoop.Constants;
-import org.apache.tinkerpop.gremlin.process.computer.ComputerResult;
-import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.ClusterCountMapReduce;
-import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.PeerPressureVertexProgram;
-import org.apache.tinkerpop.gremlin.spark.AbstractSparkTest;
-import org.apache.tinkerpop.gremlin.spark.process.computer.SparkGraphComputer;
-import org.apache.tinkerpop.gremlin.spark.process.computer.SparkHadoopGraphProvider;
-import org.apache.tinkerpop.gremlin.structure.Graph;
-import org.apache.tinkerpop.gremlin.structure.io.Storage;
-import org.apache.tinkerpop.gremlin.structure.util.GraphFactory;
-import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
-import org.junit.Test;
-
-import java.util.UUID;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-public class SparkContextStorageTest extends AbstractSparkTest {
-
-    @Test
-    public void shouldPersistGraphAndMemory() throws Exception {
-        final String outputLocation = "target/test-output/" + UUID.randomUUID();
-        final Configuration configuration = getBaseConfiguration(SparkHadoopGraphProvider.PATHS.get("tinkerpop-modern.kryo"));
-        configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
-        configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, outputLocation);
-        configuration.setProperty(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, true);
-        /////
-        Graph graph = GraphFactory.open(configuration);
-        final ComputerResult result = graph.compute(SparkGraphComputer.class).program(PeerPressureVertexProgram.build().create(graph)).mapReduce(ClusterCountMapReduce.build().memoryKey("clusterCount").create()).submit().get();
-        /////
-        final Storage storage = SparkContextStorage.open("local[4]");
-
-        assertEquals(2, storage.ls().size());
-        // TEST GRAPH PERSISTENCE
-        assertTrue(storage.exists(Constants.getGraphLocation(outputLocation)));
-        assertEquals(6, IteratorUtils.count(storage.head(outputLocation, PersistedInputRDD.class)));
-        assertEquals(6, result.graph().traversal().V().count().next().longValue());
-        assertEquals(0, result.graph().traversal().E().count().next().longValue());
-        assertEquals(6, result.graph().traversal().V().values("name").count().next().longValue());
-        assertEquals(6, result.graph().traversal().V().values(PeerPressureVertexProgram.CLUSTER).count().next().longValue());
-        /////
-        // TEST MEMORY PERSISTENCE
-        assertEquals(2, (int) result.memory().get("clusterCount"));
-        assertTrue(storage.exists(Constants.getMemoryLocation(outputLocation, "clusterCount")));
-        assertEquals(2, storage.head(outputLocation, "clusterCount", PersistedInputRDD.class).next().getValue());
-    }
-
-}


[05/30] incubator-tinkerpop git commit: Greatly greatly simplified Hadoop OLTP and interactions with HDFS and SparkContext. The trend -- dir/~g for graphs and dir/x for memory. A consistent persistence schema makes everything so much simpler. I always as

Posted by dk...@apache.org.
Greatly greatly simplified Hadoop OLTP and interactions with HDFS and SparkContext. The trend -- dir/~g for graphs and dir/x for memory. A consistent persistence schema makes everything so much simpler. I always assumed this would be all generalized/blah/blah. Never actually did it so, hell, stick with a consistent schema and watch the code just fall away.


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

Branch: refs/heads/TINKERPOP-320
Commit: b4d8e9608d4eca3ae177b28fe588518a9d77506c
Parents: 2c0d327
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Dec 9 15:58:50 2015 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Wed Dec 9 15:58:50 2015 -0700

----------------------------------------------------------------------
 .../tinkerpop/gremlin/structure/io/Storage.java |  44 +++-----
 .../tinkerpop/gremlin/hadoop/Constants.java     |  16 ++-
 .../groovy/plugin/HadoopGremlinPlugin.java      |   6 +-
 .../gremlin/hadoop/structure/HadoopGraph.java   |   4 +-
 .../hadoop/structure/hdfs/HDFSTools.java        | 113 -------------------
 .../structure/hdfs/HadoopEdgeIterator.java      |  83 --------------
 .../structure/hdfs/HadoopElementIterator.java   |  74 ------------
 .../structure/hdfs/HadoopVertexIterator.java    |  82 --------------
 .../hadoop/structure/hdfs/HiddenFileFilter.java |  44 --------
 .../hadoop/structure/hdfs/TextIterator.java     |  91 ---------------
 .../hadoop/structure/io/FileSystemStorage.java  | 106 +++++++++++++----
 .../hadoop/structure/io/HadoopEdgeIterator.java |  79 +++++++++++++
 .../structure/io/HadoopElementIterator.java     |  75 ++++++++++++
 .../structure/io/HadoopVertexIterator.java      |  78 +++++++++++++
 .../hadoop/structure/io/HiddenFileFilter.java   |  44 ++++++++
 .../structure/io/ObjectWritableIterator.java    |  12 +-
 .../hadoop/structure/io/TextIterator.java       |  91 +++++++++++++++
 .../structure/io/VertexWritableIterator.java    |  10 +-
 .../groovy/plugin/FileSystemStorageCheck.java   |  63 +++++++++++
 .../groovy/plugin/GraphMemoryHDFSCheck.java     |  64 -----------
 .../groovy/plugin/HadoopGremlinPluginCheck.java |   4 +-
 .../hadoop/groovy/plugin/HadoopPluginSuite.java |   2 +-
 .../spark/groovy/plugin/SparkGremlinPlugin.java |   3 +-
 .../process/computer/SparkGraphComputer.java    |   4 +
 .../spark/structure/io/PersistedOutputRDD.java  |   3 +-
 .../spark/structure/io/SparkContextStorage.java |  53 ++++++++-
 .../structure/io/GraphMemorySparkTest.java      |  75 ------------
 .../io/PersistedInputOutputRDDTest.java         |  14 +--
 .../structure/io/SparkContextStorageTest.java   |  74 ++++++++++++
 29 files changed, 699 insertions(+), 712 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
index 1f1bcf4..3b69ff2 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
@@ -19,6 +19,9 @@
 
 package org.apache.tinkerpop.gremlin.structure.io;
 
+import org.apache.tinkerpop.gremlin.process.computer.KeyValue;
+import org.apache.tinkerpop.gremlin.structure.Vertex;
+
 import java.util.Iterator;
 import java.util.List;
 
@@ -41,42 +44,21 @@ public interface Storage {
 
     public boolean rmr(final String location);
 
-    public <V> Iterator<V> head(final String location, final int totalLines, final Class<V> objectClass);
+    public Iterator<String> head(final String location, final int totalLines);
 
-    public default Iterator<Object> head(final String location) {
-        return this.head(location, Object.class);
+    public default Iterator<String> head(final String location) {
+        return this.head(location, Integer.MAX_VALUE);
     }
 
-    public default Iterator<Object> head(final String location, final int totalLines) {
-        return this.head(location, totalLines, Object.class);
-    }
+    public Iterator<Vertex> headGraph(final String location, final int totalLines, final Class parserClass);
 
-    public default <V> Iterator<V> head(final String location, final Class<V> objectClass) {
-        return this.head(location, Integer.MAX_VALUE, objectClass);
+    public default Iterator<Vertex> headGraph(final String location, final Class parserClass) {
+        return this.headGraph(location, Integer.MAX_VALUE, parserClass);
     }
 
-  /*
-
-        FileSystem.metaClass.copyToLocal = { final String from, final String to ->
-            return ((FileSystem) delegate).copyToLocalFile(new Path(from), new Path(to));
-        }
-
-        FileSystem.metaClass.copyFromLocal = { final String from, final String to ->
-            return ((FileSystem) delegate).copyFromLocalFile(new Path(from), new Path(to));
-        }
+    public <K, V> Iterator<KeyValue<K, V>> headMemory(final String location, final String memoryKey, final int totalLines, final Class parserClass);
 
-        FileSystem.metaClass.mergeToLocal = { final String from, final String to ->
-            final FileSystem fs = (FileSystem) delegate;
-            final FileSystem local = FileSystem.getLocal(new Configuration());
-            final FSDataOutputStream outA = local.create(new Path(to));
-
-            HDFSTools.getAllFilePaths(fs, new Path(from), HiddenFileFilter.instance()).each {
-                final FSDataInputStream inA = fs.open(it);
-                IOUtils.copyBytes(inA, outA, 8192);
-                inA.close();
-            }
-            outA.close();
-        }
-
-     */
+    public default <K, V> Iterator<KeyValue<K, V>> headMemory(final String location, final String memoryKey, final Class parserClass) {
+        return this.headMemory(location, memoryKey, Integer.MAX_VALUE, parserClass);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/Constants.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/Constants.java
index 8678441..4a91106 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/Constants.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/Constants.java
@@ -19,6 +19,9 @@
 package org.apache.tinkerpop.gremlin.hadoop;
 
 import org.apache.tinkerpop.gremlin.structure.Graph;
+import org.apache.tinkerpop.gremlin.structure.io.Storage;
+
+import java.util.Optional;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
@@ -54,10 +57,19 @@ public final class Constants {
     public static final String GREMLIN_SPARK_PERSIST_CONTEXT = "gremlin.spark.persistContext";
 
     public static String getGraphLocation(final String location) {
-        return location + "/" + Constants.HIDDEN_G;
+        return location.endsWith("/") ? location + Constants.HIDDEN_G : location + "/" + Constants.HIDDEN_G;
     }
 
     public static String getMemoryLocation(final String location, final String memoryKey) {
-        return location + "/" + memoryKey;
+        return location.endsWith("/") ? location + memoryKey : location + "/" + memoryKey;
+    }
+
+    public static Optional<String> getSearchGraphLocation(final String location, final Storage storage) {
+        if (storage.exists(getGraphLocation(location)))
+            return Optional.of(getGraphLocation(location));
+        else if (storage.exists(location))
+            return Optional.of(location);
+        else
+            return Optional.empty();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java
index 55ba020..c50d226 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPlugin.java
@@ -29,7 +29,6 @@ import org.apache.tinkerpop.gremlin.hadoop.Constants;
 import org.apache.tinkerpop.gremlin.hadoop.process.computer.mapreduce.MapReduceGraphComputer;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopConfiguration;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
-import org.apache.tinkerpop.gremlin.hadoop.structure.hdfs.HDFSTools;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.FileSystemStorage;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.graphson.GraphSONInputFormat;
@@ -65,7 +64,6 @@ public final class HadoopGremlinPlugin extends AbstractGremlinPlugin {
         add(IMPORT_SPACE + GryoInputFormat.class.getPackage().getName() + DOT_STAR);
         add(IMPORT_SPACE + GraphSONInputFormat.class.getPackage().getName() + DOT_STAR);
         add(IMPORT_SPACE + ScriptInputFormat.class.getPackage().getName() + DOT_STAR);
-        add(IMPORT_SPACE + HDFSTools.class.getPackage().getName() + DOT_STAR);
         ////
         add(IMPORT_SPACE + MapReduceGraphComputer.class.getPackage().getName() + DOT_STAR);
     }};
@@ -79,8 +77,8 @@ public final class HadoopGremlinPlugin extends AbstractGremlinPlugin {
     public void afterPluginTo(final PluginAcceptor pluginAcceptor) throws PluginInitializationException, IllegalEnvironmentException {
         pluginAcceptor.addImports(IMPORTS);
         try {
-            pluginAcceptor.addBinding("hdfs", new FileSystemStorage(FileSystem.get(new Configuration())));
-            pluginAcceptor.addBinding("local", new FileSystemStorage(FileSystem.getLocal(new Configuration())));
+            pluginAcceptor.addBinding("hdfs", FileSystemStorage.open(FileSystem.get(new Configuration())));
+            pluginAcceptor.addBinding("local", FileSystemStorage.open(FileSystem.getLocal(new Configuration())));
             if (null == System.getenv(Constants.HADOOP_GREMLIN_LIBS))
                 HadoopGraph.LOGGER.warn("Be sure to set the environmental variable: " + Constants.HADOOP_GREMLIN_LIBS);
             else

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/HadoopGraph.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/HadoopGraph.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/HadoopGraph.java
index a9f758c..22f42f4 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/HadoopGraph.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/HadoopGraph.java
@@ -26,8 +26,8 @@ import org.apache.hadoop.mapred.OutputFormat;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.tinkerpop.gremlin.hadoop.Constants;
 import org.apache.tinkerpop.gremlin.hadoop.process.computer.AbstractHadoopGraphComputer;
-import org.apache.tinkerpop.gremlin.hadoop.structure.hdfs.HadoopEdgeIterator;
-import org.apache.tinkerpop.gremlin.hadoop.structure.hdfs.HadoopVertexIterator;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.HadoopEdgeIterator;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.HadoopVertexIterator;
 import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
 import org.apache.tinkerpop.gremlin.process.computer.GraphComputer;
 import org.apache.tinkerpop.gremlin.structure.Edge;

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HDFSTools.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HDFSTools.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HDFSTools.java
deleted file mode 100644
index e4da530..0000000
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HDFSTools.java
+++ /dev/null
@@ -1,113 +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.tinkerpop.gremlin.hadoop.structure.hdfs;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-public final class HDFSTools {
-
-    private static final String FORWARD_SLASH = "/";
-    private static final String FORWARD_ASTERISK = "/*";
-
-    private HDFSTools() {
-    }
-
-    public static long getFileSize(final FileSystem fs, final Path path, final PathFilter filter) throws IOException {
-        long totalSize = 0l;
-        for (final Path p : getAllFilePaths(fs, path, filter)) {
-            totalSize = totalSize + fs.getFileStatus(p).getLen();
-        }
-        return totalSize;
-    }
-
-    public static List<Path> getAllFilePaths(final FileSystem fs, Path path, final PathFilter filter) throws IOException {
-        if (null == path) path = fs.getHomeDirectory();
-        if (path.toString().equals(FORWARD_SLASH)) path = new Path("");
-
-        final List<Path> paths = new ArrayList<Path>();
-        if (fs.isFile(path))
-            paths.add(path);
-        else {
-            for (final FileStatus status : fs.globStatus(new Path(path + FORWARD_ASTERISK), filter)) {
-                final Path next = status.getPath();
-                paths.addAll(getAllFilePaths(fs, next, filter));
-            }
-        }
-        return paths;
-    }
-
-
-    public static void decompressPath(final FileSystem fs, final String in, final String out, final String compressedFileSuffix, final boolean deletePrevious) throws IOException {
-        final Path inPath = new Path(in);
-
-        if (fs.isFile(inPath))
-            HDFSTools.decompressFile(fs, in, out, deletePrevious);
-        else {
-            final Path outPath = new Path(out);
-            if (!fs.exists(outPath))
-                fs.mkdirs(outPath);
-            for (final Path path : FileUtil.stat2Paths(fs.globStatus(new Path(in + FORWARD_ASTERISK)))) {
-                if (path.getName().endsWith(compressedFileSuffix))
-                    HDFSTools.decompressFile(fs, path.toString(), outPath.toString() + FORWARD_SLASH + path.getName().split("\\.")[0], deletePrevious);
-            }
-        }
-    }
-
-    public static void decompressFile(final FileSystem fs, final String inFile, final String outFile, boolean deletePrevious) throws IOException {
-        final Path inPath = new Path(inFile);
-        final Path outPath = new Path(outFile);
-        final CompressionCodecFactory factory = new CompressionCodecFactory(new Configuration());
-        final CompressionCodec codec = factory.getCodec(inPath);
-        final OutputStream out = fs.create(outPath);
-        final InputStream in = codec.createInputStream(fs.open(inPath));
-        IOUtils.copyBytes(in, out, 8192);
-        IOUtils.closeStream(in);
-        IOUtils.closeStream(out);
-
-        if (deletePrevious)
-            fs.delete(new Path(inFile), true);
-
-    }
-
-    public static boolean globDelete(final FileSystem fs, final String path, final boolean recursive) throws IOException {
-        boolean deleted = false;
-        for (final Path p : FileUtil.stat2Paths(fs.globStatus(new Path(path)))) {
-            fs.delete(p, recursive);
-            deleted = true;
-        }
-        return deleted;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HadoopEdgeIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HadoopEdgeIterator.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HadoopEdgeIterator.java
deleted file mode 100644
index 59a4d2c..0000000
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HadoopEdgeIterator.java
+++ /dev/null
@@ -1,83 +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.tinkerpop.gremlin.hadoop.structure.hdfs;
-
-import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopEdge;
-import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
-import org.apache.tinkerpop.gremlin.process.traversal.util.FastNoSuchElementException;
-import org.apache.tinkerpop.gremlin.structure.Direction;
-import org.apache.tinkerpop.gremlin.structure.Edge;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Iterator;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-public final class HadoopEdgeIterator extends HadoopElementIterator<Edge> {
-
-    private Iterator<Edge> edgeIterator = Collections.emptyIterator();
-
-    public HadoopEdgeIterator(final HadoopGraph graph) throws IOException {
-        super(graph);
-    }
-
-    @Override
-    public Edge next() {
-        try {
-            while (true) {
-                if (this.edgeIterator.hasNext())
-                    return new HadoopEdge(this.edgeIterator.next(), this.graph);
-                if (this.readers.isEmpty())
-                    throw FastNoSuchElementException.instance();
-                if (this.readers.peek().nextKeyValue()) {
-                    this.edgeIterator = this.readers.peek().getCurrentValue().get().edges(Direction.OUT);
-                } else {
-                    this.readers.remove();
-                }
-            }
-        } catch (Exception e) {
-            throw new RuntimeException(e.getMessage(), e);
-        }
-    }
-
-    @Override
-    public boolean hasNext() {
-        try {
-            while (true) {
-                if (this.edgeIterator.hasNext())
-                    return true;
-                if (this.readers.isEmpty())
-                    return false;
-                if (this.readers.peek().nextKeyValue()) {
-                    this.edgeIterator = this.readers.peek().getCurrentValue().get().edges(Direction.OUT);
-                } else {
-                    this.readers.remove();
-                }
-            }
-        } catch (Exception e) {
-            throw new RuntimeException(e.getMessage(), e);
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HadoopElementIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HadoopElementIterator.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HadoopElementIterator.java
deleted file mode 100644
index 45f3c55..0000000
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HadoopElementIterator.java
+++ /dev/null
@@ -1,74 +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.tinkerpop.gremlin.hadoop.structure.hdfs;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobID;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.task.JobContextImpl;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.apache.tinkerpop.gremlin.hadoop.Constants;
-import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
-import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
-import org.apache.tinkerpop.gremlin.structure.Element;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Queue;
-import java.util.UUID;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-public abstract class HadoopElementIterator<E extends Element> implements Iterator<E> {
-
-    protected final HadoopGraph graph;
-    protected final Queue<RecordReader<NullWritable, VertexWritable>> readers = new LinkedList<>();
-
-    public HadoopElementIterator(final HadoopGraph graph) throws IOException {
-        try {
-            this.graph = graph;
-            final Configuration configuration = ConfUtil.makeHadoopConfiguration(this.graph.configuration());
-            final InputFormat<NullWritable, VertexWritable> inputFormat = this.graph.configuration().getGraphInputFormat().getConstructor().newInstance();
-            if (inputFormat instanceof FileInputFormat) {
-                if (!this.graph.configuration().containsKey(Constants.GREMLIN_HADOOP_INPUT_LOCATION))
-                    return; // there is no input location and thus, no data (empty graph)
-                if (!FileSystem.get(configuration).exists(new Path(this.graph.configuration().getInputLocation())))
-                    return; // there is no data at the input location (empty graph)
-                configuration.set(Constants.MAPREDUCE_INPUT_FILEINPUTFORMAT_INPUTDIR, this.graph.configuration().getInputLocation());
-            }
-            final List<InputSplit> splits = inputFormat.getSplits(new JobContextImpl(configuration, new JobID(UUID.randomUUID().toString(), 1)));
-            for (final InputSplit split : splits) {
-                this.readers.add(inputFormat.createRecordReader(split, new TaskAttemptContextImpl(configuration, new TaskAttemptID())));
-            }
-        } catch (Exception e) {
-            throw new IllegalStateException(e.getMessage(), e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HadoopVertexIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HadoopVertexIterator.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HadoopVertexIterator.java
deleted file mode 100644
index 8f13c59..0000000
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HadoopVertexIterator.java
+++ /dev/null
@@ -1,82 +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.tinkerpop.gremlin.hadoop.structure.hdfs;
-
-import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
-import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopVertex;
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
-import org.apache.tinkerpop.gremlin.process.traversal.util.FastNoSuchElementException;
-import org.apache.tinkerpop.gremlin.structure.Vertex;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.InputFormat;
-
-import java.io.IOException;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-public final class HadoopVertexIterator extends HadoopElementIterator<Vertex> {
-
-    private HadoopVertex nextVertex = null;
-
-    public HadoopVertexIterator(final HadoopGraph graph) throws IOException {
-        super(graph);
-    }
-
-    @Override
-    public Vertex next() {
-        try {
-            if (this.nextVertex != null) {
-                final Vertex temp = this.nextVertex;
-                this.nextVertex = null;
-                return temp;
-            } else {
-                while (!this.readers.isEmpty()) {
-                    if (this.readers.peek().nextKeyValue())
-                        return new HadoopVertex(this.readers.peek().getCurrentValue().get(), this.graph);
-                    else
-                        this.readers.remove();
-                }
-            }
-            throw FastNoSuchElementException.instance();
-        } catch (final Exception e) {
-            throw new IllegalStateException(e.getMessage(), e);
-        }
-    }
-
-    @Override
-    public boolean hasNext() {
-        try {
-            if (null != this.nextVertex) return true;
-            else {
-                while (!this.readers.isEmpty()) {
-                    if (this.readers.peek().nextKeyValue()) {
-                        this.nextVertex = new HadoopVertex(this.readers.peek().getCurrentValue().get(), this.graph);
-                        return true;
-                    } else
-                        this.readers.remove();
-                }
-            }
-        } catch (final Exception e) {
-            throw new IllegalStateException(e.getMessage(), e);
-        }
-        return false;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HiddenFileFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HiddenFileFilter.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HiddenFileFilter.java
deleted file mode 100644
index 4ea0958..0000000
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/HiddenFileFilter.java
+++ /dev/null
@@ -1,44 +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.tinkerpop.gremlin.hadoop.structure.hdfs;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-public final class HiddenFileFilter implements PathFilter {
-
-    private static final HiddenFileFilter INSTANCE = new HiddenFileFilter();
-
-    private HiddenFileFilter() {
-
-    }
-
-    @Override
-    public boolean accept(final Path path) {
-        final String name = path.getName();
-        return !name.startsWith("_") && !name.startsWith(".");
-    }
-
-    public static HiddenFileFilter instance() {
-        return INSTANCE;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/TextIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/TextIterator.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/TextIterator.java
deleted file mode 100644
index c8424f6..0000000
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/hdfs/TextIterator.java
+++ /dev/null
@@ -1,91 +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.tinkerpop.gremlin.hadoop.structure.hdfs;
-
-import org.apache.tinkerpop.gremlin.process.traversal.util.FastNoSuchElementException;
-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 java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.Queue;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-public final class TextIterator implements Iterator<String> {
-
-    private String line;
-    private boolean available = false;
-    private final Queue<BufferedReader> readers = new LinkedList<>();
-
-    public TextIterator(final Configuration configuration, final Path path) throws IOException {
-        final FileSystem fs = FileSystem.get(configuration);
-        for (final FileStatus status : fs.listStatus(path, HiddenFileFilter.instance())) {
-            this.readers.add(new BufferedReader(new InputStreamReader(fs.open(status.getPath()))));
-        }
-    }
-
-    @Override
-    public boolean hasNext() {
-        try {
-            if (this.available) {
-                return true;
-            } else {
-                while (true) {
-                    if (this.readers.isEmpty())
-                        return false;
-                    if ((this.line = this.readers.peek().readLine()) != null) {
-                        this.available = true;
-                        return true;
-                    } else
-                        this.readers.remove();
-                }
-            }
-        } catch (final IOException e) {
-            throw new IllegalStateException(e.getMessage(), e);
-        }
-    }
-
-    @Override
-    public String next() {
-        try {
-            if (this.available) {
-                this.available = false;
-                return this.line;
-            } else {
-                while (true) {
-                    if (this.readers.isEmpty())
-                        throw FastNoSuchElementException.instance();
-                    if ((this.line = this.readers.peek().readLine()) != null) {
-                        return this.line;
-                    } else
-                        this.readers.remove();
-                }
-            }
-        } catch (final IOException e) {
-            throw new IllegalStateException(e.getMessage(), e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
index 56dfe52..5d3995c 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
@@ -19,6 +19,7 @@
 
 package org.apache.tinkerpop.gremlin.hadoop.structure.io;
 
+import org.apache.commons.configuration.BaseConfiguration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -26,16 +27,20 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.Writable;
-import org.apache.tinkerpop.gremlin.hadoop.structure.hdfs.HDFSTools;
-import org.apache.tinkerpop.gremlin.hadoop.structure.hdfs.HiddenFileFilter;
-import org.apache.tinkerpop.gremlin.hadoop.structure.hdfs.TextIterator;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.tinkerpop.gremlin.hadoop.Constants;
+import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
+import org.apache.tinkerpop.gremlin.process.computer.KeyValue;
+import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.structure.io.Storage;
 import org.apache.tinkerpop.gremlin.structure.util.StringFactory;
 import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.stream.Collectors;
@@ -48,13 +53,31 @@ public final class FileSystemStorage implements Storage {
 
     private static final String SPACE = " ";
     private static final String D_SPACE = "(D) ";
+    private static final String FORWARD_SLASH = "/";
+    private static final String FORWARD_ASTERISK = "/*";
 
     private final FileSystem fs;
 
-    public FileSystemStorage(final FileSystem fileSystem) {
+    private FileSystemStorage(final FileSystem fileSystem) {
         this.fs = fileSystem;
     }
 
+    public static FileSystemStorage open() {
+        return FileSystemStorage.open(new Configuration());
+    }
+
+    public static FileSystemStorage open(final Configuration configuration) {
+        try {
+            return new FileSystemStorage(FileSystem.get(configuration));
+        } catch (final IOException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+
+    public static FileSystemStorage open(final FileSystem fileSystem) {
+        return new FileSystemStorage(fileSystem);
+    }
+
     private static String fileStatusString(final FileStatus status) {
         StringBuilder s = new StringBuilder();
         s.append(status.getPermission()).append(" ");
@@ -113,7 +136,7 @@ public final class FileSystemStorage implements Storage {
     @Override
     public boolean rm(final String location) {
         try {
-            return HDFSTools.globDelete(this.fs, location, false);
+            return FileSystemStorage.globDelete(this.fs, location, false);
         } catch (final IOException e) {
             throw new IllegalStateException(e.getMessage(), e);
         }
@@ -122,35 +145,53 @@ public final class FileSystemStorage implements Storage {
     @Override
     public boolean rmr(final String location) {
         try {
-            return HDFSTools.globDelete(this.fs, location, true);
+            return FileSystemStorage.globDelete(this.fs, location, true);
         } catch (final IOException e) {
             throw new IllegalStateException(e.getMessage(), e);
         }
     }
 
     @Override
-    public <V> Iterator<V> head(final String location, final int totalLines, final Class<V> objectClass) {
-        return headMaker(this.fs, location, totalLines, (Class<? extends Writable>) objectClass);
+    public Iterator<String> head(final String location, final int totalLines) {
+        try {
+            return IteratorUtils.limit((Iterator) new TextIterator(fs.getConf(), new Path(location)), totalLines);
+        } catch (final IOException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
     }
 
     @Override
-    public String toString() {
-        return StringFactory.storageString(this.fs.toString());
+    public Iterator<Vertex> headGraph(final String location, final int totalLines, final Class parserClass) {
+        final org.apache.commons.configuration.Configuration configuration = new BaseConfiguration();
+        configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, Constants.getSearchGraphLocation(location, this).get());
+        configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, parserClass.getCanonicalName());
+        try {
+            if (InputFormat.class.isAssignableFrom(parserClass))
+                return IteratorUtils.limit(new HadoopVertexIterator(HadoopGraph.open(configuration)), totalLines);
+        } catch (final IOException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+        throw new IllegalArgumentException("The provided parser class must be an " + InputFormat.class.getCanonicalName() + ": " + parserClass.getCanonicalName());
+
     }
 
-    private static Iterator headMaker(final FileSystem fs, final String path, final int totalLines, final Class<? extends Writable> writableClass) {
+    @Override
+    public <K, V> Iterator<KeyValue<K, V>> headMemory(final String location, final String memoryKey, final int totalLines, final Class parserClass) {
+        if (!parserClass.equals(SequenceFileInputFormat.class))
+            throw new IllegalArgumentException("Only " + SequenceFileInputFormat.class.getCanonicalName() + " memories are supported");
+        final Configuration configuration = new Configuration();
         try {
-            if (writableClass.equals(ObjectWritable.class))
-                return IteratorUtils.limit(new ObjectWritableIterator(fs.getConf(), new Path(path)), totalLines);
-            else if (writableClass.equals(VertexWritable.class))
-                return IteratorUtils.limit(new VertexWritableIterator(fs.getConf(), new Path(path)), totalLines);
-            else
-                return IteratorUtils.limit(new TextIterator(fs.getConf(), new Path(path)), totalLines);
+            return IteratorUtils.limit((Iterator) new ObjectWritableIterator(configuration, new Path(Constants.getMemoryLocation(location, memoryKey))), totalLines);
         } catch (final IOException e) {
             throw new IllegalStateException(e.getMessage(), e);
         }
     }
 
+    @Override
+    public String toString() {
+        return StringFactory.storageString(this.fs.toString());
+    }
+
     /////////
 
     public void copyToLocal(final String fromLocation, final String toLocation) {
@@ -173,7 +214,7 @@ public final class FileSystemStorage implements Storage {
         try {
             final FileSystem local = FileSystem.getLocal(new Configuration());
             final FSDataOutputStream outA = local.create(new Path(toLocation));
-            for (final Path path : HDFSTools.getAllFilePaths(fs, new Path(fromLocation), HiddenFileFilter.instance())) {
+            for (final Path path : FileSystemStorage.getAllFilePaths(fs, new Path(fromLocation), HiddenFileFilter.instance())) {
                 final FSDataInputStream inA = fs.open(path);
                 IOUtils.copyBytes(inA, outA, 8192);
                 inA.close();
@@ -183,4 +224,31 @@ public final class FileSystemStorage implements Storage {
             throw new IllegalStateException(e.getMessage(), e);
         }
     }
+
+    ////////////
+
+    private static boolean globDelete(final FileSystem fs, final String path, final boolean recursive) throws IOException {
+        boolean deleted = false;
+        for (final Path p : FileUtil.stat2Paths(fs.globStatus(new Path(path)))) {
+            fs.delete(p, recursive);
+            deleted = true;
+        }
+        return deleted;
+    }
+
+    private static List<Path> getAllFilePaths(final FileSystem fs, Path path, final PathFilter filter) throws IOException {
+        if (null == path) path = fs.getHomeDirectory();
+        if (path.toString().equals(FORWARD_SLASH)) path = new Path("");
+
+        final List<Path> paths = new ArrayList<Path>();
+        if (fs.isFile(path))
+            paths.add(path);
+        else {
+            for (final FileStatus status : fs.globStatus(new Path(path + FORWARD_ASTERISK), filter)) {
+                final Path next = status.getPath();
+                paths.addAll(getAllFilePaths(fs, next, filter));
+            }
+        }
+        return paths;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopEdgeIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopEdgeIterator.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopEdgeIterator.java
new file mode 100644
index 0000000..8f5452f
--- /dev/null
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopEdgeIterator.java
@@ -0,0 +1,79 @@
+/*
+ * 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.tinkerpop.gremlin.hadoop.structure.io;
+
+import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopEdge;
+import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
+import org.apache.tinkerpop.gremlin.process.traversal.util.FastNoSuchElementException;
+import org.apache.tinkerpop.gremlin.structure.Direction;
+import org.apache.tinkerpop.gremlin.structure.Edge;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public final class HadoopEdgeIterator extends HadoopElementIterator<Edge> {
+
+    private Iterator<Edge> edgeIterator = Collections.emptyIterator();
+
+    public HadoopEdgeIterator(final HadoopGraph graph) throws IOException {
+        super(graph);
+    }
+
+    @Override
+    public Edge next() {
+        try {
+            while (true) {
+                if (this.edgeIterator.hasNext())
+                    return new HadoopEdge(this.edgeIterator.next(), this.graph);
+                if (this.readers.isEmpty())
+                    throw FastNoSuchElementException.instance();
+                if (this.readers.peek().nextKeyValue()) {
+                    this.edgeIterator = this.readers.peek().getCurrentValue().get().edges(Direction.OUT);
+                } else {
+                    this.readers.remove().close();
+                }
+            }
+        } catch (final Exception e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public boolean hasNext() {
+        try {
+            while (true) {
+                if (this.edgeIterator.hasNext())
+                    return true;
+                if (this.readers.isEmpty())
+                    return false;
+                if (this.readers.peek().nextKeyValue()) {
+                    this.edgeIterator = this.readers.peek().getCurrentValue().get().edges(Direction.OUT);
+                } else {
+                    this.readers.remove().close();
+                }
+            }
+        } catch (final Exception e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopElementIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopElementIterator.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopElementIterator.java
new file mode 100644
index 0000000..9196ce3
--- /dev/null
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopElementIterator.java
@@ -0,0 +1,75 @@
+/*
+ * 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.tinkerpop.gremlin.hadoop.structure.io;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.task.JobContextImpl;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.tinkerpop.gremlin.hadoop.Constants;
+import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.FileSystemStorage;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
+import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
+import org.apache.tinkerpop.gremlin.structure.Element;
+import org.apache.tinkerpop.gremlin.structure.io.Storage;
+
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+import java.util.UUID;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public abstract class HadoopElementIterator<E extends Element> implements Iterator<E> {
+
+    protected final HadoopGraph graph;
+    protected final Queue<RecordReader<NullWritable, VertexWritable>> readers = new LinkedList<>();
+
+    public HadoopElementIterator(final HadoopGraph graph) {
+        try {
+            this.graph = graph;
+            final Configuration configuration = ConfUtil.makeHadoopConfiguration(this.graph.configuration());
+            final InputFormat<NullWritable, VertexWritable> inputFormat = this.graph.configuration().getGraphInputFormat().getConstructor().newInstance();
+            if (inputFormat instanceof FileInputFormat) {
+                final Storage storage = FileSystemStorage.open(configuration);
+
+                if (!this.graph.configuration().containsKey(Constants.GREMLIN_HADOOP_INPUT_LOCATION))
+                    return; // there is no input location and thus, no data (empty graph)
+                if (!Constants.getSearchGraphLocation(this.graph.configuration().getInputLocation(), storage).isPresent())
+                    return; // there is no data at the input location (empty graph)
+                configuration.set(Constants.MAPREDUCE_INPUT_FILEINPUTFORMAT_INPUTDIR, Constants.getSearchGraphLocation(this.graph.configuration().getInputLocation(), storage).get());
+            }
+            final List<InputSplit> splits = inputFormat.getSplits(new JobContextImpl(configuration, new JobID(UUID.randomUUID().toString(), 1)));
+            for (final InputSplit split : splits) {
+                this.readers.add(inputFormat.createRecordReader(split, new TaskAttemptContextImpl(configuration, new TaskAttemptID())));
+            }
+        } catch (final Exception e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopVertexIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopVertexIterator.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopVertexIterator.java
new file mode 100644
index 0000000..45b0cad
--- /dev/null
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopVertexIterator.java
@@ -0,0 +1,78 @@
+/*
+ * 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.tinkerpop.gremlin.hadoop.structure.io;
+
+import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
+import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopVertex;
+import org.apache.tinkerpop.gremlin.process.traversal.util.FastNoSuchElementException;
+import org.apache.tinkerpop.gremlin.structure.Vertex;
+
+import java.io.IOException;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public final class HadoopVertexIterator extends HadoopElementIterator<Vertex> {
+
+    private HadoopVertex nextVertex = null;
+
+    public HadoopVertexIterator(final HadoopGraph graph) throws IOException {
+        super(graph);
+    }
+
+    @Override
+    public Vertex next() {
+        try {
+            if (this.nextVertex != null) {
+                final Vertex temp = this.nextVertex;
+                this.nextVertex = null;
+                return temp;
+            } else {
+                while (!this.readers.isEmpty()) {
+                    if (this.readers.peek().nextKeyValue())
+                        return new HadoopVertex(this.readers.peek().getCurrentValue().get(), this.graph);
+                    else
+                        this.readers.remove().close();
+                }
+            }
+            throw FastNoSuchElementException.instance();
+        } catch (final Exception e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public boolean hasNext() {
+        try {
+            if (null != this.nextVertex) return true;
+            else {
+                while (!this.readers.isEmpty()) {
+                    if (this.readers.peek().nextKeyValue()) {
+                        this.nextVertex = new HadoopVertex(this.readers.peek().getCurrentValue().get(), this.graph);
+                        return true;
+                    } else
+                        this.readers.remove().close();
+                }
+            }
+        } catch (final Exception e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+        return false;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HiddenFileFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HiddenFileFilter.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HiddenFileFilter.java
new file mode 100644
index 0000000..8b01571
--- /dev/null
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HiddenFileFilter.java
@@ -0,0 +1,44 @@
+/*
+ * 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.tinkerpop.gremlin.hadoop.structure.io;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public final class HiddenFileFilter implements PathFilter {
+
+    private static final HiddenFileFilter INSTANCE = new HiddenFileFilter();
+
+    private HiddenFileFilter() {
+
+    }
+
+    @Override
+    public boolean accept(final Path path) {
+        final String name = path.getName();
+        return !name.startsWith("_") && !name.startsWith(".");
+    }
+
+    public static HiddenFileFilter instance() {
+        return INSTANCE;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritableIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritableIterator.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritableIterator.java
index 60bf930..676ca07 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritableIterator.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritableIterator.java
@@ -19,11 +19,10 @@
 package org.apache.tinkerpop.gremlin.hadoop.structure.io;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.SequenceFile;
-import org.apache.tinkerpop.gremlin.hadoop.structure.hdfs.HDFSTools;
-import org.apache.tinkerpop.gremlin.hadoop.structure.hdfs.HiddenFileFilter;
 import org.apache.tinkerpop.gremlin.process.computer.KeyValue;
 import org.apache.tinkerpop.gremlin.process.traversal.util.FastNoSuchElementException;
 
@@ -43,9 +42,8 @@ public final class ObjectWritableIterator implements Iterator<KeyValue> {
     private final Queue<SequenceFile.Reader> readers = new LinkedList<>();
 
     public ObjectWritableIterator(final Configuration configuration, final Path path) throws IOException {
-        final FileSystem fs = FileSystem.get(configuration);
-        for (final Path path2 : HDFSTools.getAllFilePaths(fs, path, HiddenFileFilter.instance())) {
-            this.readers.add(new SequenceFile.Reader(configuration, SequenceFile.Reader.file(path2)));
+        for (final FileStatus status : FileSystem.get(configuration).listStatus(path, HiddenFileFilter.instance())) {
+            this.readers.add(new SequenceFile.Reader(configuration, SequenceFile.Reader.file(status.getPath())));
         }
     }
 
@@ -62,7 +60,7 @@ public final class ObjectWritableIterator implements Iterator<KeyValue> {
                         this.available = true;
                         return true;
                     } else
-                        this.readers.remove();
+                        this.readers.remove().close();
                 }
             }
         } catch (final IOException e) {
@@ -83,7 +81,7 @@ public final class ObjectWritableIterator implements Iterator<KeyValue> {
                     if (this.readers.peek().next(this.key, this.value)) {
                         return new KeyValue<>(this.key.get(), this.value.get());
                     } else
-                        this.readers.remove();
+                        this.readers.remove().close();
                 }
             }
         } catch (final IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/TextIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/TextIterator.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/TextIterator.java
new file mode 100644
index 0000000..ecb6ede
--- /dev/null
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/TextIterator.java
@@ -0,0 +1,91 @@
+/*
+ * 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.tinkerpop.gremlin.hadoop.structure.io;
+
+import org.apache.tinkerpop.gremlin.process.traversal.util.FastNoSuchElementException;
+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 java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Queue;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public final class TextIterator implements Iterator<String> {
+
+    private String line;
+    private boolean available = false;
+    private final Queue<BufferedReader> readers = new LinkedList<>();
+
+    public TextIterator(final Configuration configuration, final Path path) throws IOException {
+        final FileSystem fs = FileSystem.get(configuration);
+        for (final FileStatus status : fs.listStatus(path, HiddenFileFilter.instance())) {
+            this.readers.add(new BufferedReader(new InputStreamReader(fs.open(status.getPath()))));
+        }
+    }
+
+    @Override
+    public boolean hasNext() {
+        try {
+            if (this.available) {
+                return true;
+            } else {
+                while (true) {
+                    if (this.readers.isEmpty())
+                        return false;
+                    if ((this.line = this.readers.peek().readLine()) != null) {
+                        this.available = true;
+                        return true;
+                    } else
+                        this.readers.remove().close();
+                }
+            }
+        } catch (final IOException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public String next() {
+        try {
+            if (this.available) {
+                this.available = false;
+                return this.line;
+            } else {
+                while (true) {
+                    if (this.readers.isEmpty())
+                        throw FastNoSuchElementException.instance();
+                    if ((this.line = this.readers.peek().readLine()) != null) {
+                        return this.line;
+                    } else
+                        this.readers.remove().close();
+                }
+            }
+        } catch (final IOException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritableIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritableIterator.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritableIterator.java
index daaffb8..d3e1fd0 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritableIterator.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritableIterator.java
@@ -23,7 +23,6 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.SequenceFile;
-import org.apache.tinkerpop.gremlin.hadoop.structure.hdfs.HiddenFileFilter;
 import org.apache.tinkerpop.gremlin.process.traversal.util.FastNoSuchElementException;
 import org.apache.tinkerpop.gremlin.structure.Vertex;
 
@@ -42,9 +41,8 @@ public final class VertexWritableIterator implements Iterator<Vertex> {
     private final Queue<SequenceFile.Reader> readers = new LinkedList<>();
 
     public VertexWritableIterator(final Configuration configuration, final Path path) throws IOException {
-        final FileSystem fs = FileSystem.get(configuration);
-        for (final FileStatus status : fs.listStatus(path, HiddenFileFilter.instance())) {
-            this.readers.add(new SequenceFile.Reader(fs, status.getPath(), configuration));
+        for (final FileStatus status : FileSystem.get(configuration).listStatus(path, HiddenFileFilter.instance())) {
+            this.readers.add(new SequenceFile.Reader(configuration, SequenceFile.Reader.file(status.getPath())));
         }
     }
 
@@ -61,7 +59,7 @@ public final class VertexWritableIterator implements Iterator<Vertex> {
                         this.available = true;
                         return true;
                     } else
-                        this.readers.remove();
+                        this.readers.remove().close();
                 }
             }
         } catch (final IOException e) {
@@ -82,7 +80,7 @@ public final class VertexWritableIterator implements Iterator<Vertex> {
                     if (this.readers.peek().next(this.value)) {
                         return this.value.get();
                     } else
-                        this.readers.remove();
+                        this.readers.remove().close();
                 }
             }
         } catch (final IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java
new file mode 100644
index 0000000..a8c5307
--- /dev/null
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java
@@ -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.tinkerpop.gremlin.hadoop.groovy.plugin;
+
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.tinkerpop.gremlin.AbstractGremlinTest;
+import org.apache.tinkerpop.gremlin.LoadGraphWith;
+import org.apache.tinkerpop.gremlin.hadoop.Constants;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.FileSystemStorage;
+import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
+import org.apache.tinkerpop.gremlin.process.computer.ComputerResult;
+import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.ClusterCountMapReduce;
+import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.PeerPressureVertexProgram;
+import org.apache.tinkerpop.gremlin.structure.io.Storage;
+import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public class FileSystemStorageCheck extends AbstractGremlinTest {
+
+    @Test
+    @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+    public void shouldPersistGraphAndMemory() throws Exception {
+        final ComputerResult result = graph.compute(graphComputerClass.get()).program(PeerPressureVertexProgram.build().create(graph)).mapReduce(ClusterCountMapReduce.build().memoryKey("clusterCount").create()).submit().get();
+        /////
+        final Storage storage = FileSystemStorage.open(ConfUtil.makeHadoopConfiguration(graph.configuration()));
+        // TEST GRAPH PERSISTENCE
+        assertTrue(storage.exists(Constants.getGraphLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION))));
+        assertEquals(6, result.graph().traversal().V().count().next().longValue());
+        assertEquals(0, result.graph().traversal().E().count().next().longValue());
+        assertEquals(6, result.graph().traversal().V().values("name").count().next().longValue());
+        assertEquals(6, result.graph().traversal().V().values(PeerPressureVertexProgram.CLUSTER).count().next().longValue());
+        assertEquals(2, result.graph().traversal().V().values(PeerPressureVertexProgram.CLUSTER).dedup().count().next().longValue());
+        /////
+        // TEST MEMORY PERSISTENCE
+        assertEquals(2, (int) result.memory().get("clusterCount"));
+        assertTrue(storage.exists(Constants.getMemoryLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount")));
+        assertEquals(1, IteratorUtils.count(storage.headMemory(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount", SequenceFileInputFormat.class)));
+        assertEquals(2, storage.headMemory(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount", SequenceFileInputFormat.class).next().getValue());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/GraphMemoryHDFSCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/GraphMemoryHDFSCheck.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/GraphMemoryHDFSCheck.java
deleted file mode 100644
index d47ce43..0000000
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/GraphMemoryHDFSCheck.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.tinkerpop.gremlin.hadoop.groovy.plugin;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.tinkerpop.gremlin.AbstractGremlinTest;
-import org.apache.tinkerpop.gremlin.LoadGraphWith;
-import org.apache.tinkerpop.gremlin.hadoop.Constants;
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.FileSystemStorage;
-import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
-import org.apache.tinkerpop.gremlin.process.computer.ComputerResult;
-import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.ClusterCountMapReduce;
-import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.PeerPressureVertexProgram;
-import org.apache.tinkerpop.gremlin.structure.io.Storage;
-import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-public class GraphMemoryHDFSCheck extends AbstractGremlinTest {
-
-    @Test
-    @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
-    public void shouldPersistGraphAndMemory() throws Exception {
-        final ComputerResult result = graph.compute(graphComputerClass.get()).program(PeerPressureVertexProgram.build().create(graph)).mapReduce(ClusterCountMapReduce.build().memoryKey("clusterCount").create()).submit().get();
-        /////
-        final Storage storage = new FileSystemStorage(FileSystem.get(ConfUtil.makeHadoopConfiguration(graph.configuration())));
-        // TEST GRAPH PERSISTENCE
-        assertTrue(storage.exists(Constants.getGraphLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION))));
-        assertEquals(6, result.graph().traversal().V().count().next().longValue());
-        assertEquals(0, result.graph().traversal().E().count().next().longValue());
-        assertEquals(6, result.graph().traversal().V().values("name").count().next().longValue());
-        assertEquals(6, result.graph().traversal().V().values(PeerPressureVertexProgram.CLUSTER).count().next().longValue());
-        assertEquals(2, result.graph().traversal().V().values(PeerPressureVertexProgram.CLUSTER).dedup().count().next().longValue());
-        /////
-        // TEST MEMORY PERSISTENCE
-        assertEquals(2, (int) result.memory().get("clusterCount"));
-        assertTrue(storage.exists(Constants.getMemoryLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount")));
- //       System.out.println(IteratorUtils.list(storage.head(Constants.getMemoryLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount"))));
-//        assertEquals(1, IteratorUtils.count(storage.head(Constants.getMemoryLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount"))));
-        assertEquals(2, storage.head(Constants.getMemoryLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount")).next());
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
index 4108e35..b558169 100644
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
@@ -137,7 +137,7 @@ public class HadoopGremlinPluginCheck extends AbstractGremlinTest {
         AbstractGremlinProcessTest.checkResults(Arrays.asList("ripple", "lop"), traversal);
         assertTrue((Boolean) this.console.eval("hdfs.exists('target/test-output/m')"));
         assertTrue((Boolean) this.console.eval("hdfs.exists('target/test-output/" + TraverserMapReduce.TRAVERSERS + "')"));
-        final List<KeyValue<Integer, Collection<String>>> mList = IteratorUtils.asList(this.console.eval("hdfs.head('target/test-output/m',ObjectWritable)"));
+        final List<KeyValue<Integer, Collection<String>>> mList = IteratorUtils.asList(this.console.eval("hdfs.headMemory('target/test-output','m',SequenceFileInputFormat)"));
         assertEquals(4, mList.size());
         mList.forEach(keyValue -> {
             if (keyValue.getKey().equals(29))
@@ -151,7 +151,7 @@ public class HadoopGremlinPluginCheck extends AbstractGremlinTest {
             else
                 throw new IllegalStateException("The provided key/value is unknown: " + keyValue);
         });
-        final List<KeyValue<MapReduce.NullObject, Traverser<String>>> traversersList = IteratorUtils.asList(this.console.eval("hdfs.head('target/test-output/" + TraverserMapReduce.TRAVERSERS + "',ObjectWritable)"));
+        final List<KeyValue<MapReduce.NullObject, Traverser<String>>> traversersList = IteratorUtils.asList(this.console.eval("hdfs.headMemory('target/test-output/'," + "'" + TraverserMapReduce.TRAVERSERS + "',SequenceFileInputFormat)"));
         assertEquals(2, traversersList.size());
         traversersList.forEach(keyValue -> {
             assertEquals(MapReduce.NullObject.instance(), keyValue.getKey());

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java
index 7dc8143..16d654f 100644
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java
@@ -29,6 +29,6 @@ import org.junit.runners.model.RunnerBuilder;
  */
 public class HadoopPluginSuite extends AbstractGremlinSuite {
     public HadoopPluginSuite(final Class<?> klass, final RunnerBuilder builder) throws InitializationError {
-        super(klass, builder, new Class<?>[]{HadoopGremlinPluginCheck.class, GraphMemoryHDFSCheck.class}, new Class<?>[]{HadoopGremlinPluginCheck.class, GraphMemoryHDFSCheck.class}, true, TraversalEngine.Type.COMPUTER);
+        super(klass, builder, new Class<?>[]{HadoopGremlinPluginCheck.class, FileSystemStorageCheck.class}, new Class<?>[]{HadoopGremlinPluginCheck.class, FileSystemStorageCheck.class}, true, TraversalEngine.Type.COMPUTER);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java
index fcb234c..a7e333c 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java
@@ -42,7 +42,6 @@ public final class SparkGremlinPlugin extends AbstractGremlinPlugin {
     protected static final Set<String> IMPORTS = new HashSet<String>() {{
         add(IMPORT_SPACE + SparkGraphComputer.class.getPackage().getName() + DOT_STAR);
         add(IMPORT_SPACE + Spark.class.getPackage().getName() + DOT_STAR);
-        add(IMPORT_SPACE + SparkContextStorage.class.getPackage().getName() + DOT_STAR);
     }};
 
     @Override
@@ -54,7 +53,7 @@ public final class SparkGremlinPlugin extends AbstractGremlinPlugin {
     public void afterPluginTo(final PluginAcceptor pluginAcceptor) throws PluginInitializationException, IllegalEnvironmentException {
         pluginAcceptor.addImports(IMPORTS);
         try {
-            pluginAcceptor.eval("spark = SparkContextStorage.open()");
+            pluginAcceptor.addBinding("spark", SparkContextStorage.open());
         } catch (final Exception e) {
             throw new PluginInitializationException(e.getMessage(), e);
         }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
index 03b3016..f96fd15 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
@@ -239,6 +239,10 @@ public final class SparkGraphComputer extends AbstractHadoopGraphComputer {
                 // unpersist the graphRDD if it will no longer be used
                 if (!PersistedOutputRDD.class.equals(hadoopConfiguration.getClass(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, null)) || this.persist.equals(GraphComputer.Persist.NOTHING)) {
                     graphRDD.unpersist();
+                    if (apacheConfiguration.containsKey(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION)) {
+                        Spark.removeRDD(apacheConfiguration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));
+                        Spark.removeRDD(Constants.getGraphLocation(apacheConfiguration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION)));
+                    }
                 }
                 // update runtime and return the newly computed graph
                 finalMemory.setRuntime(System.currentTimeMillis() - startTime);

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java
index 5cb9edf..b78caa9 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java
@@ -46,6 +46,7 @@ public final class PersistedOutputRDD implements OutputRDD {
             throw new IllegalArgumentException("There is no provided " + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + " to write the persisted RDD to");
         final String graphRDDName = Constants.getGraphLocation(configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));
         Spark.removeRDD(graphRDDName);  // this might be bad cause it unpersists the job RDD
+        Constants.getSearchGraphLocation(configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), SparkContextStorage.open(configuration)).ifPresent(Spark::removeRDD);  // this might be bad cause it unpersists the job RDD
         if (!configuration.getBoolean(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT_HAS_EDGES, true))
             graphRDD.mapValues(vertex -> {
                 vertex.get().dropEdges();
@@ -62,7 +63,7 @@ public final class PersistedOutputRDD implements OutputRDD {
             LOGGER.warn("The SparkContext should be persisted in order for the RDD to persist across jobs. To do so, set " + Constants.GREMLIN_SPARK_PERSIST_CONTEXT + " to true");
         if (!configuration.containsKey(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION))
             throw new IllegalArgumentException("There is no provided " + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + " to write the persisted RDD to");
-        final String sideEffectRDDName = Constants.getMemoryLocation(configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), memoryKey);
+        final String sideEffectRDDName = configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION) + "/" + memoryKey;
         Spark.removeRDD(sideEffectRDDName);
         memoryRDD.setName(sideEffectRDDName).cache();
         return IteratorUtils.map(memoryRDD.toLocalIterator(), tuple -> new KeyValue<>(tuple._1(), tuple._2()));


[09/30] incubator-tinkerpop git commit: merged master into branch. Minor tweaks given @spmallette new work on TestDirectory stuffs.

Posted by dk...@apache.org.
merged master into branch. Minor tweaks given @spmallette new work on TestDirectory stuffs.


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

Branch: refs/heads/TINKERPOP-320
Commit: dbd4a5360a75d562df64eecd91cc8c12550adb10
Parents: 55165a5 46c7189
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Tue Jan 5 15:54:14 2016 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Tue Jan 5 15:54:14 2016 -0700

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              | 235 ++++-----
 README.asciidoc                                 |  12 +-
 bin/validate-distribution.sh                    |   4 +-
 docs/src/dev/developer/contributing.asciidoc    |  35 +-
 docs/src/dev/developer/meetings.asciidoc        | 194 ++++----
 docs/src/dev/developer/release.asciidoc         |   4 +-
 .../src/reference/gremlin-applications.asciidoc | 107 +++-
 docs/src/reference/implementations.asciidoc     |   5 +
 docs/src/reference/intro.asciidoc               |  23 +-
 docs/src/reference/the-graph.asciidoc           |  40 +-
 docs/src/reference/the-graphcomputer.asciidoc   |  11 +-
 docs/src/reference/the-traversal.asciidoc       |   2 +-
 .../tutorials/getting-started/index.asciidoc    |  52 +-
 .../upgrade/release-3.0.x-incubating.asciidoc   |  28 +-
 .../upgrade/release-3.1.x-incubating.asciidoc   | 100 ++--
 docs/static/images/business-gremlin.png         | Bin 0 -> 412520 bytes
 .../images/modern-edge-1-to-3-1-gremlin.png     | Bin 11607 -> 12720 bytes
 docs/static/images/modern-edge-1-to-3-1.png     | Bin 3210 -> 3887 bytes
 .../images/modern-edge-1-to-3-2-gremlin.png     | Bin 17029 -> 18052 bytes
 docs/static/images/modern-edge-1-to-3-2.png     | Bin 4420 -> 3784 bytes
 .../images/modern-edge-1-to-3-3-gremlin.png     | Bin 20948 -> 22258 bytes
 docs/static/images/modern-edge-1-to-3-3.png     | Bin 7372 -> 5800 bytes
 docs/static/images/quantum-gremlin-full.png     | Bin 0 -> 1877990 bytes
 .../tinkerpop/gremlin/console/Console.groovy    |  20 +-
 .../computer/bulkloading/BulkLoader.java        |   8 +-
 .../bulkloading/BulkLoaderVertexProgram.java    |  11 +-
 .../bulkloading/IncrementalBulkLoader.java      |   8 +-
 .../computer/bulkloading/OneTimeBulkLoader.java | 109 +++++
 .../tinkerpop/gremlin/structure/Element.java    |   2 +-
 .../structure/io/gryo/GryoClassResolver.java    |   2 +-
 .../gremlin/structure/io/gryo/GryoMapper.java   |  25 +-
 .../apache/tinkerpop/gremlin/TestHelper.java    |  76 ++-
 .../sideEffect/LambdaSideEffectStepTest.java    |   7 +-
 .../tinkerpop/gremlin/structure/io/IoX.java     |   5 +
 .../gremlin/structure/io/IoXIoRegistry.java     |  57 +++
 .../tinkerpop/gremlin/structure/io/IoY.java     |   5 +
 .../gremlin/structure/io/IoYIoRegistry.java     |  39 ++
 .../structure/io/gryo/GryoMapperTest.java       | 113 +++++
 .../AbstractGraphSONMessageSerializerV1d0.java  |   5 +-
 .../driver/ser/AbstractMessageSerializer.java   |  82 ++++
 .../driver/ser/GryoMessageSerializerV1d0.java   |  49 +-
 ...raphSONMessageSerializerGremlinV1d0Test.java |   5 +-
 .../ser/GraphSONMessageSerializerV1d0Test.java  |  60 +++
 .../ser/GryoMessageSerializerV1d0Test.java      | 115 ++++-
 .../traversal/step/map/GroovySelectTest.groovy  |   2 +-
 .../gremlin/groovy/GroovyEnvironmentSuite.java  |   2 +
 ...remlinGroovyScriptEngineFileSandboxTest.java | 117 +++++
 .../GremlinGroovyScriptEngineIntegrateTest.java |  11 +-
 .../process/GroovyProcessComputerSuite.java     |   1 -
 .../gremlin/groovy/jsr223/sandbox.yaml          |  58 +++
 .../customizer/AbstractSandboxExtension.groovy  |  90 ++++
 .../customizer/FileSandboxExtension.groovy      | 107 ++++
 .../PropertyExpressionEvaluator.groovy          |  66 +++
 .../jsr223/customizer/SandboxExtension.groovy   |   3 +
 .../jsr223/customizer/SandboxHelper.groovy      |  57 +++
 .../customizer/SimpleSandboxExtension.groovy    |  30 +-
 .../customizer/TinkerPopSandboxExtension.groovy |  44 +-
 .../gremlin/groovy/engine/GremlinExecutor.java  |   3 +-
 .../jsr223/GremlinGroovyScriptEngineTest.java   |   9 +-
 .../groovy/util/DependencyGrabberTest.java      |   3 +-
 gremlin-server/conf/gremlin-server-classic.yaml |   1 +
 gremlin-server/conf/gremlin-server-min.yaml     |   1 +
 .../conf/gremlin-server-modern-readonly.yaml    |   1 +
 gremlin-server/conf/gremlin-server-modern.yaml  |   1 +
 gremlin-server/conf/gremlin-server-neo4j.yaml   |   1 +
 .../conf/gremlin-server-rest-modern.yaml        |   1 +
 .../conf/gremlin-server-rest-secure.yaml        |   1 +
 gremlin-server/conf/gremlin-server-secure.yaml  |   1 +
 gremlin-server/conf/gremlin-server-spark.yaml   |   1 +
 gremlin-server/conf/gremlin-server.yaml         |   1 +
 .../scripts/empty-sample-secure.groovy          |   4 +-
 .../tinkerpop/gremlin/server/GraphManager.java  |  46 ++
 .../tinkerpop/gremlin/server/GremlinServer.java |   2 +-
 .../tinkerpop/gremlin/server/Settings.java      |   9 +
 .../server/auth/AllowAllAuthenticator.java      |   2 +-
 .../gremlin/server/auth/Authenticator.java      |   2 +-
 .../server/auth/SimpleAuthenticator.java        |   4 +-
 .../gremlin/server/channel/HttpChannelizer.java |   2 +-
 .../handler/HttpGremlinEndpointHandler.java     |  37 +-
 .../server/handler/OpSelectorHandler.java       |   2 +-
 .../server/op/AbstractEvalOpProcessor.java      |  52 +-
 .../server/op/control/ControlOpProcessor.java   |  18 +-
 .../server/op/session/SessionOpProcessor.java   |   7 +-
 .../server/op/standard/StandardOpProcessor.java |  31 +-
 .../AbstractGremlinServerIntegrationTest.java   |   7 +-
 .../server/GremlinAdditionPerformanceTest.java  |   5 +-
 .../server/GremlinDriverIntegrateTest.java      | 113 ++++-
 .../GremlinServerAuthOldIntegrateTest.java      |   2 +-
 .../server/GremlinServerHttpIntegrateTest.java  |  36 ++
 .../server/GremlinServerIntegrateTest.java      |  90 ++--
 .../server/gremlin-server-integration.yaml      |   1 +
 .../server/gremlin-server-performance.yaml      |   1 +
 .../gremlin/AbstractGraphProvider.java          |  14 +-
 .../tinkerpop/gremlin/AbstractGremlinSuite.java |  32 +-
 .../tinkerpop/gremlin/AbstractGremlinTest.java  |  20 +-
 .../apache/tinkerpop/gremlin/GraphManager.java  | 115 ++++-
 .../apache/tinkerpop/gremlin/GraphProvider.java |  10 +
 .../apache/tinkerpop/gremlin/TestHelper.java    |  51 +-
 .../generator/CommunityGeneratorTest.java       |   5 +-
 .../process/AbstractGremlinProcessTest.java     |   7 +-
 .../gremlin/process/GremlinProcessRunner.java   |   6 +-
 .../BulkLoaderVertexProgramTest.java            |  39 +-
 .../process/traversal/step/filter/TailTest.java |   2 +-
 .../process/traversal/step/map/FlatMapTest.java |   2 +-
 .../process/traversal/step/map/MapTest.java     |   2 +-
 .../process/traversal/step/map/SelectTest.java  |   6 +-
 .../ElementIdStrategyProcessTest.java           |   7 +-
 .../gremlin/structure/FeatureSupportTest.java   |   7 +-
 .../tinkerpop/gremlin/structure/GraphTest.java  |  16 +-
 .../gremlin/structure/PropertyTest.java         |   4 +-
 .../tinkerpop/gremlin/structure/VertexTest.java |  18 +-
 .../gremlin/structure/io/IoIntegrateTest.java   |  10 +-
 .../structure/ExceptionCoverageTest.java        |   5 +-
 .../gremlin/hadoop/HadoopGraphProvider.java     |   2 +-
 .../groovy/plugin/HadoopGremlinPluginCheck.java |  55 ++-
 .../structure/io/RecordReaderWriterTest.java    |   7 +-
 .../gremlin/neo4j/NativeNeo4jSuite.java         |  18 +-
 .../neo4j/process/NativeNeo4jCypherCheck.java   | 214 ++++++++
 .../neo4j/process/NativeNeo4jCypherTest.java    | 211 --------
 .../neo4j/structure/NativeNeo4jIndexCheck.java  | 371 ++++++++++++++
 .../neo4j/structure/NativeNeo4jIndexTest.java   | 368 --------------
 .../structure/NativeNeo4jStructureCheck.java    | 483 +++++++++++++++++++
 .../structure/NativeNeo4jStructureTest.java     | 483 -------------------
 pom.xml                                         |   3 -
 .../gremlin/spark/AbstractSparkTest.java        |   5 +-
 .../process/computer/LocalPropertyTest.java     |   9 +-
 .../groovy/plugin/SparkGremlinPluginTest.java   |  35 +-
 .../gremlin/spark/structure/SparkTest.java      |   6 +-
 .../spark/structure/io/InputOutputRDDTest.java  |   4 +-
 .../spark/structure/io/InputRDDTest.java        |   5 +-
 .../spark/structure/io/OutputRDDTest.java       |   3 +-
 .../io/PersistedInputOutputRDDTest.java         |  58 +--
 .../structure/TinkerGraphPlayTest.java          |  55 ++-
 .../tinkergraph/structure/TinkerGraphTest.java  |  18 +-
 134 files changed, 3713 insertions(+), 1819 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/dbd4a536/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
----------------------------------------------------------------------
diff --cc hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
index 95c19ed,6c58c52..26acee9
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
@@@ -26,22 -27,23 +27,21 @@@ import org.apache.tinkerpop.gremlin.gro
  import org.apache.tinkerpop.gremlin.hadoop.Constants;
  import org.apache.tinkerpop.gremlin.hadoop.HadoopGraphProvider;
  import org.apache.tinkerpop.gremlin.process.AbstractGremlinProcessTest;
--import org.apache.tinkerpop.gremlin.process.computer.KeyValue;
--import org.apache.tinkerpop.gremlin.process.computer.MapReduce;
  import org.apache.tinkerpop.gremlin.process.computer.traversal.step.sideEffect.mapreduce.TraverserMapReduce;
  import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
--import org.apache.tinkerpop.gremlin.process.traversal.Traverser;
  import org.apache.tinkerpop.gremlin.structure.Vertex;
  import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
  import org.junit.Before;
  import org.junit.Test;
  
+ import java.io.File;
  import java.util.Arrays;
--import java.util.Collection;
  import java.util.List;
--import java.util.UUID;
  
--import static org.junit.Assert.*;
++import static org.junit.Assert.assertEquals;
++import static org.junit.Assert.assertFalse;
++import static org.junit.Assert.assertNotNull;
++import static org.junit.Assert.assertTrue;
  
  /**
   * This is an test that is mean to be used in the context of the {@link HadoopPluginSuite} and shouldn't be
@@@ -135,8 -140,30 +138,30 @@@ public class HadoopGremlinPluginCheck e
          this.remote.connect(Arrays.asList("graph", "g"));
          Traversal<Vertex, String> traversal = (Traversal<Vertex, String>) this.remote.submit(Arrays.asList("g.V().hasLabel('person').group('m').by('age').by('name').out('knows').out('created').values('name')"));
          AbstractGremlinProcessTest.checkResults(Arrays.asList("ripple", "lop"), traversal);
-         assertFalse((Boolean) this.console.eval("hdfs.exists('target/test-output/m')"));
-         assertFalse((Boolean) this.console.eval("hdfs.exists('target/test-output/" + TraverserMapReduce.TRAVERSERS + "')"));
 -        assertTrue((Boolean) this.console.eval("hdfs.exists('" + hadoopGraphProviderDataDir + "m')"));
 -        assertTrue((Boolean) this.console.eval("hdfs.exists('" + hadoopGraphProviderDataDir + TraverserMapReduce.TRAVERSERS + "')"));
 -        final List<KeyValue<Integer, Collection<String>>> mList = IteratorUtils.asList(this.console.eval("hdfs.head('" + hadoopGraphProviderDataDir + "m',ObjectWritable)"));
++//        assertFalse((Boolean) this.console.eval("hdfs.exists('" + hadoopGraphProviderDataDir + "m')"));
++//        assertFalse((Boolean) this.console.eval("hdfs.exists('" + hadoopGraphProviderDataDir + TraverserMapReduce.TRAVERSERS + "')"));
++        /*final List<KeyValue<Integer, Collection<String>>> mList = IteratorUtils.asList(this.console.eval("hdfs.head('" + hadoopGraphProviderDataDir + "m',ObjectWritable)"));
+         assertEquals(4, mList.size());
+         mList.forEach(keyValue -> {
+             if (keyValue.getKey().equals(29))
+                 assertTrue(keyValue.getValue().contains("marko"));
+             else if (keyValue.getKey().equals(35))
+                 assertTrue(keyValue.getValue().contains("peter"));
+             else if (keyValue.getKey().equals(32))
+                 assertTrue(keyValue.getValue().contains("josh"));
+             else if (keyValue.getKey().equals(27))
+                 assertTrue(keyValue.getValue().contains("vadas"));
+             else
+                 throw new IllegalStateException("The provided key/value is unknown: " + keyValue);
+         });
+         final List<KeyValue<MapReduce.NullObject, Traverser<String>>> traversersList = IteratorUtils.asList(this.console.eval("hdfs.head('" + hadoopGraphProviderDataDir + TraverserMapReduce.TRAVERSERS + "',ObjectWritable)"));
+         assertEquals(2, traversersList.size());
+         traversersList.forEach(keyValue -> {
+             assertEquals(MapReduce.NullObject.instance(), keyValue.getKey());
+             final String name = keyValue.getValue().get();
+             assertTrue(name.equals("ripple") || name.equals("lop"));
 -        });
++        });*/
+         ////////////////
      }
  
      @Test

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/dbd4a536/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/AbstractSparkTest.java
----------------------------------------------------------------------
diff --cc spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/AbstractSparkTest.java
index 3fc2a59,9f3b87a..2ad0c14
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/AbstractSparkTest.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/AbstractSparkTest.java
@@@ -24,18 -22,11 +24,20 @@@ import org.apache.commons.configuration
  import org.apache.spark.SparkConf;
  import org.apache.spark.SparkContext;
  import org.apache.spark.api.java.JavaSparkContext;
 +import org.apache.tinkerpop.gremlin.hadoop.Constants;
 +import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
 +import org.apache.tinkerpop.gremlin.hadoop.structure.io.graphson.GraphSONInputFormat;
 +import org.apache.tinkerpop.gremlin.hadoop.structure.io.gryo.GryoInputFormat;
 +import org.apache.tinkerpop.gremlin.spark.process.computer.SparkHadoopGraphProvider;
  import org.apache.tinkerpop.gremlin.spark.structure.Spark;
 +import org.apache.tinkerpop.gremlin.spark.structure.io.PersistedInputRDD;
 +import org.apache.tinkerpop.gremlin.spark.structure.io.PersistedOutputRDD;
 +import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.GryoSerializer;
 +import org.apache.tinkerpop.gremlin.structure.Graph;
  import org.junit.After;
  import org.junit.Before;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
  
  /**
   * @author Marko A. Rodriguez (http://markorodriguez.com)
@@@ -52,25 -44,6 +55,25 @@@ public abstract class AbstractSparkTes
          sparkContext.close();
          Spark.create(sparkContext.sc());
          Spark.close();
-         System.out.println("SparkContext has been closed for " + this.getClass().getCanonicalName() + "-setupTest");
+         logger.info("SparkContext has been closed for " + this.getClass().getCanonicalName() + "-setupTest");
      }
 +
 +    protected Configuration getBaseConfiguration(final String inputLocation) {
 +        final BaseConfiguration configuration = new BaseConfiguration();
 +        configuration.setDelimiterParsingDisabled(true);
 +        configuration.setProperty("spark.master", "local[4]");
 +        configuration.setProperty("spark.serializer", GryoSerializer.class.getCanonicalName());
 +        configuration.setProperty(Graph.GRAPH, HadoopGraph.class.getName());
 +        configuration.setProperty(Constants.GREMLIN_HADOOP_JARS_IN_DISTRIBUTED_CACHE, false);
 +        configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, SparkHadoopGraphProvider.PATHS.get("tinkerpop-modern.kryo"));
 +        if (inputLocation.contains(".kryo"))
 +            configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, GryoInputFormat.class.getCanonicalName());
 +        else if (inputLocation.contains(".json"))
 +            configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, GraphSONInputFormat.class.getCanonicalName());
 +        else
 +            configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_INPUT_RDD, PersistedInputRDD.class.getCanonicalName());
 +
 +        configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
 +        return configuration;
 +    }
  }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/dbd4a536/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/LocalPropertyTest.java
----------------------------------------------------------------------
diff --cc spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/LocalPropertyTest.java
index 1bf5338,f535012..65d6ce4
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/LocalPropertyTest.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/LocalPropertyTest.java
@@@ -78,10 -79,10 +79,10 @@@ public class LocalPropertyTest extends 
          JavaSparkContext sparkContext = new JavaSparkContext(SparkContext.getOrCreate(sparkConfiguration));
          JavaSparkStatusTracker statusTracker = sparkContext.statusTracker();
          assertTrue(statusTracker.getJobIdsForGroup("22").length >= 1);
-         assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
 -        assertTrue(Spark.hasRDD(rddName));
++        assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddName)));
          ///////
          configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_INPUT_RDD, PersistedInputRDD.class.getCanonicalName());
-         configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, rddLocation);
+         configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, rddName);
          configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, null);
          configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, null);
          configuration.setProperty(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, false);

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/dbd4a536/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java
----------------------------------------------------------------------
diff --cc spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java
index 7574908,a6741da..4eadd97
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/groovy/plugin/SparkGremlinPluginTest.java
@@@ -67,7 -66,8 +68,9 @@@ public class SparkGremlinPluginTest ext
  
      @Test
      public void shouldSupportBasicRDDOperations() throws Exception {
-         String rddLocation = "target/test-output/graph-1";
++
+         final String root = TestHelper.makeTestDataDirectory(SparkGremlinPluginTest.class, "shouldSupportBasicRDDOperations");
+         final String rddName1 = TestHelper.makeTestDataDirectory(SparkGremlinPluginTest.class, "shouldSupportBasicRDDOperations", "graph-1");
          final Configuration configuration = new BaseConfiguration();
          configuration.setProperty("spark.master", "local[4]");
          configuration.setProperty("spark.serializer", GryoSerializer.class.getCanonicalName());
@@@ -87,40 -87,38 +90,38 @@@
          this.console.addBinding("graph", graph);
          this.console.eval("graph.compute(SparkGraphComputer).program(PageRankVertexProgram.build().iterations(1).create()).submit().get()");
          assertEquals(1, ((List<String>) this.console.eval("spark.ls()")).size());
-         assertEquals(Constants.getGraphLocation(rddLocation) + " [Memory Deserialized 1x Replicated]", ((List<String>) this.console.eval("spark.ls()")).get(0));
 -        assertEquals(rddName1 + " [Memory Deserialized 1x Replicated]", ((List<String>) this.console.eval("spark.ls()")).get(0));
++        assertEquals(Constants.getGraphLocation(rddName1) + " [Memory Deserialized 1x Replicated]", ((List<String>) this.console.eval("spark.ls()")).get(0));
  
-         rddLocation = "target/test-output/graph-2";
-         this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + rddLocation + "')");
+         final String rddName2 = TestHelper.makeTestDataDirectory(SparkGremlinPluginTest.class, "shouldSupportBasicRDDOperations", "graph-2");
+         this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + rddName2 + "')");
          this.console.eval("graph.compute(SparkGraphComputer).program(PageRankVertexProgram.build().iterations(1).create()).submit().get()");
          assertEquals(2, ((List<String>) this.console.eval("spark.ls()")).size());
-         assertTrue(((List<String>) this.console.eval("spark.ls()")).contains(Constants.getGraphLocation(rddLocation) + " [Memory Deserialized 1x Replicated]"));
 -        assertTrue(((List<String>) this.console.eval("spark.ls()")).contains(rddName2 + " [Memory Deserialized 1x Replicated]"));
++        assertTrue(((List<String>) this.console.eval("spark.ls()")).contains(Constants.getGraphLocation(rddName2) + " [Memory Deserialized 1x Replicated]"));
  
-         this.console.eval("spark.rm('target/test-output/graph-2/~g')");
 -        this.console.eval("spark.rm('" + rddName2 + "')");
++        this.console.eval("spark.rm('" + Constants.getGraphLocation(rddName2) + "')");
          assertEquals(1, ((List<String>) this.console.eval("spark.ls()")).size());
-         assertTrue(((List<String>) this.console.eval("spark.ls()")).contains("target/test-output/graph-1/~g [Memory Deserialized 1x Replicated]"));
 -        assertTrue(((List<String>) this.console.eval("spark.ls()")).contains(rddName1 + " [Memory Deserialized 1x Replicated]"));
++        assertTrue(((List<String>) this.console.eval("spark.ls()")).contains(Constants.getGraphLocation(rddName1) + " [Memory Deserialized 1x Replicated]"));
  
-         assertEquals(6, IteratorUtils.count(((Iterator<Object>) this.console.eval("spark.head('target/test-output/graph-1/~g')"))));
 -        assertEquals(6, ((List<Object>) this.console.eval("spark.head('" + rddName1 + "')")).size());
++        assertEquals(6, IteratorUtils.count(((Iterator<Object>) this.console.eval("spark.head('" + Constants.getGraphLocation(rddName1) + "')"))));
  
-         this.console.eval("spark.rmr('target/test-output/graph-*')");
 -        this.console.eval("spark.rm('"+ root + "graph-*')");
++        this.console.eval("spark.rmr('" + root + "graph-*')");
          assertEquals(0, ((List<String>) this.console.eval("spark.ls()")).size());
  
          //////
-         rddLocation = "target/test-output/graph-1";
-         this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + rddLocation + "')");
 -        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + rddName1 + "')");
++        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + Constants.getGraphLocation(rddName1) + "')");
          this.console.eval("graph.compute(SparkGraphComputer).program(PageRankVertexProgram.build().iterations(1).create()).submit().get()");
  
-         rddLocation = "target/test-output/graph-2";
-         this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + rddLocation + "')");
 -        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + rddName2 + "')");
++        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + Constants.getGraphLocation(rddName2) + "')");
          this.console.eval("graph.compute(SparkGraphComputer).program(PageRankVertexProgram.build().iterations(1).create()).submit().get()");
  
-         rddLocation = "target/test-output/x";
-         this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + rddLocation + "')");
+         final String rddName3 = TestHelper.makeTestDataDirectory(SparkGremlinPluginTest.class, "shouldSupportBasicRDDOperations", "x");
 -        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + rddName3 + "')");
++        this.console.eval("graph.configuration().setProperty('" + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + "','" + Constants.getGraphLocation(rddName3) + "')");
          this.console.eval("graph.compute(SparkGraphComputer).program(PageRankVertexProgram.build().iterations(1).create()).submit().get()");
  
          assertEquals(3, ((List<String>) this.console.eval("spark.ls()")).size());
-         this.console.eval("spark.rmr('target/test-output/graph-*')");
 -        this.console.eval("spark.rm('"+ root + "graph-*')");
++        this.console.eval("spark.rmr('" + root + "graph-*')");
          assertEquals(1, ((List<String>) this.console.eval("spark.ls()")).size());
 -        this.console.eval("spark.rm('*')");
 +        this.console.eval("spark.rmr('*')");
          assertEquals(0, ((List<String>) this.console.eval("spark.ls()")).size());
  
          //

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/dbd4a536/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/SparkTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/dbd4a536/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputOutputRDDTest.java
----------------------------------------------------------------------
diff --cc spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputOutputRDDTest.java
index 44e0949,4028fc6..e153c4e
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputOutputRDDTest.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputOutputRDDTest.java
@@@ -21,6 -21,10 +21,7 @@@ package org.apache.tinkerpop.gremlin.sp
  
  import org.apache.commons.configuration.BaseConfiguration;
  import org.apache.commons.configuration.Configuration;
 -import org.apache.spark.SparkConf;
 -import org.apache.spark.SparkContext;
 -import org.apache.spark.api.java.JavaSparkContext;
+ import org.apache.tinkerpop.gremlin.TestHelper;
  import org.apache.tinkerpop.gremlin.hadoop.Constants;
  import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
  import org.apache.tinkerpop.gremlin.hadoop.structure.io.gryo.GryoInputFormat;
@@@ -76,13 -79,13 +77,14 @@@ public class PersistedInputOutputRDDTes
                                  "g.V()").create(graph)).submit().get();
          ////////
          Spark.create("local[4]");
-         assertFalse(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
 -        assertFalse(Spark.hasRDD(rddName));
++        assertFalse(Spark.hasRDD(Constants.getGraphLocation(rddName)));
          Spark.close();
      }
  
      @Test
      public void shouldPersistRDDAcrossJobs() throws Exception {
-         final String rddLocation = "target/test-output/" + UUID.randomUUID();
++
+         final String rddName = TestHelper.makeTestDataDirectory(PersistedInputOutputRDDTest.class, UUID.randomUUID().toString());
          final Configuration configuration = new BaseConfiguration();
          configuration.setProperty("spark.master", "local[4]");
          configuration.setProperty("spark.serializer", GryoSerializer.class.getCanonicalName());
@@@ -102,10 -105,10 +104,10 @@@
                                  "gremlin-groovy",
                                  "g.V()").create(graph)).submit().get();
          ////////
-         assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
 -        assertTrue(Spark.hasRDD(rddName));
++        assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddName)));
          ///////
          configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_INPUT_RDD, PersistedInputRDD.class.getCanonicalName());
-         configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, rddLocation);
+         configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, rddName);
          configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, null);
          configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, null);
          graph = GraphFactory.open(configuration);
@@@ -151,7 -154,7 +153,7 @@@
                  .program(BulkLoaderVertexProgram.build().userSuppliedIds(true).writeGraph(writeConfiguration).create(bulkLoaderGraph))
                  .submit().get();
          ////
-         assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
 -        assertTrue(Spark.hasRDD(rddName));
++        assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddName)));
          ////
          final Graph graph = TinkerGraph.open();
          final GraphTraversalSource g = graph.traversal();
@@@ -194,7 -197,7 +196,7 @@@
                  .submit().get();
          ////
          Spark.create(readConfiguration);
-         assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
 -        assertTrue(Spark.hasRDD(rddName));
++        assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddName)));
          ////
          final Graph graph = TinkerGraph.open();
          final GraphTraversalSource g = graph.traversal();
@@@ -220,11 -223,9 +222,11 @@@
          configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, GryoInputFormat.class.getCanonicalName());
          configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, SparkHadoopGraphProvider.PATHS.get("tinkerpop-modern.kryo"));
          configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
-         configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddLocation);
+         configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddName);
          configuration.setProperty(Constants.GREMLIN_HADOOP_JARS_IN_DISTRIBUTED_CACHE, false);
          configuration.setProperty(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, true);
 +
-         assertFalse(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
++        assertFalse(Spark.hasRDD(Constants.getGraphLocation(rddName)));
          Graph graph = GraphFactory.open(configuration);
          graph = graph.compute(SparkGraphComputer.class).persist(GraphComputer.Persist.EDGES).program(PageRankVertexProgram.build().iterations(2).create(graph)).submit().get().graph();
          GraphTraversalSource g = graph.traversal();
@@@ -233,12 -234,12 +235,12 @@@
          assertEquals(6l, g.V().values(PageRankVertexProgram.PAGE_RANK).count().next().longValue());
          assertEquals(6l, g.V().values(PageRankVertexProgram.EDGE_COUNT).count().next().longValue());
          ////
-         assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
 -        assertTrue(Spark.hasRDD(rddName));
++        assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddName)));
          ////
          configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_INPUT_RDD, PersistedInputRDD.class.getCanonicalName());
-         configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, rddLocation);
+         configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, rddName);
          configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
-         configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddLocation);
+         configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddName);
          ////
          graph = GraphFactory.open(configuration);
          graph = graph.compute(SparkGraphComputer.class).persist(GraphComputer.Persist.EDGES).program(PageRankVertexProgram.build().iterations(2).create(graph)).submit().get().graph();
@@@ -248,7 -249,7 +250,7 @@@
          assertEquals(6l, g.V().values(PageRankVertexProgram.PAGE_RANK).count().next().longValue());
          assertEquals(6l, g.V().values(PageRankVertexProgram.EDGE_COUNT).count().next().longValue());
          ////
-         assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
 -        assertTrue(Spark.hasRDD(rddName));
++        assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddName)));
          ////
          graph = GraphFactory.open(configuration);
          graph = graph.compute(SparkGraphComputer.class).persist(GraphComputer.Persist.VERTEX_PROPERTIES).program(PageRankVertexProgram.build().iterations(2).create(graph)).submit().get().graph();
@@@ -258,18 -259,18 +260,18 @@@
          assertEquals(6l, g.V().values(PageRankVertexProgram.PAGE_RANK).count().next().longValue());
          assertEquals(6l, g.V().values(PageRankVertexProgram.EDGE_COUNT).count().next().longValue());
          ////
-         assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
 -        assertTrue(Spark.hasRDD(rddName));
++        assertTrue(Spark.hasRDD(Constants.getGraphLocation(rddName)));
          ////
          graph = GraphFactory.open(configuration);
          graph.compute(SparkGraphComputer.class).persist(GraphComputer.Persist.NOTHING).program(PageRankVertexProgram.build().iterations(2).create(graph)).submit().get().graph();
-         assertFalse(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
 -        assertFalse(Spark.hasRDD(rddName));
++        assertFalse(Spark.hasRDD(Constants.getGraphLocation(rddName)));
          g = graph.traversal();
          assertEquals(0l, g.V().count().next().longValue());
          assertEquals(0l, g.E().count().next().longValue());
          assertEquals(0l, g.V().values(PageRankVertexProgram.PAGE_RANK).count().next().longValue());
          assertEquals(0l, g.V().values(PageRankVertexProgram.EDGE_COUNT).count().next().longValue());
          ////
-         assertFalse(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
 -        assertFalse(Spark.hasRDD(rddName));
++        assertFalse(Spark.hasRDD(Constants.getGraphLocation(rddName)));
          Spark.close();
      }
  }


[07/30] incubator-tinkerpop git commit: migrated GiraphGraphComputer over to the new Storage model via FileSystemStorage for HDFS.

Posted by dk...@apache.org.
migrated GiraphGraphComputer over to the new Storage model via FileSystemStorage for HDFS.


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/74b9c8ec
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/74b9c8ec
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/74b9c8ec

Branch: refs/heads/TINKERPOP-320
Commit: 74b9c8ecfe787ead99d79c127fd85a4fccd926ec
Parents: 3fff8f5
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Dec 9 18:27:29 2015 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Wed Dec 9 18:27:29 2015 -0700

----------------------------------------------------------------------
 .../process/computer/GiraphGraphComputer.java   | 36 +++++++++-----------
 .../process/computer/util/MapReduceHelper.java  |  8 ++---
 .../hadoop/structure/io/FileSystemStorage.java  |  4 ++-
 .../hadoop/structure/io/InputOutputHelper.java  |  2 +-
 .../groovy/plugin/HadoopGremlinPluginCheck.java | 26 ++------------
 .../process/computer/SparkGraphComputer.java    |  1 +
 .../gremlin/spark/structure/Spark.java          |  2 ++
 .../spark/structure/io/SparkContextStorage.java |  2 +-
 8 files changed, 31 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/74b9c8ec/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphGraphComputer.java
----------------------------------------------------------------------
diff --git a/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphGraphComputer.java b/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphGraphComputer.java
index 646b707..7e3de5e 100644
--- a/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphGraphComputer.java
+++ b/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphGraphComputer.java
@@ -41,6 +41,7 @@ import org.apache.tinkerpop.gremlin.hadoop.process.computer.AbstractHadoopGraphC
 import org.apache.tinkerpop.gremlin.hadoop.process.computer.util.ComputerSubmissionHelper;
 import org.apache.tinkerpop.gremlin.hadoop.process.computer.util.MapReduceHelper;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.FileSystemStorage;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.InputOutputHelper;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritableIterator;
@@ -52,6 +53,7 @@ import org.apache.tinkerpop.gremlin.process.computer.MapReduce;
 import org.apache.tinkerpop.gremlin.process.computer.VertexProgram;
 import org.apache.tinkerpop.gremlin.process.computer.util.DefaultComputerResult;
 import org.apache.tinkerpop.gremlin.process.computer.util.MapMemory;
+import org.apache.tinkerpop.gremlin.structure.io.Storage;
 import org.apache.tinkerpop.gremlin.util.Gremlin;
 
 import java.io.File;
@@ -126,7 +128,6 @@ public final class GiraphGraphComputer extends AbstractHadoopGraphComputer imple
             try {
                 final FileSystem fs = FileSystem.get(this.giraphConfiguration);
                 this.loadJars(fs);
-                fs.delete(new Path(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION)), true);
                 ToolRunner.run(this, new String[]{});
             } catch (final Exception e) {
                 //e.printStackTrace();
@@ -140,6 +141,8 @@ public final class GiraphGraphComputer extends AbstractHadoopGraphComputer imple
 
     @Override
     public int run(final String[] args) {
+        final Storage storage = FileSystemStorage.open(this.giraphConfiguration);
+        storage.rmr(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));
         this.giraphConfiguration.setBoolean(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT_HAS_EDGES, this.persist.equals(Persist.EDGES));
         try {
             // it is possible to run graph computer without a vertex program (and thus, only map reduce jobs if they exist)
@@ -170,13 +173,10 @@ public final class GiraphGraphComputer extends AbstractHadoopGraphComputer imple
                 }
                 // handle input paths (if any)
                 if (FileInputFormat.class.isAssignableFrom(this.giraphConfiguration.getClass(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, InputFormat.class))) {
-                    final Path inputPath = new Path(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_INPUT_LOCATION));
-                    if (!FileSystem.get(this.giraphConfiguration).exists(inputPath))  // TODO: what about when the input is not a file input?
-                        throw new IllegalArgumentException("The provided input path does not exist: " + inputPath);
-                    FileInputFormat.setInputPaths(job.getInternalJob(), inputPath);
+                    FileInputFormat.setInputPaths(job.getInternalJob(), Constants.getSearchGraphLocation(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_INPUT_LOCATION), storage).get());
                 }
                 // handle output paths
-                final Path outputPath = new Path(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION) + "/" + Constants.HIDDEN_G);
+                final Path outputPath = new Path(Constants.getGraphLocation(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION)));
                 FileOutputFormat.setOutputPath(job.getInternalJob(), outputPath);
                 job.getInternalJob().setJarByClass(GiraphGraphComputer.class);
                 this.logger.info(Constants.GREMLIN_HADOOP_GIRAPH_JOB_PREFIX + this.vertexProgram);
@@ -184,17 +184,18 @@ public final class GiraphGraphComputer extends AbstractHadoopGraphComputer imple
                 if (!job.run(true))
                     throw new IllegalStateException("The GiraphGraphComputer job failed -- aborting all subsequent MapReduce jobs");  // how do I get the exception that occured?
                 // add vertex program memory values to the return memory
-                for (final String key : this.vertexProgram.getMemoryComputeKeys()) {
-                    final Path path = new Path(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION) + "/" + key);
-                    final ObjectWritableIterator iterator = new ObjectWritableIterator(this.giraphConfiguration, path);
-                    if (iterator.hasNext()) {
-                        this.memory.set(key, iterator.next().getValue());
+                for (final String memoryKey : this.vertexProgram.getMemoryComputeKeys()) {
+                    if (storage.exists(Constants.getMemoryLocation(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), memoryKey))) {
+                        final ObjectWritableIterator iterator = new ObjectWritableIterator(this.giraphConfiguration, new Path(Constants.getMemoryLocation(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), memoryKey)));
+                        if (iterator.hasNext()) {
+                            this.memory.set(memoryKey, iterator.next().getValue());
+                        }
+                        storage.rmr(Constants.getMemoryLocation(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), memoryKey));
                     }
-                    FileSystem.get(this.giraphConfiguration).delete(path, true);
                 }
-                final Path path = new Path(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION) + "/" + Constants.HIDDEN_ITERATION);
+                final Path path = new Path(Constants.getMemoryLocation(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), Constants.HIDDEN_ITERATION));
                 this.memory.setIteration((Integer) new ObjectWritableIterator(this.giraphConfiguration, path).next().getValue());
-                FileSystem.get(this.giraphConfiguration).delete(path, true);
+                storage.rmr(Constants.getMemoryLocation(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), Constants.HIDDEN_ITERATION));
             }
             // do map reduce jobs
             this.giraphConfiguration.setBoolean(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT_HAS_EDGES, this.giraphConfiguration.getBoolean(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT_HAS_EDGES, true));
@@ -204,11 +205,8 @@ public final class GiraphGraphComputer extends AbstractHadoopGraphComputer imple
             }
 
             // if no persistence, delete the map reduce output
-            if (this.persist.equals(Persist.NOTHING)) {
-                final Path outputPath = new Path(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION) + "/" + Constants.HIDDEN_G);
-                if (FileSystem.get(this.giraphConfiguration).exists(outputPath))      // TODO: what about when the output is not a file output?
-                    FileSystem.get(this.giraphConfiguration).delete(outputPath, true);
-            }
+            if (this.persist.equals(Persist.NOTHING))
+                storage.rmr(Constants.getGraphLocation(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION)));
         } catch (final Exception e) {
             throw new IllegalStateException(e.getMessage(), e);
         }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/74b9c8ec/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/util/MapReduceHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/util/MapReduceHelper.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/util/MapReduceHelper.java
index 4c92abe..6e0cd9e 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/util/MapReduceHelper.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/util/MapReduceHelper.java
@@ -65,7 +65,7 @@ public final class MapReduceHelper {
         mapReduce.storeState(apacheConfiguration);
         ConfUtil.mergeApacheIntoHadoopConfiguration(apacheConfiguration, newConfiguration);
         if (!mapReduce.doStage(MapReduce.Stage.MAP)) {
-            final Path memoryPath = new Path(configuration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION) + "/" + mapReduce.getMemoryKey());
+            final Path memoryPath = new Path(Constants.getMemoryLocation(configuration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), mapReduce.getMemoryKey()));
             mapReduce.addResultToMemory(memory, new ObjectWritableIterator(configuration, memoryPath));
         } else {
             final Optional<Comparator<?>> mapSort = mapReduce.getMapKeySort();
@@ -100,9 +100,9 @@ public final class MapReduceHelper {
             job.setOutputFormatClass(SequenceFileOutputFormat.class);
             // if there is no vertex program, then grab the graph from the input location
             final Path graphPath = vertexProgramExists ?
-                    new Path(newConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION) + "/" + Constants.HIDDEN_G) :
+                    new Path(Constants.getGraphLocation(newConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION))) :
                     new Path(newConfiguration.get(Constants.GREMLIN_HADOOP_INPUT_LOCATION));
-            Path memoryPath = new Path(newConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION) + "/" + (reduceSort.isPresent() ? mapReduce.getMemoryKey() + "-temp" : mapReduce.getMemoryKey()));
+            Path memoryPath = new Path(Constants.getMemoryLocation(newConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), (reduceSort.isPresent() ? mapReduce.getMemoryKey() + "-temp" : mapReduce.getMemoryKey())));
             if (FileSystem.get(newConfiguration).exists(memoryPath)) {
                 FileSystem.get(newConfiguration).delete(memoryPath, true);
             }
@@ -124,7 +124,7 @@ public final class MapReduceHelper {
                 reduceSortJob.setOutputFormatClass(SequenceFileOutputFormat.class);
                 reduceSortJob.setNumReduceTasks(1); // todo: is this necessary to ensure sorted order?
                 FileInputFormat.setInputPaths(reduceSortJob, memoryPath);
-                final Path sortedMemoryPath = new Path(newConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION) + "/" + mapReduce.getMemoryKey());
+                final Path sortedMemoryPath = new Path(Constants.getMemoryLocation(newConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), mapReduce.getMemoryKey()));
                 FileOutputFormat.setOutputPath(reduceSortJob, sortedMemoryPath);
                 reduceSortJob.waitForCompletion(true);
                 FileSystem.get(newConfiguration).delete(memoryPath, true); // delete the temporary memory path

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/74b9c8ec/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
index b34f7a3..330227e 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
@@ -154,7 +154,7 @@ public final class FileSystemStorage implements Storage {
     @Override
     public Iterator<String> head(final String location, final int totalLines) {
         try {
-            return IteratorUtils.limit((Iterator) new TextIterator(fs.getConf(), new Path(location)), totalLines);
+            return IteratorUtils.limit((Iterator) new TextIterator(this.fs.getConf(), new Path(location)), totalLines);
         } catch (final IOException e) {
             throw new IllegalStateException(e.getMessage(), e);
         }
@@ -228,6 +228,8 @@ public final class FileSystemStorage implements Storage {
     ////////////
 
     private static boolean globDelete(final FileSystem fs, final String path, final boolean recursive) throws IOException {
+        if (!fs.exists(new Path(path)))
+            return false;
         boolean deleted = false;
         for (final Path p : FileUtil.stat2Paths(fs.globStatus(new Path(path)))) {
             fs.delete(p, recursive);

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/74b9c8ec/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/InputOutputHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/InputOutputHelper.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/InputOutputHelper.java
index 04097c1..48c2ad4 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/InputOutputHelper.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/InputOutputHelper.java
@@ -77,7 +77,7 @@ public final class InputOutputHelper {
         final BaseConfiguration newConfiguration = new BaseConfiguration();
         newConfiguration.copy(hadoopConfiguration);
         if (resultGraph.equals(GraphComputer.ResultGraph.NEW)) {
-            newConfiguration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, hadoopConfiguration.getOutputLocation() + "/" + Constants.HIDDEN_G);
+            newConfiguration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, hadoopConfiguration.getOutputLocation());
             if (hadoopConfiguration.containsKey(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT))
                 newConfiguration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, InputOutputHelper.getInputFormat(hadoopConfiguration.getGraphOutputFormat()).getCanonicalName());
             newConfiguration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, hadoopConfiguration.getOutputLocation() + "_");

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/74b9c8ec/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
index 4e4353d..95c19ed 100644
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopGremlinPluginCheck.java
@@ -135,30 +135,8 @@ public class HadoopGremlinPluginCheck extends AbstractGremlinTest {
         this.remote.connect(Arrays.asList("graph", "g"));
         Traversal<Vertex, String> traversal = (Traversal<Vertex, String>) this.remote.submit(Arrays.asList("g.V().hasLabel('person').group('m').by('age').by('name').out('knows').out('created').values('name')"));
         AbstractGremlinProcessTest.checkResults(Arrays.asList("ripple", "lop"), traversal);
-        assertTrue((Boolean) this.console.eval("hdfs.exists('target/test-output/m')"));
-        assertTrue((Boolean) this.console.eval("hdfs.exists('target/test-output/" + TraverserMapReduce.TRAVERSERS + "')"));
-        final List<KeyValue<Integer, Collection<String>>> mList = IteratorUtils.asList(this.console.eval("hdfs.head('target/test-output','m',SequenceFileInputFormat)"));
-        assertEquals(4, mList.size());
-        mList.forEach(keyValue -> {
-            if (keyValue.getKey().equals(29))
-                assertTrue(keyValue.getValue().contains("marko"));
-            else if (keyValue.getKey().equals(35))
-                assertTrue(keyValue.getValue().contains("peter"));
-            else if (keyValue.getKey().equals(32))
-                assertTrue(keyValue.getValue().contains("josh"));
-            else if (keyValue.getKey().equals(27))
-                assertTrue(keyValue.getValue().contains("vadas"));
-            else
-                throw new IllegalStateException("The provided key/value is unknown: " + keyValue);
-        });
-        final List<KeyValue<MapReduce.NullObject, Traverser<String>>> traversersList = IteratorUtils.asList(this.console.eval("hdfs.head('target/test-output/'," + "'" + TraverserMapReduce.TRAVERSERS + "',SequenceFileInputFormat)"));
-        assertEquals(2, traversersList.size());
-        traversersList.forEach(keyValue -> {
-            assertEquals(MapReduce.NullObject.instance(), keyValue.getKey());
-            final String name = keyValue.getValue().get();
-            assertTrue(name.equals("ripple") || name.equals("lop"));
-        });
-        ////////////////
+        assertFalse((Boolean) this.console.eval("hdfs.exists('target/test-output/m')"));
+        assertFalse((Boolean) this.console.eval("hdfs.exists('target/test-output/" + TraverserMapReduce.TRAVERSERS + "')"));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/74b9c8ec/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
index 3f7efaa..ecd9573 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
@@ -24,6 +24,7 @@ import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.spark.SparkConf;

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/74b9c8ec/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/Spark.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/Spark.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/Spark.java
index 2761efb..0bf679b 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/Spark.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/Spark.java
@@ -34,6 +34,8 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.stream.Collectors;
 
 /**
+ * This is a static cache the prevents Spark from garbage collecting unreferenced RDDs.
+ *
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
 public class Spark {

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/74b9c8ec/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
index 2ea0fc3..97a26f1 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
@@ -75,7 +75,7 @@ public final class SparkContextStorage implements Storage {
     @Override
     public List<String> ls(final String location) {
         final List<String> rdds = new ArrayList<>();
-        final String wildCardLocation = location.replace(".", "\\.").replace("*", ".*");
+        final String wildCardLocation = (location.endsWith("*") ? location : location + "*").replace(".", "\\.").replace("*", ".*");
         for (final RDD<?> rdd : Spark.getRDDs()) {
             if (rdd.name().matches(wildCardLocation))
                 rdds.add(rdd.name() + " [" + rdd.getStorageLevel().description() + "]");


[20/30] incubator-tinkerpop git commit: merged master -- CHANGELOG updates and spark/groovy version bumps.

Posted by dk...@apache.org.
merged master -- CHANGELOG updates and spark/groovy version bumps.


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/11b154e3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/11b154e3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/11b154e3

Branch: refs/heads/TINKERPOP-320
Commit: 11b154e3a700f201f123c36aadb1ba838feac914
Parents: 9f5879e 78b1056
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Thu Jan 7 17:55:11 2016 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Thu Jan 7 17:55:11 2016 -0700

----------------------------------------------------------------------
 CHANGELOG.asciidoc    | 2 ++
 pom.xml               | 2 +-
 spark-gremlin/pom.xml | 2 +-
 3 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/11b154e3/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --cc CHANGELOG.asciidoc
index fad0630,0990ef1..3185290
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@@ -26,11 -26,8 +26,13 @@@ image::https://raw.githubusercontent.co
  TinkerPop 3.1.1 (NOT OFFICIALLY RELEASED YET)
  ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
  
 +* It is possible to completely avoid using HDFS with Spark if `PersistedInputRDD` and `PersistedOutpuRDD` are leveraged.
 +* `InputRDD` and `OutputRDD` can now process both graphs and memory (i.e. sideEffects).
 +* Removed Groovy specific meta-programming overloads for handling Hadoop `FileSystem` (instead, its all accessible via `FileSystemStorage`).
 +* Added `FileSystemStorage` and `SparkContextStorage` which both implement the new `Storage` API.
 +* Added `Storage` to the gremlin-core io-package which providers can implement to allow conventional access to data sources (e.g. `ls()`, `rm()`, `cp()`, etc.).
+ * Bumped to Spark 1.5.2.
+ * Bumped to Groovy 2.4.5.
  * Execute the `LifeCycle.beforeEval()` in the same thread that `eval()` is executed in for `GremlinExecutor`.
  * Improved error handling of Gremlin Console initialization scripts to better separate errors in initialization script I/O versus execution of the script itself.
  * Fixed a bug in `Graph.OptOut` when trying to opt-out of certain test cases with the `method` property set to "*".


[21/30] incubator-tinkerpop git commit: Merge branch 'TINKERPOP-1033'

Posted by dk...@apache.org.
Merge branch 'TINKERPOP-1033'


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/114609df
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/114609df
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/114609df

Branch: refs/heads/TINKERPOP-320
Commit: 114609df01b79f2e7d9dbd11d9bcf8c74aa70d24
Parents: 8749d3d 11b154e
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Fri Jan 8 07:20:11 2016 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Fri Jan 8 07:20:11 2016 -0700

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |   5 +
 docs/src/reference/implementations.asciidoc     | 104 ++++----
 .../upgrade/release-3.1.x-incubating.asciidoc   |  26 ++
 .../process/computer/GiraphGraphComputer.java   |  43 ++--
 .../GiraphHadoopGremlinIntegrateTest.java       |  33 +++
 .../computer/GiraphHadoopGraphProvider.java     |   6 +-
 .../GiraphHadoopGremlinPluginIntegrateTest.java |  33 ---
 .../peerpressure/ClusterCountMapReduce.java     |   7 +-
 .../tinkerpop/gremlin/structure/io/Storage.java | 139 +++++++++++
 .../gremlin/structure/util/StringFactory.java   |  18 +-
 .../process/computer/GraphComputerTest.java     |  54 ++--
 .../conf/hadoop-grateful-gryo.properties        |   6 +-
 .../hadoop/groovy/plugin/HadoopLoader.groovy    | 138 -----------
 .../tinkerpop/gremlin/hadoop/Constants.java     |  20 ++
 .../groovy/plugin/HadoopGremlinPlugin.java      |  11 +-
 .../process/computer/util/MapReduceHelper.java  |   8 +-
 .../hadoop/structure/HadoopConfiguration.java   |   5 +
 .../gremlin/hadoop/structure/HadoopGraph.java   |   4 +-
 .../hadoop/structure/hdfs/HDFSTools.java        | 113 ---------
 .../structure/hdfs/HadoopEdgeIterator.java      |  83 -------
 .../structure/hdfs/HadoopElementIterator.java   |  74 ------
 .../structure/hdfs/HadoopVertexIterator.java    |  82 -------
 .../hadoop/structure/hdfs/HiddenFileFilter.java |  44 ----
 .../hadoop/structure/hdfs/TextIterator.java     |  91 -------
 .../hadoop/structure/io/FileSystemStorage.java  | 246 +++++++++++++++++++
 .../hadoop/structure/io/HadoopEdgeIterator.java |  79 ++++++
 .../structure/io/HadoopElementIterator.java     |  75 ++++++
 .../structure/io/HadoopVertexIterator.java      |  78 ++++++
 .../hadoop/structure/io/HiddenFileFilter.java   |  44 ++++
 .../hadoop/structure/io/InputOutputHelper.java  |   2 +-
 .../structure/io/ObjectWritableIterator.java    |  12 +-
 .../hadoop/structure/io/TextIterator.java       |  91 +++++++
 .../structure/io/VertexWritableIterator.java    |  10 +-
 .../gremlin/hadoop/HadoopGremlinSuite.java      |  36 +++
 .../groovy/plugin/HadoopGremlinPluginCheck.java |  71 +-----
 .../hadoop/groovy/plugin/HadoopPluginSuite.java |  34 ---
 .../structure/io/AbstractStorageCheck.java      | 145 +++++++++++
 .../structure/io/FileSystemStorageCheck.java    |  88 +++++++
 .../spark/groovy/plugin/SparkLoader.groovy      |  68 -----
 .../spark/groovy/plugin/SparkGremlinPlugin.java |   5 +-
 .../spark/process/computer/SparkExecutor.java   |  28 ---
 .../process/computer/SparkGraphComputer.java    |  18 +-
 .../gremlin/spark/structure/Spark.java          |   2 +
 .../spark/structure/io/InputFormatRDD.java      |  15 ++
 .../spark/structure/io/InputOutputHelper.java   |   3 -
 .../gremlin/spark/structure/io/InputRDD.java    |  17 +-
 .../spark/structure/io/OutputFormatRDD.java     |  30 ++-
 .../gremlin/spark/structure/io/OutputRDD.java   |  21 ++
 .../spark/structure/io/PersistedInputRDD.java   |  14 +-
 .../spark/structure/io/PersistedOutputRDD.java  |  25 +-
 .../spark/structure/io/SparkContextStorage.java | 164 +++++++++++++
 .../gremlin/spark/AbstractSparkTest.java        |  30 +++
 .../gremlin/spark/SparkGremlinSuite.java        |  35 +++
 .../gremlin/spark/SparkGremlinTest.java         |  33 +++
 .../gremlin/spark/SparkHadoopGremlinTest.java   |  33 +++
 .../process/computer/LocalPropertyTest.java     |   2 +-
 .../computer/SparkHadoopGraphProvider.java      |  12 +-
 .../groovy/plugin/SparkGremlinPluginTest.java   | 126 ----------
 .../plugin/SparkHadoopGremlinPluginTest.java    |  33 ---
 .../gremlin/spark/structure/SparkTest.java      |  10 +-
 .../spark/structure/io/ExampleInputRDD.java     |   5 +
 .../spark/structure/io/ExampleOutputRDD.java    |   6 +
 .../io/PersistedInputOutputRDDTest.java         |  29 +--
 .../structure/io/SparkContextStorageCheck.java  |  74 ++++++
 .../spark/structure/io/ToyGraphInputRDD.java    |   7 +
 65 files changed, 1801 insertions(+), 1202 deletions(-)
----------------------------------------------------------------------



[14/30] incubator-tinkerpop git commit: Merge branch 'master' into TINKERPOP-1033

Posted by dk...@apache.org.
Merge branch 'master' into TINKERPOP-1033


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

Branch: refs/heads/TINKERPOP-320
Commit: a7db52bda732810fc8d5d3a8279a4f7095285d3d
Parents: 5c9e81b b44253d
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Jan 6 16:03:59 2016 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Wed Jan 6 16:03:59 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/tinkerpop/gremlin/driver/Cluster.java    | 3 +++
 .../main/java/org/apache/tinkerpop/gremlin/driver/Settings.java   | 2 ++
 2 files changed, 5 insertions(+)
----------------------------------------------------------------------



[25/30] incubator-tinkerpop git commit: Merge branch 'TINKERPOP-1075-fix-profile-duration-cap-step' of https://github.com/rjbriody/incubator-tinkerpop

Posted by dk...@apache.org.
Merge branch 'TINKERPOP-1075-fix-profile-duration-cap-step' of https://github.com/rjbriody/incubator-tinkerpop


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/1e5dcaf6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/1e5dcaf6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/1e5dcaf6

Branch: refs/heads/TINKERPOP-320
Commit: 1e5dcaf67e3b4aed2ea03ba59b902481d775118c
Parents: b4cb00d 2089106
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Mon Jan 11 08:44:46 2016 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Mon Jan 11 08:44:46 2016 -0700

----------------------------------------------------------------------
 .../traversal/util/DependantMutableMetrics.java | 34 +++++++++++---------
 .../process/traversal/util/MutableMetrics.java  |  3 +-
 2 files changed, 21 insertions(+), 16 deletions(-)
----------------------------------------------------------------------



[04/30] incubator-tinkerpop git commit: Greatly greatly simplified Hadoop OLTP and interactions with HDFS and SparkContext. The trend -- dir/~g for graphs and dir/x for memory. A consistent persistence schema makes everything so much simpler. I always as

Posted by dk...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
index 2db267f..887e2f9 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
@@ -19,11 +19,18 @@
 
 package org.apache.tinkerpop.gremlin.spark.structure.io;
 
+import org.apache.commons.configuration.BaseConfiguration;
 import org.apache.commons.configuration.Configuration;
+import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.spark.SparkContext;
+import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.rdd.RDD;
+import org.apache.tinkerpop.gremlin.hadoop.Constants;
+import org.apache.tinkerpop.gremlin.process.computer.KeyValue;
 import org.apache.tinkerpop.gremlin.spark.structure.Spark;
+import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.structure.io.Storage;
+import org.apache.tinkerpop.gremlin.structure.util.StringFactory;
 import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 import scala.collection.JavaConversions;
 
@@ -114,11 +121,53 @@ public final class SparkContextStorage implements Storage {
     }
 
     @Override
-    public <V> Iterator<V> head(final String location, final int totalLines, final Class<V> objectClass) {
-        return IteratorUtils.limit((Iterator) JavaConversions.asJavaIterator(Spark.getRDD(location).toLocalIterator()), totalLines);
+    public Iterator<Vertex> headGraph(final String location, int totalLines, final Class parserClass) {
+        final Configuration configuration = new BaseConfiguration();
+        configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, Constants.getSearchGraphLocation(location, this).get());
+        configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_INPUT_RDD, parserClass.getCanonicalName());
+        configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, parserClass.getCanonicalName());
+        try {
+            if (InputRDD.class.isAssignableFrom(parserClass)) {
+                return IteratorUtils.limit(IteratorUtils.map(((InputRDD) parserClass.getConstructor().newInstance()).readGraphRDD(configuration, new JavaSparkContext(Spark.getContext())).toLocalIterator(), tuple -> tuple._2().get()), totalLines);
+            } else if (InputFormat.class.isAssignableFrom(parserClass)) {
+                return IteratorUtils.limit(IteratorUtils.map(new InputFormatRDD().readGraphRDD(configuration, new JavaSparkContext(Spark.getContext())).toLocalIterator(), tuple -> tuple._2().get()), totalLines);
+            }
+        } catch (final Exception e) {
+            throw new IllegalArgumentException(e.getMessage(), e);
+        }
+        throw new IllegalArgumentException("The provided parserClass must be an " + InputFormat.class.getCanonicalName() + " or a " + InputRDD.class.getCanonicalName() + ": " + parserClass.getCanonicalName());
+    }
+
+    @Override
+    public <K, V> Iterator<KeyValue<K, V>> headMemory(final String location, final String memoryKey, int totalLines, Class parserClass) {
+        final Configuration configuration = new BaseConfiguration();
+        configuration.setProperty(Constants.GREMLIN_HADOOP_INPUT_LOCATION, Constants.getMemoryLocation(location, memoryKey));
+        configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_INPUT_RDD, parserClass.getCanonicalName());
+        configuration.setProperty(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, parserClass.getCanonicalName());
+        try {
+            if (InputRDD.class.isAssignableFrom(parserClass)) {
+                return IteratorUtils.limit(IteratorUtils.map(((InputRDD) parserClass.getConstructor().newInstance()).readMemoryRDD(configuration, memoryKey, new JavaSparkContext(Spark.getContext())).toLocalIterator(), tuple -> new KeyValue(tuple._1(), tuple._2())), totalLines);
+            } else if (InputFormat.class.isAssignableFrom(parserClass)) {
+                return IteratorUtils.limit(IteratorUtils.map(new InputFormatRDD().readMemoryRDD(configuration, memoryKey, new JavaSparkContext(Spark.getContext())).toLocalIterator(), tuple -> new KeyValue(tuple._1(), tuple._2())), totalLines);
+            }
+        } catch (final Exception e) {
+            throw new IllegalArgumentException(e.getMessage(), e);
+        }
+        throw new IllegalArgumentException("The provided parserClass must be an " + InputFormat.class.getCanonicalName() + " or a " + InputRDD.class.getCanonicalName() + ": " + parserClass.getCanonicalName());
     }
 
+    @Override
+    public Iterator<String> head(final String location, final int totalLines) {
+        return IteratorUtils.limit(IteratorUtils.map(JavaConversions.asJavaIterator(Spark.getRDD(location).toLocalIterator()), Object::toString), totalLines);
+    }
+
+    // TODO: @Override
     public String describe(final String location) {
         return Spark.getRDD(location).toDebugString();
     }
+
+    @Override
+    public String toString() {
+        return StringFactory.storageString(Spark.getContext().master());
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/GraphMemorySparkTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/GraphMemorySparkTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/GraphMemorySparkTest.java
deleted file mode 100644
index 10153b0..0000000
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/GraphMemorySparkTest.java
+++ /dev/null
@@ -1,75 +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.tinkerpop.gremlin.spark.structure.io;
-
-import org.apache.commons.configuration.Configuration;
-import org.apache.tinkerpop.gremlin.hadoop.Constants;
-import org.apache.tinkerpop.gremlin.process.computer.ComputerResult;
-import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.ClusterCountMapReduce;
-import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.PeerPressureVertexProgram;
-import org.apache.tinkerpop.gremlin.spark.AbstractSparkTest;
-import org.apache.tinkerpop.gremlin.spark.process.computer.SparkGraphComputer;
-import org.apache.tinkerpop.gremlin.spark.process.computer.SparkHadoopGraphProvider;
-import org.apache.tinkerpop.gremlin.structure.Graph;
-import org.apache.tinkerpop.gremlin.structure.io.Storage;
-import org.apache.tinkerpop.gremlin.structure.util.GraphFactory;
-import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
-import org.junit.Test;
-import scala.Tuple2;
-
-import java.util.UUID;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-public class GraphMemorySparkTest extends AbstractSparkTest {
-
-    @Test
-    public void shouldPersistGraphAndMemory() throws Exception {
-        final String outputLocation = "target/test-output/" + UUID.randomUUID();
-        final Configuration configuration = getBaseConfiguration(SparkHadoopGraphProvider.PATHS.get("tinkerpop-modern.kryo"));
-        configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
-        configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, outputLocation);
-        configuration.setProperty(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, true);
-        /////
-        Graph graph = GraphFactory.open(configuration);
-        final ComputerResult result = graph.compute(SparkGraphComputer.class).program(PeerPressureVertexProgram.build().create(graph)).mapReduce(ClusterCountMapReduce.build().memoryKey("clusterCount").create()).submit().get();
-        /////
-        final Storage storage = SparkContextStorage.open("local[4]");
-
-        assertEquals(2, storage.ls().size());
-        // TEST GRAPH PERSISTENCE
-        assertTrue(storage.exists(Constants.getGraphLocation(outputLocation)));
-        assertEquals(6, IteratorUtils.count(storage.head(Constants.getGraphLocation(outputLocation), Tuple2.class)));
-        assertEquals(6, result.graph().traversal().V().count().next().longValue());
-        assertEquals(0, result.graph().traversal().E().count().next().longValue());
-        assertEquals(6, result.graph().traversal().V().values("name").count().next().longValue());
-        assertEquals(6, result.graph().traversal().V().values(PeerPressureVertexProgram.CLUSTER).count().next().longValue());
-        /////
-        // TEST MEMORY PERSISTENCE
-        assertEquals(2, (int) result.memory().get("clusterCount"));
-        assertTrue(storage.exists(Constants.getMemoryLocation(outputLocation, "clusterCount")));
-        assertEquals(2, storage.head(Constants.getMemoryLocation(outputLocation, "clusterCount"), Tuple2.class).next()._2());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputOutputRDDTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputOutputRDDTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputOutputRDDTest.java
index 954cdfe..44e0949 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputOutputRDDTest.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedInputOutputRDDTest.java
@@ -21,13 +21,9 @@ package org.apache.tinkerpop.gremlin.spark.structure.io;
 
 import org.apache.commons.configuration.BaseConfiguration;
 import org.apache.commons.configuration.Configuration;
-import org.apache.spark.SparkConf;
-import org.apache.spark.SparkContext;
-import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.tinkerpop.gremlin.hadoop.Constants;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.gryo.GryoInputFormat;
-import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
 import org.apache.tinkerpop.gremlin.process.computer.GraphComputer;
 import org.apache.tinkerpop.gremlin.process.computer.bulkloading.BulkLoaderVertexProgram;
 import org.apache.tinkerpop.gremlin.process.computer.ranking.pagerank.PageRankVertexProgram;
@@ -47,7 +43,9 @@ import org.junit.Test;
 
 import java.util.UUID;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
@@ -214,7 +212,7 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
     public void testComplexChain() throws Exception {
         Spark.create("local[4]");
 
-        final String rddLocation = "target/test-output/graphRDD";
+        final String rddLocation = "target/test-output/" + UUID.randomUUID();
         final Configuration configuration = new BaseConfiguration();
         configuration.setProperty("spark.master", "local[4]");
         configuration.setProperty("spark.serializer", GryoSerializer.class.getCanonicalName());
@@ -225,6 +223,8 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
         configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, rddLocation);
         configuration.setProperty(Constants.GREMLIN_HADOOP_JARS_IN_DISTRIBUTED_CACHE, false);
         configuration.setProperty(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, true);
+
+        assertFalse(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
         Graph graph = GraphFactory.open(configuration);
         graph = graph.compute(SparkGraphComputer.class).persist(GraphComputer.Persist.EDGES).program(PageRankVertexProgram.build().iterations(2).create(graph)).submit().get().graph();
         GraphTraversalSource g = graph.traversal();
@@ -262,7 +262,7 @@ public class PersistedInputOutputRDDTest extends AbstractSparkTest {
         ////
         graph = GraphFactory.open(configuration);
         graph.compute(SparkGraphComputer.class).persist(GraphComputer.Persist.NOTHING).program(PageRankVertexProgram.build().iterations(2).create(graph)).submit().get().graph();
-        assertFalse(Spark.hasRDD(rddLocation));
+        assertFalse(Spark.hasRDD(Constants.getGraphLocation(rddLocation)));
         g = graph.traversal();
         assertEquals(0l, g.V().count().next().longValue());
         assertEquals(0l, g.E().count().next().longValue());

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4d8e960/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageTest.java
new file mode 100644
index 0000000..43e8508
--- /dev/null
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.tinkerpop.gremlin.spark.structure.io;
+
+import org.apache.commons.configuration.Configuration;
+import org.apache.tinkerpop.gremlin.hadoop.Constants;
+import org.apache.tinkerpop.gremlin.process.computer.ComputerResult;
+import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.ClusterCountMapReduce;
+import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.PeerPressureVertexProgram;
+import org.apache.tinkerpop.gremlin.spark.AbstractSparkTest;
+import org.apache.tinkerpop.gremlin.spark.process.computer.SparkGraphComputer;
+import org.apache.tinkerpop.gremlin.spark.process.computer.SparkHadoopGraphProvider;
+import org.apache.tinkerpop.gremlin.structure.Graph;
+import org.apache.tinkerpop.gremlin.structure.io.Storage;
+import org.apache.tinkerpop.gremlin.structure.util.GraphFactory;
+import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public class SparkContextStorageTest extends AbstractSparkTest {
+
+    @Test
+    public void shouldPersistGraphAndMemory() throws Exception {
+        final String outputLocation = "target/test-output/" + UUID.randomUUID();
+        final Configuration configuration = getBaseConfiguration(SparkHadoopGraphProvider.PATHS.get("tinkerpop-modern.kryo"));
+        configuration.setProperty(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, PersistedOutputRDD.class.getCanonicalName());
+        configuration.setProperty(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, outputLocation);
+        configuration.setProperty(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, true);
+        /////
+        Graph graph = GraphFactory.open(configuration);
+        final ComputerResult result = graph.compute(SparkGraphComputer.class).program(PeerPressureVertexProgram.build().create(graph)).mapReduce(ClusterCountMapReduce.build().memoryKey("clusterCount").create()).submit().get();
+        /////
+        final Storage storage = SparkContextStorage.open("local[4]");
+
+        assertEquals(2, storage.ls().size());
+        // TEST GRAPH PERSISTENCE
+        assertTrue(storage.exists(Constants.getGraphLocation(outputLocation)));
+        assertEquals(6, IteratorUtils.count(storage.headGraph(outputLocation, PersistedInputRDD.class)));
+        assertEquals(6, result.graph().traversal().V().count().next().longValue());
+        assertEquals(0, result.graph().traversal().E().count().next().longValue());
+        assertEquals(6, result.graph().traversal().V().values("name").count().next().longValue());
+        assertEquals(6, result.graph().traversal().V().values(PeerPressureVertexProgram.CLUSTER).count().next().longValue());
+        /////
+        // TEST MEMORY PERSISTENCE
+        assertEquals(2, (int) result.memory().get("clusterCount"));
+        assertTrue(storage.exists(Constants.getMemoryLocation(outputLocation, "clusterCount")));
+        assertEquals(2, storage.headMemory(outputLocation, "clusterCount", PersistedInputRDD.class).next().getValue());
+    }
+
+}


[18/30] incubator-tinkerpop git commit: Add another test to validate TinkerGraph serialization.

Posted by dk...@apache.org.
Add another test to validate TinkerGraph serialization.

Test type coersion again but this time with a TinkerGraph and with embedding in other objects. CTR


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/8749d3d9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/8749d3d9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/8749d3d9

Branch: refs/heads/TINKERPOP-320
Commit: 8749d3d9d7837b8d2f887a7a4fe414b2bae59fbd
Parents: 78b1056
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Thu Jan 7 17:09:43 2016 -0500
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Jan 7 17:10:49 2016 -0500

----------------------------------------------------------------------
 .../tinkergraph/structure/TinkerIoRegistry.java |   1 +
 .../tinkergraph/structure/TinkerGraphTest.java  | 129 ++++++++++++++++++-
 2 files changed, 129 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/8749d3d9/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerIoRegistry.java
----------------------------------------------------------------------
diff --git a/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerIoRegistry.java b/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerIoRegistry.java
index abdef24..9080c94 100644
--- a/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerIoRegistry.java
+++ b/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerIoRegistry.java
@@ -23,6 +23,7 @@ import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.structure.io.AbstractIoRegistry;
 import org.apache.tinkerpop.gremlin.structure.io.GraphReader;
 import org.apache.tinkerpop.gremlin.structure.io.GraphWriter;
+import org.apache.tinkerpop.gremlin.structure.io.IoCore;
 import org.apache.tinkerpop.gremlin.structure.io.IoRegistry;
 import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONIo;
 import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONTokens;

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/8749d3d9/tinkergraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraphTest.java
----------------------------------------------------------------------
diff --git a/tinkergraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraphTest.java b/tinkergraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraphTest.java
index 47ba642..fca1275 100644
--- a/tinkergraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraphTest.java
+++ b/tinkergraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraphTest.java
@@ -22,8 +22,10 @@ import org.apache.commons.configuration.BaseConfiguration;
 import org.apache.commons.configuration.Configuration;
 import org.apache.tinkerpop.gremlin.TestHelper;
 import org.apache.tinkerpop.gremlin.process.traversal.P;
+import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversalSource;
 import org.apache.tinkerpop.gremlin.structure.Edge;
 import org.apache.tinkerpop.gremlin.structure.Graph;
+import org.apache.tinkerpop.gremlin.structure.T;
 import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.structure.io.Io;
 import org.apache.tinkerpop.gremlin.structure.io.GraphReader;
@@ -34,15 +36,34 @@ import org.apache.tinkerpop.gremlin.structure.io.IoTest;
 import org.apache.tinkerpop.gremlin.structure.io.Mapper;
 import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONReader;
 import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONWriter;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoClassResolver;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoMapper;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoWriter;
+import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 import org.apache.tinkerpop.shaded.jackson.databind.ObjectMapper;
+import org.apache.tinkerpop.shaded.kryo.ClassResolver;
+import org.apache.tinkerpop.shaded.kryo.Kryo;
+import org.apache.tinkerpop.shaded.kryo.Registration;
+import org.apache.tinkerpop.shaded.kryo.Serializer;
+import org.apache.tinkerpop.shaded.kryo.io.Input;
+import org.apache.tinkerpop.shaded.kryo.io.Output;
 import org.junit.Test;
 
+import java.awt.Color;
 import java.io.BufferedOutputStream;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.FileOutputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
 import java.util.Set;
+import java.util.function.Supplier;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -281,7 +302,8 @@ public class TinkerGraphTest {
         final TinkerGraph graph = TinkerFactory.createModern();
         try (final ByteArrayOutputStream out = new ByteArrayOutputStream()) {
             graph.io(IoCore.gryo()).writer().create().writeObject(out, graph);
-            try (final ByteArrayInputStream inputStream = new ByteArrayInputStream(out.toByteArray())) {
+            final byte[] b = out.toByteArray();
+            try (final ByteArrayInputStream inputStream = new ByteArrayInputStream(b)) {
                 final TinkerGraph target = graph.io(IoCore.gryo()).reader().create().readObject(inputStream, TinkerGraph.class);
                 IoTest.assertModernGraph(target, true, false);
             }
@@ -448,6 +470,111 @@ public class TinkerGraphTest {
         assertEquals(TestIoBuilder.calledCreate, 1);
     }
 
+    @Test
+    public void shouldSerializeWithColorClassResolverToTinkerGraph() throws Exception {
+        final Map<String,Color> colors = new HashMap<>();
+        colors.put("red", Color.RED);
+        colors.put("green", Color.GREEN);
+
+        final ArrayList<Color> colorList = new ArrayList<>(Arrays.asList(Color.RED, Color.GREEN));
+
+        final Supplier<ClassResolver> classResolver = new CustomClassResolverSupplier();
+        final GryoMapper mapper = GryoMapper.build().addRegistry(TinkerIoRegistry.getInstance()).classResolver(classResolver).create();
+        final Kryo kryo = mapper.createMapper();
+        try (final ByteArrayOutputStream stream = new ByteArrayOutputStream()) {
+            final Output out = new Output(stream);
+
+            kryo.writeObject(out, colorList);
+            out.flush();
+            final byte[] b = stream.toByteArray();
+
+            try (final InputStream inputStream = new ByteArrayInputStream(b)) {
+                final Input input = new Input(inputStream);
+                final List m = kryo.readObject(input, ArrayList.class);
+                final TinkerGraph readX = (TinkerGraph) m.get(0);
+                assertEquals(104, IteratorUtils.count(readX.vertices()));
+                assertEquals(102, IteratorUtils.count(readX.edges()));
+            }
+        }
+    }
+
+    /**
+     * Coerces a {@code Color} to a {@link TinkerGraph} during serialization.  Demonstrates how custom serializers
+     * can be developed that can coerce one value to another during serialization.
+     */
+    public final static class ColorToTinkerGraphSerializer extends Serializer<Color> {
+        public ColorToTinkerGraphSerializer() {
+        }
+
+        @Override
+        public void write(final Kryo kryo, final Output output, final Color color) {
+            final TinkerGraph graph = TinkerGraph.open();
+            final Vertex v = graph.addVertex(T.id, 1, T.label, "color", "name", color.toString());
+            final Vertex vRed = graph.addVertex(T.id, 2, T.label, "primary", "name", "red");
+            final Vertex vGreen = graph.addVertex(T.id, 3, T.label, "primary", "name", "green");
+            final Vertex vBlue = graph.addVertex(T.id, 4, T.label, "primary", "name", "blue");
+
+            v.addEdge("hasComponent", vRed, "amount", color.getRed());
+            v.addEdge("hasComponent", vGreen, "amount", color.getGreen());
+            v.addEdge("hasComponent", vBlue, "amount", color.getBlue());
+
+            // make some junk so the graph is kinda big
+            generate(graph);
+
+            try (final ByteArrayOutputStream stream = new ByteArrayOutputStream()) {
+                GryoWriter.build().mapper(() -> kryo).create().writeGraph(stream, graph);
+                final byte[] bytes = stream.toByteArray();
+                output.writeInt(bytes.length);
+                output.write(bytes);
+            } catch (Exception ex) {
+                ex.printStackTrace();
+            }
+        }
+
+        @Override
+        public Color read(final Kryo kryo, final Input input, final Class<Color> colorClass) {
+            throw new UnsupportedOperationException("IoX writes to DetachedVertex and can't be read back in as IoX");
+        }
+
+        private static void generate(final Graph graph) {
+            final int size = 100;
+            final List<Object> ids = new ArrayList<>();
+            final Vertex v = graph.addVertex("sin", 0.0f, "cos", 1.0f, "ii", 0f);
+            ids.add(v.id());
+
+            final GraphTraversalSource g = graph.traversal();
+
+            final Random rand = new Random();
+            for (int ii = 1; ii < size; ii++) {
+                final Vertex t = graph.addVertex("ii", ii, "sin", Math.sin(ii / 5.0f), "cos", Math.cos(ii / 5.0f));
+                final Vertex u = g.V(ids.get(rand.nextInt(ids.size()))).next();
+                t.addEdge("linked", u);
+                ids.add(u.id());
+                ids.add(v.id());
+            }
+        }
+    }
+
+    public static class CustomClassResolverSupplier implements Supplier<ClassResolver> {
+        @Override
+        public ClassResolver get() {
+            return new CustomClassResolver();
+        }
+    }
+
+    public static class CustomClassResolver extends GryoClassResolver {
+        private ColorToTinkerGraphSerializer colorToGraphSerializer = new ColorToTinkerGraphSerializer();
+
+        public Registration getRegistration(final Class clazz) {
+            if (Color.class.isAssignableFrom(clazz)) {
+                final Registration registration = super.getRegistration(TinkerGraph.class);
+                return new Registration(registration.getType(), colorToGraphSerializer, registration.getId());
+            } else {
+                return super.getRegistration(clazz);
+            }
+        }
+    }
+
     public static class TestIoBuilder implements Io.Builder{
 
         static int calledRegistry, calledGraph, calledCreate;


[27/30] incubator-tinkerpop git commit: fixed a bug in ObjectWritable comparable that showed up with Giraph. If the objects are not comparable and you say they are equal, then equals() is not used for comparison. Weak. I simple -1 for non-comparable obje

Posted by dk...@apache.org.
fixed a bug in ObjectWritable comparable that showed up with Giraph. If the objects are not comparable and you say they are equal, then equals() is not used for comparison. Weak. I simple -1 for non-comparable objects solves the problem.


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/4ad66170
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/4ad66170
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/4ad66170

Branch: refs/heads/TINKERPOP-320
Commit: 4ad66170a792b2768cf95d18271bb5ccd62fee2d
Parents: d7b295b
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Mon Jan 11 09:31:00 2016 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Mon Jan 11 09:31:00 2016 -0700

----------------------------------------------------------------------
 .../tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java      | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/4ad66170/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
index a4cde0e..765b6d5 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
@@ -97,7 +97,7 @@ public final class ObjectWritable<T> implements WritableComparable<ObjectWritabl
 
     @Override
     public int compareTo(final ObjectWritable objectWritable) {
-        return this.t instanceof Comparable ? ((Comparable) this.t).compareTo(objectWritable.get()) : 0;
+        return this.t instanceof Comparable ? ((Comparable) this.t).compareTo(objectWritable.get()) : -1;
     }
 
     public boolean isEmpty() {


[16/30] incubator-tinkerpop git commit: bumped to Groovy 2.4.5 and Spark 1.5.2. Both minor release bumps and all tests pass (as well as Spark integration tests).

Posted by dk...@apache.org.
bumped to Groovy 2.4.5 and Spark 1.5.2. Both minor release bumps and all tests pass (as well as Spark integration tests).


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/78b10569
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/78b10569
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/78b10569

Branch: refs/heads/TINKERPOP-320
Commit: 78b10569755070b088c460341bb473112dfe3ffe
Parents: b44253d
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Thu Jan 7 11:00:18 2016 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Thu Jan 7 11:00:18 2016 -0700

----------------------------------------------------------------------
 CHANGELOG.asciidoc    | 2 ++
 pom.xml               | 2 +-
 spark-gremlin/pom.xml | 2 +-
 3 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/78b10569/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index 43fb4b6..0990ef1 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -26,6 +26,8 @@ image::https://raw.githubusercontent.com/apache/incubator-tinkerpop/master/docs/
 TinkerPop 3.1.1 (NOT OFFICIALLY RELEASED YET)
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
+* Bumped to Spark 1.5.2.
+* Bumped to Groovy 2.4.5.
 * Execute the `LifeCycle.beforeEval()` in the same thread that `eval()` is executed in for `GremlinExecutor`.
 * Improved error handling of Gremlin Console initialization scripts to better separate errors in initialization script I/O versus execution of the script itself.
 * Fixed a bug in `Graph.OptOut` when trying to opt-out of certain test cases with the `method` property set to "*".

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/78b10569/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6e7cdfa..405d03e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -125,7 +125,7 @@ limitations under the License.
         <url>https://git-wip-us.apache.org/repos/asf?p=incubator-tinkerpop.git</url>
     </scm>
     <properties>
-        <groovy.version>2.4.1</groovy.version>
+        <groovy.version>2.4.5</groovy.version>
         <junit.version>4.12</junit.version>
         <metrics.version>3.0.2</metrics.version>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/78b10569/spark-gremlin/pom.xml
----------------------------------------------------------------------
diff --git a/spark-gremlin/pom.xml b/spark-gremlin/pom.xml
index 63ac42f..76b616f 100644
--- a/spark-gremlin/pom.xml
+++ b/spark-gremlin/pom.xml
@@ -104,7 +104,7 @@
         <dependency>
             <groupId>org.apache.spark</groupId>
             <artifactId>spark-core_2.10</artifactId>
-            <version>1.5.1</version>
+            <version>1.5.2</version>
             <exclusions>
                 <!-- self conflicts -->
                 <exclusion>


[19/30] incubator-tinkerpop git commit: updated upgrade docs to say that HDFSTools and HadoopLoader are gone. SparkLoader never existed until this SNAPSHOT so no reason to mention it.

Posted by dk...@apache.org.
updated upgrade docs to say that HDFSTools and HadoopLoader are gone. SparkLoader never existed until this SNAPSHOT so no reason to mention it.


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/9f5879e9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/9f5879e9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/9f5879e9

Branch: refs/heads/TINKERPOP-320
Commit: 9f5879e93a2199292a1edb68232b1fe0213f41a2
Parents: cc7cfff
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Thu Jan 7 17:33:57 2016 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Thu Jan 7 17:33:57 2016 -0700

----------------------------------------------------------------------
 docs/src/upgrade/release-3.1.x-incubating.asciidoc | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9f5879e9/docs/src/upgrade/release-3.1.x-incubating.asciidoc
----------------------------------------------------------------------
diff --git a/docs/src/upgrade/release-3.1.x-incubating.asciidoc b/docs/src/upgrade/release-3.1.x-incubating.asciidoc
index f026d02..29631c8 100644
--- a/docs/src/upgrade/release-3.1.x-incubating.asciidoc
+++ b/docs/src/upgrade/release-3.1.x-incubating.asciidoc
@@ -53,6 +53,8 @@ Given that HDFS (and now Spark) interactions are possible via `Storage` and no l
 developers can use these `Storage` implementations in their Java code. In fact, `Storage` has greatly simplified
 complex file/RDD operations in both `GiraphGraphComputer` and `SparkGraphComputer`.
 
+Finally, note that the following low-level/internal classes have been removed: `HadoopLoader` and `HDFSTools`.
+
 See: link:https://issues.apache.org/jira/browse/TINKERPOP-1033[TINKERPOP-1033],
 link:https://issues.apache.org/jira/browse/TINKERPOP-1023[TINKERPOP-1023]
 


[10/30] incubator-tinkerpop git commit: breaking commit. ignore.

Posted by dk...@apache.org.
breaking commit. ignore.


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/53e57a73
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/53e57a73
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/53e57a73

Branch: refs/heads/TINKERPOP-320
Commit: 53e57a73aa5316b44d5ef4917347a6ba8934a102
Parents: dbd4a53
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Jan 6 08:02:33 2016 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Wed Jan 6 08:02:33 2016 -0700

----------------------------------------------------------------------
 .../hadoop/structure/io/FileSystemStorage.java  |  4 +--
 .../groovy/plugin/FileSystemStorageCheck.java   | 35 ++++++++++++++++----
 .../hadoop/groovy/plugin/HadoopPluginSuite.java |  2 +-
 .../computer/SparkHadoopGraphProvider.java      |  3 +-
 4 files changed, 33 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/53e57a73/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
index 330227e..68a6cf8 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
@@ -177,8 +177,8 @@ public final class FileSystemStorage implements Storage {
 
     @Override
     public <K, V> Iterator<KeyValue<K, V>> head(final String location, final String memoryKey, final Class parserClass, final int totalLines) {
-        if (!parserClass.equals(SequenceFileInputFormat.class))
-            throw new IllegalArgumentException("Only " + SequenceFileInputFormat.class.getCanonicalName() + " memories are supported");
+        if (!parserClass.equals(SequenceFileInputFormat.class) && !parserClass.equals(ObjectWritable.class))
+            throw new IllegalArgumentException("Only " + SequenceFileInputFormat.class.getCanonicalName() + " and " + ObjectWritable.class.getCanonicalName() + " memories are supported");
         final Configuration configuration = new Configuration();
         try {
             return IteratorUtils.limit((Iterator) new ObjectWritableIterator(configuration, new Path(Constants.getMemoryLocation(location, memoryKey))), totalLines);

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/53e57a73/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java
index b0517ad..aea7db7 100644
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/FileSystemStorageCheck.java
@@ -24,6 +24,7 @@ import org.apache.tinkerpop.gremlin.AbstractGremlinTest;
 import org.apache.tinkerpop.gremlin.LoadGraphWith;
 import org.apache.tinkerpop.gremlin.hadoop.Constants;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.FileSystemStorage;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable;
 import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
 import org.apache.tinkerpop.gremlin.process.computer.ComputerResult;
 import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.ClusterCountMapReduce;
@@ -43,11 +44,27 @@ public class FileSystemStorageCheck extends AbstractGremlinTest {
     @Test
     @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
     public void shouldPersistGraphAndMemory() throws Exception {
-        final ComputerResult result = graph.compute(graphComputerClass.get()).program(PeerPressureVertexProgram.build().create(graph)).mapReduce(ClusterCountMapReduce.build().memoryKey("clusterCount").create()).submit().get();
-        /////
         final Storage storage = FileSystemStorage.open(ConfUtil.makeHadoopConfiguration(graph.configuration()));
-        // TEST GRAPH PERSISTENCE
-        assertTrue(storage.exists(Constants.getGraphLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION))));
+        final String inputLocation = Constants.getSearchGraphLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION), storage).get();
+        final String outputLocation = graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION);
+
+        // TEST INPUT GRAPH
+        assertTrue(storage.exists(inputLocation));
+        // assertFalse(storage.exists(outputLocation)); AbstractGremlinTest will create this automatically.
+        if (inputLocation.endsWith(".json")) { // gryo is not text readable
+            assertEquals(6, IteratorUtils.count(storage.head(inputLocation)));
+            for (int i = 0; i < 7; i++) {
+                assertEquals(i, IteratorUtils.count(storage.head(inputLocation, i)));
+            }
+            assertEquals(6, IteratorUtils.count(storage.head(inputLocation, 10)));
+        }
+
+        ////////////////////
+
+        final ComputerResult result = graph.compute(graphComputerClass.get()).program(PeerPressureVertexProgram.build().create(graph)).mapReduce(ClusterCountMapReduce.build().memoryKey("clusterCount").create()).submit().get();
+        // TEST OUTPUT GRAPH
+        assertTrue(storage.exists(outputLocation));
+        assertTrue(storage.exists(Constants.getGraphLocation(outputLocation)));
         assertEquals(6, result.graph().traversal().V().count().next().longValue());
         assertEquals(0, result.graph().traversal().E().count().next().longValue());
         assertEquals(6, result.graph().traversal().V().values("name").count().next().longValue());
@@ -56,8 +73,12 @@ public class FileSystemStorageCheck extends AbstractGremlinTest {
         /////
         // TEST MEMORY PERSISTENCE
         assertEquals(2, (int) result.memory().get("clusterCount"));
-        assertTrue(storage.exists(Constants.getMemoryLocation(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount")));
-        assertEquals(1, IteratorUtils.count(storage.head(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount", SequenceFileInputFormat.class)));
-        assertEquals(2, storage.head(graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), "clusterCount", SequenceFileInputFormat.class).next().getValue());
+        assertTrue(storage.exists(Constants.getMemoryLocation(outputLocation, "clusterCount")));
+        assertEquals(1, IteratorUtils.count(storage.head(outputLocation, "clusterCount", SequenceFileInputFormat.class)));
+        assertEquals(2, storage.head(outputLocation, "clusterCount", SequenceFileInputFormat.class).next().getValue());
+        //// backwards compatibility
+        assertEquals(1, IteratorUtils.count(storage.head(outputLocation, "clusterCount", ObjectWritable.class)));
+        assertEquals(2, storage.head(outputLocation, "clusterCount", ObjectWritable.class).next().getValue());
+
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/53e57a73/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java
index 16d654f..f062156 100644
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/groovy/plugin/HadoopPluginSuite.java
@@ -29,6 +29,6 @@ import org.junit.runners.model.RunnerBuilder;
  */
 public class HadoopPluginSuite extends AbstractGremlinSuite {
     public HadoopPluginSuite(final Class<?> klass, final RunnerBuilder builder) throws InitializationError {
-        super(klass, builder, new Class<?>[]{HadoopGremlinPluginCheck.class, FileSystemStorageCheck.class}, new Class<?>[]{HadoopGremlinPluginCheck.class, FileSystemStorageCheck.class}, true, TraversalEngine.Type.COMPUTER);
+        super(klass, builder, new Class<?>[]{FileSystemStorageCheck.class}, new Class<?>[]{FileSystemStorageCheck.class}, true, TraversalEngine.Type.COMPUTER);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/53e57a73/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphProvider.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphProvider.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphProvider.java
index f14cf7e..484c55a 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphProvider.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphProvider.java
@@ -22,6 +22,7 @@ import org.apache.tinkerpop.gremlin.GraphProvider;
 import org.apache.tinkerpop.gremlin.LoadGraphWith;
 import org.apache.tinkerpop.gremlin.hadoop.Constants;
 import org.apache.tinkerpop.gremlin.hadoop.HadoopGraphProvider;
+import org.apache.tinkerpop.gremlin.hadoop.groovy.plugin.FileSystemStorageCheck;
 import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversalSource;
 import org.apache.tinkerpop.gremlin.process.traversal.engine.ComputerTraversalEngine;
 import org.apache.tinkerpop.gremlin.spark.structure.io.ToyGraphInputRDD;
@@ -43,7 +44,7 @@ public final class SparkHadoopGraphProvider extends HadoopGraphProvider {
     public Map<String, Object> getBaseConfiguration(final String graphName, final Class<?> test, final String testMethodName, final LoadGraphWith.GraphData loadGraphWith) {
         final Map<String, Object> config = super.getBaseConfiguration(graphName, test, testMethodName, loadGraphWith);
         config.put(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, true);  // this makes the test suite go really fast
-        if (null != loadGraphWith && RANDOM.nextBoolean()) {
+        if (!test.equals(FileSystemStorageCheck.class) && null != loadGraphWith && RANDOM.nextBoolean()) {
             config.put(Constants.GREMLIN_SPARK_GRAPH_INPUT_RDD, ToyGraphInputRDD.class.getCanonicalName());
             // config.put(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, InputRDDFormat.class.getCanonicalName());
         }


[28/30] incubator-tinkerpop git commit: Made a safer comparator for non-comparable objects where if they are equal, then the comparison is 0. CTR.

Posted by dk...@apache.org.
Made a safer comparator for non-comparable objects where if they are equal, then the comparison is 0. CTR.


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/1e989b78
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/1e989b78
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/1e989b78

Branch: refs/heads/TINKERPOP-320
Commit: 1e989b78b36cb9bc6a1ea8c1e7e720048fd1d00f
Parents: 4ad6617
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Mon Jan 11 09:39:57 2016 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Mon Jan 11 09:39:57 2016 -0700

----------------------------------------------------------------------
 .../tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/1e989b78/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
index 765b6d5..a371bc3 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
@@ -97,7 +97,12 @@ public final class ObjectWritable<T> implements WritableComparable<ObjectWritabl
 
     @Override
     public int compareTo(final ObjectWritable objectWritable) {
-        return this.t instanceof Comparable ? ((Comparable) this.t).compareTo(objectWritable.get()) : -1;
+        if (this.t instanceof Comparable)
+            return ((Comparable) this.t).compareTo(objectWritable.get());
+        else if (this.t.equals(objectWritable.get()))
+            return 0;
+        else
+            return -1;
     }
 
     public boolean isEmpty() {


[13/30] incubator-tinkerpop git commit: added documentation, upgrade docs, JavaDoc, more test cases, and fixed up some random inconsistencies in BulkLoaderVertexProgram documentation examples.

Posted by dk...@apache.org.
added documentation,  upgrade docs, JavaDoc, more test cases, and fixed up some random inconsistencies in BulkLoaderVertexProgram documentation examples.


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/5c9e81b0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/5c9e81b0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/5c9e81b0

Branch: refs/heads/TINKERPOP-320
Commit: 5c9e81b0cebd8c3841e2442a8ef13b3d23d44295
Parents: b0f3e4a
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Jan 6 15:58:18 2016 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Wed Jan 6 15:58:18 2016 -0700

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |   5 +
 docs/src/reference/implementations.asciidoc     | 107 +++++++++----------
 .../upgrade/release-3.1.x-incubating.asciidoc   |  24 +++++
 .../process/computer/GiraphGraphComputer.java   |  13 ++-
 .../tinkerpop/gremlin/structure/io/Storage.java |  96 ++++++++++++++---
 .../conf/hadoop-grateful-gryo.properties        |   6 +-
 .../hadoop/structure/HadoopConfiguration.java   |   5 +
 .../hadoop/structure/io/FileSystemStorage.java  |  16 +--
 .../structure/io/AbstractStorageCheck.java      |  16 ++-
 .../structure/io/FileSystemStorageCheck.java    |   4 +-
 .../spark/groovy/plugin/SparkGremlinPlugin.java |   1 +
 .../process/computer/SparkGraphComputer.java    |   4 +-
 .../spark/structure/io/PersistedOutputRDD.java  |   2 +-
 .../spark/structure/io/SparkContextStorage.java |  19 +---
 .../structure/io/SparkContextStorageCheck.java  |   4 +-
 15 files changed, 197 insertions(+), 125 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/5c9e81b0/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index 43fb4b6..fad0630 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -26,6 +26,11 @@ image::https://raw.githubusercontent.com/apache/incubator-tinkerpop/master/docs/
 TinkerPop 3.1.1 (NOT OFFICIALLY RELEASED YET)
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
+* It is possible to completely avoid using HDFS with Spark if `PersistedInputRDD` and `PersistedOutpuRDD` are leveraged.
+* `InputRDD` and `OutputRDD` can now process both graphs and memory (i.e. sideEffects).
+* Removed Groovy specific meta-programming overloads for handling Hadoop `FileSystem` (instead, its all accessible via `FileSystemStorage`).
+* Added `FileSystemStorage` and `SparkContextStorage` which both implement the new `Storage` API.
+* Added `Storage` to the gremlin-core io-package which providers can implement to allow conventional access to data sources (e.g. `ls()`, `rm()`, `cp()`, etc.).
 * Execute the `LifeCycle.beforeEval()` in the same thread that `eval()` is executed in for `GremlinExecutor`.
 * Improved error handling of Gremlin Console initialization scripts to better separate errors in initialization script I/O versus execution of the script itself.
 * Fixed a bug in `Graph.OptOut` when trying to opt-out of certain test cases with the `method` property set to "*".

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/5c9e81b0/docs/src/reference/implementations.asciidoc
----------------------------------------------------------------------
diff --git a/docs/src/reference/implementations.asciidoc b/docs/src/reference/implementations.asciidoc
index c3a1df6..add8555 100644
--- a/docs/src/reference/implementations.asciidoc
+++ b/docs/src/reference/implementations.asciidoc
@@ -1213,30 +1213,8 @@ Using a Persisted Context
 
 It is possible to persist the graph RDD between jobs within the `SparkContext` (e.g. SparkServer) by leveraging `PersistedOutputRDD`.
 Note that `gremlin.spark.persistContext` should be set to `true` or else the persisted RDD will be destroyed when the `SparkContext` closes.
-The persisted RDD is named by the `gremlin.hadoop.outputLocation` configuration.
-Similarly, `PersistedInputRDD` is used with respective  `gremlin.hadoop.inputLocation` to retrieve the persisted RDD from the `SparkContext`.
-
-There is a static `spark` object that can be used to manage persisted RDDs much like `hdfs` is used to manage HDFS files (see <<interacting-with-hdfs, Interacting with HDFS>>).
-
-[gremlin-groovy]
-----
-spark.create('local[4]') // the SparkContext location (master)
-graph = GraphFactory.open('conf/hadoop/hadoop-gryo.properties')
-graph.configuration().setProperty('gremlin.spark.persistContext',true)
-graph.configuration().setProperty('gremlin.spark.graphOutputRDD','org.apache.tinkerpop.gremlin.spark.structure.io.PersistedOutputRDD')
-graph.configuration().setProperty('gremlin.hadoop.outputLocation','pageRankGraph')
-graph.compute(SparkGraphComputer).program(PageRankVertexProgram.build().create()).submit().get()
-spark.ls()
-graph.configuration().setProperty('gremlin.hadoop.outputLocation','peerPressureGraph')
-graph.compute(SparkGraphComputer).program(PeerPressureVertexProgram.build().create()).submit().get()
-spark.ls()
-spark.rm('pageRankGraph')
-spark.head('peerPressureGraph')
-spark.describe('peerPressureGraph')
-spark.rm('peerPressureGraph')
-spark.ls()
-spark.close()
-----
+The persisted RDD is named by the `gremlin.hadoop.outputLocation` configuration. Similarly, `PersistedInputRDD` is used with respective
+`gremlin.hadoop.inputLocation` to retrieve the persisted RDD from the `SparkContext`.
 
 When using a persistent `SparkContext` the configuration used by the original Spark Configuration will be inherited by all threaded
 references to that Spark Context. The exception to this rule are those properties which have a specific thread local effect.
@@ -1247,6 +1225,8 @@ references to that Spark Context. The exception to this rule are those propertie
 . spark.job.interruptOnCancel
 . spark.scheduler.pool
 
+Finally, there is a `spark` object that can be used to manage persisted RDDs (see <<interacting-with-spark, Interacting with Spark>>).
+
 Loading with BulkLoaderVertexProgram
 ++++++++++++++++++++++++++++++++++++
 
@@ -1256,7 +1236,7 @@ Grateful Dead graph from HadoopGraph into TinkerGraph over Spark:
 
 [gremlin-groovy]
 ----
-hdfs.copyFromLocal('data/grateful-dead.kryo', 'data/grateful-dead.kryo')
+hdfs.copyFromLocal('data/grateful-dead.kryo', 'grateful-dead.kryo')
 readGraph = GraphFactory.open('conf/hadoop/hadoop-grateful-gryo.properties')
 writeGraph = 'conf/tinkergraph-gryo.properties'
 blvp = BulkLoaderVertexProgram.build().
@@ -1279,10 +1259,8 @@ graph.close()
 #
 gremlin.graph=org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph
 gremlin.hadoop.graphInputFormat=org.apache.tinkerpop.gremlin.hadoop.structure.io.gryo.GryoInputFormat
-gremlin.hadoop.memoryOutputFormat=org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat
-gremlin.hadoop.inputLocation=data/grateful-dead.kryo
+gremlin.hadoop.inputLocation=grateful-dead.kryo
 gremlin.hadoop.outputLocation=output
-gremlin.hadoop.deriveMemory=false
 gremlin.hadoop.jarsInDistributedCache=true
 
 #
@@ -1385,7 +1363,7 @@ the Grateful Dead graph from HadoopGraph into TinkerGraph over Giraph:
 
 [gremlin-groovy]
 ----
-hdfs.copyFromLocal('data/grateful-dead.kryo', 'data/grateful-dead.kryo')
+hdfs.copyFromLocal('data/grateful-dead.kryo', 'grateful-dead.kryo')
 readGraph = GraphFactory.open('conf/hadoop/hadoop-grateful-gryo.properties')
 writeGraph = 'conf/tinkergraph-gryo.properties'
 blvp = BulkLoaderVertexProgram.build().
@@ -1409,10 +1387,8 @@ graph.close()
 gremlin.graph=org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph
 gremlin.hadoop.graphInputFormat=org.apache.tinkerpop.gremlin.hadoop.structure.io.gryo.GryoInputFormat
 gremlin.hadoop.graphOutputFormat=org.apache.hadoop.mapreduce.lib.output.NullOutputFormat
-gremlin.hadoop.memoryOutputFormat=org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat
-gremlin.hadoop.inputLocation=data/grateful-dead.kryo
+gremlin.hadoop.inputLocation=grateful-dead.kryo
 gremlin.hadoop.outputLocation=output
-gremlin.hadoop.deriveMemory=false
 gremlin.hadoop.jarsInDistributedCache=true
 
 #
@@ -1477,12 +1453,14 @@ simple (easy to create and parse).
 The data below represents an adjacency list representation of the classic TinkerGraph toy graph in GraphSON format.
 
 [source,json]
+----
 {"id":1,"label":"person","outE":{"created":[{"id":9,"inV":3,"properties":{"weight":0.4}}],"knows":[{"id":7,"inV":2,"properties":{"weight":0.5}},{"id":8,"inV":4,"properties":{"weight":1.0}}]},"properties":{"name":[{"id":0,"value":"marko"}],"age":[{"id":1,"value":29}]}}
 {"id":2,"label":"person","inE":{"knows":[{"id":7,"outV":1,"properties":{"weight":0.5}}]},"properties":{"name":[{"id":2,"value":"vadas"}],"age":[{"id":3,"value":27}]}}
 {"id":3,"label":"software","inE":{"created":[{"id":9,"outV":1,"properties":{"weight":0.4}},{"id":11,"outV":4,"properties":{"weight":0.4}},{"id":12,"outV":6,"properties":{"weight":0.2}}]},"properties":{"name":[{"id":4,"value":"lop"}],"lang":[{"id":5,"value":"java"}]}}
 {"id":4,"label":"person","inE":{"knows":[{"id":8,"outV":1,"properties":{"weight":1.0}}]},"outE":{"created":[{"id":10,"inV":5,"properties":{"weight":1.0}},{"id":11,"inV":3,"properties":{"weight":0.4}}]},"properties":{"name":[{"id":6,"value":"josh"}],"age":[{"id":7,"value":32}]}}
 {"id":5,"label":"software","inE":{"created":[{"id":10,"outV":4,"properties":{"weight":1.0}}]},"properties":{"name":[{"id":8,"value":"ripple"}],"lang":[{"id":9,"value":"java"}]}}
 {"id":6,"label":"person","outE":{"created":[{"id":12,"inV":3,"properties":{"weight":0.2}}]},"properties":{"name":[{"id":10,"value":"peter"}],"age":[{"id":11,"value":35}]}}
+----
 
 [[script-io-format]]
 Script I/O Format
@@ -1575,45 +1553,58 @@ def stringify(vertex) {
     return [v, outE].join('\t')
 }
 
+
+
+Storage Systems
+~~~~~~~~~~~~~~~
+
+Hadoop-Gremlin provides two implementations of the `Storage` API:
+
+* `FileSystemStorage`: Access HDFS and local file system data.
+* `SparkContextStorage`: Access Spark persisted RDD data.
+
 [[interacting-with-hdfs]]
 Interacting with HDFS
-~~~~~~~~~~~~~~~~~~~~~
+^^^^^^^^^^^^^^^^^^^^^
 
 The distributed file system of Hadoop is called link:http://en.wikipedia.org/wiki/Apache_Hadoop#Hadoop_distributed_file_system[HDFS].
-The results of any OLAP operation are stored in HDFS accessible via `hdfs`.
+The results of any OLAP operation are stored in HDFS accessible via `hdfs`. For local file system access, there is `local`.
 
 [gremlin-groovy]
 ----
 graph = GraphFactory.open('conf/hadoop/hadoop-gryo.properties')
-g = graph.traversal(computer(SparkGraphComputer))
-:remote connect tinkerpop.hadoop graph g
-:> g.V().group().by{it.value('name')[1]}.by('name')
+graph.compute(SparkGraphComputer).program(PeerPressureVertexProgram.build().create(graph)).mapReduce(ClusterCountMapReduce.build().memoryKey('clusterCount').create()).submit().get();
 hdfs.ls()
 hdfs.ls('output')
-hdfs.ls('output/~reducing')
-hdfs.head('output/~reducing', SequenceFileInputFormat)
+hdfs.head('output', GryoInputFormat)
+hdfs.head('output', 'clusterCount', SequenceFileInputFormat)
+hdfs.rm('output')
+hdfs.ls()
 ----
 
-A list of the HDFS methods available are itemized below. Note that these methods are also available for the 'local' variable:
+[[interacting-with-spark]]
+Interacting with Spark
+^^^^^^^^^^^^^^^^^^^^^^
 
-[width="100%",cols="13,10",options="header"]
-|=========================================================
-| Method| Description
-|hdfs.ls(String path)| List the contents of the supplied directory.
-|hdfs.cp(String from, String to)| Copy the specified path to the specified path.
-|hdfs.exists(String path)| Whether the specified path exists.
-|hdfs.rm(String path)| Remove the specified path.
-|hdfs.rmr(String path)| Remove the specified path and its contents recurssively.
-|hdfs.copyToLocal(String from, String to)| Copy the specified HDFS path to the specified local path.
-|hdfs.copyFromLocal(String from, String to)| Copy the specified local path to the specified HDFS path.
-|hdfs.mergeToLocal(String from, String to)| Merge the files in path to the specified local path.
-|hdfs.head(String path)| Display the data in the path as text.
-|hdfs.head(String path, int lineCount)| Text display only the first `lineCount`-number of lines in the path.
-|hdfs.head(String path, int totalKeyValues, Class<InputFormat> inputFormatClass)| Parse and display the data using the InputFormat.
-|hdfs.head(String path, Class<InputFormat> inputFormatClass)| Parse and display the data using the InputFormat.
-|hdfs.head(String path, String memoryKey, Class<InputFormat> inputFormatClass, int totalKeyValues) | Parse and display memory data using the InputFormat.
-|hdfs.head(String path, String memoryKey, Class<InputFormat> inputFormatClass) | Parse and display memory data using the InputFormat.
-|=========================================================
+If a Spark context is persisted, then Spark RDDs will remain the Spark cache and accessible over subsequent jobs.
+RDDs are retrieved and saved to the `SparkContext` via `PersistedInputRDD` and `PersistedOutputRDD` respectivly.
+Persisted RDDs can be accessed using `spark`.
+
+[gremlin-groovy]
+----
+Spark.create('local[4]')
+graph = GraphFactory.open('conf/hadoop/hadoop-gryo.properties')
+graph.configuration().setProperty('gremlin.spark.graphOutputRDD', PersistedOutputRDD.class.getCanonicalName())
+graph.configuration().clearProperty('gremlin.hadoop.graphOutputFormat')
+graph.configuration().setProperty('gremlin.spark.persistContext',true)
+graph.compute(SparkGraphComputer).program(PeerPressureVertexProgram.build().create(graph)).mapReduce(ClusterCountMapReduce.build().memoryKey('clusterCount').create()).submit().get();
+spark.ls()
+spark.ls('output')
+spark.head('output', PersistedInputRDD)
+spark.head('output', 'clusterCount', PersistedInputRDD)
+spark.rm('output')
+spark.ls()
+----
 
 A Command Line Example
 ~~~~~~~~~~~~~~~~~~~~~~

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/5c9e81b0/docs/src/upgrade/release-3.1.x-incubating.asciidoc
----------------------------------------------------------------------
diff --git a/docs/src/upgrade/release-3.1.x-incubating.asciidoc b/docs/src/upgrade/release-3.1.x-incubating.asciidoc
index 8f487a2..f026d02 100644
--- a/docs/src/upgrade/release-3.1.x-incubating.asciidoc
+++ b/docs/src/upgrade/release-3.1.x-incubating.asciidoc
@@ -32,6 +32,30 @@ Please see the link:https://github.com/apache/incubator-tinkerpop/blob/3.1.1-inc
 Upgrading for Users
 ~~~~~~~~~~~~~~~~~~~
 
+Storage I/O
+^^^^^^^^^^^
+
+The `gremlin-core` io-package now has a `Storage` interface. The methods that were available via `hdfs`
+(e.g. `rm()`, `ls()`, `head()`, etc.) are now part of `Storage`. Both HDFS and Spark implement `Storage` via
+`FileSystemStorage` and `SparkContextStorage`, respectively.  `SparkContextStorage` adds support for interacting with
+persisted RDDs in the Spark cache.
+
+This update changed a few of the file handling methods. As it stands, these changes only effect manual Gremlin Console
+usage as HDFS support was previously provided via Groovy meta-programing. Thus, these are not "code-based" breaking changes.
+
+* `hdfs.rmr()` no longer exists. `hdfs.rm()` is now recursive. Simply change all references to `rmr()` to `rm()` for identical behavior.
+* `hdfs.head(location,lines,writableClass)` no longer exists.
+** For graph locations, use `hdfs.head(location,writableClass,lines)`.
+** For memory locations, use `hdfs.head(location,memoryKey,writableClass,lines)`.
+* `hdfs.head(...,ObjectWritable)` no longer exists. Use `SequenceFileInputFormat` as an input format is the parsing class.
+
+Given that HDFS (and now Spark) interactions are possible via `Storage` and no longer via Groovy meta-programming,
+developers can use these `Storage` implementations in their Java code. In fact, `Storage` has greatly simplified
+complex file/RDD operations in both `GiraphGraphComputer` and `SparkGraphComputer`.
+
+See: link:https://issues.apache.org/jira/browse/TINKERPOP-1033[TINKERPOP-1033],
+link:https://issues.apache.org/jira/browse/TINKERPOP-1023[TINKERPOP-1023]
+
 Gremlin Server Transaction Management
 ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/5c9e81b0/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphGraphComputer.java
----------------------------------------------------------------------
diff --git a/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphGraphComputer.java b/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphGraphComputer.java
index 7e3de5e..dfe8e8c 100644
--- a/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphGraphComputer.java
+++ b/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphGraphComputer.java
@@ -117,7 +117,6 @@ public final class GiraphGraphComputer extends AbstractHadoopGraphComputer imple
     @Override
     public Future<ComputerResult> submit() {
         super.validateStatePriorToExecution();
-
         return ComputerSubmissionHelper.runWithBackgroundThread(this::submitWithExecutor, "GiraphSubmitter");
     }
 
@@ -142,7 +141,7 @@ public final class GiraphGraphComputer extends AbstractHadoopGraphComputer imple
     @Override
     public int run(final String[] args) {
         final Storage storage = FileSystemStorage.open(this.giraphConfiguration);
-        storage.rmr(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));
+        storage.rm(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));
         this.giraphConfiguration.setBoolean(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT_HAS_EDGES, this.persist.equals(Persist.EDGES));
         try {
             // it is possible to run graph computer without a vertex program (and thus, only map reduce jobs if they exist)
@@ -190,12 +189,12 @@ public final class GiraphGraphComputer extends AbstractHadoopGraphComputer imple
                         if (iterator.hasNext()) {
                             this.memory.set(memoryKey, iterator.next().getValue());
                         }
-                        storage.rmr(Constants.getMemoryLocation(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), memoryKey));
+                        storage.rm(Constants.getMemoryLocation(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), memoryKey));
                     }
                 }
                 final Path path = new Path(Constants.getMemoryLocation(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), Constants.HIDDEN_ITERATION));
                 this.memory.setIteration((Integer) new ObjectWritableIterator(this.giraphConfiguration, path).next().getValue());
-                storage.rmr(Constants.getMemoryLocation(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), Constants.HIDDEN_ITERATION));
+                storage.rm(Constants.getMemoryLocation(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION), Constants.HIDDEN_ITERATION));
             }
             // do map reduce jobs
             this.giraphConfiguration.setBoolean(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT_HAS_EDGES, this.giraphConfiguration.getBoolean(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT_HAS_EDGES, true));
@@ -204,9 +203,9 @@ public final class GiraphGraphComputer extends AbstractHadoopGraphComputer imple
                 MapReduceHelper.executeMapReduceJob(mapReduce, this.memory, this.giraphConfiguration);
             }
 
-            // if no persistence, delete the map reduce output
+            // if no persistence, delete the graph output
             if (this.persist.equals(Persist.NOTHING))
-                storage.rmr(Constants.getGraphLocation(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION)));
+                storage.rm(Constants.getGraphLocation(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION)));
         } catch (final Exception e) {
             throw new IllegalStateException(e.getMessage(), e);
         }
@@ -244,7 +243,7 @@ public final class GiraphGraphComputer extends AbstractHadoopGraphComputer imple
                                 } catch (final Exception e) {
                                     throw new RuntimeException(e.getMessage(), e);
                                 }
-                            } catch (Exception e) {
+                            } catch (final Exception e) {
                                 throw new IllegalStateException(e.getMessage(), e);
                             }
                         });

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/5c9e81b0/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
index d9c6927..1779b38 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Storage.java
@@ -26,43 +26,113 @@ import java.util.Iterator;
 import java.util.List;
 
 /**
+ * Storage is a standard API that providers can implement to allow "file-system"-based access to data sources.
+ * The methods provided by Storage are similar in form and behavior to standard Linux operating system commands.
+ *
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
 public interface Storage {
 
+    /**
+     * List all the data sources in the root directory.
+     *
+     * @return the data sources in the root directory
+     */
     public List<String> ls();
 
+    /**
+     * List all the data sources at the specified location.
+     *
+     * @param location a location
+     * @return the data sources at the specified location
+     */
     public List<String> ls(final String location);
 
-    public boolean mkdir(final String location);
-
-    public boolean cp(final String fromLocation, final String toLocation);
-
+    /**
+     * Recursively copy all the data sources from the source location to the target location.
+     *
+     * @param sourceLocation the source location
+     * @param targetLocation the target location
+     * @return whether data sources were copied
+     */
+    public boolean cp(final String sourceLocation, final String targetLocation);
+
+    /**
+     * Determine whether the specified location has a data source.
+     *
+     * @param location a location to check
+     * @return whether that location has a data source.
+     */
     public boolean exists(final String location);
 
+    /**
+     * Recursively remove the data source at the specified location.
+     *
+     * @param location the location of the data source
+     * @return whether a data source was removed.
+     */
     public boolean rm(final String location);
 
-    public boolean rmr(final String location);
-
-    public Iterator<String> head(final String location, final int totalLines);
-
+    /**
+     * Get a string representation of the specified number of lines at the data source location.
+     *
+     * @param location the data source location
+     * @return an iterator of lines
+     */
     public default Iterator<String> head(final String location) {
         return this.head(location, Integer.MAX_VALUE);
     }
 
-    public Iterator<Vertex> head(final String location, final Class parserClass, final int totalLines);
+    /**
+     * Get a string representation of the specified number of lines at the data source location.
+     *
+     * @param location   the data source location
+     * @param totalLines the total number of lines to retrieve
+     * @return an iterator of lines.
+     */
+    public Iterator<String> head(final String location, final int totalLines);
 
-    @Deprecated
-    public default Iterator<Vertex> head(final String location, final int totalLines, final Class parserClass) {
-       return this.head(location,parserClass,totalLines);
-    }
+    /**
+     * Get the vertices at the specified graph location.
+     *
+     * @param location    the location of the graph (or the root location and search will be made)
+     * @param parserClass the class of the parser that understands the graph format
+     * @param totalLines  the total number of lines of the graph to return
+     * @return an iterator of vertices.
+     */
+    public Iterator<Vertex> head(final String location, final Class parserClass, final int totalLines);
 
+    /**
+     * Get the vertices at the specified graph location.
+     *
+     * @param location    the location of the graph (or the root location and search will be made)
+     * @param parserClass the class of the parser that understands the graph format
+     * @return an iterator of vertices.
+     */
     public default Iterator<Vertex> head(final String location, final Class parserClass) {
         return this.head(location, parserClass, Integer.MAX_VALUE);
     }
 
+    /**
+     * Get the {@link KeyValue} data at the specified memory location.
+     *
+     * @param location    the root location of the data
+     * @param memoryKey   the memory key
+     * @param parserClass the class of the parser that understands the memory format
+     * @param totalLines  the total number of key-values to return
+     * @return an iterator of key-values.
+     */
     public <K, V> Iterator<KeyValue<K, V>> head(final String location, final String memoryKey, final Class parserClass, final int totalLines);
 
+
+    /**
+     * Get the {@link KeyValue} data at the specified memory location.
+     *
+     * @param location    the root location of the data
+     * @param memoryKey   the memory key
+     * @param parserClass the class of the parser that understands the memory format
+     * @return an iterator of key-values.
+     */
     public default <K, V> Iterator<KeyValue<K, V>> head(final String location, final String memoryKey, final Class parserClass) {
         return this.head(location, memoryKey, parserClass, Integer.MAX_VALUE);
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/5c9e81b0/hadoop-gremlin/conf/hadoop-grateful-gryo.properties
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/conf/hadoop-grateful-gryo.properties b/hadoop-gremlin/conf/hadoop-grateful-gryo.properties
index 0554fcc..e247c2a 100644
--- a/hadoop-gremlin/conf/hadoop-grateful-gryo.properties
+++ b/hadoop-gremlin/conf/hadoop-grateful-gryo.properties
@@ -21,10 +21,8 @@
 gremlin.graph=org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph
 gremlin.hadoop.graphInputFormat=org.apache.tinkerpop.gremlin.hadoop.structure.io.gryo.GryoInputFormat
 gremlin.hadoop.graphOutputFormat=org.apache.tinkerpop.gremlin.hadoop.structure.io.gryo.GryoOutputFormat
-gremlin.hadoop.memoryOutputFormat=org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat
-gremlin.hadoop.inputLocation=data/grateful-dead.kryo
+gremlin.hadoop.inputLocation=grateful-dead.kryo
 gremlin.hadoop.outputLocation=output
-gremlin.hadoop.deriveMemory=false
 gremlin.hadoop.jarsInDistributedCache=true
 
 #
@@ -45,5 +43,5 @@ giraph.maxMessagesInMemory=100000
 #
 spark.master=local[1]
 spark.executor.memory=1g
-spark.serializer=org.apache.spark.serializer.KryoSerializer
+spark.serializer=org.apache.tinkerpop.gremlin.spark.structure.io.gryo.GryoSerializer
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/5c9e81b0/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/HadoopConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/HadoopConfiguration.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/HadoopConfiguration.java
index 25b01ff..d4578b4 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/HadoopConfiguration.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/HadoopConfiguration.java
@@ -50,6 +50,11 @@ public final class HadoopConfiguration extends AbstractConfiguration implements
         this.properties.put(key, value);
     }
 
+    @Override
+    protected void clearPropertyDirect(final String key) {
+        this.properties.remove(key);
+    }
+
     public HadoopConfiguration(final Configuration configuration) {
         this();
         this.copy(configuration);

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/5c9e81b0/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
index 4f648ee..ed112f7 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorage.java
@@ -106,7 +106,6 @@ public final class FileSystemStorage implements Storage {
         }
     }
 
-    @Override
     public boolean mkdir(final String location) {
         try {
             return this.fs.mkdirs(new Path(location));
@@ -116,9 +115,9 @@ public final class FileSystemStorage implements Storage {
     }
 
     @Override
-    public boolean cp(final String fromLocation, final String toLocation) {
+    public boolean cp(final String sourceLocation, final String targetLocation) {
         try {
-            return FileUtil.copy(this.fs, new Path(fromLocation), this.fs, new Path(toLocation), false, new Configuration());
+            return FileUtil.copy(this.fs, new Path(sourceLocation), this.fs, new Path(targetLocation), false, new Configuration());
         } catch (final IOException e) {
             throw new IllegalStateException(e.getMessage(), e);
         }
@@ -136,15 +135,6 @@ public final class FileSystemStorage implements Storage {
     @Override
     public boolean rm(final String location) {
         try {
-            return FileSystemStorage.globDelete(this.fs, location, false);
-        } catch (final IOException e) {
-            throw new IllegalStateException(e.getMessage(), e);
-        }
-    }
-
-    @Override
-    public boolean rmr(final String location) {
-        try {
             return FileSystemStorage.globDelete(this.fs, location, true);
         } catch (final IOException e) {
             throw new IllegalStateException(e.getMessage(), e);
@@ -177,7 +167,7 @@ public final class FileSystemStorage implements Storage {
 
     @Override
     public <K, V> Iterator<KeyValue<K, V>> head(final String location, final String memoryKey, final Class parserClass, final int totalLines) {
-        if (!parserClass.equals(SequenceFileInputFormat.class) && !parserClass.equals(ObjectWritable.class)) // object writable support for backwards compatibility
+        if (!parserClass.equals(SequenceFileInputFormat.class))
             throw new IllegalArgumentException("Only " + SequenceFileInputFormat.class.getCanonicalName() + " memories are supported");
         final Configuration configuration = new Configuration();
         try {

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/5c9e81b0/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/AbstractStorageCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/AbstractStorageCheck.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/AbstractStorageCheck.java
index 195f50d..bec9c72 100644
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/AbstractStorageCheck.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/AbstractStorageCheck.java
@@ -77,14 +77,14 @@ public abstract class AbstractStorageCheck extends AbstractGremlinTest {
         assertTrue(storage.exists(Constants.getGraphLocation(outputLocation)));
         assertTrue(storage.exists(Constants.getMemoryLocation(outputLocation, "clusterCount")));
         assertEquals(2, storage.ls(outputLocation).size());
-        assertTrue(storage.rmr(Constants.getGraphLocation(outputLocation)));
+        assertTrue(storage.rm(Constants.getGraphLocation(outputLocation)));
         assertEquals(1, storage.ls(outputLocation).size());
-        assertTrue(storage.rmr(Constants.getMemoryLocation(outputLocation, "clusterCount")));
+        assertTrue(storage.rm(Constants.getMemoryLocation(outputLocation, "clusterCount")));
         assertEquals(0, storage.ls(outputLocation).size());
         assertFalse(storage.exists(Constants.getGraphLocation(outputLocation)));
         assertFalse(storage.exists(Constants.getMemoryLocation(outputLocation, "clusterCount")));
         if (storage.exists(outputLocation))
-            assertTrue(storage.rmr(outputLocation));
+            assertTrue(storage.rm(outputLocation));
         assertFalse(storage.exists(outputLocation));
 
         ////////////////
@@ -94,12 +94,12 @@ public abstract class AbstractStorageCheck extends AbstractGremlinTest {
         assertTrue(storage.exists(Constants.getGraphLocation(outputLocation)));
         assertTrue(storage.exists(Constants.getMemoryLocation(outputLocation, "clusterCount")));
         assertEquals(2, storage.ls(outputLocation).size());
-        assertTrue(storage.rmr(outputLocation));
+        assertTrue(storage.rm(outputLocation));
         assertFalse(storage.exists(outputLocation));
         assertEquals(0, storage.ls(outputLocation).size());
     }
 
-    public void checkCopyMethods(final Storage storage, final String outputLocation, final String newOutputLocation) throws Exception {
+    public void checkCopyMethods(final Storage storage, final String outputLocation, final String newOutputLocation, final Class outputGraphParserClass, final Class outputMemoryParserClass) throws Exception {
         graph.compute(graphComputerClass.get()).program(PeerPressureVertexProgram.build().create(graph)).mapReduce(ClusterCountMapReduce.build().memoryKey("clusterCount").create()).submit().get();
         assertTrue(storage.exists(outputLocation));
         assertTrue(storage.exists(Constants.getGraphLocation(outputLocation)));
@@ -113,5 +113,11 @@ public abstract class AbstractStorageCheck extends AbstractGremlinTest {
         assertTrue(storage.exists(newOutputLocation));
         assertTrue(storage.exists(Constants.getGraphLocation(newOutputLocation)));
         assertTrue(storage.exists(Constants.getMemoryLocation(newOutputLocation, "clusterCount")));
+
+        assertEquals(2, storage.ls(newOutputLocation).size());
+        assertEquals(6, IteratorUtils.count(storage.head(outputLocation, outputGraphParserClass)));
+        assertEquals(6, IteratorUtils.count(storage.head(newOutputLocation, outputGraphParserClass)));
+        assertEquals(1, IteratorUtils.count(storage.head(outputLocation, "clusterCount", outputMemoryParserClass)));
+        assertEquals(1, IteratorUtils.count(storage.head(newOutputLocation, "clusterCount", outputMemoryParserClass)));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/5c9e81b0/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorageCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorageCheck.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorageCheck.java
index f528df9..1b2c04e 100644
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorageCheck.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorageCheck.java
@@ -47,8 +47,6 @@ public class FileSystemStorageCheck extends AbstractStorageCheck {
         final String outputLocation = graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION);
         // TestHelper creates the directory and we need it not to exist
         deleteDirectory(outputLocation);
-        super.checkHeadMethods(storage, inputLocation, outputLocation, InputOutputHelper.getInputFormat((Class) Class.forName(graph.configuration().getString(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT))), ObjectWritable.class);
-        deleteDirectory(outputLocation);
         super.checkHeadMethods(storage, inputLocation, outputLocation, InputOutputHelper.getInputFormat((Class) Class.forName(graph.configuration().getString(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT))), SequenceFileInputFormat.class);
     }
 
@@ -68,7 +66,7 @@ public class FileSystemStorageCheck extends AbstractStorageCheck {
         final String newOutputLocation = TestHelper.makeTestDataDirectory(FileSystemStorageCheck.class, "new-location-for-copy");
         // TestHelper creates the directory and we need it not to exist
         deleteDirectory(newOutputLocation);
-        super.checkCopyMethods(storage, outputLocation, newOutputLocation);
+        super.checkCopyMethods(storage, outputLocation, newOutputLocation, InputOutputHelper.getInputFormat((Class) Class.forName(graph.configuration().getString(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT))), SequenceFileInputFormat.class);
 
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/5c9e81b0/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java
index a7e333c..1fe23e3 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/groovy/plugin/SparkGremlinPlugin.java
@@ -42,6 +42,7 @@ public final class SparkGremlinPlugin extends AbstractGremlinPlugin {
     protected static final Set<String> IMPORTS = new HashSet<String>() {{
         add(IMPORT_SPACE + SparkGraphComputer.class.getPackage().getName() + DOT_STAR);
         add(IMPORT_SPACE + Spark.class.getPackage().getName() + DOT_STAR);
+        add(IMPORT_SPACE + SparkContextStorage.class.getPackage().getName() + DOT_STAR);
     }};
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/5c9e81b0/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
index 82c4331..b48fac5 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
@@ -243,11 +243,11 @@ public final class SparkGraphComputer extends AbstractHadoopGraphComputer {
                 if (!PersistedOutputRDD.class.equals(hadoopConfiguration.getClass(Constants.GREMLIN_SPARK_GRAPH_OUTPUT_RDD, null)) || this.persist.equals(GraphComputer.Persist.NOTHING)) {
                     graphRDD.unpersist();
                     if (apacheConfiguration.containsKey(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION))
-                        SparkContextStorage.open().rmr(apacheConfiguration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));
+                        SparkContextStorage.open().rm(apacheConfiguration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));
                 }
                 // delete any file system output if persist nothing
                 if (FileOutputFormat.class.isAssignableFrom(hadoopConfiguration.getClass(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT, FileInputFormat.class)) && this.persist.equals(GraphComputer.Persist.NOTHING))
-                    FileSystemStorage.open(hadoopConfiguration).rmr(apacheConfiguration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));
+                    FileSystemStorage.open(hadoopConfiguration).rm(apacheConfiguration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));
                 // update runtime and return the newly computed graph
                 finalMemory.setRuntime(System.currentTimeMillis() - startTime);
                 return new DefaultComputerResult(InputOutputHelper.getOutputGraph(apacheConfiguration, this.resultGraph, this.persist), finalMemory.asImmutable());

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/5c9e81b0/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java
index 7833701..27b87f5 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/PersistedOutputRDD.java
@@ -44,7 +44,7 @@ public final class PersistedOutputRDD implements OutputRDD {
             LOGGER.warn("The SparkContext should be persisted in order for the RDD to persist across jobs. To do so, set " + Constants.GREMLIN_SPARK_PERSIST_CONTEXT + " to true");
         if (!configuration.containsKey(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION))
             throw new IllegalArgumentException("There is no provided " + Constants.GREMLIN_HADOOP_OUTPUT_LOCATION + " to write the persisted RDD to");
-        SparkContextStorage.open(configuration).rmr(configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));  // this might be bad cause it unpersists the job RDD
+        SparkContextStorage.open(configuration).rm(configuration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));  // this might be bad cause it unpersists the job RDD
         if (!configuration.getBoolean(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT_HAS_EDGES, true))
             graphRDD.mapValues(vertex -> {
                 vertex.get().dropEdges();

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/5c9e81b0/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
index d0fc984..6801304 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorage.java
@@ -84,17 +84,12 @@ public final class SparkContextStorage implements Storage {
     }
 
     @Override
-    public boolean mkdir(final String location) {
-        throw new UnsupportedOperationException("This operation does not make sense for a persisted SparkContext");
-    }
-
-    @Override
-    public boolean cp(final String fromLocation, final String toLocation) {
-        final List<String> rdds = Spark.getRDDs().stream().filter(r -> r.name().startsWith(fromLocation)).map(RDD::name).collect(Collectors.toList());
+    public boolean cp(final String sourceLocation, final String targetLocation) {
+        final List<String> rdds = Spark.getRDDs().stream().filter(r -> r.name().startsWith(sourceLocation)).map(RDD::name).collect(Collectors.toList());
         if (rdds.size() == 0)
             return false;
         for (final String rdd : rdds) {
-            Spark.getRDD(rdd).toJavaRDD().filter(a -> false).setName(rdd.equals(fromLocation) ? toLocation : rdd.replace(fromLocation, toLocation)).cache().count();
+            Spark.getRDD(rdd).toJavaRDD().filter(a -> true).setName(rdd.equals(sourceLocation) ? targetLocation : rdd.replace(sourceLocation, targetLocation)).cache().count();
         }
         return true;
     }
@@ -106,14 +101,6 @@ public final class SparkContextStorage implements Storage {
 
     @Override
     public boolean rm(final String location) {
-        if (!Spark.hasRDD(location))
-            return false;
-        Spark.removeRDD(location);
-        return true;
-    }
-
-    @Override
-    public boolean rmr(final String location) {
         final List<String> rdds = new ArrayList<>();
         final String wildCardLocation = (location.endsWith("*") ? location : location + "*").replace(".", "\\.").replace("*", ".*");
         for (final RDD<?> rdd : Spark.getRDDs()) {

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/5c9e81b0/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageCheck.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageCheck.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageCheck.java
index 60b44ef..9d9fa37 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageCheck.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageCheck.java
@@ -23,8 +23,6 @@ import org.apache.tinkerpop.gremlin.LoadGraphWith;
 import org.apache.tinkerpop.gremlin.hadoop.Constants;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.AbstractStorageCheck;
 import org.apache.tinkerpop.gremlin.spark.structure.Spark;
-import org.apache.tinkerpop.gremlin.spark.structure.io.PersistedInputRDD;
-import org.apache.tinkerpop.gremlin.spark.structure.io.SparkContextStorage;
 import org.apache.tinkerpop.gremlin.structure.io.Storage;
 import org.junit.Before;
 import org.junit.Test;
@@ -66,6 +64,6 @@ public class SparkContextStorageCheck extends AbstractStorageCheck {
         final Storage storage = SparkContextStorage.open("local[4]");
         final String outputLocation = graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION);
         final String newOutputLocation = "new-location-for-copy";
-        super.checkCopyMethods(storage, outputLocation, newOutputLocation);
+        super.checkCopyMethods(storage, outputLocation, newOutputLocation, PersistedInputRDD.class, PersistedInputRDD.class);
     }
 }
\ No newline at end of file



[26/30] incubator-tinkerpop git commit: updated CHANGELOG.

Posted by dk...@apache.org.
updated CHANGELOG.


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

Branch: refs/heads/TINKERPOP-320
Commit: d7b295be6ee0dd21c8d5f7d6a2300787460b3a77
Parents: 1e5dcaf
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Mon Jan 11 08:46:10 2016 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Mon Jan 11 08:46:10 2016 -0700

----------------------------------------------------------------------
 CHANGELOG.asciidoc | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/d7b295be/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index 3185290..fdbfb90 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -26,6 +26,7 @@ image::https://raw.githubusercontent.com/apache/incubator-tinkerpop/master/docs/
 TinkerPop 3.1.1 (NOT OFFICIALLY RELEASED YET)
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
+* Fixed a bug around duration calculations of `cap()`-step during profiling.
 * It is possible to completely avoid using HDFS with Spark if `PersistedInputRDD` and `PersistedOutpuRDD` are leveraged.
 * `InputRDD` and `OutputRDD` can now process both graphs and memory (i.e. sideEffects).
 * Removed Groovy specific meta-programming overloads for handling Hadoop `FileSystem` (instead, its all accessible via `FileSystemStorage`).


[24/30] incubator-tinkerpop git commit: Fixed a bug in the SimpleSandbox for Gremlin Server

Posted by dk...@apache.org.
Fixed a bug in the SimpleSandbox for Gremlin Server

Not sure how this every slipped through, but added an integration test to prevent future regressions.


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

Branch: refs/heads/TINKERPOP-320
Commit: b4cb00dd7aa567c14c94318fe65d9ef99c761f1d
Parents: 114609d
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Mon Jan 11 08:54:30 2016 -0500
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Mon Jan 11 08:54:30 2016 -0500

----------------------------------------------------------------------
 .../customizer/SimpleSandboxExtension.groovy    |  5 ++--
 .../server/GremlinServerIntegrateTest.java      | 30 ++++++++++++++++++++
 2 files changed, 33 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4cb00dd/gremlin-groovy/src/main/groovy/org/apache/tinkerpop/gremlin/groovy/jsr223/customizer/SimpleSandboxExtension.groovy
----------------------------------------------------------------------
diff --git a/gremlin-groovy/src/main/groovy/org/apache/tinkerpop/gremlin/groovy/jsr223/customizer/SimpleSandboxExtension.groovy b/gremlin-groovy/src/main/groovy/org/apache/tinkerpop/gremlin/groovy/jsr223/customizer/SimpleSandboxExtension.groovy
index fe6cfd2..e6a8046 100644
--- a/gremlin-groovy/src/main/groovy/org/apache/tinkerpop/gremlin/groovy/jsr223/customizer/SimpleSandboxExtension.groovy
+++ b/gremlin-groovy/src/main/groovy/org/apache/tinkerpop/gremlin/groovy/jsr223/customizer/SimpleSandboxExtension.groovy
@@ -43,9 +43,10 @@ class SimpleSandboxExtension extends GroovyTypeCheckingExtensionSupport.TypeChec
         }
 
         onMethodSelection { expr, MethodNode methodNode ->
-            def descriptor = toMethodDescriptor(methodNode)
-            if (null == descriptor.declaringClass || descriptor.declaringClass.name != 'java.lang.System')
+            if (null == methodNode.declaringClass || methodNode.declaringClass.name == 'java.lang.System') {
+                def descriptor = SandboxHelper.toMethodDescriptor(methodNode)
                 addStaticTypeError("Not authorized to call this method: $descriptor", expr)
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b4cb00dd/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerIntegrateTest.java
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerIntegrateTest.java b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerIntegrateTest.java
index d1376a1..df515bd 100644
--- a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerIntegrateTest.java
+++ b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerIntegrateTest.java
@@ -33,6 +33,8 @@ import org.apache.tinkerpop.gremlin.driver.simple.NioClient;
 import org.apache.tinkerpop.gremlin.driver.simple.SimpleClient;
 import org.apache.tinkerpop.gremlin.driver.simple.WebSocketClient;
 import org.apache.tinkerpop.gremlin.groovy.jsr223.GremlinGroovyScriptEngine;
+import org.apache.tinkerpop.gremlin.groovy.jsr223.customizer.CompileStaticCustomizerProvider;
+import org.apache.tinkerpop.gremlin.groovy.jsr223.customizer.SimpleSandboxExtension;
 import org.apache.tinkerpop.gremlin.structure.T;
 import org.apache.tinkerpop.gremlin.server.channel.NioChannelizer;
 import org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor;
@@ -56,6 +58,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+import static org.hamcrest.CoreMatchers.containsString;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.*;
 import static org.junit.Assume.assumeThat;
@@ -127,12 +130,39 @@ public class GremlinServerIntegrateTest extends AbstractGremlinServerIntegration
                 deleteDirectory(new File("/tmp/neo4j"));
                 settings.graphs.put("graph", "conf/neo4j-empty.properties");
                 break;
+            case "shouldUseSimpleSandbox":
+                final Map<String,Object> scriptEngineConf = new HashMap<>();
+                final Map<String,Object> compilerCustomizerProviderConf = new HashMap<>();
+                final List<String> sandboxes = new ArrayList<>();
+                sandboxes.add(SimpleSandboxExtension.class.getName());
+                compilerCustomizerProviderConf.put(CompileStaticCustomizerProvider.class.getName(), sandboxes);
+                scriptEngineConf.put("compilerCustomizerProviders", compilerCustomizerProviderConf);
+                settings.scriptEngines.get("gremlin-groovy").config = scriptEngineConf;
+                break;
         }
 
         return settings;
     }
 
     @Test
+    public void shouldUseSimpleSandbox() throws Exception {
+        final Cluster cluster = Cluster.open();
+        final Client client = cluster.connect();
+
+        assertEquals(2, client.submit("1+1").all().get().get(0).getInt());
+
+        try {
+            // this should return "nothing" - there should be no exception
+            client.submit("java.lang.System.exit(0)").all().get();
+            fail("The above should not have executed in any successful way as sandboxing is enabled");
+        } catch (Exception ex) {
+            assertThat(ex.getCause().getMessage(), containsString("[Static type checking] - Not authorized to call this method: java.lang.System#exit(int)"));
+        } finally {
+            cluster.close();
+        }
+    }
+
+    @Test
     public void shouldStartWithDefaultSettings() {
         // just quickly validate that results are returning given defaults. no graphs are config'd with defaults
         // so just eval a groovy script.


[08/30] incubator-tinkerpop git commit: cleanup HDFS if Persist.NOTHING.

Posted by dk...@apache.org.
cleanup HDFS if Persist.NOTHING.


Project: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/commit/55165a57
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/tree/55165a57
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/diff/55165a57

Branch: refs/heads/TINKERPOP-320
Commit: 55165a572f5d07e1ca20be13b064843da18fc8e6
Parents: 74b9c8e
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Dec 9 19:11:33 2015 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Wed Dec 9 19:11:33 2015 -0700

----------------------------------------------------------------------
 .../gremlin/spark/process/computer/SparkGraphComputer.java     | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/55165a57/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
index ecd9573..82c4331 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGraphComputer.java
@@ -24,9 +24,9 @@ import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.spark.SparkConf;
 import org.apache.spark.SparkContext;
 import org.apache.spark.api.java.JavaPairRDD;
@@ -37,6 +37,7 @@ import org.apache.tinkerpop.gremlin.hadoop.process.computer.AbstractHadoopGraphC
 import org.apache.tinkerpop.gremlin.hadoop.process.computer.util.ComputerSubmissionHelper;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopConfiguration;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.FileSystemStorage;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
 import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
 import org.apache.tinkerpop.gremlin.process.computer.ComputerResult;
@@ -244,6 +245,9 @@ public final class SparkGraphComputer extends AbstractHadoopGraphComputer {
                     if (apacheConfiguration.containsKey(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION))
                         SparkContextStorage.open().rmr(apacheConfiguration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));
                 }
+                // delete any file system output if persist nothing
+                if (FileOutputFormat.class.isAssignableFrom(hadoopConfiguration.getClass(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT, FileInputFormat.class)) && this.persist.equals(GraphComputer.Persist.NOTHING))
+                    FileSystemStorage.open(hadoopConfiguration).rmr(apacheConfiguration.getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));
                 // update runtime and return the newly computed graph
                 finalMemory.setRuntime(System.currentTimeMillis() - startTime);
                 return new DefaultComputerResult(InputOutputHelper.getOutputGraph(apacheConfiguration, this.resultGraph, this.persist), finalMemory.asImmutable());


[12/30] incubator-tinkerpop git commit: Add protocol to the list of Settings for the driver.

Posted by dk...@apache.org.
Add protocol to the list of Settings for the driver.

This was a minor oversight from previous work where the "protocol" value was not being passed in from yaml file. CTR


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

Branch: refs/heads/TINKERPOP-320
Commit: b44253d949c0e07e0cd1e1c3568783faaf382a78
Parents: 46c7189
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Wed Jan 6 13:13:44 2016 -0500
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Wed Jan 6 13:13:44 2016 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/tinkerpop/gremlin/driver/Cluster.java    | 3 +++
 .../main/java/org/apache/tinkerpop/gremlin/driver/Settings.java   | 2 ++
 2 files changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b44253d9/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/Cluster.java
----------------------------------------------------------------------
diff --git a/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/Cluster.java b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/Cluster.java
index ddc6081..c8d3bd6 100644
--- a/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/Cluster.java
+++ b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/Cluster.java
@@ -145,6 +145,9 @@ public final class Cluster {
         if (settings.jaasEntry != null)
             builder.jaasEntry(settings.jaasEntry);
 
+        if (settings.protocol != null)
+            builder.protocol(settings.protocol);
+
         // the first address was added above in the constructor, so skip it if there are more
         if (addresses.size() > 1)
             addresses.stream().skip(1).forEach(builder::addContactPoint);

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b44253d9/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/Settings.java
----------------------------------------------------------------------
diff --git a/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/Settings.java b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/Settings.java
index 7faa377..7a40d4f 100644
--- a/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/Settings.java
+++ b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/Settings.java
@@ -53,6 +53,8 @@ final class Settings {
 
     public String jaasEntry = null;
 
+    public String protocol = null;
+
     /**
      * Read configuration from a file into a new {@link Settings} object.
      *


[29/30] incubator-tinkerpop git commit: updated CHANGELOG and reference docs

Posted by dk...@apache.org.
updated CHANGELOG and reference docs


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

Branch: refs/heads/TINKERPOP-320
Commit: d7ae9236bd2a08212f0aa6ac1547bd7b3da95d68
Parents: 53f28d4
Author: Daniel Kuppitz <da...@hotmail.com>
Authored: Mon Jan 11 17:46:23 2016 +0100
Committer: Daniel Kuppitz <da...@hotmail.com>
Committed: Mon Jan 11 17:46:23 2016 +0100

----------------------------------------------------------------------
 CHANGELOG.asciidoc                            |  1 +
 docs/src/reference/implementations.asciidoc   | 18 ++++++++++++++++++
 docs/src/reference/the-graphcomputer.asciidoc |  9 +++++++++
 3 files changed, 28 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/d7ae9236/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index 3185290..27003b6 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -26,6 +26,7 @@ image::https://raw.githubusercontent.com/apache/incubator-tinkerpop/master/docs/
 TinkerPop 3.1.1 (NOT OFFICIALLY RELEASED YET)
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
+* Added `BulkDumperVertex` that allows to dump a whole graph in any of the supported IO formats (GraphSON, Gryo, Script).
 * It is possible to completely avoid using HDFS with Spark if `PersistedInputRDD` and `PersistedOutpuRDD` are leveraged.
 * `InputRDD` and `OutputRDD` can now process both graphs and memory (i.e. sideEffects).
 * Removed Groovy specific meta-programming overloads for handling Hadoop `FileSystem` (instead, its all accessible via `FileSystemStorage`).

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/d7ae9236/docs/src/reference/implementations.asciidoc
----------------------------------------------------------------------
diff --git a/docs/src/reference/implementations.asciidoc b/docs/src/reference/implementations.asciidoc
index add8555..b55e36b 100644
--- a/docs/src/reference/implementations.asciidoc
+++ b/docs/src/reference/implementations.asciidoc
@@ -1227,6 +1227,24 @@ references to that Spark Context. The exception to this rule are those propertie
 
 Finally, there is a `spark` object that can be used to manage persisted RDDs (see <<interacting-with-spark, Interacting with Spark>>).
 
+[[bulkdumpervertexprogramusingspark]]
+Exporting with BulkDumperVertexProgram
+++++++++++++++++++++++++++++++++++++++
+
+The <<bulkdumpervertexprogram, BulkDumperVertexProgram>> exports a whole graph in any of the supported Hadoop GraphOutputFormats (`GraphSONOutputFormat`,
+`GryoOutputFormat` or `ScriptOutputFormat`). The example below takes a Hadoop graph as the input (in `GryoInputFormat`) and exports it as a GraphSON file
+(`GraphSONOutputFormat`).
+
+[gremlin-groovy]
+----
+hdfs.copyFromLocal('data/tinkerpop-modern.kryo', 'tinkerpop-modern.kryo')
+graph = GraphFactory.open('conf/hadoop/hadoop-gryo.properties')
+graph.configuration().setProperty('gremlin.hadoop.graphOutputFormat', 'org.apache.tinkerpop.gremlin.hadoop.structure.io.graphson.GraphSONOutputFormat')
+graph.compute(SparkGraphComputer).program(BulkDumperVertexProgram.build().create()).submit().get()
+hdfs.ls('output')
+hdfs.head('output/~g')
+----
+
 Loading with BulkLoaderVertexProgram
 ++++++++++++++++++++++++++++++++++++
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/d7ae9236/docs/src/reference/the-graphcomputer.asciidoc
----------------------------------------------------------------------
diff --git a/docs/src/reference/the-graphcomputer.asciidoc b/docs/src/reference/the-graphcomputer.asciidoc
index d2a429a..f9116fb 100644
--- a/docs/src/reference/the-graphcomputer.asciidoc
+++ b/docs/src/reference/the-graphcomputer.asciidoc
@@ -319,6 +319,15 @@ same cluster. The algorithm proceeds in the following manner.
   .. If there is a tie, then the cluster with the lowest `toString()` comparison is selected.
  . Steps 3 and 4 repeat until either a max number of iterations has occurred or no vertex has adjusted its cluster anymore.
 
+[[bulkdumpervertexprogram]]
+BulkDumperVertexProgram
+~~~~~~~~~~~~~~~~~~~~~~~
+
+The `BulkDumperVertexProgram` can be used to export a whole graph in any of the provided Hadoop GraphOutputFormats (e.g.
+`GraphSONOutputFormat`, `GryoOutputFormat` or `ScriptOutputFormat`). The input can be any Hadoop GraphInputFormat
+(e.g. `GraphSONInputFormat`, `GryoInputFormat` or `ScriptInputFormat`). An <<bulkdumpervertexprogramusingspark,example>>
+is provided in the SparkGraphComputer section.
+
 [[bulkloadervertexprogram]]
 BulkLoaderVertexProgram
 ~~~~~~~~~~~~~~~~~~~~~~~


[17/30] incubator-tinkerpop git commit: Fix the way DependantMutableMetrics profiler durations are calculated.

Posted by dk...@apache.org.
Fix the way DependantMutableMetrics profiler durations are calculated.


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

Branch: refs/heads/TINKERPOP-320
Commit: 208910695b3b37bfc7a80c3cdfe945c931edf2c5
Parents: b44253d
Author: rjbriody <bo...@datastax.com>
Authored: Thu Jan 7 15:52:19 2016 -0500
Committer: rjbriody <bo...@datastax.com>
Committed: Thu Jan 7 15:52:19 2016 -0500

----------------------------------------------------------------------
 .../traversal/util/DependantMutableMetrics.java | 34 +++++++++++---------
 .../process/traversal/util/MutableMetrics.java  |  3 +-
 2 files changed, 21 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/20891069/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/util/DependantMutableMetrics.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/util/DependantMutableMetrics.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/util/DependantMutableMetrics.java
index fd580fe..0ce939a 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/util/DependantMutableMetrics.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/util/DependantMutableMetrics.java
@@ -18,9 +18,11 @@
  */
 package org.apache.tinkerpop.gremlin.process.traversal.util;
 
+import java.util.concurrent.TimeUnit;
+
 /**
  * This Metrics class handles a metrics chain in which durations are "double counted" by upstream metrics. Durations are
- * corrected on-the-fly by subtracting upstream durations on every call to stop().
+ * corrected upon retrieval by subtracting upstream durations.
  *
  * @author Bob Briody (http://bobbriody.com)
  */
@@ -38,22 +40,24 @@ public class DependantMutableMetrics extends MutableMetrics {
         this.upStreamMetrics = upStreamMetrics;
     }
 
-    public void start() {
-        super.start();
-    }
-
-    public void stop() {
-        super.stop();
-        // root step will not have an upstream metrics
-        if (upStreamMetrics != null) {
-            // subtract time that is "double counted" by upstream metrics
-            super.durationNs -= upStreamMetrics.getAndResetIncrementalDur();
+    /**
+     * Returns the actual duration taken by this Metrics by subtracting the duration taken by the upstream Step, if one exists.
+     * @param unit
+     * @return
+     */
+    @Override
+    public long getDuration(final TimeUnit unit) {
+        if (upStreamMetrics == null){
+           return unit.convert(super.durationNs, unit);
+        } else {
+           // upStreamMetrics exists. Subtract that duration since it is time not spent in this step.
+           return unit.convert(super.durationNs - upStreamMetrics.durationNs, unit);
         }
     }
 
-    public long getAndResetIncrementalDur() {
-        long incrementalDur = super.durationNs - prevDur;
-        prevDur = super.durationNs;
-        return incrementalDur;
+    @Override
+    protected void copyMembers(final ImmutableMetrics clone) {
+        super.copyMembers(clone);
+        clone.durationNs = this.getDuration(TimeUnit.NANOSECONDS);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/20891069/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/util/MutableMetrics.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/util/MutableMetrics.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/util/MutableMetrics.java
index 5305020..72c1076 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/util/MutableMetrics.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/util/MutableMetrics.java
@@ -157,9 +157,10 @@ public class MutableMetrics extends ImmutableMetrics implements Cloneable {
         return clone;
     }
 
-    private void copyMembers(final ImmutableMetrics clone) {
+    protected void copyMembers(final ImmutableMetrics clone) {
         clone.id = this.id;
         clone.name = this.name;
+        // Note: This value is overwritten in the DependantMutableMetrics overridden copyMembers method.
         clone.durationNs = this.durationNs;
         for (Map.Entry<String, AtomicLong> c : this.counts.entrySet()) {
             clone.counts.put(c.getKey(), new AtomicLong(c.getValue().get()));


[15/30] incubator-tinkerpop git commit: added a Storage test case to ensure that residual data for Persist.NOTHING is consistent for both Giraph and Spark. Giraph always left behind sideEffects (memory) on disk. Spark doesn't. Decided NOTHING means destr

Posted by dk...@apache.org.
added a Storage test case to ensure that residual data for Persist.NOTHING is consistent for both Giraph and Spark. Giraph always left behind sideEffects (memory) on disk. Spark doesn't. Decided NOTHING means destroy persisted memory as well. For 3.2.0, we need to have contracts for all this specified. Made a ticket. Rebuilt docs and ran test suites, all is still golden.


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

Branch: refs/heads/TINKERPOP-320
Commit: cc7cfffe47ae284ee6ecd67afffb34a6fe3b0b42
Parents: a7db52b
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Thu Jan 7 09:43:07 2016 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Thu Jan 7 09:43:07 2016 -0700

----------------------------------------------------------------------
 .../process/computer/GiraphGraphComputer.java   |  6 +++---
 .../computer/GiraphHadoopGraphProvider.java     |  6 +++---
 .../structure/io/AbstractStorageCheck.java      | 22 ++++++++++++++++++++
 .../structure/io/FileSystemStorageCheck.java    |  8 +++++++
 .../structure/io/SparkContextStorageCheck.java  | 11 +++++++---
 5 files changed, 44 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/cc7cfffe/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphGraphComputer.java
----------------------------------------------------------------------
diff --git a/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphGraphComputer.java b/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphGraphComputer.java
index dfe8e8c..82b3ec1 100644
--- a/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphGraphComputer.java
+++ b/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphGraphComputer.java
@@ -120,7 +120,7 @@ public final class GiraphGraphComputer extends AbstractHadoopGraphComputer imple
         return ComputerSubmissionHelper.runWithBackgroundThread(this::submitWithExecutor, "GiraphSubmitter");
     }
 
-    private Future<ComputerResult> submitWithExecutor(Executor exec) {
+    private Future<ComputerResult> submitWithExecutor(final Executor exec) {
         final long startTime = System.currentTimeMillis();
         final Configuration apacheConfiguration = ConfUtil.makeApacheConfiguration(this.giraphConfiguration);
         return CompletableFuture.<ComputerResult>supplyAsync(() -> {
@@ -203,9 +203,9 @@ public final class GiraphGraphComputer extends AbstractHadoopGraphComputer imple
                 MapReduceHelper.executeMapReduceJob(mapReduce, this.memory, this.giraphConfiguration);
             }
 
-            // if no persistence, delete the graph output
+            // if no persistence, delete the graph and memory output
             if (this.persist.equals(Persist.NOTHING))
-                storage.rm(Constants.getGraphLocation(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION)));
+                storage.rm(this.giraphConfiguration.get(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION));
         } catch (final Exception e) {
             throw new IllegalStateException(e.getMessage(), e);
         }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/cc7cfffe/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphHadoopGraphProvider.java
----------------------------------------------------------------------
diff --git a/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphHadoopGraphProvider.java b/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphHadoopGraphProvider.java
index 9257006..07b7897 100644
--- a/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphHadoopGraphProvider.java
+++ b/giraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphHadoopGraphProvider.java
@@ -39,13 +39,13 @@ public final class GiraphHadoopGraphProvider extends HadoopGraphProvider {
     @Override
     public Map<String, Object> getBaseConfiguration(final String graphName, final Class<?> test, final String testMethodName, final LoadGraphWith.GraphData loadGraphWith) {
         final Map<String, Object> config = super.getBaseConfiguration(graphName, test, testMethodName, loadGraphWith);
-        config.put("mapreduce.job.reduces", 4);
+        config.put("mapreduce.job.reduces", 2);
         /// giraph configuration
         config.put(GiraphConstants.LOCAL_TEST_MODE.getKey(), true); // local testing can only spawn one worker
         config.put(GiraphConstants.MIN_WORKERS, 1);
         config.put(GiraphConstants.MAX_WORKERS, 1);
         config.put(GiraphConstants.SPLIT_MASTER_WORKER.getKey(), false);
-        config.put(GiraphConstants.ZOOKEEPER_SERVER_PORT.getKey(), 2181);  // you must have a local zookeeper running on this port
+        config.put(GiraphConstants.ZOOKEEPER_IS_EXTERNAL.getKey(), false);
         config.put(GiraphConstants.NETTY_SERVER_USE_EXECUTION_HANDLER.getKey(), false); // this prevents so many integration tests running out of threads
         config.put(GiraphConstants.NETTY_CLIENT_USE_EXECUTION_HANDLER.getKey(), false); // this prevents so many integration tests running out of threads
         config.put(GiraphConstants.NETTY_USE_DIRECT_MEMORY.getKey(), true);
@@ -53,7 +53,7 @@ public final class GiraphHadoopGraphProvider extends HadoopGraphProvider {
         config.put(GiraphConstants.NUM_COMPUTE_THREADS.getKey(), 2);
         config.put(GiraphConstants.MAX_MASTER_SUPERSTEP_WAIT_MSECS.getKey(), TimeUnit.MINUTES.toMillis(60L));
         config.put(GiraphConstants.VERTEX_OUTPUT_FORMAT_THREAD_SAFE.getKey(), false);
-        config.put(GiraphConstants.NUM_OUTPUT_THREADS.getKey(), 2);
+        config.put(GiraphConstants.NUM_OUTPUT_THREADS.getKey(), 1);
         return config;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/cc7cfffe/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/AbstractStorageCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/AbstractStorageCheck.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/AbstractStorageCheck.java
index bec9c72..1a73093 100644
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/AbstractStorageCheck.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/AbstractStorageCheck.java
@@ -24,9 +24,14 @@ import org.apache.tinkerpop.gremlin.hadoop.Constants;
 import org.apache.tinkerpop.gremlin.process.computer.ComputerResult;
 import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.ClusterCountMapReduce;
 import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.PeerPressureVertexProgram;
+import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversal;
+import org.apache.tinkerpop.gremlin.structure.Graph;
+import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.structure.io.Storage;
 import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 
+import java.util.Map;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -120,4 +125,21 @@ public abstract class AbstractStorageCheck extends AbstractGremlinTest {
         assertEquals(1, IteratorUtils.count(storage.head(outputLocation, "clusterCount", outputMemoryParserClass)));
         assertEquals(1, IteratorUtils.count(storage.head(newOutputLocation, "clusterCount", outputMemoryParserClass)));
     }
+
+    public void checkResidualDataInStorage(final Storage storage, final String outputLocation) throws Exception {
+        final GraphTraversal<Vertex, Long> traversal = g.V().both("knows").groupCount("m").by("age").count();
+        assertEquals(4l, traversal.next().longValue());
+        assertFalse(storage.exists(outputLocation));
+        assertFalse(storage.exists(Constants.getMemoryLocation(outputLocation, "m")));
+        assertFalse(storage.exists(Constants.getMemoryLocation(outputLocation, Graph.Hidden.hide("reducing"))));
+        assertFalse(storage.exists(Constants.getGraphLocation(outputLocation)));
+        ///
+        assertEquals(3, traversal.asAdmin().getSideEffects().<Map<Integer, Long>>get("m").get().size());
+        assertEquals(1, traversal.asAdmin().getSideEffects().<Map<Integer, Long>>get("m").get().get(27).longValue());
+        assertEquals(2, traversal.asAdmin().getSideEffects().<Map<Integer, Long>>get("m").get().get(29).longValue());
+        assertEquals(1, traversal.asAdmin().getSideEffects().<Map<Integer, Long>>get("m").get().get(32).longValue());
+        ///
+        assertEquals(4l, traversal.asAdmin().getSideEffects().<Long>get(Graph.Hidden.hide("reducing")).get().longValue());
+
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/cc7cfffe/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorageCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorageCheck.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorageCheck.java
index 1b2c04e..846582e 100644
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorageCheck.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/FileSystemStorageCheck.java
@@ -70,6 +70,14 @@ public class FileSystemStorageCheck extends AbstractStorageCheck {
 
     }
 
+    @Test
+    @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+    public void shouldNotHaveResidualDataInStorage() throws Exception {
+        final Storage storage = FileSystemStorage.open(ConfUtil.makeHadoopConfiguration(graph.configuration()));
+        final String outputLocation = graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION);
+        super.checkResidualDataInStorage(storage, outputLocation);
+    }
+
     private static void deleteDirectory(final String location) throws IOException {
         // TestHelper creates the directory and we need it not to exist
         assertTrue(new File(location).isDirectory());

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/cc7cfffe/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageCheck.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageCheck.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageCheck.java
index 9d9fa37..c5746b6 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageCheck.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/SparkContextStorageCheck.java
@@ -27,8 +27,6 @@ import org.apache.tinkerpop.gremlin.structure.io.Storage;
 import org.junit.Before;
 import org.junit.Test;
 
-import static org.junit.Assert.assertFalse;
-
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
@@ -46,7 +44,6 @@ public class SparkContextStorageCheck extends AbstractStorageCheck {
     public void shouldSupportHeadMethods() throws Exception {
         final Storage storage = SparkContextStorage.open("local[4]");
         final String outputLocation = graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION);
-        assertFalse(storage.exists(outputLocation));
         super.checkHeadMethods(storage, graph.configuration().getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION), outputLocation, PersistedInputRDD.class, PersistedInputRDD.class);
     }
 
@@ -66,4 +63,12 @@ public class SparkContextStorageCheck extends AbstractStorageCheck {
         final String newOutputLocation = "new-location-for-copy";
         super.checkCopyMethods(storage, outputLocation, newOutputLocation, PersistedInputRDD.class, PersistedInputRDD.class);
     }
+
+    @Test
+    @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+    public void shouldNotHaveResidualDataInStorage() throws Exception {
+        final Storage storage = SparkContextStorage.open("local[4]");
+        final String outputLocation = graph.configuration().getString(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION);
+        super.checkResidualDataInStorage(storage, outputLocation);
+    }
 }
\ No newline at end of file