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 2018/05/31 21:29:30 UTC

[2/2] tinkerpop git commit: Implemented `ShortestPathVertexProgram` and `ShortestPathVertexProgramStep`.

Implemented `ShortestPathVertexProgram` and `ShortestPathVertexProgramStep`.


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

Branch: refs/heads/shortest-path-wip
Commit: 27cf8e555fbc59e0446c8b8ed27c483b9578c97f
Parents: 939674a
Author: Daniel Kuppitz <da...@hotmail.com>
Authored: Tue May 22 07:59:32 2018 -0700
Committer: Daniel Kuppitz <da...@hotmail.com>
Committed: Thu May 31 14:29:17 2018 -0700

----------------------------------------------------------------------
 .../tinkerpop/gremlin/jsr223/CoreImports.java   |   4 +
 .../gremlin/jsr223/JavaTranslator.java          |  37 --
 .../search/path/ShortestPathVertexProgram.java  | 566 +++++++++++++++++++
 .../step/map/ShortestPathVertexProgramStep.java | 257 +++++++++
 .../gremlin/process/remote/RemoteGraph.java     |   8 +
 .../gremlin/process/traversal/Bytecode.java     |  16 +-
 .../gremlin/process/traversal/Traversal.java    |  16 +-
 .../traversal/dsl/graph/GraphTraversal.java     |  23 +-
 .../traversal/lambda/ColumnTraversal.java       |   8 +
 .../traversal/lambda/ConstantTraversal.java     |   9 +-
 .../traversal/lambda/ElementValueTraversal.java |  10 +-
 .../traversal/lambda/IdentityTraversal.java     |   7 +-
 .../process/traversal/lambda/LoopTraversal.java |   6 +
 .../traversal/lambda/TokenTraversal.java        |   8 +
 .../process/traversal/lambda/TrueTraversal.java |   5 +
 .../traversal/step/StepConfiguration.java       |  51 --
 .../step/util/DefaultStepConfiguration.java     | 158 ------
 .../step/util/StepConfigurationProxy.java       |  50 --
 .../structure/io/graphson/GraphSONModule.java   |   8 -
 .../io/graphson/TraversalSerializersV2d0.java   |  37 +-
 .../io/graphson/TraversalSerializersV3d0.java   |  34 +-
 .../structure/io/gryo/GryoSerializersV1d0.java  |  32 --
 .../structure/io/gryo/GryoSerializersV3d0.java  |   2 +-
 .../gremlin/structure/io/gryo/GryoVersion.java  |   7 +-
 .../traversal/dsl/graph/GraphTraversalTest.java |   2 +-
 .../process/traversal/step/StepTest.java        |   2 +-
 .../step/util/DefaultStepConfigurationTest.java | 167 ------
 .../GraphSONMapperPartialEmbeddedTypeTest.java  |  10 -
 .../structure/io/gryo/GryoMapperTest.java       |   7 -
 gremlin-dotnet/glv/generate.groovy              |   1 -
 .../Process/Traversal/GraphTraversal.cs         |   9 +
 .../Process/Traversal/IStepConfiguration.cs     |  32 --
 .../gremlin/groovy/jsr223/GroovyTranslator.java |   9 +-
 .../lib/process/graph-traversal.js              |  10 +
 gremlin-python/glv/generate.groovy              |   1 -
 .../gremlin_python/process/graph_traversal.py   |   4 +
 .../tinkerpop/gremlin/AbstractGremlinTest.java  |   4 +
 .../gremlin/process/ProcessComputerSuite.java   |   5 +
 .../search/path/ShortestPathTestHelper.java     |  72 +++
 .../path/ShortestPathVertexProgramTest.java     | 264 +++++++++
 .../traversal/step/map/ShortestPathTest.java    | 310 ++++++++++
 pom.xml                                         |   3 +
 42 files changed, 1607 insertions(+), 664 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/jsr223/CoreImports.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/jsr223/CoreImports.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/jsr223/CoreImports.java
index 368b92d..d853b8f 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/jsr223/CoreImports.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/jsr223/CoreImports.java
@@ -46,9 +46,11 @@ import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.Clu
 import org.apache.tinkerpop.gremlin.process.computer.clustering.peerpressure.PeerPressureVertexProgram;
 import org.apache.tinkerpop.gremlin.process.computer.ranking.pagerank.PageRankMapReduce;
 import org.apache.tinkerpop.gremlin.process.computer.ranking.pagerank.PageRankVertexProgram;
+import org.apache.tinkerpop.gremlin.process.computer.search.path.ShortestPathVertexProgram;
 import org.apache.tinkerpop.gremlin.process.computer.traversal.MemoryTraversalSideEffects;
 import org.apache.tinkerpop.gremlin.process.computer.traversal.TraversalVertexProgram;
 import org.apache.tinkerpop.gremlin.process.computer.traversal.step.map.PageRankVertexProgramStep;
+import org.apache.tinkerpop.gremlin.process.computer.traversal.step.map.ShortestPathVertexProgramStep;
 import org.apache.tinkerpop.gremlin.process.computer.traversal.strategy.decoration.VertexProgramStrategy;
 import org.apache.tinkerpop.gremlin.process.computer.traversal.strategy.optimization.GraphFilterStrategy;
 import org.apache.tinkerpop.gremlin.process.remote.RemoteConnection;
@@ -262,6 +264,8 @@ public final class CoreImports {
         CLASS_IMPORTS.add(PageRankMapReduce.class);
         CLASS_IMPORTS.add(PageRankVertexProgram.class);
         CLASS_IMPORTS.add(PageRankVertexProgramStep.PageRank.class);
+        CLASS_IMPORTS.add(ShortestPathVertexProgram.class);
+        CLASS_IMPORTS.add(ShortestPathVertexProgramStep.ShortestPath.class);
         CLASS_IMPORTS.add(GraphFilterStrategy.class);
         CLASS_IMPORTS.add(TraversalVertexProgram.class);
         CLASS_IMPORTS.add(VertexProgramStrategy.class);

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/jsr223/JavaTranslator.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/jsr223/JavaTranslator.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/jsr223/JavaTranslator.java
index 4813fab..7480974 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/jsr223/JavaTranslator.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/jsr223/JavaTranslator.java
@@ -25,10 +25,7 @@ import org.apache.tinkerpop.gremlin.process.traversal.Bytecode;
 import org.apache.tinkerpop.gremlin.process.traversal.Translator;
 import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
 import org.apache.tinkerpop.gremlin.process.traversal.TraversalSource;
-import org.apache.tinkerpop.gremlin.process.traversal.step.StepConfiguration;
 import org.apache.tinkerpop.gremlin.process.traversal.step.util.BulkSet;
-import org.apache.tinkerpop.gremlin.process.traversal.step.util.DefaultStepConfiguration;
-import org.apache.tinkerpop.gremlin.process.traversal.step.util.StepConfigurationProxy;
 import org.apache.tinkerpop.gremlin.process.traversal.step.util.Tree;
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.TraversalStrategyProxy;
 import org.apache.tinkerpop.gremlin.structure.util.StringFactory;
@@ -122,16 +119,6 @@ public final class JavaTranslator<S extends TraversalSource, T extends Traversal
             final Configuration configuration = ((TraversalStrategyProxy) object).getConfiguration();
             configuration.getKeys().forEachRemaining(key -> map.put(key, translateObject(configuration.getProperty(key))));
             return invokeStrategyCreationMethod(object, map);
-        } else if (object instanceof DefaultStepConfiguration) {
-            final Map<String, Object> map = new LinkedHashMap<>();
-            final Configuration configuration = ((DefaultStepConfiguration) object).getConfiguration();
-            configuration.getKeys().forEachRemaining(key -> map.put(key, translateObject(configuration.getProperty(key))));
-            return invokeStepConfigurationCreationMethod(object, map);
-        }else if (object instanceof StepConfigurationProxy) {
-            final Map<String, Object> map = new LinkedHashMap<>();
-            final Configuration configuration = ((StepConfigurationProxy) object).getConfiguration();
-            configuration.getKeys().forEachRemaining(key -> map.put(key, translateObject(configuration.getProperty(key))));
-            return invokeStepConfigurationCreationMethod(object, map);
         } else if (object instanceof Map) {
             final Map<Object, Object> map = object instanceof Tree ?
                     new Tree() :
@@ -197,30 +184,6 @@ public final class JavaTranslator<S extends TraversalSource, T extends Traversal
         }
     }
 
-    private Object invokeStepConfigurationCreationMethod(final Object delegate, final Map<String, Object> map) {
-        final Class<?> stepConfigurationClass = delegate instanceof DefaultStepConfiguration ? delegate.getClass() : ((StepConfigurationProxy) delegate).getStepConfigurationClass();
-        final Map<String, Method> methodCache = localMethodCache.computeIfAbsent(stepConfigurationClass, k -> {
-            final Map<String, Method> cacheEntry = new HashMap<>();
-
-            try {
-                cacheEntry.put("create", stepConfigurationClass.getMethod("create", Configuration.class));
-            } catch (NoSuchMethodException ignored) {
-                // nothing - the StepConfiguration may not be constructed this way
-            }
-
-            if (cacheEntry.isEmpty())
-                throw new IllegalStateException(String.format("%s does can only be constructed with create(Configuration)", stepConfigurationClass.getSimpleName()));
-
-            return cacheEntry;
-        });
-
-        try {
-            return methodCache.get("create").invoke(null, new MapConfiguration(map));
-        } catch (final InvocationTargetException | IllegalAccessException e) {
-            throw new IllegalStateException(e.getMessage(), e);
-        }
-    }
-
     private Object invokeMethod(final Object delegate, final Class<?> returnType, final String methodName, final Object... arguments) {
         // populate method cache for fast access to methods in subsequent calls
         final Map<String, List<Method>> methodCache = GLOBAL_METHOD_CACHE.getOrDefault(delegate.getClass(), new HashMap<>());

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/search/path/ShortestPathVertexProgram.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/search/path/ShortestPathVertexProgram.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/search/path/ShortestPathVertexProgram.java
new file mode 100644
index 0000000..37d647e
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/search/path/ShortestPathVertexProgram.java
@@ -0,0 +1,566 @@
+/*
+ * 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.search.path;
+
+import org.apache.commons.configuration.Configuration;
+import org.apache.tinkerpop.gremlin.process.computer.*;
+import org.apache.tinkerpop.gremlin.process.computer.traversal.TraversalVertexProgram;
+import org.apache.tinkerpop.gremlin.process.computer.traversal.step.map.ProgramVertexProgramStep;
+import org.apache.tinkerpop.gremlin.process.computer.traversal.step.map.VertexProgramStep;
+import org.apache.tinkerpop.gremlin.process.computer.util.AbstractVertexProgramBuilder;
+import org.apache.tinkerpop.gremlin.process.traversal.*;
+import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__;
+import org.apache.tinkerpop.gremlin.process.traversal.lambda.ConstantTraversal;
+import org.apache.tinkerpop.gremlin.process.traversal.lambda.ElementValueTraversal;
+import org.apache.tinkerpop.gremlin.process.traversal.lambda.IdentityTraversal;
+import org.apache.tinkerpop.gremlin.process.traversal.step.util.ImmutablePath;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.IndexedTraverserSet;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.TraverserSet;
+import org.apache.tinkerpop.gremlin.process.traversal.util.PureTraversal;
+import org.apache.tinkerpop.gremlin.structure.*;
+import org.apache.tinkerpop.gremlin.structure.util.StringFactory;
+import org.apache.tinkerpop.gremlin.util.NumberHelper;
+import org.javatuples.Pair;
+import org.javatuples.Triplet;
+
+import java.util.*;
+import java.util.function.Function;
+
+/**
+ * @author Daniel Kuppitz (http://gremlin.guru)
+ */
+public class ShortestPathVertexProgram implements VertexProgram<Triplet<Path, Edge, Number>> {
+
+    public static final String SHORTEST_PATHS = "gremlin.shortestPathVertexProgram.shortestPaths";
+
+    private static final String SOURCE_VERTEX_FILTER = "gremlin.shortestPathVertexProgram.sourceVertexFilter";
+    private static final String TARGET_VERTEX_FILTER = "gremlin.shortestPathVertexProgram.targetVertexFilter";
+    private static final String EDGE_TRAVERSAL = "gremlin.shortestPathVertexProgram.edgeTraversal";
+    private static final String DISTANCE_TRAVERSAL = "gremlin.shortestPathVertexProgram.distanceTraversal";
+    private static final String MAX_DISTANCE = "gremlin.shortestPathVertexProgram.maxDistance";
+    private static final String INCLUDE_EDGES = "gremlin.shortestPathVertexProgram.includeEdges";
+
+    private static final String STATE = "gremlin.shortestPathVertexProgram.state";
+    private static final String PATHS = "gremlin.shortestPathVertexProgram.paths";
+    private static final String VOTE_TO_HALT = "gremlin.shortestPathVertexProgram.voteToHalt";
+
+    public static final PureTraversal<Vertex, ?> DEFAULT_VERTEX_FILTER_TRAVERSAL = new PureTraversal<>(new IdentityTraversal<>());
+    public static final PureTraversal<Vertex, Edge> DEFAULT_EDGE_TRAVERSAL = new PureTraversal<>(__.bothE().asAdmin());
+    public static final PureTraversal<Edge, Number> DEFAULT_DISTANCE_TRAVERSAL = new PureTraversal<>(new ConstantTraversal<>(1));
+
+    private TraverserSet<Vertex> haltedTraversers;
+    private IndexedTraverserSet<Vertex, Vertex> haltedTraversersIndex;
+    private PureTraversal<?, ?> traversal;
+    private PureTraversal<Vertex, ?> sourceVertexFilterTraversal = DEFAULT_VERTEX_FILTER_TRAVERSAL.clone();
+    private PureTraversal<Vertex, ?> targetVertexFilterTraversal = DEFAULT_VERTEX_FILTER_TRAVERSAL.clone();
+    private PureTraversal<Vertex, Edge> edgeTraversal = DEFAULT_EDGE_TRAVERSAL.clone();
+    private PureTraversal<Edge, Number> distanceTraversal = DEFAULT_DISTANCE_TRAVERSAL.clone();
+    private Step<Vertex, Path> programStep;
+    private Number maxDistance;
+    private boolean distanceEqualsNumberOfHops;
+    private boolean includeEdges;
+    private boolean standalone;
+
+    private static final Set<VertexComputeKey> VERTEX_COMPUTE_KEYS = new HashSet<>(Arrays.asList(
+            VertexComputeKey.of(PATHS, true),
+            VertexComputeKey.of(TraversalVertexProgram.HALTED_TRAVERSERS, false)));
+
+    private final Set<MemoryComputeKey> memoryComputeKeys = new HashSet<>(Arrays.asList(
+            MemoryComputeKey.of(VOTE_TO_HALT, Operator.and, false, true),
+            MemoryComputeKey.of(STATE, Operator.assign, true, true)));
+
+    private ShortestPathVertexProgram() {
+
+    }
+
+    @Override
+    public void loadState(final Graph graph, final Configuration configuration) {
+
+        if (configuration.containsKey(SOURCE_VERTEX_FILTER))
+            this.sourceVertexFilterTraversal = PureTraversal.loadState(configuration, SOURCE_VERTEX_FILTER, graph);
+
+        if (configuration.containsKey(TARGET_VERTEX_FILTER))
+            this.targetVertexFilterTraversal = PureTraversal.loadState(configuration, TARGET_VERTEX_FILTER, graph);
+
+        if (configuration.containsKey(EDGE_TRAVERSAL))
+            this.edgeTraversal = PureTraversal.loadState(configuration, EDGE_TRAVERSAL, graph);
+
+        if (configuration.containsKey(DISTANCE_TRAVERSAL))
+            this.distanceTraversal = PureTraversal.loadState(configuration, DISTANCE_TRAVERSAL, graph);
+
+        if (configuration.containsKey(MAX_DISTANCE))
+            this.maxDistance = (Number) configuration.getProperty(MAX_DISTANCE);
+
+        this.distanceEqualsNumberOfHops = this.distanceTraversal.equals(DEFAULT_DISTANCE_TRAVERSAL);
+        this.includeEdges = configuration.getBoolean(INCLUDE_EDGES, false);
+        this.standalone = !configuration.containsKey(VertexProgramStep.ROOT_TRAVERSAL);
+
+        if (!this.standalone) {
+            this.traversal = PureTraversal.loadState(configuration, VertexProgramStep.ROOT_TRAVERSAL, graph);
+            final String programStepId = configuration.getString(ProgramVertexProgramStep.STEP_ID);
+            for (final Step step : this.traversal.get().getSteps()) {
+                if (step.getId().equals(programStepId)) {
+                    //noinspection unchecked
+                    this.programStep = step;
+                    break;
+                }
+            }
+        }
+
+        this.haltedTraversers = TraversalVertexProgram.loadHaltedTraversers(configuration);
+        this.memoryComputeKeys.add(MemoryComputeKey.of(SHORTEST_PATHS, Operator.addAll, true, !standalone));
+    }
+
+    @Override
+    public void storeState(final Configuration configuration) {
+        VertexProgram.super.storeState(configuration);
+        this.sourceVertexFilterTraversal.storeState(configuration, SOURCE_VERTEX_FILTER);
+        this.targetVertexFilterTraversal.storeState(configuration, TARGET_VERTEX_FILTER);
+        this.edgeTraversal.storeState(configuration, EDGE_TRAVERSAL);
+        this.distanceTraversal.storeState(configuration, DISTANCE_TRAVERSAL);
+        configuration.setProperty(INCLUDE_EDGES, this.includeEdges);
+        if (this.maxDistance != null)
+            configuration.setProperty(MAX_DISTANCE, maxDistance);
+        if (this.traversal != null) {
+            this.traversal.storeState(configuration, ProgramVertexProgramStep.ROOT_TRAVERSAL);
+            configuration.setProperty(ProgramVertexProgramStep.STEP_ID, this.programStep.getId());
+        }
+    }
+
+    @Override
+    public Set<VertexComputeKey> getVertexComputeKeys() {
+        return VERTEX_COMPUTE_KEYS;
+    }
+
+    @Override
+    public Set<MemoryComputeKey> getMemoryComputeKeys() {
+        return memoryComputeKeys;
+    }
+
+    @Override
+    public Set<MessageScope> getMessageScopes(final Memory memory) {
+        return Collections.emptySet();
+    }
+
+    @Override
+    public VertexProgram<Triplet<Path, Edge, Number>> clone() {
+        try {
+            final ShortestPathVertexProgram clone = (ShortestPathVertexProgram) super.clone();
+            if (null != this.edgeTraversal)
+                clone.edgeTraversal = this.edgeTraversal.clone();
+            if (null != this.sourceVertexFilterTraversal)
+                clone.sourceVertexFilterTraversal = this.sourceVertexFilterTraversal.clone();
+            if (null != this.targetVertexFilterTraversal)
+                clone.targetVertexFilterTraversal = this.targetVertexFilterTraversal.clone();
+            if (null != this.distanceTraversal)
+                clone.distanceTraversal = this.distanceTraversal.clone();
+            if (null != this.traversal) {
+                clone.traversal = this.traversal.clone();
+                for (final Step step : clone.traversal.get().getSteps()) {
+                    if (step.getId().equals(this.programStep.getId())) {
+                        //noinspection unchecked
+                        clone.programStep = step;
+                        break;
+                    }
+                }
+            }
+            return clone;
+        } catch (final CloneNotSupportedException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public GraphComputer.ResultGraph getPreferredResultGraph() {
+        return GraphComputer.ResultGraph.ORIGINAL;
+    }
+
+    @Override
+    public GraphComputer.Persist getPreferredPersist() {
+        return GraphComputer.Persist.NOTHING;
+    }
+
+    @Override
+    public void setup(final Memory memory) {
+
+        this.haltedTraversersIndex = new IndexedTraverserSet<>(v -> v);
+        for (final Traverser.Admin<Vertex> traverser : this.haltedTraversers) {
+            this.haltedTraversersIndex.add(traverser.split());
+        }
+        this.haltedTraversers.clear();
+
+        memory.set(VOTE_TO_HALT, true);
+        memory.set(STATE, State.SEARCH);
+    }
+
+    @Override
+    public void execute(final Vertex vertex, final Messenger<Triplet<Path, Edge, Number>> messenger, final Memory memory) {
+
+        switch (memory.<State>get(STATE)) {
+
+            case COLLECT_PATHS:
+                collectShortestPaths(vertex, memory);
+                return;
+
+            case UPDATE_HALTED_TRAVERSERS:
+                updateHaltedTraversers(vertex, memory);
+                return;
+        }
+
+        boolean voteToHalt = true;
+
+        if (memory.isInitialIteration()) {
+
+            copyHaltedTraversersFromMemory(vertex);
+
+            if (!isStartVertex(vertex)) return;
+
+            final Map<Vertex, Pair<Number, Set<Path>>> paths = new HashMap<>();
+            final Path path;
+            final Set<Path> pathSet = new HashSet<>();
+
+            pathSet.add(path = makePath(vertex));
+            paths.put(vertex, Pair.with(0, pathSet));
+
+            vertex.property(VertexProperty.Cardinality.single, PATHS, paths);
+
+            processEdges(vertex, path, 0, messenger);
+
+            voteToHalt = false;
+
+        } else {
+
+            final Map<Vertex, Pair<Number, Set<Path>>> paths =
+                    vertex.<Map<Vertex, Pair<Number, Set<Path>>>>property(PATHS).orElseGet(HashMap::new);
+            final Iterator<Triplet<Path, Edge, Number>> iterator = messenger.receiveMessages();
+
+            while (iterator.hasNext()) {
+
+                final Triplet<Path, Edge, Number> triplet = iterator.next();
+                final Path sourcePath = triplet.getValue0();
+                final Number distance = triplet.getValue2();
+                final Vertex sourceVertex = sourcePath.get(0);
+
+                Path newPath = null;
+
+                if (paths.containsKey(sourceVertex)) {
+
+                    final Number currentShortestDistance = paths.get(sourceVertex).getValue0();
+                    final int cmp = NumberHelper.compare(distance, currentShortestDistance);
+
+                    if (cmp <= 0) {
+                        newPath = extendPath(sourcePath, triplet.getValue1(), vertex);
+                        if (cmp < 0) {
+                            final Set<Path> pathSet = new HashSet<>();
+                            pathSet.add(newPath);
+                            paths.put(sourceVertex, Pair.with(distance, pathSet));
+                        } else {
+                            paths.get(sourceVertex).getValue1().add(newPath);
+                        }
+                    }
+                } else if (!exceedsMaxDistance(distance)) {
+                    final Set<Path> pathSet = new HashSet<>();
+                    pathSet.add(newPath = extendPath(sourcePath, triplet.getValue1(), vertex));
+                    paths.put(sourceVertex, Pair.with(distance, pathSet));
+                }
+
+                if (newPath != null) {
+                    vertex.property(VertexProperty.Cardinality.single, PATHS, paths);
+                    processEdges(vertex, newPath, distance, messenger);
+                    voteToHalt = false;
+                }
+            }
+        }
+
+        memory.add(VOTE_TO_HALT, voteToHalt);
+    }
+
+    @Override
+    public boolean terminate(final Memory memory) {
+        if (memory.isInitialIteration() && this.haltedTraversersIndex != null) {
+            this.haltedTraversersIndex.clear();
+        }
+        final boolean voteToHalt = memory.get(VOTE_TO_HALT);
+        if (voteToHalt) {
+            final State state = memory.get(STATE);
+            if (state.equals(State.COLLECT_PATHS)) {
+                if (this.standalone) return true;
+                memory.set(STATE, State.UPDATE_HALTED_TRAVERSERS);
+                return false;
+            }
+            if (state.equals(State.UPDATE_HALTED_TRAVERSERS)) return true;
+            if (state.equals(State.COLLECT_PATHS)) memory.set(STATE, State.UPDATE_HALTED_TRAVERSERS);
+            else memory.set(STATE, State.COLLECT_PATHS);
+            return false;
+        } else {
+            memory.set(VOTE_TO_HALT, true);
+            return false;
+        }
+    }
+
+    @Override
+    public String toString() {
+
+        final List<String> options = new ArrayList<>();
+        final Function<String, String> shortName = name -> name.substring(name.lastIndexOf(".") + 1);
+
+        if (!this.sourceVertexFilterTraversal.equals(DEFAULT_VERTEX_FILTER_TRAVERSAL)) {
+            options.add(shortName.apply(SOURCE_VERTEX_FILTER) + "=" + this.sourceVertexFilterTraversal.get());
+        }
+
+        if (!this.targetVertexFilterTraversal.equals(DEFAULT_VERTEX_FILTER_TRAVERSAL)) {
+            options.add(shortName.apply(TARGET_VERTEX_FILTER) + "=" + this.targetVertexFilterTraversal.get());
+        }
+
+        if (!this.edgeTraversal.equals(DEFAULT_EDGE_TRAVERSAL)) {
+            options.add(shortName.apply(EDGE_TRAVERSAL) + "=" + this.edgeTraversal.get());
+        }
+
+        if (!this.distanceTraversal.equals(DEFAULT_DISTANCE_TRAVERSAL)) {
+            options.add(shortName.apply(DISTANCE_TRAVERSAL) + "=" + this.distanceTraversal.get());
+        }
+
+        options.add(shortName.apply(INCLUDE_EDGES) + "=" + this.includeEdges);
+
+        return StringFactory.vertexProgramString(this, String.join(", ", options));
+    }
+
+    //////////////////////////////
+
+    private void copyHaltedTraversersFromMemory(final Vertex vertex) {
+        final Collection<Traverser.Admin<Vertex>> traversers = this.haltedTraversersIndex.get(vertex);
+        if (traversers != null) {
+            final TraverserSet<Vertex> newHaltedTraversers = new TraverserSet<>();
+            newHaltedTraversers.addAll(traversers);
+            vertex.property(VertexProperty.Cardinality.single, TraversalVertexProgram.HALTED_TRAVERSERS, newHaltedTraversers);
+        }
+    }
+
+
+    private static Path makePath(final Vertex newVertex) {
+        return extendPath(null, newVertex);
+    }
+
+    private static Path extendPath(final Path currentPath, final Element... elements) {
+        Path result = ImmutablePath.make();
+        if (currentPath != null) {
+            for (final Object o : currentPath.objects()) {
+                result = result.extend(o, Collections.emptySet());
+            }
+        }
+        for (final Element element : elements) {
+            if (element != null) result = result.extend(element, Collections.emptySet());
+        }
+        return result;
+    }
+
+    private boolean isStartVertex(final Vertex vertex) {
+        if (this.standalone) {
+            final Traversal.Admin<Vertex, ?> filterTraversal = this.sourceVertexFilterTraversal.getPure();
+            filterTraversal.addStart(filterTraversal.getTraverserGenerator().generate(vertex, filterTraversal.getStartStep(), 1));
+            return filterTraversal.hasNext();
+        }
+        if (vertex.property(TraversalVertexProgram.HALTED_TRAVERSERS).isPresent()) return true;
+        for (final Traverser<?> traverser : this.haltedTraversers) {
+            if (vertex.equals(traverser.get())) return true;
+        }
+        return false;
+    }
+
+    private boolean isEndVertex(final Vertex vertex, final Vertex startVertex) {
+        final Traversal.Admin<Vertex, ?> filterTraversal = this.targetVertexFilterTraversal.getPure();
+        //noinspection unchecked
+        final Step<Vertex, Vertex> startStep = (Step<Vertex, Vertex>) filterTraversal.getStartStep();
+        if (this.standalone) {
+            filterTraversal.addStart(filterTraversal.getTraverserGenerator().generate(vertex, startStep, 1));
+        } else {
+            final Property<TraverserSet<Vertex>> haltedTraversers = startVertex.property(TraversalVertexProgram.HALTED_TRAVERSERS);
+            if (haltedTraversers.isPresent()) {
+                for (final Traverser.Admin<Vertex> traverser : haltedTraversers.value()) {
+                    filterTraversal.addStart(traverser.split(vertex, startStep));
+                }
+            }
+        }
+        return filterTraversal.hasNext();
+    }
+
+    private void processEdges(final Vertex vertex, final Path currentPath, final Number currentDistance,
+                              final Messenger<Triplet<Path, Edge, Number>> messenger) {
+
+        final Traversal.Admin<Vertex, Edge> edgeTraversal = this.edgeTraversal.getPure();
+        edgeTraversal.addStart(edgeTraversal.getTraverserGenerator().generate(vertex, edgeTraversal.getStartStep(), 1));
+
+        while (edgeTraversal.hasNext()) {
+            final Edge edge = edgeTraversal.next();
+            final Number distance = getDistance(edge);
+
+            Vertex otherV = edge.inVertex();
+            if (otherV.equals(vertex))
+                otherV = edge.outVertex();
+
+            if (!currentPath.objects().contains(otherV)) {
+                messenger.sendMessage(MessageScope.Global.of(otherV),
+                        Triplet.with(currentPath, this.includeEdges ? edge : null,
+                                NumberHelper.add(currentDistance, distance)));
+            }
+        }
+    }
+
+    private void updateHaltedTraversers(final Vertex vertex, final Memory memory) {
+        if (isStartVertex(vertex)) {
+            final List<Path> paths = memory.get(SHORTEST_PATHS);
+            if (vertex.property(TraversalVertexProgram.HALTED_TRAVERSERS).isPresent()) {
+                final TraverserSet<Vertex> haltedTraversers = vertex.value(TraversalVertexProgram.HALTED_TRAVERSERS);
+                final TraverserSet<Path> newHaltedTraversers = new TraverserSet<>();
+                for (final Traverser.Admin<Vertex> traverser : haltedTraversers) {
+                    final Vertex v = traverser.get();
+                    for (final Path path : paths) {
+                        if (path.get(0).equals(v)) {
+                            newHaltedTraversers.add(traverser.split(path, this.programStep));
+                        }
+                    }
+                }
+                vertex.property(VertexProperty.Cardinality.single, TraversalVertexProgram.HALTED_TRAVERSERS, newHaltedTraversers);
+            }
+        }
+    }
+
+    private Number getDistance(final Edge edge) {
+        if (this.distanceEqualsNumberOfHops) return 1;
+        final Traversal.Admin<Edge, Number> traversal = this.distanceTraversal.getPure();
+        traversal.addStart(traversal.getTraverserGenerator().generate(edge, traversal.getStartStep(), 1));
+        return traversal.tryNext().orElse(0);
+    }
+
+    private boolean exceedsMaxDistance(final Number distance) {
+        // This method is used to stop the message sending for paths that exceed the specified maximum distance. Since
+        // custom distances can be negative, this method should only return true if the distance is calculated based on
+        // the number of hops.
+        return this.distanceEqualsNumberOfHops && this.maxDistance != null
+                && NumberHelper.compare(distance, this.maxDistance) > 0;
+    }
+
+    private void collectShortestPaths(final Vertex vertex, final Memory memory) {
+
+        final VertexProperty<Map<Vertex, Pair<Number, Set<Path>>>> pathProperty = vertex.property(PATHS);
+
+        if (pathProperty.isPresent()) {
+
+            final Map<Vertex, Pair<Number, Set<Path>>> paths = pathProperty.value();
+            final List<Path> result = new ArrayList<>();
+
+            for (final Pair<Number, Set<Path>> pair : paths.values()) {
+                for (final Path path : pair.getValue1()) {
+                    if (isEndVertex(path.get(path.size() - 1), path.get(0))) {
+                        result.add(path);
+                    }
+                }
+            }
+
+            pathProperty.remove();
+
+            memory.add(SHORTEST_PATHS, result);
+        }
+    }
+
+    //////////////////////////////
+
+    public static Builder build() {
+        return new Builder();
+    }
+
+    @SuppressWarnings("WeakerAccess")
+    public static final class Builder extends AbstractVertexProgramBuilder<Builder> {
+
+
+        private Builder() {
+            super(ShortestPathVertexProgram.class);
+        }
+
+        public Builder source(final Traversal<Vertex, ?> sourceVertexFilter) {
+            if (null == sourceVertexFilter) throw Graph.Exceptions.argumentCanNotBeNull("sourceVertexFilter");
+            PureTraversal.storeState(this.configuration, SOURCE_VERTEX_FILTER, sourceVertexFilter.asAdmin());
+            return this;
+        }
+
+        public Builder target(final Traversal<Vertex, ?> targetVertexFilter) {
+            if (null == targetVertexFilter) throw Graph.Exceptions.argumentCanNotBeNull("targetVertexFilter");
+            PureTraversal.storeState(this.configuration, TARGET_VERTEX_FILTER, targetVertexFilter.asAdmin());
+            return this;
+        }
+
+        public Builder edgeDirection(final Direction direction) {
+            if (null == direction) throw Graph.Exceptions.argumentCanNotBeNull("direction");
+            return edgeTraversal(__.toE(direction));
+        }
+
+        public Builder edgeTraversal(final Traversal<Vertex, Edge> edgeTraversal) {
+            if (null == edgeTraversal) throw Graph.Exceptions.argumentCanNotBeNull("edgeTraversal");
+            PureTraversal.storeState(this.configuration, EDGE_TRAVERSAL, edgeTraversal.asAdmin());
+            return this;
+        }
+
+        public Builder distanceProperty(final String distance) {
+            //noinspection unchecked
+            return distance != null
+                    ? distanceTraversal((Traversal) new ElementValueTraversal<>(distance))
+                    : distanceTraversal(DEFAULT_DISTANCE_TRAVERSAL.getPure());
+        }
+
+        public Builder distanceTraversal(final Traversal<Edge, Number> distanceTraversal) {
+            if (null == distanceTraversal) throw Graph.Exceptions.argumentCanNotBeNull("distanceTraversal");
+            PureTraversal.storeState(this.configuration, DISTANCE_TRAVERSAL, distanceTraversal.asAdmin());
+            return this;
+        }
+
+        public Builder maxDistance(final Number distance) { // todo: implement test
+            if (null != distance)
+                this.configuration.setProperty(MAX_DISTANCE, distance);
+            else
+                this.configuration.clearProperty(MAX_DISTANCE);
+            return this;
+        }
+
+        public Builder includeEdges(final boolean include) {
+            this.configuration.setProperty(INCLUDE_EDGES, include);
+            return this;
+        }
+    }
+
+    ////////////////////////////
+
+    @Override
+    public Features getFeatures() {
+        return new Features() {
+            @Override
+            public boolean requiresGlobalMessageScopes() {
+                return true;
+            }
+
+            @Override
+            public boolean requiresVertexPropertyAddition() {
+                return true;
+            }
+        };
+    }
+
+    private enum State {
+        SEARCH, COLLECT_PATHS, UPDATE_HALTED_TRAVERSERS
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/traversal/step/map/ShortestPathVertexProgramStep.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/traversal/step/map/ShortestPathVertexProgramStep.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/traversal/step/map/ShortestPathVertexProgramStep.java
new file mode 100644
index 0000000..c9fee35
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/traversal/step/map/ShortestPathVertexProgramStep.java
@@ -0,0 +1,257 @@
+/*
+ * 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.traversal.step.map;
+
+import org.apache.tinkerpop.gremlin.process.computer.*;
+import org.apache.tinkerpop.gremlin.process.computer.search.path.ShortestPathVertexProgram;
+import org.apache.tinkerpop.gremlin.process.computer.traversal.TraversalVertexProgram;
+import org.apache.tinkerpop.gremlin.process.traversal.*;
+import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversal;
+import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__;
+import org.apache.tinkerpop.gremlin.process.traversal.step.*;
+import org.apache.tinkerpop.gremlin.process.traversal.step.util.Parameters;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.TraverserRequirement;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.TraverserSet;
+import org.apache.tinkerpop.gremlin.process.traversal.util.PureTraversal;
+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.Vertex;
+import org.apache.tinkerpop.gremlin.structure.util.StringFactory;
+import org.apache.tinkerpop.gremlin.util.Serializer;
+
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * @author Daniel Kuppitz (http://gremlin.guru)
+ */
+public final class ShortestPathVertexProgramStep extends VertexProgramStep implements TraversalParent, Configuring {
+
+    private Parameters parameters = new Parameters();
+    private PureTraversal<Vertex, ?> targetVertexFilter = ShortestPathVertexProgram.DEFAULT_VERTEX_FILTER_TRAVERSAL.clone();
+    private PureTraversal<Vertex, Edge> edgeTraversal = ShortestPathVertexProgram.DEFAULT_EDGE_TRAVERSAL.clone();
+    private PureTraversal<Edge, Number> distanceTraversal = ShortestPathVertexProgram.DEFAULT_DISTANCE_TRAVERSAL.clone();
+    private Number maxDistance;
+    private boolean includeEdges;
+
+    public ShortestPathVertexProgramStep(final Traversal.Admin<?, ?> traversal) {
+        super(traversal);
+    }
+
+    private void setTargetVertexFilter(final Traversal filterTraversal) {
+        this.targetVertexFilter = new PureTraversal<>(this.integrateChild(filterTraversal.asAdmin()));
+    }
+
+    private void setEdgeTraversal(final Traversal edgeTraversal) {
+        this.edgeTraversal = new PureTraversal<>(this.integrateChild(edgeTraversal.asAdmin()));
+    }
+
+    private void setDistanceTraversal(final Traversal distanceTraversal) {
+        this.distanceTraversal = new PureTraversal<>(this.integrateChild(distanceTraversal.asAdmin()));
+    }
+
+    private void setMaxDistance(final Number maxDistance) {
+        this.maxDistance = maxDistance;
+    }
+
+    private void setIncludeEdges(final boolean includeEdges) {
+        this.includeEdges = includeEdges;
+    }
+
+    @Override
+    public void configure(final Object... keyValues) {
+        if (!ShortestPath.configure(this, (String) keyValues[0], keyValues[1])) {
+            this.parameters.set(this, keyValues);
+        }
+    }
+
+    @Override
+    public Parameters getParameters() {
+        return parameters;
+    }
+
+    @Override
+    protected Traverser.Admin<ComputerResult> processNextStart() throws NoSuchElementException {
+        return super.processNextStart();
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public List<Traversal.Admin<?, ?>> getLocalChildren() {
+        return Arrays.asList(
+                this.targetVertexFilter.get(),
+                this.edgeTraversal.get(),
+                this.distanceTraversal.get());
+    }
+
+    @Override
+    public String toString() {
+        return StringFactory.stepString(this, this.targetVertexFilter.get(), this.edgeTraversal.get(),
+                this.distanceTraversal.get(), this.maxDistance, this.includeEdges, new GraphFilter(this.computer));
+    }
+
+    @Override
+    public ShortestPathVertexProgram generateProgram(final Graph graph, final Memory memory) {
+
+        final ShortestPathVertexProgram.Builder builder = ShortestPathVertexProgram.build()
+                .target(this.targetVertexFilter.getPure())
+                .edgeTraversal(this.edgeTraversal.getPure())
+                .distanceTraversal(this.distanceTraversal.getPure())
+                .maxDistance(this.maxDistance)
+                .includeEdges(this.includeEdges);
+
+        //noinspection unchecked
+        final PureTraversal pureRootTraversal = new PureTraversal<>(this.traversal);
+        Object rootTraversalValue;
+        try {
+            rootTraversalValue = Base64.getEncoder().encodeToString(Serializer.serializeObject(pureRootTraversal));
+        } catch (final IOException ignored) {
+            rootTraversalValue = pureRootTraversal;
+        }
+
+        builder.configure(
+                ProgramVertexProgramStep.ROOT_TRAVERSAL, rootTraversalValue,
+                ProgramVertexProgramStep.STEP_ID, this.id);
+
+        if (memory.exists(TraversalVertexProgram.HALTED_TRAVERSERS)) {
+            final TraverserSet<?> haltedTraversers = memory.get(TraversalVertexProgram.HALTED_TRAVERSERS);
+            if (!haltedTraversers.isEmpty()) {
+                Object haltedTraversersValue;
+                try {
+                    haltedTraversersValue = Base64.getEncoder().encodeToString(Serializer.serializeObject(haltedTraversers));
+                } catch (final IOException ignored) {
+                    haltedTraversersValue = haltedTraversers;
+                }
+                builder.configure(TraversalVertexProgram.HALTED_TRAVERSERS, haltedTraversersValue);
+            }
+        }
+
+        return builder.create(graph);
+    }
+
+    @Override
+    public Set<TraverserRequirement> getRequirements() {
+        return TraversalParent.super.getSelfAndChildRequirements();
+    }
+
+    @Override
+    public ShortestPathVertexProgramStep clone() {
+        final ShortestPathVertexProgramStep clone = (ShortestPathVertexProgramStep) super.clone();
+        clone.targetVertexFilter = this.targetVertexFilter.clone();
+        clone.edgeTraversal = this.edgeTraversal.clone();
+        clone.distanceTraversal = this.distanceTraversal.clone();
+        return clone;
+    }
+
+    @Override
+    public void setTraversal(final Traversal.Admin<?, ?> parentTraversal) {
+        super.setTraversal(parentTraversal);
+        this.integrateChild(this.targetVertexFilter.get());
+        this.integrateChild(this.edgeTraversal.get());
+        this.integrateChild(this.distanceTraversal.get());
+    }
+
+    @Override
+    public int hashCode() {
+        return super.hashCode();
+    }
+
+    /**
+     * Configuration options to be passed to the {@link GraphTraversal#with(String, Object)} step.
+     */
+    public static class ShortestPath {
+
+        /**
+         * Configures the traversal to use to filter the target vertices for all shortest paths.
+         */
+        public static final String TARGET = Graph.Hidden.hide("tinkerpop.shortestPath.target");
+
+        /**
+         * Configures the direction or traversal to use to filter the edges traversed during the shortest path search phase.
+         */
+        public static final String EDGES = Graph.Hidden.hide("tinkerpop.shortestPath.edges");
+
+        /**
+         * Configures the edge property or traversal to use for shortest path distance calculations.
+         */
+        public static final String DISTANCE = Graph.Hidden.hide("tinkerpop.shortestPath.distance");
+
+        /**
+         * Configures the maximum distance for all shortest paths. Any path with a distance greater than the specified
+         * value will not be returned.
+         */
+        public static final String MAX_DISTANCE = Graph.Hidden.hide("tinkerpop.shortestPath.maxDistance");
+
+        /**
+         * Configures the inclusion of edges in the shortest path computation result.
+         */
+        public static final String INCLUDE_EDGES = Graph.Hidden.hide("tinkerpop.shortestPath.includeEdges");
+
+        private static boolean configure(final ShortestPathVertexProgramStep step, final String key, final Object value) {
+
+            if (TARGET.equals(key)) {
+                if (value instanceof Traversal) {
+                    step.setTargetVertexFilter((Traversal) value);
+                    return true;
+                }
+                else throw new IllegalArgumentException("ShortestPath.TARGET requires a Traversal as its argument");
+            }
+            else if (EDGES.equals(key)) {
+                if (value instanceof Traversal) {
+                    step.setEdgeTraversal((Traversal) value);
+                    return true;
+                }
+                else if (value instanceof Direction) {
+                    step.setEdgeTraversal(__.toE((Direction) value));
+                    return true;
+                }
+                else throw new IllegalArgumentException("ShortestPath.EDGES requires a Traversal or a Direction as its argument");
+            }
+            else if (DISTANCE.equals(key)) {
+                if (value instanceof Traversal) {
+                    step.setDistanceTraversal((Traversal) value);
+                    return true;
+                }
+                else if (value instanceof String) {
+                    // todo: new ElementValueTraversal((String) value)
+                    step.setDistanceTraversal(__.values((String) value));
+                    return true;
+                }
+                else throw new IllegalArgumentException("ShortestPath.DISTANCE requires a Traversal or a property name as its argument");
+            }
+            else if (MAX_DISTANCE.equals(key)) {
+                if (value instanceof Number) {
+                    step.setMaxDistance((Number) value);
+                    return true;
+                }
+                else throw new IllegalArgumentException("ShortestPath.MAX_DISTANCE requires a Number as its argument");
+            }
+            else if (INCLUDE_EDGES.equals(key)) {
+                if (value instanceof Boolean) {
+                    step.setIncludeEdges((Boolean) value);
+                    return true;
+                }
+                else throw new IllegalArgumentException("ShortestPath.INCLUDE_EDGES requires a Boolean as its argument");
+            }
+            return false;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/remote/RemoteGraph.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/remote/RemoteGraph.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/remote/RemoteGraph.java
index da12114..8874f09 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/remote/RemoteGraph.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/remote/RemoteGraph.java
@@ -61,6 +61,10 @@ import java.util.Iterator;
         method = "*",
         reason = "hmmmm")
 @Graph.OptOut(
+        test = "org.apache.tinkerpop.gremlin.process.traversal.step.map.ShortestPathTest",
+        method = "*",
+        reason = "hmmmm")
+@Graph.OptOut(
         test = "org.apache.tinkerpop.gremlin.process.traversal.strategy.decoration.TranslationStrategyProcessTest",
         method = "*",
         reason = "hmmmm")
@@ -89,6 +93,10 @@ import java.util.Iterator;
         method = "*",
         reason = "RemoteGraph does not support direct Graph.compute() access")
 @Graph.OptOut(
+        test = "org.apache.tinkerpop.gremlin.process.computer.search.path.ShortestPathVertexProgramTest",
+        method = "*",
+        reason = "RemoteGraph does not support direct Graph.compute() access")
+@Graph.OptOut(
         test = "org.apache.tinkerpop.gremlin.process.computer.bulkloading.BulkLoaderVertexProgramTest",
         method = "*",
         reason = "RemoteGraph does not support direct Graph.compute() access")

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/Bytecode.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/Bytecode.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/Bytecode.java
index ad50c5c..a3ba9f0 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/Bytecode.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/Bytecode.java
@@ -19,8 +19,6 @@
 
 package org.apache.tinkerpop.gremlin.process.traversal;
 
-import org.apache.tinkerpop.gremlin.process.traversal.step.StepConfiguration;
-import org.apache.tinkerpop.gremlin.process.traversal.step.util.DefaultStepConfiguration;
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.TraversalStrategyProxy;
 import org.apache.tinkerpop.gremlin.structure.util.StringFactory;
 import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
@@ -278,19 +276,7 @@ public final class Bytecode implements Cloneable, Serializable {
         //
         if (argument instanceof Traversal)
             return ((Traversal) argument).asAdmin().getBytecode();
-        else if (argument instanceof StepConfiguration) {
-            final StepConfiguration stepConfig = (StepConfiguration) argument;
-            final LinkedHashMap<String,List<Object>> m = new LinkedHashMap<>();
-            final org.apache.commons.configuration.Configuration conf = stepConfig.getConfiguration();
-            final java.util.Iterator<String> keys = conf.getKeys();
-            while (keys.hasNext()) {
-                final String key = keys.next();
-                final List<Object> args = new ArrayList<>();
-                conf.getList(key).forEach(x -> args.add(convertArgument(x, true)));
-                m.put(key, args);
-            }
-            return new DefaultStepConfiguration(m);
-        } else if (argument instanceof Map) {
+        else if (argument instanceof Map) {
             final Map<Object, Object> map = new LinkedHashMap<>(((Map) argument).size());
             for (final Map.Entry<?, ?> entry : ((Map<?, ?>) argument).entrySet()) {
                 map.put(convertArgument(entry.getKey(), true), convertArgument(entry.getValue(), true));

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/Traversal.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/Traversal.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/Traversal.java
index 220c995..30435ab 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/Traversal.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/Traversal.java
@@ -496,15 +496,17 @@ public interface Traversal<S, E> extends Iterator<E>, Serializable, Cloneable, A
         public void setGraph(final Graph graph);
 
         public default boolean equals(final Traversal.Admin<S, E> other) {
-            final List<Step> steps = this.getSteps();
-            final List<Step> otherSteps = other.getSteps();
-            if (steps.size() == otherSteps.size()) {
-                for (int i = 0; i < steps.size(); i++) {
-                    if (!steps.get(i).equals(otherSteps.get(i))) {
-                        return false;
+            if (this.getClass().equals(other.getClass())) {
+                final List<Step> steps = this.getSteps();
+                final List<Step> otherSteps = other.getSteps();
+                if (steps.size() == otherSteps.size()) {
+                    for (int i = 0; i < steps.size(); i++) {
+                        if (!steps.get(i).equals(otherSteps.get(i))) {
+                            return false;
+                        }
                     }
+                    return true;
                 }
-                return true;
             }
             return false;
         }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/dsl/graph/GraphTraversal.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/dsl/graph/GraphTraversal.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/dsl/graph/GraphTraversal.java
index afc4767..28798ee 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/dsl/graph/GraphTraversal.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/dsl/graph/GraphTraversal.java
@@ -22,6 +22,7 @@ import org.apache.tinkerpop.gremlin.process.computer.VertexProgram;
 import org.apache.tinkerpop.gremlin.process.computer.traversal.step.map.PageRankVertexProgramStep;
 import org.apache.tinkerpop.gremlin.process.computer.traversal.step.map.PeerPressureVertexProgramStep;
 import org.apache.tinkerpop.gremlin.process.computer.traversal.step.map.ProgramVertexProgramStep;
+import org.apache.tinkerpop.gremlin.process.computer.traversal.step.map.ShortestPathVertexProgramStep;
 import org.apache.tinkerpop.gremlin.process.traversal.Order;
 import org.apache.tinkerpop.gremlin.process.traversal.P;
 import org.apache.tinkerpop.gremlin.process.traversal.Path;
@@ -41,8 +42,6 @@ import org.apache.tinkerpop.gremlin.process.traversal.step.FromToModulating;
 import org.apache.tinkerpop.gremlin.process.traversal.step.Mutating;
 import org.apache.tinkerpop.gremlin.process.traversal.step.TimesModulating;
 import org.apache.tinkerpop.gremlin.process.traversal.step.TraversalOptionParent;
-import org.apache.tinkerpop.gremlin.process.traversal.step.WithModulating;
-import org.apache.tinkerpop.gremlin.process.traversal.step.WithModulation;
 import org.apache.tinkerpop.gremlin.process.traversal.step.branch.BranchStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.branch.ChooseStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.branch.LocalStep;
@@ -2424,6 +2423,24 @@ public interface GraphTraversal<S, E> extends Traversal<S, E> {
     }
 
     /**
+     * Executes a Shortest Path algorithm over the graph.
+     *
+     * @return the traversal with the appended {@link ShortestPathVertexProgramStep}
+     * @see <a href="http://tinkerpop.apache.org/docs/${project.version}/reference/#shortestpath-step" target="_blank">Reference Documentation - ShortestPath Step</a>
+     */
+    public default GraphTraversal<S, Path> shortestPath() {
+        if (this.asAdmin().getEndStep() instanceof GraphStep) {
+            // This is very unfortunate, but I couldn't find another way to make it work. Without the additional
+            // IdentityStep, TraversalVertexProgram doesn't handle halted traversers as expected (it ignores both:
+            // HALTED_TRAVERSER stored in memory and stored as vertex properties); instead it just emits all vertices.
+            this.identity();
+        }
+        this.asAdmin().getBytecode().addStep(Symbols.shortestPath);
+        return (GraphTraversal<S, Path>) ((Traversal.Admin) this.asAdmin())
+                .addStep(new ShortestPathVertexProgramStep(this.asAdmin()));
+    }
+
+    /**
      * Executes an arbitrary {@link VertexProgram} over the graph.
      *
      * @return the traversal with the appended {@link ProgramVertexProgramStep}
@@ -2819,6 +2836,7 @@ public interface GraphTraversal<S, E> extends Traversal<S, E> {
 
         public static final String pageRank = "pageRank";
         public static final String peerPressure = "peerPressure";
+        public static final String shortestPath = "shortestPath";
         public static final String program = "program";
 
         public static final String by = "by";
@@ -2826,7 +2844,6 @@ public interface GraphTraversal<S, E> extends Traversal<S, E> {
         public static final String times = "times";
         public static final String as = "as";
         public static final String option = "option";
-        public static final String with = "with";
 
     }
 }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/ColumnTraversal.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/ColumnTraversal.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/ColumnTraversal.java
index 5023805..9e2f31c 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/ColumnTraversal.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/ColumnTraversal.java
@@ -22,6 +22,8 @@ package org.apache.tinkerpop.gremlin.process.traversal.lambda;
 import org.apache.tinkerpop.gremlin.process.traversal.Traverser;
 import org.apache.tinkerpop.gremlin.structure.Column;
 
+import java.util.Objects;
+
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
@@ -57,4 +59,10 @@ public final class ColumnTraversal extends AbstractLambdaTraversal {
     public int hashCode() {
         return this.getClass().hashCode() ^ this.column.hashCode();
     }
+
+    @Override
+    public boolean equals(final Object other) {
+        return other instanceof ColumnTraversal
+                && Objects.equals(((ColumnTraversal) other).column, this.column);
+    }
 }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/ConstantTraversal.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/ConstantTraversal.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/ConstantTraversal.java
index 91973b9..bcf82d4 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/ConstantTraversal.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/ConstantTraversal.java
@@ -18,6 +18,8 @@
  */
 package org.apache.tinkerpop.gremlin.process.traversal.lambda;
 
+import java.util.Objects;
+
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
@@ -43,5 +45,10 @@ public final class ConstantTraversal<S, E> extends AbstractLambdaTraversal<S, E>
     public int hashCode() {
         return this.getClass().hashCode() ^ this.end.hashCode();
     }
-}
 
+    @Override
+    public boolean equals(final Object other) {
+        return other instanceof ConstantTraversal
+                && Objects.equals(((ConstantTraversal) other).end, this.end);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/ElementValueTraversal.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/ElementValueTraversal.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/ElementValueTraversal.java
index 2e9b26c..fbfff62 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/ElementValueTraversal.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/ElementValueTraversal.java
@@ -22,6 +22,8 @@ import org.apache.tinkerpop.gremlin.process.traversal.Traverser;
 import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalUtil;
 import org.apache.tinkerpop.gremlin.structure.Element;
 
+import java.util.Objects;
+
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
@@ -62,4 +64,10 @@ public final class ElementValueTraversal<V> extends AbstractLambdaTraversal<Elem
     public int hashCode() {
         return super.hashCode() ^ this.propertyKey.hashCode();
     }
-}
+
+    @Override
+    public boolean equals(final Object other) {
+        return other instanceof ElementValueTraversal
+                && Objects.equals(((ElementValueTraversal) other).propertyKey, this.propertyKey);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/IdentityTraversal.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/IdentityTraversal.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/IdentityTraversal.java
index 98f85c0..1ed5749 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/IdentityTraversal.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/IdentityTraversal.java
@@ -41,4 +41,9 @@ public final class IdentityTraversal<S> extends AbstractLambdaTraversal<S, S> {
     public String toString() {
         return "identity";
     }
-}
+
+    @Override
+    public boolean equals(final Object other) {
+        return other instanceof IdentityTraversal;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/LoopTraversal.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/LoopTraversal.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/LoopTraversal.java
index ae03c94..68b6b18 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/LoopTraversal.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/LoopTraversal.java
@@ -55,4 +55,10 @@ public final class LoopTraversal<S> extends AbstractLambdaTraversal<S, S> {
     public int hashCode() {
         return this.getClass().hashCode() ^ Long.hashCode(this.maxLoops);
     }
+
+    @Override
+    public boolean equals(final Object other) {
+        return other instanceof LoopTraversal
+                && ((LoopTraversal) other).maxLoops == this.maxLoops;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/TokenTraversal.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/TokenTraversal.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/TokenTraversal.java
index 4f98a54..d41c402 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/TokenTraversal.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/TokenTraversal.java
@@ -22,6 +22,8 @@ import org.apache.tinkerpop.gremlin.process.traversal.Traverser;
 import org.apache.tinkerpop.gremlin.structure.Element;
 import org.apache.tinkerpop.gremlin.structure.T;
 
+import java.util.Objects;
+
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
@@ -57,4 +59,10 @@ public final class TokenTraversal<S extends Element, E> extends AbstractLambdaTr
     public int hashCode() {
         return this.getClass().hashCode() ^ this.t.hashCode();
     }
+
+    @Override
+    public boolean equals(final Object other) {
+        return other instanceof TokenTraversal
+                && Objects.equals(((TokenTraversal) other).t, this.t);
+    }
 }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/TrueTraversal.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/TrueTraversal.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/TrueTraversal.java
index 84c0db6..71580ce 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/TrueTraversal.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/lambda/TrueTraversal.java
@@ -43,4 +43,9 @@ public final class TrueTraversal<S> extends AbstractLambdaTraversal<S, S> {
     public TrueTraversal<S> clone() {
         return INSTANCE;
     }
+
+    @Override
+    public boolean equals(final Object other) {
+        return other instanceof TrueTraversal;
+    }
 }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/StepConfiguration.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/StepConfiguration.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/StepConfiguration.java
deleted file mode 100644
index 3a9d4a7..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/StepConfiguration.java
+++ /dev/null
@@ -1,51 +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.process.traversal.step;
-
-import org.apache.commons.configuration.BaseConfiguration;
-import org.apache.commons.configuration.Configuration;
-import org.apache.tinkerpop.gremlin.process.traversal.Step;
-import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversal;
-import org.apache.tinkerpop.gremlin.process.traversal.step.util.DefaultStepConfiguration;
-
-import java.util.function.Consumer;
-
-/**
- * A {@code WithOption} can be supplied to {@link GraphTraversal#with(StepConfiguration)} and is designed to modulate a
- * {@link Step} in some way. As {@code WithStep} is a {@code Consumer} that accepts a {@link Step}, the implementation
- * can modify that step in any way it deems necessary. Typical usage for those adding to the Gremlin language in some
- * way would be to provide an expression that returns a {@link DefaultStepConfiguration}.
- * <p/>
- * To work properly with TinkerPop serialization, implementations should provide a static
- * {@code StepConfiguration create(Configuration)} method.
- *
- * @author Stephen Mallette (http://stephen.genoprime.com)
- */
-public interface StepConfiguration<S extends Step> extends Consumer<S> {
-
-    /**
-     * Get the configuration representation of this strategy. This is useful for converting a strategy into a
-     * serialized form.
-     *
-     * @return the configuration used to create this strategy
-     */
-    public default Configuration getConfiguration() {
-        return new BaseConfiguration();
-    }
-}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/DefaultStepConfiguration.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/DefaultStepConfiguration.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/DefaultStepConfiguration.java
deleted file mode 100644
index b75358f..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/DefaultStepConfiguration.java
+++ /dev/null
@@ -1,158 +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.process.traversal.step.util;
-
-import org.apache.commons.configuration.Configuration;
-import org.apache.commons.configuration.MapConfiguration;
-import org.apache.commons.lang.reflect.MethodUtils;
-import org.apache.tinkerpop.gremlin.process.remote.traversal.strategy.decoration.RemoteStrategy;
-import org.apache.tinkerpop.gremlin.process.traversal.Step;
-import org.apache.tinkerpop.gremlin.process.traversal.step.StepConfiguration;
-import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-
-/**
- * A basic {@link StepConfiguration} implementation that uses reflection to set methods on the step to which the
- * configuration will be applied. While use of reflection isn't quite as nice as direct application of configuration
- * options to a step, this implementation is serialization ready and thus requires no additional work from the
- * developer to get a step option ready for usage. If using this implementation, it is of extreme importance that
- * the developer implement solid test coverage to ensure that reflection calls will work at runtime as compilation
- * errors will not be raised under this approach.
- *
- * @author Stephen Mallette (http://stephen.genoprime.com)
- */
-public class DefaultStepConfiguration implements StepConfiguration<Step> {
-
-    private final Map<String, List<Object>> conf;
-    private final Class<? extends Step> expects;
-
-    /**
-     * Creates a new {@code DefaultStepConfiguration}.
-     *
-     * @param method to call on the step
-     * @param args the arguments to pass to the method
-     */
-    public DefaultStepConfiguration(final String method, final Object... args) {
-        this(null, method, args);
-    }
-
-    /**
-     * Creates a new {@code DefaultStepConfiguration}.
-     *
-     * @param methods a map of methods to call when configuring a step where the keys are the method names and the
-     *                values are the list of arguments to apply to that method
-     */
-    public DefaultStepConfiguration(final LinkedHashMap<String, List<Object>> methods) {
-        this(null, methods);
-    }
-
-    /**
-     * Creates a new {@code DefaultStepConfiguration} with a validation option to ensure that the configuration is
-     * applied to the right type of step.
-     *
-     * @param expects the step type that this configuration should be applied to
-     * @param method to call on the step
-     * @param args the arguments to pass to the method
-     */
-    public DefaultStepConfiguration(final Class<? extends Step> expects, final String method, final Object... args) {
-        if (null == method || method.isEmpty()) throw new IllegalArgumentException("method may not be null or empty");
-        conf = new LinkedHashMap<>();
-        conf.put(method, Arrays.asList(args));
-        this.expects = expects;
-    }
-
-    /**
-     * Creates a new {@code DefaultStepConfiguration} with a validation option to ensure that the configuration is
-     * applied to the right type of step.
-     *
-     * @param expects the step type that this configuration should be applied to
-     * @param methods a map of methods to call when configuring a step where the keys are the method names and the
-     *                values are the list of arguments to apply to that method
-     */
-    public DefaultStepConfiguration(final Class<? extends Step> expects, final LinkedHashMap<String, List<Object>> methods) {
-        if (null == methods || methods.isEmpty()) throw new IllegalArgumentException("methods may not be null or empty");
-        if (IteratorUtils.anyMatch(methods.keySet().iterator(), k -> null == k || k.isEmpty())) throw new IllegalArgumentException("no key of methods map may be null or empty");
-        conf = methods;
-        this.expects = expects;
-    }
-
-    private DefaultStepConfiguration() {
-        // for gyro's sake.........
-        conf = Collections.emptyMap();
-        expects = null;
-    }
-
-    @Override
-    public void accept(final Step step) {
-        final Optional<Class<? extends Step>> opt = Optional.ofNullable(expects);
-        if (opt.isPresent() && !opt.get().isAssignableFrom(step.getClass())) {
-            throw new IllegalStateException(String.format("Could not apply step configuration of %s to %s", conf, step.getClass().getName()));
-        }
-
-        for (Map.Entry<String, List<Object>> kv : conf.entrySet()) {
-            try {
-                MethodUtils.invokeMethod(step, kv.getKey(), kv.getValue().toArray());
-            } catch (NoSuchMethodException nsme) {
-                if (!step.getTraversal().asAdmin().getStrategies().getStrategy(RemoteStrategy.class).isPresent())
-                    throw new IllegalStateException(String.format("Step configuration of %s with args of %s cannot be applied to %s",
-                            kv.getKey(), kv.getValue(), step.getClass().getName()), nsme);
-            } catch (Exception ex) {
-                throw new IllegalStateException(String.format("Step configuration of %s with args of %s cannot be applied to %s",
-                        kv.getKey(), kv.getValue(), step.getClass().getName()), ex);
-            }
-        }
-    }
-
-    public static StepConfiguration create(final Configuration conf) {
-        final LinkedHashMap<String,List<Object>> m = new LinkedHashMap<>();
-        final Iterator<String> keys = conf.getKeys();
-        while (keys.hasNext()) {
-            final String key = keys.next();
-            m.put(key, conf.getList(key));
-        }
-        return new DefaultStepConfiguration(m);
-    }
-
-    @Override
-    public Configuration getConfiguration() {
-        return new MapConfiguration(conf);
-    }
-
-    @Override
-    public boolean equals(final Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-
-        final DefaultStepConfiguration that = (DefaultStepConfiguration) o;
-
-        return conf.equals(that.conf);
-    }
-
-    @Override
-    public int hashCode() {
-        return conf.hashCode();
-    }
-}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/StepConfigurationProxy.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/StepConfigurationProxy.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/StepConfigurationProxy.java
deleted file mode 100644
index 6226900..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/StepConfigurationProxy.java
+++ /dev/null
@@ -1,50 +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.process.traversal.step.util;
-
-import org.apache.commons.configuration.Configuration;
-import org.apache.tinkerpop.gremlin.process.traversal.step.StepConfiguration;
-
-import java.io.Serializable;
-
-/**
- * @author Stephen Mallette (http://stephen.genoprime.com)
- */
-public class StepConfigurationProxy<T extends StepConfiguration> implements Serializable {
-
-    private final Configuration configuration;
-    private final Class<T> stepConfigurationClass;
-
-    public StepConfigurationProxy(final T stepConfiguration) {
-        this((Class<T>) stepConfiguration.getClass(), stepConfiguration.getConfiguration());
-    }
-
-    public StepConfigurationProxy(final Class<T> stepConfigurationClass, final Configuration configuration) {
-        this.configuration = configuration;
-        this.stepConfigurationClass = stepConfigurationClass;
-    }
-
-    public Configuration getConfiguration() {
-        return this.configuration;
-    }
-
-    public Class<T> getStepConfigurationClass() {
-        return this.stepConfigurationClass;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java
index 4253c15..1bccd7c 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java
@@ -31,9 +31,7 @@ import org.apache.tinkerpop.gremlin.process.traversal.Scope;
 import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
 import org.apache.tinkerpop.gremlin.process.traversal.TraversalStrategy;
 import org.apache.tinkerpop.gremlin.process.traversal.Traverser;
-import org.apache.tinkerpop.gremlin.process.traversal.step.StepConfiguration;
 import org.apache.tinkerpop.gremlin.process.traversal.step.TraversalOptionParent;
-import org.apache.tinkerpop.gremlin.process.traversal.step.util.DefaultStepConfiguration;
 import org.apache.tinkerpop.gremlin.process.traversal.step.util.Tree;
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.decoration.ConnectiveStrategy;
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.decoration.ElementIdStrategy;
@@ -186,7 +184,6 @@ abstract class GraphSONModule extends TinkerPopJacksonModule {
                             GraphFilterStrategy.class,
                             VertexProgramStrategy.class
                     ).forEach(strategy -> put(strategy, strategy.getSimpleName()));
-                    Arrays.asList(DefaultStepConfiguration.class).forEach(stepConfiguration-> put(stepConfiguration, stepConfiguration.getSimpleName()));
                 }});
 
         /**
@@ -237,7 +234,6 @@ abstract class GraphSONModule extends TinkerPopJacksonModule {
             addSerializer(Bytecode.Binding.class, new TraversalSerializersV3d0.BindingJacksonSerializer());
             addSerializer(Traverser.class, new TraversalSerializersV3d0.TraverserJacksonSerializer());
             addSerializer(TraversalStrategy.class, new TraversalSerializersV3d0.TraversalStrategyJacksonSerializer());
-            addSerializer(StepConfiguration.class, new TraversalSerializersV3d0.StepConfigurationJacksonSerializer());
 
             /////////////////////// DESERIALIZERS ////////////////////////////
 
@@ -304,7 +300,6 @@ abstract class GraphSONModule extends TinkerPopJacksonModule {
                     GraphFilterStrategy.class,
                     VertexProgramStrategy.class
             ).forEach(strategy -> addDeserializer(strategy, new TraversalSerializersV3d0.TraversalStrategyProxyJacksonDeserializer(strategy)));
-            Arrays.asList(DefaultStepConfiguration.class).forEach(stepConf -> addDeserializer(stepConf, new TraversalSerializersV3d0.StepConfigurationProxyJacksonDeserializer(stepConf)));
         }
 
         public static Builder build() {
@@ -403,7 +398,6 @@ abstract class GraphSONModule extends TinkerPopJacksonModule {
                             GraphFilterStrategy.class,
                             VertexProgramStrategy.class
                     ).forEach(strategy -> put(strategy, strategy.getSimpleName()));
-                    Arrays.asList(DefaultStepConfiguration.class).forEach(stepConfiguration-> put(stepConfiguration, stepConfiguration.getSimpleName()));
                 }});
 
         /**
@@ -451,7 +445,6 @@ abstract class GraphSONModule extends TinkerPopJacksonModule {
             addSerializer(Bytecode.Binding.class, new TraversalSerializersV2d0.BindingJacksonSerializer());
             addSerializer(Traverser.class, new TraversalSerializersV2d0.TraverserJacksonSerializer());
             addSerializer(TraversalStrategy.class, new TraversalSerializersV2d0.TraversalStrategyJacksonSerializer());
-            addSerializer(StepConfiguration.class, new TraversalSerializersV2d0.StepConfigurationJacksonSerializer());
 
             /////////////////////// DESERIALIZERS ////////////////////////////
 
@@ -513,7 +506,6 @@ abstract class GraphSONModule extends TinkerPopJacksonModule {
                     GraphFilterStrategy.class,
                     VertexProgramStrategy.class
             ).forEach(strategy -> addDeserializer(strategy, new TraversalSerializersV2d0.TraversalStrategyProxyJacksonDeserializer(strategy)));
-            Arrays.asList(DefaultStepConfiguration.class).forEach(stepConf -> addDeserializer(stepConf, new TraversalSerializersV2d0.StepConfigurationProxyJacksonDeserializer(stepConf)));
         }
 
         public static Builder build() {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/27cf8e55/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/TraversalSerializersV2d0.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/TraversalSerializersV2d0.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/TraversalSerializersV2d0.java
index bc105e6..040fd1d 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/TraversalSerializersV2d0.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/TraversalSerializersV2d0.java
@@ -27,8 +27,6 @@ import org.apache.tinkerpop.gremlin.process.traversal.P;
 import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
 import org.apache.tinkerpop.gremlin.process.traversal.TraversalStrategy;
 import org.apache.tinkerpop.gremlin.process.traversal.Traverser;
-import org.apache.tinkerpop.gremlin.process.traversal.step.StepConfiguration;
-import org.apache.tinkerpop.gremlin.process.traversal.step.util.StepConfigurationProxy;
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.TraversalStrategyProxy;
 import org.apache.tinkerpop.gremlin.process.traversal.util.AndP;
 import org.apache.tinkerpop.gremlin.process.traversal.util.ConnectiveP;
@@ -39,14 +37,18 @@ import org.apache.tinkerpop.shaded.jackson.core.JsonParser;
 import org.apache.tinkerpop.shaded.jackson.core.JsonProcessingException;
 import org.apache.tinkerpop.shaded.jackson.core.JsonToken;
 import org.apache.tinkerpop.shaded.jackson.databind.DeserializationContext;
+import org.apache.tinkerpop.shaded.jackson.databind.JavaType;
+import org.apache.tinkerpop.shaded.jackson.databind.JsonNode;
 import org.apache.tinkerpop.shaded.jackson.databind.SerializerProvider;
 import org.apache.tinkerpop.shaded.jackson.databind.deser.std.StdDeserializer;
 import org.apache.tinkerpop.shaded.jackson.databind.jsontype.TypeSerializer;
 import org.apache.tinkerpop.shaded.jackson.databind.ser.std.StdScalarSerializer;
 import org.apache.tinkerpop.shaded.jackson.databind.ser.std.StdSerializer;
+import org.apache.tinkerpop.shaded.jackson.databind.type.TypeFactory;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
@@ -241,23 +243,6 @@ final class TraversalSerializersV2d0 {
         }
     }
 
-    final static class StepConfigurationJacksonSerializer extends StdScalarSerializer<StepConfiguration> {
-
-        public StepConfigurationJacksonSerializer() {
-            super(StepConfiguration.class);
-        }
-
-        @Override
-        public void serialize(final StepConfiguration stepConfiguration, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
-                throws IOException {
-            jsonGenerator.writeStartObject();
-            for (final Map.Entry<Object, Object> entry : ConfigurationConverter.getMap(stepConfiguration.getConfiguration()).entrySet()) {
-                jsonGenerator.writeObjectField((String) entry.getKey(), entry.getValue());
-            }
-            jsonGenerator.writeEndObject();
-        }
-    }
-
     ///////////////////
     // DESERIALIZERS //
     //////////////////
@@ -505,18 +490,4 @@ final class TraversalSerializersV2d0 {
             return new TraversalStrategyProxy<>(this.clazz, new MapConfiguration(data));
         }
     }
-
-    final static class StepConfigurationProxyJacksonDeserializer<T extends StepConfiguration> extends AbstractObjectDeserializer<StepConfigurationProxy> {
-        private final Class<T> clazz;
-
-        public StepConfigurationProxyJacksonDeserializer(final Class<T> clazz) {
-            super(StepConfigurationProxy.class);
-            this.clazz = clazz;
-        }
-
-        @Override
-        public StepConfigurationProxy<T> createObject(final Map<String, Object> data) {
-            return new StepConfigurationProxy<>(this.clazz, new MapConfiguration(data));
-        }
-    }
 }