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/04/23 17:58:57 UTC

[1/2] incubator-tinkerpop git commit: Added Host interface and altered readEdge() signature.

Repository: incubator-tinkerpop
Updated Branches:
  refs/heads/refactor-io 6bb950fc0 -> 4fb573988


Added Host interface and altered readEdge() signature.

This was a bit of a model commit to see how usage of Host and Attachable in IO would look.


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

Branch: refs/heads/refactor-io
Commit: b7a1a7e98f04e99536720c2e8526605d54ce87d9
Parents: 30d0388
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Thu Apr 23 11:57:39 2015 -0400
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Apr 23 11:57:39 2015 -0400

----------------------------------------------------------------------
 .../tinkerpop/gremlin/structure/Graph.java      |  2 +-
 .../tinkerpop/gremlin/structure/Host.java       | 29 +++++++++++++++++++
 .../tinkerpop/gremlin/structure/Vertex.java     |  2 +-
 .../gremlin/structure/io/GraphReader.java       |  9 ++++--
 .../structure/io/graphml/GraphMLReader.java     |  5 +++-
 .../structure/io/graphson/GraphSONReader.java   | 14 +++++++--
 .../io/graphson/LegacyGraphSONReader.java       |  5 +++-
 .../gremlin/structure/io/gryo/GryoReader.java   |  9 ++++--
 .../gremlin/structure/util/Attachable.java      |  9 +++---
 .../tinkerpop/gremlin/structure/IoTest.java     | 30 +++++++++++++-------
 10 files changed, 89 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b7a1a7e9/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/Graph.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/Graph.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/Graph.java
index ffc5bdc..5f8816b 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/Graph.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/Graph.java
@@ -52,7 +52,7 @@ import java.util.stream.Collectors;
  * @author Stephen Mallette (http://stephen.genoprime.com)
  * @author Pieter Martin
  */
-public interface Graph extends AutoCloseable {
+public interface Graph extends AutoCloseable, Host {
 
     public static final String GRAPH = "gremlin.graph";
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b7a1a7e9/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/Host.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/Host.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/Host.java
new file mode 100644
index 0000000..f8a2d1b
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/Host.java
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+import org.apache.tinkerpop.gremlin.structure.util.Attachable;
+
+/**
+ * A marker interface that identifies an object as something that an {@link Attachable} can connect to.
+ *
+ * @author Stephen Mallette (http://stephen.genoprime.com)
+ */
+public interface Host {
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b7a1a7e9/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/Vertex.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/Vertex.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/Vertex.java
index ca71361..3e23a7e 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/Vertex.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/Vertex.java
@@ -33,7 +33,7 @@ import java.util.Iterator;
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  * @author Stephen Mallette (http://stephen.genoprime.com)
  */
-public interface Vertex extends Element {
+public interface Vertex extends Element, Host {
 
     /**
      * The default label to use for a vertex.

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b7a1a7e9/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/GraphReader.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/GraphReader.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/GraphReader.java
index e4064b0..1a86311 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/GraphReader.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/GraphReader.java
@@ -21,7 +21,9 @@ package org.apache.tinkerpop.gremlin.structure.io;
 import org.apache.tinkerpop.gremlin.structure.Direction;
 import org.apache.tinkerpop.gremlin.structure.Edge;
 import org.apache.tinkerpop.gremlin.structure.Graph;
+import org.apache.tinkerpop.gremlin.structure.Host;
 import org.apache.tinkerpop.gremlin.structure.Vertex;
+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;
 
@@ -29,6 +31,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.Iterator;
+import java.util.function.BiFunction;
 import java.util.function.Function;
 
 /**
@@ -91,11 +94,13 @@ public interface GraphReader {
      * Reads a single edge from an {@link InputStream}.
      *
      * @param inputStream a stream containing a single vertex as defined by the accompanying {@link GraphWriter}
-     * @param edgeMaker   a function that creates an edge from the stream where the first argument is the edge
+     * @param host        the object to be passed to the {@code reattach} function when coordinating the re-attachment
+     *                    of the {@link Edge}.
+     * @param reattach    a function that creates an edge from the stream where the first argument is the edge
      *                    identifier, the second argument is the out vertex id, the third is the in vertex id,
      *                    the fourth is the label, and the fifth is the list of properties as key/value pairs.
      */
-    public Edge readEdge(final InputStream inputStream, final Function<DetachedEdge, Edge> edgeMaker) throws IOException;
+    public Edge readEdge(final InputStream inputStream, final Host host, final BiFunction<Attachable<Edge>, Host, Edge> reattach) throws IOException;
 
     /**
      * Reads an arbitrary object using the standard serializers.

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b7a1a7e9/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphml/GraphMLReader.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphml/GraphMLReader.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphml/GraphMLReader.java
index a2faa05..205df93 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphml/GraphMLReader.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphml/GraphMLReader.java
@@ -18,6 +18,7 @@
  */
 package org.apache.tinkerpop.gremlin.structure.io.graphml;
 
+import org.apache.tinkerpop.gremlin.structure.Host;
 import org.apache.tinkerpop.gremlin.structure.T;
 import org.apache.tinkerpop.gremlin.structure.Direction;
 import org.apache.tinkerpop.gremlin.structure.Edge;
@@ -25,6 +26,7 @@ import org.apache.tinkerpop.gremlin.structure.Graph;
 import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.structure.io.GraphReader;
 import org.apache.tinkerpop.gremlin.structure.io.Io;
+import org.apache.tinkerpop.gremlin.structure.util.Attachable;
 import org.apache.tinkerpop.gremlin.structure.util.batch.BatchGraph;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedEdge;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedVertex;
@@ -39,6 +41,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Optional;
+import java.util.function.BiFunction;
 import java.util.function.Function;
 import java.util.stream.Stream;
 
@@ -80,7 +83,7 @@ public class GraphMLReader implements GraphReader {
     }
 
     @Override
-    public Edge readEdge(final InputStream inputStream, final Function<DetachedEdge, Edge> edgeMaker) throws IOException {
+    public Edge readEdge(final InputStream inputStream, final Host host, final BiFunction<Attachable<Edge>, Host, Edge> edgeMaker) throws IOException {
         throw Io.Exceptions.readerFormatIsForFullGraphSerializationOnly(this.getClass());
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b7a1a7e9/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONReader.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONReader.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONReader.java
index 7c69203..21a4431 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONReader.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONReader.java
@@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.core.JsonToken;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.tinkerpop.gremlin.structure.Host;
 import org.apache.tinkerpop.gremlin.structure.T;
 import org.apache.tinkerpop.gremlin.structure.Direction;
 import org.apache.tinkerpop.gremlin.structure.Edge;
@@ -30,6 +31,7 @@ import org.apache.tinkerpop.gremlin.structure.Graph;
 import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.structure.VertexProperty;
 import org.apache.tinkerpop.gremlin.structure.io.GraphReader;
+import org.apache.tinkerpop.gremlin.structure.util.Attachable;
 import org.apache.tinkerpop.gremlin.structure.util.batch.BatchGraph;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedEdge;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedVertex;
@@ -46,6 +48,7 @@ import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.function.BiFunction;
 import java.util.function.Function;
 
 /**
@@ -143,9 +146,16 @@ public class GraphSONReader implements GraphReader {
     }
 
     @Override
-    public Edge readEdge(final InputStream inputStream, final Function<DetachedEdge, Edge> edgeMaker) throws IOException {
+    public Edge readEdge(final InputStream inputStream, final Host host, final BiFunction<Attachable<Edge>, Host, Edge> edgeMaker) throws IOException {
         final Map<String, Object> edgeData = mapper.readValue(inputStream, mapTypeReference);
-        return readEdgeData(edgeData, edgeMaker);
+
+        final DetachedEdge edge = new DetachedEdge(edgeData.get(GraphSONTokens.ID),
+                edgeData.get(GraphSONTokens.LABEL).toString(),
+                (Map<String, Object>) edgeData.get(GraphSONTokens.PROPERTIES),
+                Pair.with(edgeData.get(GraphSONTokens.OUT), edgeData.get(GraphSONTokens.OUT_LABEL).toString()),
+                Pair.with(edgeData.get(GraphSONTokens.IN), edgeData.get(GraphSONTokens.IN_LABEL).toString()));
+
+        return (Edge) edgeMaker.apply(edge, host);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b7a1a7e9/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/LegacyGraphSONReader.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/LegacyGraphSONReader.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/LegacyGraphSONReader.java
index fc02b50..74d8e5d 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/LegacyGraphSONReader.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/LegacyGraphSONReader.java
@@ -24,6 +24,7 @@ import com.fasterxml.jackson.core.JsonToken;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.module.SimpleModule;
+import org.apache.tinkerpop.gremlin.structure.Host;
 import org.apache.tinkerpop.gremlin.structure.T;
 import org.apache.tinkerpop.gremlin.structure.Direction;
 import org.apache.tinkerpop.gremlin.structure.Edge;
@@ -32,6 +33,7 @@ import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.structure.VertexProperty;
 import org.apache.tinkerpop.gremlin.structure.io.GraphReader;
 import org.apache.tinkerpop.gremlin.structure.io.Io;
+import org.apache.tinkerpop.gremlin.structure.util.Attachable;
 import org.apache.tinkerpop.gremlin.structure.util.batch.BatchGraph;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedEdge;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedVertex;
@@ -43,6 +45,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.function.BiFunction;
 import java.util.function.Function;
 
 /**
@@ -125,7 +128,7 @@ public class LegacyGraphSONReader implements GraphReader {
     }
 
     @Override
-    public Edge readEdge(final InputStream inputStream, final Function<DetachedEdge, Edge> edgeMaker) throws IOException {
+    public Edge readEdge(final InputStream inputStream, final Host host, final BiFunction<Attachable<Edge>, Host, Edge> edgeMaker) throws IOException {
         throw Io.Exceptions.readerFormatIsForFullGraphSerializationOnly(this.getClass());
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b7a1a7e9/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoReader.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoReader.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoReader.java
index 4039f4f..49d5090 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoReader.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoReader.java
@@ -18,6 +18,8 @@
  */
 package org.apache.tinkerpop.gremlin.structure.io.gryo;
 
+import org.apache.tinkerpop.gremlin.structure.Host;
+import org.apache.tinkerpop.gremlin.structure.util.Attachable;
 import org.apache.tinkerpop.shaded.kryo.Kryo;
 import org.apache.tinkerpop.gremlin.structure.T;
 import org.apache.tinkerpop.gremlin.structure.Direction;
@@ -44,6 +46,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
+import java.util.function.BiFunction;
 import java.util.function.Function;
 
 /**
@@ -103,11 +106,11 @@ public class GryoReader implements GraphReader {
     }
 
     @Override
-    public Edge readEdge(final InputStream inputStream, final Function<DetachedEdge, Edge> edgeMaker) throws IOException {
+    public Edge readEdge(final InputStream inputStream, final Host host, final BiFunction<Attachable<Edge>, Host, Edge> edgeMaker) throws IOException {
         final Input input = new Input(inputStream);
         readHeader(input);
-        final Object o = kryo.readClassAndObject(input);
-        return edgeMaker.apply((DetachedEdge) o);
+        final Attachable<Edge> attachable = (Attachable<Edge>) kryo.readClassAndObject(input);
+        return edgeMaker.apply(attachable, host);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b7a1a7e9/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/Attachable.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/Attachable.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/Attachable.java
index 685fe69..5cba5e5 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/Attachable.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/Attachable.java
@@ -22,6 +22,7 @@ import org.apache.tinkerpop.gremlin.structure.Direction;
 import org.apache.tinkerpop.gremlin.structure.Edge;
 import org.apache.tinkerpop.gremlin.structure.Element;
 import org.apache.tinkerpop.gremlin.structure.Graph;
+import org.apache.tinkerpop.gremlin.structure.Host;
 import org.apache.tinkerpop.gremlin.structure.Property;
 import org.apache.tinkerpop.gremlin.structure.T;
 import org.apache.tinkerpop.gremlin.structure.Vertex;
@@ -57,11 +58,11 @@ public interface Attachable<V> {
         return (V) method.apply(this, hostGraph);
     }
 
-    public enum Method implements BiFunction<Attachable, Object, Object> {
+    public enum Method implements BiFunction<Attachable, Host, Object> {
 
         GET {
             @Override
-            public Object apply(final Attachable attachable, final Object hostVertexOrGraph) {
+            public Object apply(final Attachable attachable, final Host hostVertexOrGraph) {
                 final Object base = attachable.get();
                 if (base instanceof Vertex) {
                     final Optional<Vertex> optional = hostVertexOrGraph instanceof Graph ?
@@ -98,7 +99,7 @@ public interface Attachable<V> {
 
         CREATE {
             @Override
-            public Object apply(final Attachable attachable, final Object hostVertexOrGraph) {
+            public Object apply(final Attachable attachable, final Host hostVertexOrGraph) {
                 final Object base = attachable.get();
                 if (base instanceof Vertex) {
                     return hostVertexOrGraph instanceof Graph ?
@@ -123,7 +124,7 @@ public interface Attachable<V> {
 
         GET_OR_CREATE {
             @Override
-            public Object apply(final Attachable attachable, final Object hostVertexOrGraph) {
+            public Object apply(final Attachable attachable, final Host hostVertexOrGraph) {
                 final Object base = attachable.get();
                 if (base instanceof Vertex) {
                     return (hostVertexOrGraph instanceof Graph ?

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/b7a1a7e9/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/IoTest.java
----------------------------------------------------------------------
diff --git a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/IoTest.java b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/IoTest.java
index 191696d..aa0a561 100644
--- a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/IoTest.java
+++ b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/IoTest.java
@@ -618,7 +618,8 @@ public class IoTest extends AbstractGremlinTest {
             final AtomicBoolean called = new AtomicBoolean(false);
             final GryoReader reader = graph.io(gryo).reader().workingDirectory(tempPath).create();
             try (final ByteArrayInputStream bais = new ByteArrayInputStream(os.toByteArray())) {
-                reader.readEdge(bais, detachedEdge -> {
+                reader.readEdge(bais, graph, (edge, host) -> {
+                    final Edge detachedEdge = (Edge) edge;
                     assertEquals(e.id(), detachedEdge.id());
                     assertEquals(v1.id(), detachedEdge.outVertex().id());
                     assertEquals(v2.id(), detachedEdge.inVertex().id());
@@ -654,7 +655,8 @@ public class IoTest extends AbstractGremlinTest {
             final AtomicBoolean called = new AtomicBoolean(false);
             final GryoReader reader = graph.io(gryo).reader().workingDirectory(tempPath).create();
             try (final ByteArrayInputStream bais = new ByteArrayInputStream(os.toByteArray())) {
-                reader.readEdge(bais, detachedEdge -> {
+                reader.readEdge(bais, graph, (edge, host) -> {
+                    final Edge detachedEdge = (Edge) edge;
                     assertEquals(e.id(), detachedEdge.id());
                     assertEquals(v1.id(), detachedEdge.outVertex().id());
                     assertEquals(v2.id(), detachedEdge.inVertex().id());
@@ -688,7 +690,8 @@ public class IoTest extends AbstractGremlinTest {
             final AtomicBoolean called = new AtomicBoolean(false);
             final GryoReader reader = graph.io(gryo).reader().workingDirectory(tempPath).create();
             try (final ByteArrayInputStream bais = new ByteArrayInputStream(os.toByteArray())) {
-                reader.readEdge(bais, detachedEdge -> {
+                reader.readEdge(bais, graph, (edge, host) -> {
+                    final Edge detachedEdge = (Edge) edge;
                     assertEquals(e.id(), detachedEdge.id());
                     assertEquals(v1.id(), detachedEdge.outVertex().id());
                     assertEquals(v2.id(), detachedEdge.inVertex().id());
@@ -722,7 +725,8 @@ public class IoTest extends AbstractGremlinTest {
             final AtomicBoolean called = new AtomicBoolean(false);
             final GryoReader reader = graph.io(gryo).reader().workingDirectory(tempPath).create();
             try (final ByteArrayInputStream bais = new ByteArrayInputStream(os.toByteArray())) {
-                reader.readEdge(bais, detachedEdge -> {
+                reader.readEdge(bais, graph, (edge, host) -> {
+                    final Edge detachedEdge = (Edge) edge;
                     assertEquals(e.id(), detachedEdge.id());
                     assertEquals(v1.id(), detachedEdge.outVertex().id());
                     assertEquals(v2.id(), detachedEdge.inVertex().id());
@@ -757,7 +761,8 @@ public class IoTest extends AbstractGremlinTest {
             final AtomicBoolean called = new AtomicBoolean(false);
             final GraphSONReader reader = graph.io(graphson).reader().create();
             try (final ByteArrayInputStream bais = new ByteArrayInputStream(os.toByteArray())) {
-                reader.readEdge(bais, detachedEdge -> {
+                reader.readEdge(bais, graph, (edge, host) -> {
+                    final Edge detachedEdge = (Edge) edge;
                     assertEquals(e.id(), graph.edges(detachedEdge.id().toString()).next().id());
                     assertEquals(v1.id(), graph.vertices(detachedEdge.outVertex().id().toString()).next().id());
                     assertEquals(v2.id(), graph.vertices(detachedEdge.inVertex().id().toString()).next().id());
@@ -792,7 +797,8 @@ public class IoTest extends AbstractGremlinTest {
             final AtomicBoolean called = new AtomicBoolean(false);
             final GraphSONReader reader = graph.io(graphson).reader().create();
             try (final ByteArrayInputStream bais = new ByteArrayInputStream(os.toByteArray())) {
-                reader.readEdge(bais, detachedEdge -> {
+                reader.readEdge(bais, graph, (edge, host) -> {
+                    final Edge detachedEdge = (Edge) edge;
                     assertEquals(e.id(), graph.edges(detachedEdge.id().toString()).next().id());
                     assertEquals(v1.id(), graph.vertices(detachedEdge.outVertex().id().toString()).next().id());
                     assertEquals(v2.id(), graph.vertices(detachedEdge.inVertex().id().toString()).next().id());
@@ -826,7 +832,8 @@ public class IoTest extends AbstractGremlinTest {
             final AtomicBoolean called = new AtomicBoolean(false);
             final GraphSONReader reader = graph.io(graphson).reader().create();
             try (final ByteArrayInputStream bais = new ByteArrayInputStream(os.toByteArray())) {
-                reader.readEdge(bais, detachedEdge -> {
+                reader.readEdge(bais, graph, (edge, host) -> {
+                    final Edge detachedEdge = (Edge) edge;
                     assertEquals(e.id(), graph.edges(detachedEdge.id().toString()).next().id());
                     assertEquals(v1.id(), graph.vertices(detachedEdge.outVertex().id().toString()).next().id());
                     assertEquals(v2.id(), graph.vertices(detachedEdge.inVertex().id().toString()).next().id());
@@ -862,7 +869,8 @@ public class IoTest extends AbstractGremlinTest {
             final AtomicBoolean called = new AtomicBoolean(false);
             final GraphSONReader reader = graph.io(graphson).reader().mapper(graph.io(graphson).mapper().embedTypes(true).create()).create();
             try (final ByteArrayInputStream bais = new ByteArrayInputStream(os.toByteArray())) {
-                reader.readEdge(bais, detachedEdge -> {
+                reader.readEdge(bais, graph, (edge, host) -> {
+                    final Edge detachedEdge = (Edge) edge;
                     assertEquals(e.id(), detachedEdge.id());
                     assertEquals(v1.id(), detachedEdge.outVertex().id());
                     assertEquals(v2.id(), detachedEdge.inVertex().id());
@@ -898,7 +906,8 @@ public class IoTest extends AbstractGremlinTest {
             final AtomicBoolean called = new AtomicBoolean(false);
             final GraphSONReader reader = graph.io(graphson).reader().mapper(graph.io(graphson).mapper().embedTypes(true).create()).create();
             try (final ByteArrayInputStream bais = new ByteArrayInputStream(os.toByteArray())) {
-                reader.readEdge(bais, detachedEdge -> {
+                reader.readEdge(bais, graph, (edge, host) -> {
+                    final Edge detachedEdge = (Edge) edge;
                     // a quick reminder here that the purpose of these id assertions is to ensure that those with
                     // complex ids that are not simply toString'd (i.e. are complex objects in JSON as well)
                     // properly respond to filtering in Graph.edges/vertices
@@ -938,7 +947,8 @@ public class IoTest extends AbstractGremlinTest {
             final AtomicBoolean called = new AtomicBoolean(false);
             final GryoReader reader = graph.io(gryo).reader().workingDirectory(tempPath).create();
             try (final ByteArrayInputStream bais = new ByteArrayInputStream(os.toByteArray())) {
-                reader.readEdge(bais, detachedEdge -> {
+                reader.readEdge(bais, graph, (edge, host) -> {
+                    final Edge detachedEdge = (Edge) edge;
                     assertEquals(e.id(), detachedEdge.id());
                     assertEquals(v1.id(), detachedEdge.outVertex().id());
                     assertEquals(v2.id(), detachedEdge.inVertex().id());


[2/2] incubator-tinkerpop git commit: Merge remote-tracking branch 'origin/refactor-io' into refactor-io

Posted by sp...@apache.org.
Merge remote-tracking branch 'origin/refactor-io' into refactor-io


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

Branch: refs/heads/refactor-io
Commit: 4fb57398899ea58ca50cfc548d27e527e7839826
Parents: b7a1a7e 6bb950f
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Thu Apr 23 11:58:47 2015 -0400
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Apr 23 11:58:47 2015 -0400

----------------------------------------------------------------------
 .../gremlin/structure/io/gryo/GryoMapper.java   |  4 +-
 .../gremlin/structure/util/star/StarGraph.java  | 64 ++++++++------
 .../util/star/StarGraphSerializer.java          | 26 +++---
 .../tinkerpop/gremlin/util/tools/BiMap.java     | 92 ++++++++++++++++++++
 4 files changed, 150 insertions(+), 36 deletions(-)
----------------------------------------------------------------------