You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tinkerpop.apache.org by sp...@apache.org on 2015/05/04 17:48:26 UTC

[5/8] incubator-tinkerpop git commit: GraphSONReaderReader and GryoRecordReader now directly use the StarGraph.StarVertex as it is simply fed into a VertexWritable. This makes it so we don't create a StarGraph at the XXXReader and then again at the XXXRe

GraphSONReaderReader and GryoRecordReader now directly use the StarGraph.StarVertex as it is simply fed into a VertexWritable. This makes it so we don't create a StarGraph at the XXXReader and then again at the XXXRecordReader. 1/2 the amount of objects, memory, and strain on the GC.


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

Branch: refs/heads/gs-protocol
Commit: ad3570f8f95f60d3e8677ef35d1b8d89ca092239
Parents: 4070af8
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Mon May 4 09:04:57 2015 -0600
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Mon May 4 09:04:57 2015 -0600

----------------------------------------------------------------------
 .../structure/io/graphson/GraphSONRecordReader.java  | 13 ++-----------
 .../hadoop/structure/io/gryo/GryoRecordReader.java   | 15 +--------------
 .../gremlin/hadoop/HadoopGraphProvider.java          | 10 ++++++++--
 3 files changed, 11 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ad3570f8/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/graphson/GraphSONRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/graphson/GraphSONRecordReader.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/graphson/GraphSONRecordReader.java
index 497b42f..d0460f3 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/graphson/GraphSONRecordReader.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/graphson/GraphSONRecordReader.java
@@ -26,18 +26,12 @@ import org.apache.hadoop.mapreduce.lib.input.LineRecordReader;
 import org.apache.tinkerpop.gremlin.hadoop.Constants;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
 import org.apache.tinkerpop.gremlin.structure.Direction;
-import org.apache.tinkerpop.gremlin.structure.Edge;
-import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONReader;
 import org.apache.tinkerpop.gremlin.structure.util.Attachable;
-import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedEdge;
-import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedVertex;
-import org.apache.tinkerpop.gremlin.structure.util.star.StarGraph;
 
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.function.Function;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
@@ -64,13 +58,10 @@ public class GraphSONRecordReader extends RecordReader<NullWritable, VertexWrita
         if (!this.lineRecordReader.nextKeyValue())
             return false;
 
-        final StarGraph starGraph = StarGraph.open();
-        final Function<Attachable<Vertex>, Vertex> vertexMaker = attachableVertex -> attachableVertex.attach(Attachable.Method.create(starGraph));
-        final Function<Attachable<Edge>, Edge> edgeMaker = attachableEdge -> attachableEdge.attach(Attachable.Method.create(starGraph));
         try (InputStream in = new ByteArrayInputStream(this.lineRecordReader.getCurrentValue().getBytes())) {
             this.vertexWritable.set(this.hasEdges ?
-                    this.graphsonReader.readVertex(in, vertexMaker, edgeMaker, Direction.BOTH) :
-                    this.graphsonReader.readVertex(in, vertexMaker));
+                    this.graphsonReader.readVertex(in, Attachable::get, Attachable::get, Direction.BOTH) :
+                    this.graphsonReader.readVertex(in, Attachable::get));
             return true;
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ad3570f8/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/gryo/GryoRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/gryo/GryoRecordReader.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/gryo/GryoRecordReader.java
index 53b4162..3a12087 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/gryo/GryoRecordReader.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/gryo/GryoRecordReader.java
@@ -27,22 +27,16 @@ import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.tinkerpop.gremlin.hadoop.Constants;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
-import org.apache.tinkerpop.gremlin.structure.Direction;
-import org.apache.tinkerpop.gremlin.structure.Edge;
-import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoMapper;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoReader;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.VertexTerminator;
 import org.apache.tinkerpop.gremlin.structure.util.Attachable;
-import org.apache.tinkerpop.gremlin.structure.util.star.StarGraph;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.function.Function;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
@@ -56,7 +50,6 @@ public class GryoRecordReader extends RecordReader<NullWritable, VertexWritable>
 
     private final GryoReader gryoReader = GryoReader.build().create();
     private final VertexWritable vertexWritable = new VertexWritable();
-    private boolean hasEdges;
 
     private long currentLength = 0;
     private long splitLength;
@@ -76,7 +69,6 @@ public class GryoRecordReader extends RecordReader<NullWritable, VertexWritable>
         // open the file and seek to the start of the split
         this.inputStream = file.getFileSystem(job).open(split.getPath());
         this.splitLength = split.getLength() - (seekToHeader(this.inputStream, start) - start);
-        this.hasEdges = context.getConfiguration().getBoolean(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT_HAS_EDGES, true);
     }
 
     private static long seekToHeader(final FSDataInputStream inputStream, final long start) throws IOException {
@@ -127,13 +119,8 @@ public class GryoRecordReader extends RecordReader<NullWritable, VertexWritable>
 
             terminatorLocation = ((byte) currentByte) == TERMINATOR[terminatorLocation] ? terminatorLocation + 1 : 0;
             if (terminatorLocation >= TERMINATOR.length) {
-                final StarGraph starGraph = StarGraph.open();
-                final Function<Attachable<Vertex>, Vertex> vertexMaker = attachableVertex -> attachableVertex.attach(Attachable.Method.create(starGraph));
-                final Function<Attachable<Edge>, Edge> edgeMaker = attachableEdge -> attachableEdge.attach(Attachable.Method.create(starGraph));
                 try (InputStream in = new ByteArrayInputStream(output.toByteArray())) {
-                    this.vertexWritable.set(this.hasEdges ?
-                            this.gryoReader.readVertex(in, vertexMaker, edgeMaker, Direction.BOTH) :
-                            this.gryoReader.readVertex(in, vertexMaker));
+                    this.vertexWritable.set(this.gryoReader.readVertex(in, Attachable::get)); // I know how GryoReader works, so I'm cheating here
                     return true;
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ad3570f8/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java
index dcca647..f447753 100644
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java
@@ -32,6 +32,7 @@ import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopProperty;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopVertex;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopVertexProperty;
 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.hadoop.structure.io.gryo.GryoOutputFormat;
 import org.apache.tinkerpop.gremlin.structure.Graph;
 import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONResourceAccess;
@@ -43,6 +44,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Random;
 import java.util.Set;
 
 /**
@@ -52,6 +54,9 @@ import java.util.Set;
  */
 public class HadoopGraphProvider extends AbstractGraphProvider {
 
+    private static final Random RANDOM = new Random();
+    private boolean graphSONInput = false;
+
     public static Map<String, String> PATHS = new HashMap<>();
     private static final Set<Class> IMPLEMENTATION = new HashSet<Class>() {{
         add(HadoopEdge.class);
@@ -97,9 +102,10 @@ public class HadoopGraphProvider extends AbstractGraphProvider {
 
     @Override
     public Map<String, Object> getBaseConfiguration(final String graphName, final Class<?> test, final String testMethodName, final LoadGraphWith.GraphData loadGraphWith) {
+        this.graphSONInput = RANDOM.nextBoolean();
         return new HashMap<String, Object>() {{
             put(Graph.GRAPH, HadoopGraph.class.getName());
-            put(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, GraphSONInputFormat.class.getCanonicalName());
+            put(Constants.GREMLIN_HADOOP_GRAPH_INPUT_FORMAT, graphSONInput ? GraphSONInputFormat.class.getCanonicalName() : GryoInputFormat.class.getCanonicalName());
             put(Constants.GREMLIN_HADOOP_GRAPH_OUTPUT_FORMAT, GryoOutputFormat.class.getCanonicalName());
             put(Constants.GREMLIN_HADOOP_MEMORY_OUTPUT_FORMAT, SequenceFileOutputFormat.class.getCanonicalName());
             put(Constants.GREMLIN_HADOOP_OUTPUT_LOCATION, "hadoop-gremlin/target/test-output");
@@ -141,7 +147,7 @@ public class HadoopGraphProvider extends AbstractGraphProvider {
     }
 
     public void loadGraphDataViaHadoopConfig(final Graph g, final LoadGraphWith.GraphData graphData) {
-        final String type = "json";
+        final String type = this.graphSONInput ? "json" : "kryo";
 
         if (graphData.equals(LoadGraphWith.GraphData.GRATEFUL)) {
             ((HadoopGraph) g).configuration().setInputLocation(PATHS.get("grateful-dead." + type));