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 2015/09/03 19:51:55 UTC

[28/50] [abbrv] incubator-tinkerpop git commit: Renamed ComputerVerifcationStrategy to VerificationStrategy and all VerificationStrategies use it. Added StandardVerificationStrategy which ensures that certain traversal patterns are not allowed. The first

Renamed ComputerVerifcationStrategy to VerificationStrategy and all VerificationStrategies use it. Added StandardVerificationStrategy which ensures that certain traversal patterns are not allowed. The first up -- a reducing barrier step inside of repeat(). TINKERPOP3-780


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

Branch: refs/heads/blvp
Commit: a20d06083aa7c3060a7c91a3409cb9e28993fdb6
Parents: 6902e68
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Thu Aug 27 14:15:24 2015 -0600
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Thu Aug 27 14:15:24 2015 -0600

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |  2 +
 .../process/traversal/TraversalStrategies.java  |  4 +-
 .../ComputerVerificationException.java          | 36 ---------
 .../ComputerVerificationStrategy.java           | 18 ++---
 .../verification/LambdaRestrictionStrategy.java |  6 +-
 .../strategy/verification/ReadOnlyStrategy.java | 15 ++--
 .../StandardVerificationStrategy.java           | 51 ++++++++++++
 .../verification/VerificationException.java     | 36 +++++++++
 .../ComputerVerificationStrategyTest.java       |  4 +-
 .../LambdaRestrictionStrategyTest.java          |  4 +-
 .../verification/ReadOnlyStrategyTest.java      | 10 +--
 .../StandardVerificationStrategyTest.java       | 82 ++++++++++++++++++++
 .../gremlin/process/GremlinProcessRunner.java   |  4 +-
 13 files changed, 205 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a20d0608/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index ec39683..4ee6bd6 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -25,6 +25,8 @@ image::http://www.tinkerpop.com/docs/current/images/gremlin-hindu.png[width=225]
 TinkerPop 3.0.1 (NOT OFFICIALLY RELEASED YET)
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
+* Renamed `ComputerVerificationStrategy` to `VerificationStrategy` so all the verification strategies can use it.
+* Added `StandardVerificationStrategy` that throws exceptions for illegal traversal patterns on the standard engine (which extends to `GraphComputer`).
 * Clarified semantics of `Transaction.close()` in unit tests - now refers only to closing the current transaction in the current thread.
 * `Neo4jGraph` no longer uses `OptOut` on `TransactionTest.shouldRollbackOnCloseWhenConfigured` (formerly `shouldRollbackOnShutdownWhenConfigured`)
 * Gremlin Server initialization scripts can now return a `Map` of values that will become global bindings for the server.

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a20d0608/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/TraversalStrategies.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/TraversalStrategies.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/TraversalStrategies.java
index bf56d31..6192838 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/TraversalStrategies.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/TraversalStrategies.java
@@ -28,6 +28,7 @@ import org.apache.tinkerpop.gremlin.process.traversal.strategy.optimization.Inci
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.optimization.MatchPredicateStrategy;
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.optimization.RangeByIsCountStrategy;
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.verification.ComputerVerificationStrategy;
+import org.apache.tinkerpop.gremlin.process.traversal.strategy.verification.StandardVerificationStrategy;
 import org.apache.tinkerpop.gremlin.process.traversal.traverser.TraverserGeneratorFactory;
 import org.apache.tinkerpop.gremlin.process.traversal.util.DefaultTraversalStrategies;
 import org.apache.tinkerpop.gremlin.structure.Graph;
@@ -204,7 +205,8 @@ public interface TraversalStrategies extends Serializable, Cloneable {
                     IdentityRemovalStrategy.instance(),
                     MatchPredicateStrategy.instance(),
                     RangeByIsCountStrategy.instance(),
-                    ComputerVerificationStrategy.instance());
+                    ComputerVerificationStrategy.instance(),
+                    StandardVerificationStrategy.instance());
             //LambdaRestrictionStrategy.instance(),
             //LazyBarrierStrategy.instance(),
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a20d0608/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationException.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationException.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationException.java
deleted file mode 100644
index 34c502b..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationException.java
+++ /dev/null
@@ -1,36 +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.strategy.verification;
-
-import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
-
-/**
- * @author Daniel Kuppitz (http://gremlin.guru)
- */
-public class ComputerVerificationException extends IllegalStateException {
-
-    private final Traversal traversal;
-
-    public ComputerVerificationException(final String message, final Traversal traversal) {
-        super(message);
-        this.traversal = traversal;
-    }
-
-    public Traversal getTraversal() { return this.traversal; }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a20d0608/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategy.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategy.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategy.java
index b610215..a56afd4 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategy.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategy.java
@@ -78,46 +78,46 @@ public final class ComputerVerificationStrategy extends AbstractTraversalStrateg
 
         if (traversal.getParent() instanceof EmptyStep) {
             if (!(traversal.getStartStep() instanceof GraphStep))
-                throw new ComputerVerificationException("GraphComputer does not support traversals starting from a non-GraphStep: " + traversal.getStartStep(), traversal);
+                throw new VerificationException("GraphComputer does not support traversals starting from a non-GraphStep: " + traversal.getStartStep(), traversal);
             ///
             if (endStep instanceof CollectingBarrierStep && endStep instanceof TraversalParent) {
                 if (((TraversalParent) endStep).getLocalChildren().stream().filter(t ->
                         !(t instanceof IdentityTraversal) &&
                                 !(t instanceof ConstantTraversal) &&  // for SampleStep
                                 !(t instanceof TokenTraversal && ((TokenTraversal) t).getToken().equals(T.id))).findAny().isPresent())
-                    throw new ComputerVerificationException("A final CollectingBarrierStep can not process an element beyond its id: " + endStep, traversal);
+                    throw new VerificationException("A final CollectingBarrierStep can not process an element beyond its id: " + endStep, traversal);
             }
             ///
             if (endStep instanceof RangeGlobalStep || endStep instanceof TailGlobalStep || endStep instanceof DedupGlobalStep)
                 ((Bypassing) endStep).setBypass(true);
             if (endStep instanceof DedupGlobalStep && !((DedupGlobalStep) endStep).getScopeKeys().isEmpty())
-                throw new ComputerVerificationException("Path history de-duplication is not possible in GraphComputer:" + endStep, traversal);
+                throw new VerificationException("Path history de-duplication is not possible in GraphComputer:" + endStep, traversal);
         }
 
         for (final Step<?, ?> step : traversal.getSteps()) {
             if ((step instanceof ReducingBarrierStep || step instanceof SupplyingBarrierStep || step instanceof OrderGlobalStep || step instanceof RangeGlobalStep || step instanceof TailGlobalStep || step instanceof DedupGlobalStep) && (step != endStep || !(traversal.getParent() instanceof EmptyStep)))
-                throw new ComputerVerificationException("Global traversals on GraphComputer may not contain mid-traversal barriers: " + step, traversal);
+                throw new VerificationException("Global traversals on GraphComputer may not contain mid-traversal barriers: " + step, traversal);
 
             if (step instanceof DedupGlobalStep && !((DedupGlobalStep) step).getLocalChildren().isEmpty())
-                throw new ComputerVerificationException("Global traversals on GraphComputer may not contain by()-projecting de-duplication steps: " + step, traversal);
+                throw new VerificationException("Global traversals on GraphComputer may not contain by()-projecting de-duplication steps: " + step, traversal);
 
             if (step instanceof TraversalParent) {
                 final Optional<Traversal.Admin<Object, Object>> traversalOptional = ((TraversalParent) step).getLocalChildren().stream()
                         .filter(t -> !TraversalHelper.isLocalStarGraph(t.asAdmin()))
                         .findAny();
                 if (traversalOptional.isPresent())
-                    throw new ComputerVerificationException("Local traversals on GraphComputer may not traverse past the local star-graph: " + traversalOptional.get(), traversal);
+                    throw new VerificationException("Local traversals on GraphComputer may not traverse past the local star-graph: " + traversalOptional.get(), traversal);
             }
 
             if ((step instanceof WherePredicateStep && ((WherePredicateStep) step).getStartKey().isPresent()) ||
                     (step instanceof WhereTraversalStep && TraversalHelper.getVariableLocations(((WhereTraversalStep<?>) step).getLocalChildren().get(0)).contains(Scoping.Variable.START)))
-                throw new ComputerVerificationException("A where()-step that has a start variable is not allowed because the variable value is retrieved from the path: " + step, traversal);
+                throw new VerificationException("A where()-step that has a start variable is not allowed because the variable value is retrieved from the path: " + step, traversal);
 
             if (UNSUPPORTED_STEPS.stream().filter(c -> c.isAssignableFrom(step.getClass())).findFirst().isPresent())
-                throw new ComputerVerificationException("The following step is currently not supported by GraphComputer traversals: " + step, traversal);
+                throw new VerificationException("The following step is currently not supported by GraphComputer traversals: " + step, traversal);
 
             if (step instanceof PathProcessor && ((PathProcessor) step).getMaxRequirement() != PathProcessor.ElementRequirement.ID)
-                throw new ComputerVerificationException("The following path processor step requires more than the element id: " + step + " requires " + ((PathProcessor) step).getMaxRequirement(), traversal);
+                throw new VerificationException("The following path processor step requires more than the element id: " + step + " requires " + ((PathProcessor) step).getMaxRequirement(), traversal);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a20d0608/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/LambdaRestrictionStrategy.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/LambdaRestrictionStrategy.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/LambdaRestrictionStrategy.java
index 837b6db..79c0511 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/LambdaRestrictionStrategy.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/LambdaRestrictionStrategy.java
@@ -53,14 +53,14 @@ public final class LambdaRestrictionStrategy extends AbstractTraversalStrategy<T
     @Override
     public void apply(final Traversal.Admin<?, ?> traversal) {
         if (traversal instanceof LambdaHolder)
-            throw new IllegalStateException("The provided traversal is a lambda traversal: " + traversal);
+            throw new VerificationException("The provided traversal is a lambda traversal: ", traversal);
         for (final Step<?, ?> step : traversal.getSteps()) {
             if (step instanceof LambdaHolder)
-                throw new IllegalStateException("The provided traversal contains a lambda step: " + step);
+                throw new VerificationException("The provided traversal contains a lambda step: " + step, traversal);
             if (step instanceof ComparatorHolder) {
                 for (final Comparator<?> comparator : ((ComparatorHolder<?>) step).getComparators()) {
                     if (comparator instanceof LambdaHolder || comparator.toString().contains("$$Lambda$"))
-                        throw new IllegalStateException("The provided step contains a lambda comparator: " + step);
+                        throw new VerificationException("The provided step contains a lambda comparator: " + step, traversal);
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a20d0608/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ReadOnlyStrategy.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ReadOnlyStrategy.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ReadOnlyStrategy.java
index 2cd78a3..bf7fd09 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ReadOnlyStrategy.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ReadOnlyStrategy.java
@@ -18,6 +18,7 @@
  */
 package org.apache.tinkerpop.gremlin.process.traversal.strategy.verification;
 
+import org.apache.tinkerpop.gremlin.process.traversal.Step;
 import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
 import org.apache.tinkerpop.gremlin.process.traversal.TraversalStrategy;
 import org.apache.tinkerpop.gremlin.process.traversal.step.Mutating;
@@ -28,10 +29,10 @@ import org.apache.tinkerpop.gremlin.process.traversal.strategy.AbstractTraversal
  *
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  * @example <pre>
- * __.out().addE()                 // throws an IllegalStateException
- * __.addV()                       // throws an IllegalStateException
- * __.property(key,value)          // throws an IllegalStateException
- * __.out().drop()                 // throws an IllegalStateException
+ * __.out().addE()                 // throws an VerificationException
+ * __.addV()                       // throws an VerificationException
+ * __.property(key,value)          // throws an VerificationException
+ * __.out().drop()                 // throws an VerificationException
  * </pre>
  */
 public final class ReadOnlyStrategy extends AbstractTraversalStrategy<TraversalStrategy.VerificationStrategy> implements TraversalStrategy.VerificationStrategy {
@@ -43,8 +44,10 @@ public final class ReadOnlyStrategy extends AbstractTraversalStrategy<TraversalS
 
     @Override
     public void apply(final Traversal.Admin<?, ?> traversal) {
-        if (traversal.getSteps().stream().anyMatch(step -> step instanceof Mutating))
-            throw new IllegalStateException("The provided traversal has a mutating step and thus is not read only: " + traversal);
+        for (final Step step : traversal.getSteps()) {
+            if (step instanceof Mutating)
+                throw new VerificationException("The provided traversal has a mutating step and thus is not read only: " + step, traversal);
+        }
     }
 
     public static ReadOnlyStrategy instance() {

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a20d0608/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/StandardVerificationStrategy.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/StandardVerificationStrategy.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/StandardVerificationStrategy.java
new file mode 100644
index 0000000..6ba5216
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/StandardVerificationStrategy.java
@@ -0,0 +1,51 @@
+/*
+ *
+ *  * 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.strategy.verification;
+
+import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
+import org.apache.tinkerpop.gremlin.process.traversal.TraversalStrategy;
+import org.apache.tinkerpop.gremlin.process.traversal.step.branch.RepeatStep;
+import org.apache.tinkerpop.gremlin.process.traversal.step.util.ReducingBarrierStep;
+import org.apache.tinkerpop.gremlin.process.traversal.strategy.AbstractTraversalStrategy;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public final class StandardVerificationStrategy extends AbstractTraversalStrategy<TraversalStrategy.VerificationStrategy> implements TraversalStrategy.VerificationStrategy {
+
+    private static final StandardVerificationStrategy INSTANCE = new StandardVerificationStrategy();
+
+    private StandardVerificationStrategy() {
+    }
+
+    @Override
+    public void apply(final Traversal.Admin<?, ?> traversal) {
+        traversal.getSteps().forEach(step -> {
+            if (step instanceof ReducingBarrierStep && step.getTraversal().getParent() instanceof RepeatStep)
+                throw new VerificationException("The direct parent of a ReducingBarrierStep can not be a RepeatStep: " + step, traversal);
+        });
+    }
+
+    public static StandardVerificationStrategy instance() {
+        return INSTANCE;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a20d0608/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/VerificationException.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/VerificationException.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/VerificationException.java
new file mode 100644
index 0000000..0ebb25d
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/VerificationException.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.tinkerpop.gremlin.process.traversal.strategy.verification;
+
+import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
+
+/**
+ * @author Daniel Kuppitz (http://gremlin.guru)
+ */
+public class VerificationException extends IllegalStateException {
+
+    private final Traversal traversal;
+
+    public VerificationException(final String message, final Traversal traversal) {
+        super(message);
+        this.traversal = traversal;
+    }
+
+    public Traversal getTraversal() { return this.traversal; }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a20d0608/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategyTest.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategyTest.java b/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategyTest.java
index 07d7caf..42f2ff0 100644
--- a/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategyTest.java
+++ b/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategyTest.java
@@ -68,7 +68,7 @@ public class ComputerVerificationStrategyTest {
     public Traversal traversal;
 
     @Test
-    public void shouldNotAllowLambdaSteps() {
+    public void shouldBeVerifiedIllegal() {
         try {
             final TraversalStrategies strategies = new DefaultTraversalStrategies();
             strategies.addStrategies(ComputerVerificationStrategy.instance());
@@ -76,7 +76,7 @@ public class ComputerVerificationStrategyTest {
             traversal.asAdmin().setEngine(this.traversalEngine);
             traversal.asAdmin().applyStrategies();
             fail("The strategy should not allow lambdas: " + this.traversal);
-        } catch (IllegalStateException ise) {
+        } catch (VerificationException ise) {
            assertTrue(true);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a20d0608/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/LambdaRestrictionStrategyTest.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/LambdaRestrictionStrategyTest.java b/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/LambdaRestrictionStrategyTest.java
index 09325e0..6a0dcdd 100644
--- a/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/LambdaRestrictionStrategyTest.java
+++ b/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/LambdaRestrictionStrategyTest.java
@@ -65,7 +65,7 @@ public class LambdaRestrictionStrategyTest {
     public Traversal traversal;
 
     @Test
-    public void shouldNotAllowLambdaSteps() {
+    public void shouldBeVerifiedIllegal() {
         try {
             final TraversalStrategies strategies = new DefaultTraversalStrategies();
             strategies.addStrategies(LambdaRestrictionStrategy.instance());
@@ -73,7 +73,7 @@ public class LambdaRestrictionStrategyTest {
             traversal.asAdmin().setEngine(StandardTraversalEngine.instance());
             traversal.asAdmin().applyStrategies();
             fail("The strategy should not allow lambdas: " + this.traversal);
-        } catch (IllegalStateException ise) {
+        } catch (VerificationException ise) {
             assertTrue(ise.getMessage().contains("lambda"));
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a20d0608/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ReadOnlyStrategyTest.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ReadOnlyStrategyTest.java b/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ReadOnlyStrategyTest.java
index 41ffb2a..0be9611 100644
--- a/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ReadOnlyStrategyTest.java
+++ b/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ReadOnlyStrategyTest.java
@@ -20,8 +20,6 @@ package org.apache.tinkerpop.gremlin.process.traversal.strategy.verification;
 
 import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
 import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.DefaultGraphTraversal;
-import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__;
-import org.apache.tinkerpop.gremlin.process.traversal.strategy.verification.ReadOnlyStrategy;
 import org.apache.tinkerpop.gremlin.structure.Direction;
 import org.apache.tinkerpop.gremlin.structure.VertexProperty;
 import org.apache.tinkerpop.gremlin.structure.util.empty.EmptyGraph;
@@ -31,7 +29,7 @@ import org.junit.runners.Parameterized;
 
 import java.util.Arrays;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 /**
@@ -65,12 +63,12 @@ public class ReadOnlyStrategyTest {
     public Traversal traversal;
 
     @Test
-    public void shouldPreventMutatingStepsFromBeingInTheTraversal() {
+    public void shouldBeVerifiedIllegal() {
         try {
             ReadOnlyStrategy.instance().apply(this.traversal.asAdmin());
             fail("The strategy should have found a mutating step.");
-        } catch (IllegalStateException ise) {
-            assertEquals("The provided traversal has a mutating step and thus is not read only: " + this.traversal, ise.getMessage());
+        } catch (VerificationException ise) {
+            assertTrue(ise.getMessage().startsWith("The provided traversal has a mutating step and thus is not read only: "));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a20d0608/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/StandardVerificationStrategyTest.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/StandardVerificationStrategyTest.java b/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/StandardVerificationStrategyTest.java
new file mode 100644
index 0000000..05dec36
--- /dev/null
+++ b/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/StandardVerificationStrategyTest.java
@@ -0,0 +1,82 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  * http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ */
+
+package org.apache.tinkerpop.gremlin.process.traversal.strategy.verification;
+
+import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
+import org.apache.tinkerpop.gremlin.process.traversal.TraversalEngine;
+import org.apache.tinkerpop.gremlin.process.traversal.TraversalStrategies;
+import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__;
+import org.apache.tinkerpop.gremlin.process.traversal.util.DefaultTraversalStrategies;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+@RunWith(Parameterized.class)
+public class StandardVerificationStrategyTest {
+
+    private TraversalEngine traversalEngine;
+
+    @Before
+    public void setup() {
+        this.traversalEngine = mock(TraversalEngine.class);
+        when(this.traversalEngine.getType()).thenReturn(TraversalEngine.Type.COMPUTER);
+    }
+
+    @Parameterized.Parameters(name = "{0}")
+    public static Iterable<Object[]> data() {
+        return Arrays.asList(new Object[][]{
+                {"__.repeat(out().fold().unfold()).times(2)", __.repeat(__.out().fold().unfold()).times(2)},
+                {"__.repeat(sum()).times(2)", __.repeat(__.sum()).times(2)},
+        });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public String name;
+
+    @Parameterized.Parameter(value = 1)
+    public Traversal traversal;
+
+    @Test
+    public void shouldBeVerifiedIllegal() {
+        try {
+            final TraversalStrategies strategies = new DefaultTraversalStrategies();
+            strategies.addStrategies(ComputerVerificationStrategy.instance());
+            traversal.asAdmin().setStrategies(strategies);
+            traversal.asAdmin().setEngine(this.traversalEngine);
+            traversal.asAdmin().applyStrategies();
+            fail("The strategy should not allow lambdas: " + this.traversal);
+        } catch (IllegalStateException ise) {
+            assertTrue(true);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a20d0608/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/GremlinProcessRunner.java
----------------------------------------------------------------------
diff --git a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/GremlinProcessRunner.java b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/GremlinProcessRunner.java
index e7ab77e..35ebea0 100644
--- a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/GremlinProcessRunner.java
+++ b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/GremlinProcessRunner.java
@@ -18,7 +18,7 @@
  */
 package org.apache.tinkerpop.gremlin.process;
 
-import org.apache.tinkerpop.gremlin.process.traversal.strategy.verification.ComputerVerificationException;
+import org.apache.tinkerpop.gremlin.process.traversal.strategy.verification.VerificationException;
 import org.junit.internal.AssumptionViolatedException;
 import org.junit.internal.runners.model.EachTestNotifier;
 import org.junit.runner.Description;
@@ -68,7 +68,7 @@ public class GremlinProcessRunner extends BlockJUnit4ClassRunner {
     private static boolean validateForGraphComputer(final Throwable e) {
         Throwable ex = e;
         while (ex != null) {
-            if (ex instanceof ComputerVerificationException)
+            if (ex instanceof VerificationException)
                 return true;
             else if (ex instanceof NotSerializableException)
                 return true;