You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tinkerpop.apache.org by ok...@apache.org on 2016/02/04 18:42:23 UTC

incubator-tinkerpop git commit: Moved GraphFilterTest to gremlin-groovy/ so I can use reflection and not have to make internal variables protected for testing purposes. Optional GraphReader.readVertex(InputStream, GraphFilter) now exists at the in

Repository: incubator-tinkerpop
Updated Branches:
  refs/heads/TINKERPOP-962 569496f67 -> 07f7a8c61


Moved GraphFilterTest to gremlin-groovy/ so I can use reflection and not have to make internal variables protected for testing purposes. Optional<Vertex> GraphReader.readVertex(InputStream,GraphFilter) now exists at the interface level with an UnsupportedOperationException default. GryoReader can now read vertices from a GraphFilter-perspective and only materialize those vertices/edges that are legal. Should be fairly trivial to add to GraphSONReader.


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

Branch: refs/heads/TINKERPOP-962
Commit: 07f7a8c614493de4bd13d2e75292609c5ee7183c
Parents: 569496f
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Thu Feb 4 10:42:24 2016 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Thu Feb 4 10:42:24 2016 -0700

----------------------------------------------------------------------
 .../gremlin/process/computer/GraphFilter.java   | 56 +++++++++--
 .../gremlin/structure/io/GraphReader.java       | 58 +++++++-----
 .../gremlin/structure/io/gryo/GryoReader.java   | 19 ++++
 .../util/star/StarGraphGryoSerializer.java      | 26 ++++--
 .../process/computer/GraphFilterTest.java       | 70 +-------------
 .../computer/GroovyGraphFilterTest.groovy       | 98 ++++++++++++++++++++
 .../structure/io/gryo/GryoRecordReader.java     | 10 +-
 7 files changed, 228 insertions(+), 109 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/07f7a8c6/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/GraphFilter.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/GraphFilter.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/GraphFilter.java
index e6083a1..44bde83 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/GraphFilter.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/GraphFilter.java
@@ -47,10 +47,10 @@ public final class GraphFilter implements Cloneable, Serializable {
     private Traversal.Admin<Vertex, Vertex> vertexFilter = null;
     private Traversal.Admin<Vertex, Edge> edgeFilter = null;
 
-    protected boolean allowNoEdges = false;
-    protected Direction allowedEdgeDirection = Direction.BOTH;
-    protected Set<String> allowedEdgeLabels = new HashSet<>();
-    protected boolean allowAllRemainingEdges = false;
+    private boolean allowNoEdges = false;
+    private Direction allowedEdgeDirection = Direction.BOTH;
+    private Set<String> allowedEdgeLabels = new HashSet<>();
+    private boolean allowAllRemainingEdges = false;
 
     public void setVertexFilter(final Traversal<Vertex, Vertex> vertexFilter) {
         if (!TraversalHelper.isLocalVertex(vertexFilter.asAdmin()))
@@ -62,9 +62,9 @@ public final class GraphFilter implements Cloneable, Serializable {
         if (!TraversalHelper.isLocalStarGraph(edgeFilter.asAdmin()))
             throw GraphComputer.Exceptions.edgeFilterAccessesAdjacentVertices(edgeFilter);
         this.edgeFilter = edgeFilter.asAdmin().clone();
-        if (this.edgeFilter.getEndStep() instanceof RangeGlobalStep && 0 == ((RangeGlobalStep) this.edgeFilter.getEndStep()).getHighRange()) {
+        if (this.edgeFilter.getEndStep() instanceof RangeGlobalStep && 0 == ((RangeGlobalStep) this.edgeFilter.getEndStep()).getHighRange())
             this.allowNoEdges = true;
-        } else if (this.edgeFilter.getStartStep() instanceof VertexStep) {
+        else if (this.edgeFilter.getStartStep() instanceof VertexStep) {
             this.allowedEdgeLabels.clear();
             this.allowedEdgeLabels.addAll(Arrays.asList(((VertexStep) this.edgeFilter.getStartStep()).getEdgeLabels()));
             this.allowedEdgeDirection = ((VertexStep) this.edgeFilter.getStartStep()).getDirection();
@@ -72,6 +72,13 @@ public final class GraphFilter implements Cloneable, Serializable {
         }
     }
 
+    public void applyStrategies() {
+        if (null != this.vertexFilter && !this.vertexFilter.isLocked())
+            this.vertexFilter.applyStrategies();
+        if (null != this.edgeFilter && !this.edgeFilter.isLocked())
+            this.edgeFilter.applyStrategies();
+    }
+
     public boolean legalVertex(final Vertex vertex) {
         return null == this.vertexFilter || TraversalUtil.test(vertex, this.vertexFilter);
     }
@@ -102,6 +109,36 @@ public final class GraphFilter implements Cloneable, Serializable {
         return this.vertexFilter != null;
     }
 
+    public boolean maybeLegalEdge(final Direction direction, final String label) {
+        if (null == this.edgeFilter)
+            return true;
+        else if (this.allowNoEdges)
+            return false;
+        else if (!direction.equals(Direction.BOTH) && !this.allowedEdgeDirection.equals(direction))
+            return false;
+        else if (!this.allowedEdgeLabels.isEmpty() && !this.allowedEdgeLabels.contains(label))
+            return false;
+        else
+            return true;
+    }
+
+    @Override
+    public int hashCode() {
+        return (null == this.edgeFilter ? 124 : this.edgeFilter.hashCode()) ^ (null == this.vertexFilter ? 875 : this.vertexFilter.hashCode());
+    }
+
+    @Override
+    public boolean equals(final Object object) {
+        if (!(object instanceof GraphFilter))
+            return false;
+        else if (((GraphFilter) object).hasVertexFilter() && !((GraphFilter) object).getVertexFilter().equals(this.vertexFilter))
+            return false;
+        else if (((GraphFilter) object).hasEdgeFilter() && !((GraphFilter) object).getEdgeFilter().equals(this.edgeFilter))
+            return false;
+        else
+            return true;
+    }
+
     @Override
     public GraphFilter clone() {
         try {
@@ -132,6 +169,11 @@ public final class GraphFilter implements Cloneable, Serializable {
     /////////////////////////////////////
     ////////////////////////////////////
 
+    public StarGraph applyGraphFilter(final StarGraph graph) {
+        final StarGraph.StarVertex filtered = this.applyGraphFilter(graph.getStarVertex());
+        return null == filtered ? null : (StarGraph) filtered.graph();
+    }
+
     public StarGraph.StarVertex applyGraphFilter(final StarGraph.StarVertex vertex) {
         if (!this.hasFilter())
             return vertex;
@@ -185,4 +227,4 @@ public final class GraphFilter implements Cloneable, Serializable {
             return null;
         }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/07f7a8c6/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 b548f07..a56df63 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
@@ -18,6 +18,7 @@
  */
 package org.apache.tinkerpop.gremlin.structure.io;
 
+import org.apache.tinkerpop.gremlin.process.computer.GraphFilter;
 import org.apache.tinkerpop.gremlin.structure.Direction;
 import org.apache.tinkerpop.gremlin.structure.Edge;
 import org.apache.tinkerpop.gremlin.structure.Graph;
@@ -31,6 +32,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.Iterator;
+import java.util.Optional;
 import java.util.function.Function;
 
 /**
@@ -55,19 +57,33 @@ public interface GraphReader {
      * It is up to individual implementations to manage transactions, but it is not required or enforced.  Consult
      * the documentation of an implementation to understand the approach it takes.
      *
-     * @param inputStream a stream containing an entire graph of vertices and edges as defined by the accompanying
-     *                    {@link GraphWriter#writeGraph(OutputStream, Graph)}.
+     * @param inputStream    a stream containing an entire graph of vertices and edges as defined by the accompanying
+     *                       {@link GraphWriter#writeGraph(OutputStream, Graph)}.
      * @param graphToWriteTo the graph to write to when reading from the stream.
      */
     public void readGraph(final InputStream inputStream, final Graph graphToWriteTo) throws IOException;
 
     /**
+     * Reads a single vertex from an {@link InputStream}. This method will filter the read the read vertex by the provided
+     * {@link GraphFilter}. If the graph filter will filter the vertex itself, then the returned {@link Optional} is empty.
+     *
+     * @param inputStream a stream containing at least a single vertex as defined by the accompanying
+     *                    {@link GraphWriter#writeVertex(OutputStream, Vertex)}.
+     * @param graphFilter The {@link GraphFilter} to filter the vertex and its associated edges by.
+     * @return the vertex with filtered edges or {@link Optional#empty()}  if the vertex itself was filtered.
+     * @throws IOException
+     */
+    public default Optional<Vertex> readVertex(final InputStream inputStream, final GraphFilter graphFilter) throws IOException {
+        throw new UnsupportedOperationException(this.getClass().getCanonicalName() + " currently does not support " + GraphFilter.class.getSimpleName() + " deserialization filtering");
+    }
+
+    /**
      * Reads a single vertex from an {@link InputStream}.  This method will read vertex properties but not edges.
      * It is expected that the user will manager their own transaction context with respect to this method (i.e.
      * implementations should not commit the transaction for the user).
      *
-     * @param inputStream a stream containing at least a single vertex as defined by the accompanying
-     *                    {@link GraphWriter#writeVertex(OutputStream, Vertex)}.
+     * @param inputStream        a stream containing at least a single vertex as defined by the accompanying
+     *                           {@link GraphWriter#writeVertex(OutputStream, Vertex)}.
      * @param vertexAttachMethod a function that creates re-attaches a {@link Vertex} to a {@link Host} object.
      */
     public Vertex readVertex(final InputStream inputStream, final Function<Attachable<Vertex>, Vertex> vertexAttachMethod) throws IOException;
@@ -77,10 +93,10 @@ public interface GraphReader {
      * given the direction supplied as an argument.  It is expected that the user will manager their own transaction
      * context with respect to this method (i.e. implementations should not commit the transaction for the user).
      *
-     * @param inputStream a stream containing at least one {@link Vertex} as defined by the accompanying
-     *                    {@link GraphWriter#writeVertices(OutputStream, Iterator, Direction)} method.
-     * @param vertexAttachMethod a function that creates re-attaches a {@link Vertex} to a {@link Host} object.
-     * @param edgeAttachMethod a function that creates re-attaches a {@link Edge} to a {@link Host} object.
+     * @param inputStream                a stream containing at least one {@link Vertex} as defined by the accompanying
+     *                                   {@link GraphWriter#writeVertices(OutputStream, Iterator, Direction)} method.
+     * @param vertexAttachMethod         a function that creates re-attaches a {@link Vertex} to a {@link Host} object.
+     * @param edgeAttachMethod           a function that creates re-attaches a {@link Edge} to a {@link Host} object.
      * @param attachEdgesOfThisDirection only edges of this direction are passed to the {@code edgeMaker}.
      */
     public Vertex readVertex(final InputStream inputStream,
@@ -95,11 +111,11 @@ public interface GraphReader {
      * transaction context with respect to this method (i.e. implementations should not commit the transaction for
      * the user).
      *
-     * @param inputStream a stream containing at least one {@link Vertex} as defined by the accompanying
-     *                    {@link GraphWriter#writeVertices(OutputStream, Iterator, Direction)} or
-     *                    {@link GraphWriter#writeVertices(OutputStream, Iterator)} methods.
-     * @param vertexAttachMethod a function that creates re-attaches a {@link Vertex} to a {@link Host} object.
-     * @param edgeAttachMethod a function that creates re-attaches a {@link Edge} to a {@link Host} object.
+     * @param inputStream                a stream containing at least one {@link Vertex} as defined by the accompanying
+     *                                   {@link GraphWriter#writeVertices(OutputStream, Iterator, Direction)} or
+     *                                   {@link GraphWriter#writeVertices(OutputStream, Iterator)} methods.
+     * @param vertexAttachMethod         a function that creates re-attaches a {@link Vertex} to a {@link Host} object.
+     * @param edgeAttachMethod           a function that creates re-attaches a {@link Edge} to a {@link Host} object.
      * @param attachEdgesOfThisDirection only edges of this direction are passed to the {@code edgeMaker}.
      */
     public Iterator<Vertex> readVertices(final InputStream inputStream,
@@ -112,8 +128,8 @@ public interface GraphReader {
      * transaction context with respect to this method (i.e. implementations should not commit the transaction for
      * the user).
      *
-     * @param inputStream a stream containing at least one {@link Edge} as defined by the accompanying
-     *                    {@link GraphWriter#writeEdge(OutputStream, Edge)} method.
+     * @param inputStream      a stream containing at least one {@link Edge} as defined by the accompanying
+     *                         {@link GraphWriter#writeEdge(OutputStream, Edge)} method.
      * @param edgeAttachMethod a function that creates re-attaches a {@link Edge} to a {@link Host} object.
      */
     public Edge readEdge(final InputStream inputStream, final Function<Attachable<Edge>, Edge> edgeAttachMethod) throws IOException;
@@ -123,8 +139,8 @@ public interface GraphReader {
      * transaction context with respect to this method (i.e. implementations should not commit the transaction for
      * the user).
      *
-     * @param inputStream a stream containing at least one {@link VertexProperty} as written by the accompanying
-     *                    {@link GraphWriter#writeVertexProperty(OutputStream, VertexProperty)} method.
+     * @param inputStream                a stream containing at least one {@link VertexProperty} as written by the accompanying
+     *                                   {@link GraphWriter#writeVertexProperty(OutputStream, VertexProperty)} method.
      * @param vertexPropertyAttachMethod a function that creates re-attaches a {@link VertexProperty} to a
      *                                   {@link Host} object.
      * @return the value returned by the attach method.
@@ -137,8 +153,8 @@ public interface GraphReader {
      * transaction context with respect to this method (i.e. implementations should not commit the transaction for
      * the user).
      *
-     * @param inputStream a stream containing at least one {@link Property} as written by the accompanying
-     *                    {@link GraphWriter#writeProperty(OutputStream, Property)} method.
+     * @param inputStream          a stream containing at least one {@link Property} as written by the accompanying
+     *                             {@link GraphWriter#writeProperty(OutputStream, Property)} method.
      * @param propertyAttachMethod a function that creates re-attaches a {@link Property} to a {@link Host} object.
      * @return the value returned by the attach method.
      */
@@ -148,8 +164,8 @@ public interface GraphReader {
     /**
      * Reads an arbitrary object using the registered serializers.
      *
-     * @param inputStream  a stream containing an object.
-     * @param clazz the class expected to be in the stream - may or may not be used by the underlying implementation.
+     * @param inputStream a stream containing an object.
+     * @param clazz       the class expected to be in the stream - may or may not be used by the underlying implementation.
      */
     public <C> C readObject(final InputStream inputStream, final Class<? extends C> clazz) throws IOException;
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/07f7a8c6/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 d457ffb..c34101b 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,7 @@
  */
 package org.apache.tinkerpop.gremlin.structure.io.gryo;
 
+import org.apache.tinkerpop.gremlin.process.computer.GraphFilter;
 import org.apache.tinkerpop.gremlin.structure.Direction;
 import org.apache.tinkerpop.gremlin.structure.Edge;
 import org.apache.tinkerpop.gremlin.structure.Graph;
@@ -34,6 +35,7 @@ import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedEdge;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedProperty;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedVertexProperty;
 import org.apache.tinkerpop.gremlin.structure.util.star.StarGraph;
+import org.apache.tinkerpop.gremlin.structure.util.star.StarGraphGryoSerializer;
 import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 import org.apache.tinkerpop.shaded.kryo.Kryo;
 import org.apache.tinkerpop.shaded.kryo.io.Input;
@@ -45,6 +47,7 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.Function;
 
@@ -60,6 +63,7 @@ import java.util.function.Function;
  */
 public final class GryoReader implements GraphReader {
     private final Kryo kryo;
+    private final Map<GraphFilter, StarGraphGryoSerializer> graphFilterCache = new HashMap<>();
 
     private final long batchSize;
 
@@ -108,6 +112,21 @@ public final class GryoReader implements GraphReader {
         if (supportsTx) graphToWriteTo.tx().commit();
     }
 
+    @Override
+    public Optional<Vertex> readVertex(final InputStream inputStream, final GraphFilter graphFilter) throws IOException {
+        StarGraphGryoSerializer serializer = this.graphFilterCache.get(graphFilter);
+        if (null == serializer) {
+            serializer = StarGraphGryoSerializer.withGraphFilter(graphFilter);
+            this.graphFilterCache.put(graphFilter, serializer);
+        }
+        final Input input = new Input(inputStream);
+        this.readHeader(input);
+        final StarGraph starGraph = this.kryo.readObject(input, StarGraph.class, serializer);
+        // read the terminator
+        this.kryo.readClassAndObject(input);
+        return Optional.ofNullable(starGraph == null ? null : starGraph.getStarVertex());
+    }
+
     /**
      * Read {@link Vertex} objects from output generated by any of the {@link GryoWriter} {@code writeVertex} or
      * {@code writeVertices} methods or by {@link GryoWriter#writeGraph(OutputStream, Graph)}.

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/07f7a8c6/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphGryoSerializer.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphGryoSerializer.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphGryoSerializer.java
index 5bcfe80..384f4a8 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphGryoSerializer.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphGryoSerializer.java
@@ -18,6 +18,7 @@
  */
 package org.apache.tinkerpop.gremlin.structure.util.star;
 
+import org.apache.tinkerpop.gremlin.process.computer.GraphFilter;
 import org.apache.tinkerpop.gremlin.structure.Direction;
 import org.apache.tinkerpop.gremlin.structure.Edge;
 import org.apache.tinkerpop.gremlin.structure.T;
@@ -46,6 +47,7 @@ public final class StarGraphGryoSerializer extends Serializer<StarGraph> {
     private static final Map<Direction, StarGraphGryoSerializer> CACHE = new HashMap<>();
 
     private final Direction edgeDirectionToSerialize;
+    private GraphFilter graphFilter = new GraphFilter(); // will allow all vertices/edges
 
     private final static byte VERSION_1 = Byte.MIN_VALUE;
 
@@ -68,6 +70,12 @@ public final class StarGraphGryoSerializer extends Serializer<StarGraph> {
         return CACHE.get(direction);
     }
 
+    public static StarGraphGryoSerializer withGraphFilter(final GraphFilter graphFilter) {
+        final StarGraphGryoSerializer serializer = new StarGraphGryoSerializer(Direction.BOTH);
+        serializer.graphFilter = graphFilter.clone();
+        return serializer;
+    }
+
     @Override
     public void write(final Kryo kryo, final Output output, final StarGraph starGraph) {
         output.writeByte(VERSION_1);
@@ -112,7 +120,7 @@ public final class StarGraphGryoSerializer extends Serializer<StarGraph> {
                 }
             }
         }
-        return starGraph;
+        return this.graphFilter.applyGraphFilter(starGraph);
     }
 
     private void writeEdges(final Kryo kryo, final Output output, final StarGraph starGraph, final Direction direction) {
@@ -135,7 +143,7 @@ public final class StarGraphGryoSerializer extends Serializer<StarGraph> {
         }
     }
 
-    private static void readEdges(final Kryo kryo, final Input input, final StarGraph starGraph, final Direction direction) {
+    private void readEdges(final Kryo kryo, final Input input, final StarGraph starGraph, final Direction direction) {
         if (kryo.readObject(input, Boolean.class)) {
             final int numberOfUniqueLabels = kryo.readObject(input, Integer.class);
             for (int i = 0; i < numberOfUniqueLabels; i++) {
@@ -144,12 +152,16 @@ public final class StarGraphGryoSerializer extends Serializer<StarGraph> {
                 for (int j = 0; j < numberOfEdgesWithLabel; j++) {
                     final Object edgeId = kryo.readClassAndObject(input);
                     final Object adjacentVertexId = kryo.readClassAndObject(input);
-                    if (direction.equals(Direction.OUT))
-                        starGraph.starVertex.addOutEdge(edgeLabel, starGraph.addVertex(T.id, adjacentVertexId), T.id, edgeId);
-                    else
-                        starGraph.starVertex.addInEdge(edgeLabel, starGraph.addVertex(T.id, adjacentVertexId), T.id, edgeId);
+                    if (this.graphFilter.maybeLegalEdge(direction, edgeLabel)) {
+                        if (direction.equals(Direction.OUT))
+                            starGraph.starVertex.addOutEdge(edgeLabel, starGraph.addVertex(T.id, adjacentVertexId), T.id, edgeId);
+                        else
+                            starGraph.starVertex.addInEdge(edgeLabel, starGraph.addVertex(T.id, adjacentVertexId), T.id, edgeId);
+                    } else if (null != starGraph.edgeProperties) {
+                        starGraph.edgeProperties.remove(edgeId);
+                    }
                 }
             }
         }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/07f7a8c6/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/computer/GraphFilterTest.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/computer/GraphFilterTest.java b/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/computer/GraphFilterTest.java
index 0a09155..9f04fb9 100644
--- a/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/computer/GraphFilterTest.java
+++ b/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/computer/GraphFilterTest.java
@@ -19,79 +19,11 @@
 
 package org.apache.tinkerpop.gremlin.process.computer;
 
-import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__;
-import org.apache.tinkerpop.gremlin.structure.Direction;
-import org.apache.tinkerpop.gremlin.structure.Vertex;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
 public class GraphFilterTest {
 
-    @Test
-    public void shouldHandlePreFilterCorrectly() {
-        GraphFilter graphFilter = new GraphFilter();
-        graphFilter.setEdgeFilter(__.<Vertex>bothE().limit(0));
-        assertTrue(graphFilter.allowedEdgeLabels.isEmpty());
-        assertEquals(Direction.BOTH, graphFilter.allowedEdgeDirection);
-        assertFalse(graphFilter.allowAllRemainingEdges);
-        //
-        graphFilter = new GraphFilter();
-        graphFilter.setEdgeFilter(__.<Vertex>bothE("knows").limit(0));
-        assertEquals(1, graphFilter.allowedEdgeLabels.size());
-        assertTrue(graphFilter.allowedEdgeLabels.contains("knows"));
-        assertEquals(Direction.BOTH, graphFilter.allowedEdgeDirection);
-        assertFalse(graphFilter.allowAllRemainingEdges);
-        //
-        graphFilter = new GraphFilter();
-        graphFilter.setEdgeFilter(__.<Vertex>outE("knows", "created"));
-        assertEquals(2, graphFilter.allowedEdgeLabels.size());
-        assertTrue(graphFilter.allowedEdgeLabels.contains("knows"));
-        assertTrue(graphFilter.allowedEdgeLabels.contains("created"));
-        assertEquals(Direction.OUT, graphFilter.allowedEdgeDirection);
-        assertTrue(graphFilter.allowAllRemainingEdges);
-        //
-        graphFilter = new GraphFilter();
-        graphFilter.setEdgeFilter(__.<Vertex>inE("knows", "created", "likes"));
-        assertEquals(3, graphFilter.allowedEdgeLabels.size());
-        assertTrue(graphFilter.allowedEdgeLabels.contains("knows"));
-        assertTrue(graphFilter.allowedEdgeLabels.contains("created"));
-        assertTrue(graphFilter.allowedEdgeLabels.contains("likes"));
-        assertEquals(Direction.IN, graphFilter.allowedEdgeDirection);
-        assertTrue(graphFilter.allowAllRemainingEdges);
-        //
-        graphFilter = new GraphFilter();
-        graphFilter.setEdgeFilter(__.<Vertex>inE("knows", "created", "likes"));
-        assertEquals(3, graphFilter.allowedEdgeLabels.size());
-        assertTrue(graphFilter.allowedEdgeLabels.contains("knows"));
-        assertTrue(graphFilter.allowedEdgeLabels.contains("created"));
-        assertTrue(graphFilter.allowedEdgeLabels.contains("likes"));
-        assertEquals(Direction.IN, graphFilter.allowedEdgeDirection);
-        assertTrue(graphFilter.allowAllRemainingEdges);
-        //
-        graphFilter = new GraphFilter();
-        try {
-            graphFilter.setVertexFilter(__.out("likes"));    // cannot leave local vertex
-            fail();
-        } catch (final IllegalArgumentException e) {
-            assertEquals(e.getMessage(), GraphComputer.Exceptions.vertexFilterAccessesIncidentEdges(__.out("likes")).getMessage());
-        }
-        //
-        graphFilter = new GraphFilter();
-        try {
-            graphFilter.setEdgeFilter(__.<Vertex>inE("likes").inV().outE().has("weight", 1));    // cannot leave local star graph
-            fail();
-        } catch (final IllegalArgumentException e) {
-            assertEquals(e.getMessage(), GraphComputer.Exceptions.edgeFilterAccessesAdjacentVertices(__.<Vertex>inE("likes").inV().outE().has("weight", 1)).getMessage());
-        }
-    }
-
     /*@Test
     public void shouldHandleStarGraph() {
         final StarGraph graph = StarGraph.open();
@@ -101,4 +33,4 @@ public class GraphFilterTest {
         }
         final GraphFilter graphFilter = new GraphFilter();
     }*/
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/07f7a8c6/gremlin-groovy/src/test/groovy/org/apache/tinkerpop/gremlin/process/computer/GroovyGraphFilterTest.groovy
----------------------------------------------------------------------
diff --git a/gremlin-groovy/src/test/groovy/org/apache/tinkerpop/gremlin/process/computer/GroovyGraphFilterTest.groovy b/gremlin-groovy/src/test/groovy/org/apache/tinkerpop/gremlin/process/computer/GroovyGraphFilterTest.groovy
new file mode 100644
index 0000000..7147650
--- /dev/null
+++ b/gremlin-groovy/src/test/groovy/org/apache/tinkerpop/gremlin/process/computer/GroovyGraphFilterTest.groovy
@@ -0,0 +1,98 @@
+/*
+ * 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.process.computer
+
+import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__
+import org.apache.tinkerpop.gremlin.structure.Direction
+import org.apache.tinkerpop.gremlin.structure.Vertex
+import org.junit.Test
+
+import static org.junit.Assert.*
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public class GroovyGraphFilterTest {
+
+    @Test
+    public void shouldHandlePreFilterCorrectly() {
+        GraphFilter graphFilter = new GraphFilter();
+        graphFilter.setEdgeFilter(__.<Vertex> bothE().limit(0));
+        assertTrue(graphFilter.allowNoEdges);
+        //
+        graphFilter = new GraphFilter();
+        graphFilter.setEdgeFilter(__.<Vertex> bothE("knows").limit(0));
+        assertTrue(graphFilter.allowNoEdges);
+        //
+        graphFilter = new GraphFilter();
+        graphFilter.setEdgeFilter(__.<Vertex> bothE("knows").select("a"));
+        assertEquals(1, graphFilter.allowedEdgeLabels.size());
+        assertTrue(graphFilter.allowedEdgeLabels.contains("knows"));
+        assertEquals(Direction.BOTH, graphFilter.allowedEdgeDirection);
+        assertFalse(graphFilter.allowAllRemainingEdges);
+        assertFalse(graphFilter.allowNoEdges);
+        //
+        graphFilter = new GraphFilter();
+        graphFilter.setEdgeFilter(__.<Vertex> outE("knows", "created"));
+        assertEquals(2, graphFilter.allowedEdgeLabels.size());
+        assertTrue(graphFilter.allowedEdgeLabels.contains("knows"));
+        assertTrue(graphFilter.allowedEdgeLabels.contains("created"));
+        assertEquals(Direction.OUT, graphFilter.allowedEdgeDirection);
+        assertTrue(graphFilter.allowAllRemainingEdges);
+        assertFalse(graphFilter.allowNoEdges);
+        //
+        graphFilter = new GraphFilter();
+        graphFilter.setEdgeFilter(__.<Vertex> inE("knows", "created", "likes"));
+        assertEquals(3, graphFilter.allowedEdgeLabels.size());
+        assertTrue(graphFilter.allowedEdgeLabels.contains("knows"));
+        assertTrue(graphFilter.allowedEdgeLabels.contains("created"));
+        assertTrue(graphFilter.allowedEdgeLabels.contains("likes"));
+        assertEquals(Direction.IN, graphFilter.allowedEdgeDirection);
+        assertTrue(graphFilter.allowAllRemainingEdges);
+        assertFalse(graphFilter.allowNoEdges);
+        //
+        graphFilter = new GraphFilter();
+        graphFilter.setEdgeFilter(__.<Vertex> inE("knows", "created", "likes"));
+        assertEquals(3, graphFilter.allowedEdgeLabels.size());
+        assertTrue(graphFilter.allowedEdgeLabels.contains("knows"));
+        assertTrue(graphFilter.allowedEdgeLabels.contains("created"));
+        assertTrue(graphFilter.allowedEdgeLabels.contains("likes"));
+        assertEquals(Direction.IN, graphFilter.allowedEdgeDirection);
+        assertTrue(graphFilter.allowAllRemainingEdges);
+        assertFalse(graphFilter.allowNoEdges);
+        //
+        graphFilter = new GraphFilter();
+        try {
+            graphFilter.setVertexFilter(__.out("likes"));    // cannot leave local vertex
+            fail();
+        } catch (final IllegalArgumentException e) {
+            assertEquals(e.getMessage(), GraphComputer.Exceptions.vertexFilterAccessesIncidentEdges(__.out("likes")).getMessage());
+        }
+        //
+        graphFilter = new GraphFilter();
+        try {
+            graphFilter.setEdgeFilter(__.<Vertex> inE("likes").inV().outE().has("weight", 1));
+            // cannot leave local star graph
+            fail();
+        } catch (final IllegalArgumentException e) {
+            assertEquals(e.getMessage(), GraphComputer.Exceptions.edgeFilterAccessesAdjacentVertices(__.<Vertex> inE("likes").inV().outE().has("weight", 1)).getMessage());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/07f7a8c6/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 64e0091..1b3172e 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
@@ -34,13 +34,12 @@ 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.Optional;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
@@ -61,6 +60,7 @@ public final class GryoRecordReader extends RecordReader<NullWritable, VertexWri
 
     public GryoRecordReader(final GraphFilter graphFilter) {
         this.graphFilter = graphFilter.clone();
+        this.graphFilter.applyStrategies();
     }
 
     @Override
@@ -129,9 +129,9 @@ public final class GryoRecordReader extends RecordReader<NullWritable, VertexWri
             terminatorLocation = ((byte) currentByte) == TERMINATOR[terminatorLocation] ? terminatorLocation + 1 : 0;
             if (terminatorLocation >= TERMINATOR.length) {
                 try (InputStream in = new ByteArrayInputStream(output.toByteArray())) {
-                    final Vertex vertex = this.graphFilter.applyGraphFilter((StarGraph.StarVertex) this.gryoReader.readVertex(in, Attachable::get)); // I know how GryoReader works, so I'm cheating here
-                    if (null != vertex) {
-                        this.vertexWritable.set(vertex);
+                    final Optional<Vertex> vertex = this.gryoReader.readVertex(in, this.graphFilter);
+                    if (vertex.isPresent()) {
+                        this.vertexWritable.set(vertex.get());
                         return true;
                     } else {
                         currentVertexLength = 0;