You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tinkerpop.apache.org by ok...@apache.org on 2015/06/15 20:33:57 UTC

[4/4] incubator-tinkerpop git commit: Have AND/OR nesting working in XMatchStep. I'm not too happy with the solution -- though, one could argue that it is semantically correct. In short, all OR patterns and all AND patterns are tried. OR can't only try o

Have AND/OR nesting working in XMatchStep. I'm not too happy with the solution -- though, one could argue that it is semantically correct. In short, all OR patterns and all AND patterns are tried. OR can't only try one and if pass, return -- it has to try all.


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

Branch: refs/heads/xmatch
Commit: a44bcaece56430da6bd326cdc59e8d6ea0dc527f
Parents: 7745775
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Mon Jun 15 12:33:50 2015 -0600
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Mon Jun 15 12:33:50 2015 -0600

----------------------------------------------------------------------
 .../process/traversal/TraversalStrategies.java  |   2 +
 .../traversal/dsl/graph/GraphTraversal.java     |  10 +-
 .../traversal/step/filter/exp/XMatchStep.java   | 119 +++++++++++--------
 .../decoration/MatchStartEndStrategy.java       | 107 +++++++++++++++++
 .../tinkergraph/structure/TinkerGraphTest.java  |  39 ++++--
 5 files changed, 210 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a44bcaec/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 352af05..50dd7f3 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
@@ -19,6 +19,7 @@
 package org.apache.tinkerpop.gremlin.process.traversal;
 
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.decoration.ConjunctionStrategy;
+import org.apache.tinkerpop.gremlin.process.traversal.strategy.decoration.MatchStartEndStrategy;
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.decoration.WhereStartEndStrategy;
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.finalization.EngineDependentStrategy;
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.finalization.LazyBarrierStrategy;
@@ -199,6 +200,7 @@ public interface TraversalStrategies extends Serializable, Cloneable {
             final TraversalStrategies coreStrategies = new DefaultTraversalStrategies();
             coreStrategies.addStrategies(
                     ConjunctionStrategy.instance(),
+                    MatchStartEndStrategy.instance(),
                     WhereStartEndStrategy.instance(),
                     EngineDependentStrategy.instance(),
                     ProfileStrategy.instance(),

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a44bcaec/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 885e8ac..8b9db86 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
@@ -119,8 +119,6 @@ import org.apache.tinkerpop.gremlin.process.traversal.step.util.HasContainer;
 import org.apache.tinkerpop.gremlin.process.traversal.step.util.NoOpBarrierStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.util.TraversalComparator;
 import org.apache.tinkerpop.gremlin.process.traversal.step.util.Tree;
-import org.apache.tinkerpop.gremlin.process.traversal.util.AndP;
-import org.apache.tinkerpop.gremlin.process.traversal.util.OrP;
 import org.apache.tinkerpop.gremlin.structure.Direction;
 import org.apache.tinkerpop.gremlin.structure.Edge;
 import org.apache.tinkerpop.gremlin.structure.Element;
@@ -614,11 +612,11 @@ public interface GraphTraversal<S, E> extends Traversal<S, E> {
         return this.asAdmin().addStep(new LambdaFilterStep<>(this.asAdmin(), predicate));
     }
 
-    public default GraphTraversal<S, E> or(final Traversal<?,?>... orTraversals) {
+    public default GraphTraversal<S, E> or(final Traversal<?, ?>... orTraversals) {
         return this.asAdmin().addStep(new OrStep(this.asAdmin(), orTraversals));
     }
 
-    public default GraphTraversal<S, E> and(final Traversal<?,?>... andTraversals) {
+    public default GraphTraversal<S, E> and(final Traversal<?, ?>... andTraversals) {
         return this.asAdmin().addStep(new AndStep(this.asAdmin(), andTraversals));
     }
 
@@ -1012,8 +1010,8 @@ public interface GraphTraversal<S, E> extends Traversal<S, E> {
 
     ////
 
-    public default GraphTraversal<S, E> xmatch(final Traversal<?,?>... andTraversals) {
-        return this.asAdmin().addStep(new XMatchStep<>(this.asAdmin(), andTraversals));
+    public default GraphTraversal<S, E> xmatch(final Traversal<?, ?>... andTraversals) {
+        return this.asAdmin().addStep(new XMatchStep<>(this.asAdmin(), XMatchStep.Conjunction.AND, andTraversals));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a44bcaec/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/exp/XMatchStep.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/exp/XMatchStep.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/exp/XMatchStep.java
index 439b94c..8296712 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/exp/XMatchStep.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/exp/XMatchStep.java
@@ -23,16 +23,14 @@ package org.apache.tinkerpop.gremlin.process.traversal.step.filter.exp;
 
 import org.apache.tinkerpop.gremlin.process.traversal.Path;
 import org.apache.tinkerpop.gremlin.process.traversal.Pop;
-import org.apache.tinkerpop.gremlin.process.traversal.Scope;
 import org.apache.tinkerpop.gremlin.process.traversal.Step;
 import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
 import org.apache.tinkerpop.gremlin.process.traversal.Traverser;
 import org.apache.tinkerpop.gremlin.process.traversal.step.TraversalParent;
 import org.apache.tinkerpop.gremlin.process.traversal.step.map.SelectOneStep;
-import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.StartStep;
+import org.apache.tinkerpop.gremlin.process.traversal.step.util.AbstractStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.util.ComputerAwareStep;
 import org.apache.tinkerpop.gremlin.process.traversal.traverser.TraverserRequirement;
-import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalHelper;
 import org.apache.tinkerpop.gremlin.structure.util.StringFactory;
 import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 
@@ -45,45 +43,33 @@ import java.util.NoSuchElementException;
 import java.util.Optional;
 import java.util.Set;
 import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
 public final class XMatchStep<S> extends ComputerAwareStep<S, S> implements TraversalParent {
 
-    private List<Traversal.Admin<Object, Object>> andTraversals = new ArrayList<>();
+    public enum Conjunction {AND, OR}
+
+    private List<Traversal.Admin<Object, Object>> conjunctionTraversals = new ArrayList<>();
     private boolean first = true;
     private Set<String> matchStartLabels = null;
+    private final Conjunction conjunction;
     private final MatchAlgorithm matchAlgorithm = new GreedyMatchAlgorithm();
 
-    public XMatchStep(final Traversal.Admin traversal, final Traversal... andTraversals) {
+    public XMatchStep(final Traversal.Admin traversal, final Conjunction conjunction, final Traversal... conjunctionTraversals) {
         super(traversal);
-        for (final Traversal andTraversal : andTraversals) {
-            //// START STEP
-            final Step<?, ?> startStep = andTraversal.asAdmin().getStartStep();
-            if (startStep instanceof StartStep && !startStep.getLabels().isEmpty()) {
-                if (startStep.getLabels().size() != 1)
-                    throw new IllegalArgumentException("The start step of a match()-traversal can must have one and only one label: " + startStep);
-                TraversalHelper.replaceStep(andTraversal.asAdmin().getStartStep(), new SelectOneStep<>(andTraversal.asAdmin(), Scope.global, Pop.head, startStep.getLabels().iterator().next()), andTraversal.asAdmin());
-            }
-            //// END STEP
-            final Step<?, ?> endStep = andTraversal.asAdmin().getEndStep();
-            if (endStep.getLabels().size() > 1)
-                throw new IllegalArgumentException("The end step of a match()-traversal can have at most one label: " + endStep);
-            final String label = endStep.getLabels().size() == 0 ? null : endStep.getLabels().iterator().next();
-            if (null != label) endStep.removeLabel(label);
-            final Step<?, ?> step = new XMatchEndStep(andTraversal.asAdmin(), label);
-            if (null != label) step.addLabel(label);
-            andTraversal.asAdmin().addStep(step);
-            this.andTraversals.add(this.integrateChild(andTraversal.asAdmin()));
-        }
+        this.conjunction = conjunction;
+        this.conjunctionTraversals = (List) Stream.of(conjunctionTraversals).map(Traversal::asAdmin).map(this::integrateChild).collect(Collectors.toList());
     }
 
     public Set<String> getMatchStartLabels() {
         if (null == this.matchStartLabels) {
             this.matchStartLabels = new HashSet<>();
-            for (final Traversal.Admin<Object, Object> andTraversal : this.andTraversals) {
-                this.matchStartLabels.addAll(andTraversal.getStartStep().getLabels());
+            for (final Traversal.Admin<Object, Object> andTraversal : this.conjunctionTraversals) {
+                this.matchStartLabels.addAll(andTraversal.getStartStep() instanceof XMatchStep ? ((XMatchStep) andTraversal.getStartStep()).getMatchStartLabels() : ((SelectOneStep) andTraversal.getStartStep()).getScopeKeys());
             }
             this.matchStartLabels = Collections.unmodifiableSet(this.matchStartLabels);
         }
@@ -91,12 +77,12 @@ public final class XMatchStep<S> extends ComputerAwareStep<S, S> implements Trav
     }
 
     public List<Traversal.Admin<Object, Object>> getGlobalChildren() {
-        return Collections.unmodifiableList(this.andTraversals);
+        return Collections.unmodifiableList(this.conjunctionTraversals);
     }
 
     @Override
     public String toString() {
-        return StringFactory.stepString(this, this.andTraversals);
+        return StringFactory.stepString(this, this.conjunction, this.conjunctionTraversals);
     }
 
     @Override
@@ -108,9 +94,9 @@ public final class XMatchStep<S> extends ComputerAwareStep<S, S> implements Trav
     @Override
     public XMatchStep<S> clone() {
         final XMatchStep<S> clone = (XMatchStep<S>) super.clone();
-        clone.andTraversals = new ArrayList<>();
-        for (final Traversal.Admin<Object, Object> traversal : this.andTraversals) {
-            clone.andTraversals.add(clone.integrateChild(traversal.clone()));
+        clone.conjunctionTraversals = new ArrayList<>();
+        for (final Traversal.Admin<Object, Object> traversal : this.conjunctionTraversals) {
+            clone.conjunctionTraversals.add(clone.integrateChild(traversal.clone()));
         }
         // TODO: does it need to clone the match algorithm?
         return clone;
@@ -121,17 +107,24 @@ public final class XMatchStep<S> extends ComputerAwareStep<S, S> implements Trav
         while (true) {
             Traverser.Admin traverser = null;
             if (this.first) {
-                this.matchAlgorithm.initialize(this.andTraversals);
+                this.matchAlgorithm.initialize(this.conjunction, this.conjunctionTraversals);
                 this.first = false;
             } else {
-                for (final Traversal.Admin<?, ?> andTraversal : this.andTraversals) {
-                    if (andTraversal.hasNext()) {
-                        traverser = andTraversal.getEndStep().next().asAdmin();
+                for (final Traversal.Admin<?, ?> conjunctionTraversal : this.conjunctionTraversals) {
+                    if (conjunctionTraversal.hasNext()) {
+                        traverser = conjunctionTraversal.getEndStep().next().asAdmin();
                         break;
                     }
                 }
             }
-            if (null == traverser) traverser = this.starts.next();
+            if (null == traverser) {
+                traverser = this.starts.next();
+                if (this.conjunction == Conjunction.OR) {
+                    for (final Traversal.Admin<?, ?> conjunctionTraversal : this.conjunctionTraversals) {
+                        conjunctionTraversal.addStart(traverser.split());
+                    }
+                }
+            }
             final Optional<Traversal.Admin<Object, Object>> optional = this.matchAlgorithm.apply(traverser); // determine which sub-pattern the traverser should try next
             if (optional.isPresent()) {
                 final Traversal.Admin<Object, Object> traversal = optional.get();
@@ -148,7 +141,7 @@ public final class XMatchStep<S> extends ComputerAwareStep<S, S> implements Trav
     @Override
     protected Iterator<Traverser<S>> computerAlgorithm() throws NoSuchElementException {
         if (this.first) {
-            this.matchAlgorithm.initialize(this.andTraversals);
+            this.matchAlgorithm.initialize(this.conjunction, this.conjunctionTraversals);
             this.first = false;
         }
         final Traverser.Admin traverser = this.starts.next();
@@ -169,7 +162,7 @@ public final class XMatchStep<S> extends ComputerAwareStep<S, S> implements Trav
 
     @Override
     public int hashCode() {
-        return super.hashCode() ^ this.andTraversals.hashCode();
+        return super.hashCode() ^ this.conjunctionTraversals.hashCode();
     }
 
     @Override
@@ -179,13 +172,15 @@ public final class XMatchStep<S> extends ComputerAwareStep<S, S> implements Trav
 
     //////////////////////////////
 
-    public class XMatchEndStep extends EndStep {
+    public static class XMatchEndStep<S> extends AbstractStep<S, S> {
 
         private final String matchKey;
+        private final String matchStepId;
 
-        public XMatchEndStep(final Traversal.Admin traversal, final String matchKey) {
+        public XMatchEndStep(final Traversal.Admin traversal, final XMatchStep matchStep, final String matchKey) {
             super(traversal);
             this.matchKey = matchKey;
+            this.matchStepId = matchStep.getId();
         }
 
         @Override
@@ -194,23 +189,28 @@ public final class XMatchStep<S> extends ComputerAwareStep<S, S> implements Trav
                 final Traverser.Admin<S> start = this.starts.next();
                 // no end label
                 if (null == this.matchKey) {
-                    if (this.traverserStepIdSetByChild) start.setStepId(XMatchStep.this.getId());
+                    if (this.traverserStepIdSetByChild) start.setStepId(this.matchStepId);
                     return start;
                 }
                 // side-effect check
                 final Optional<S> optional = start.getSideEffects().get(this.matchKey);
                 if (optional.isPresent() && start.get().equals(optional.get())) {
-                    if (this.traverserStepIdSetByChild) start.setStepId(XMatchStep.this.getId());
+                    if (this.traverserStepIdSetByChild) start.setStepId(this.matchStepId);
                     return start;
                 }
                 // path check
                 final Path path = start.path();
                 if (!path.hasLabel(this.matchKey) || start.get().equals(path.getSingle(Pop.head, this.matchKey))) {
-                    if (this.traverserStepIdSetByChild) start.setStepId(XMatchStep.this.getId());
+                    if (this.traverserStepIdSetByChild) start.setStepId(this.matchStepId);
                     return start;
                 }
             }
         }
+
+        @Override
+        public String toString() {
+            return StringFactory.stepString(this, this.matchKey, this.matchStepId);
+        }
     }
 
 
@@ -228,7 +228,7 @@ public final class XMatchStep<S> extends ComputerAwareStep<S, S> implements Trav
                 return Collections.emptySet();
         }
 
-        public void initialize(final List<Traversal.Admin<Object, Object>> traversals);
+        public void initialize(final Conjunction conjunction, final List<Traversal.Admin<Object, Object>> traversals);
     }
 
     public static class GreedyMatchAlgorithm implements MatchAlgorithm {
@@ -236,10 +236,12 @@ public final class XMatchStep<S> extends ComputerAwareStep<S, S> implements Trav
         private List<Traversal.Admin<Object, Object>> traversals;
         private List<String> traversalLabels = new ArrayList<>();
         private List<Set<String>> startLabels = new ArrayList<>();
+        private Conjunction conjunction;
 
         @Override
-        public void initialize(final List<Traversal.Admin<Object, Object>> traversals) {
+        public void initialize(final Conjunction conjunction, final List<Traversal.Admin<Object, Object>> traversals) {
             this.traversals = traversals;
+            this.conjunction = conjunction;
             for (final Traversal.Admin<Object, Object> traversal : this.traversals) {
                 this.traversalLabels.add(traversal.getStartStep().getId());
                 this.startLabels.add(MatchAlgorithm.getStartLabels(traversal));
@@ -249,12 +251,31 @@ public final class XMatchStep<S> extends ComputerAwareStep<S, S> implements Trav
         @Override
         public Optional<Traversal.Admin<Object, Object>> apply(final Traverser.Admin<Object> traverser) {
             final Path path = traverser.path();
-            for (int i = 0; i < this.traversals.size(); i++) {
-                if (this.startLabels.get(i).stream().filter(path::hasLabel).findAny().isPresent() && !path.hasLabel(this.traversalLabels.get(i))) {
-                    return Optional.of(this.traversals.get(i));
+            if (Conjunction.AND == this.conjunction) {
+                int count = 0;
+                for (int i = 0; i < this.traversals.size(); i++) {
+                    count++;
+                    if (this.startLabels.get(i).stream().filter(path::hasLabel).findAny().isPresent() && !path.hasLabel(this.traversalLabels.get(i))) {
+                        return Optional.of(this.traversals.get(i));
+                    }
+                }
+                if (count != this.traversals.size())
+                    throw new IllegalStateException("The provided match and-pattern is unsolvable: " + this.traversals);
+                return Optional.empty();
+            } else {
+                int count = 0;
+                for (int i = 0; i < this.traversals.size(); i++) {
+                    count++;
+                    if (path.hasLabel(this.traversalLabels.get(i)))
+                        return Optional.empty();
+                    else if (this.startLabels.get(i).stream().filter(path::hasLabel).findAny().isPresent()) {
+                        return Optional.of(this.traversals.get(i));
+                    }
                 }
+                if (count == 0)
+                    throw new IllegalStateException("The provided match or-pattern is unsolvable: " + this.traversals);
+                return Optional.empty();
             }
-            return Optional.empty();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a44bcaec/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/decoration/MatchStartEndStrategy.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/decoration/MatchStartEndStrategy.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/decoration/MatchStartEndStrategy.java
new file mode 100644
index 0000000..f7fd400
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/decoration/MatchStartEndStrategy.java
@@ -0,0 +1,107 @@
+/*
+ *
+ *  * 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.decoration;
+
+import org.apache.tinkerpop.gremlin.process.traversal.Pop;
+import org.apache.tinkerpop.gremlin.process.traversal.Scope;
+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.TraversalParent;
+import org.apache.tinkerpop.gremlin.process.traversal.step.filter.AndStep;
+import org.apache.tinkerpop.gremlin.process.traversal.step.filter.ConjunctionStep;
+import org.apache.tinkerpop.gremlin.process.traversal.step.filter.exp.XMatchStep;
+import org.apache.tinkerpop.gremlin.process.traversal.step.map.SelectOneStep;
+import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.StartStep;
+import org.apache.tinkerpop.gremlin.process.traversal.strategy.AbstractTraversalStrategy;
+import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalHelper;
+
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public class MatchStartEndStrategy extends AbstractTraversalStrategy<TraversalStrategy.DecorationStrategy> implements TraversalStrategy.DecorationStrategy {
+
+    private static final MatchStartEndStrategy INSTANCE = new MatchStartEndStrategy();
+
+    private static final Set<Class<? extends DecorationStrategy>> PRIORS = new HashSet<>();
+
+    static {
+        PRIORS.add(ConjunctionStrategy.class);
+    }
+
+
+    private MatchStartEndStrategy() {
+    }
+
+    private static void addSelectOneStartStep(final XMatchStep<?> matchStep, final Traversal.Admin<?, ?> traversal) {
+        for (final Step<?, ?> conjunction : TraversalHelper.getStepsOfAssignableClass(ConjunctionStep.class, traversal)) {
+            final XMatchStep xMatchStep = new XMatchStep(traversal, conjunction instanceof AndStep ? XMatchStep.Conjunction.AND : XMatchStep.Conjunction.OR, ((ConjunctionStep<?>) conjunction).getLocalChildren().toArray(new Traversal[((ConjunctionStep<?>) conjunction).getLocalChildren().size()]));
+            TraversalHelper.replaceStep(conjunction, xMatchStep, traversal);
+            addSelectOneStartStep(xMatchStep, traversal);
+        }
+
+        // START STEP to SelectOneStep
+        final Step<?, ?> startStep = traversal.getStartStep();
+        if (startStep instanceof StartStep && !startStep.getLabels().isEmpty()) {
+            if (startStep.getLabels().size() > 1)
+                throw new IllegalArgumentException("The start step of a match()-traversal can only have one label: " + startStep);
+            TraversalHelper.replaceStep(traversal.getStartStep(), new SelectOneStep<>(traversal, Scope.global, Pop.head, startStep.getLabels().iterator().next()), traversal);
+        }
+        // END STEP to XMatchStep
+        final Step<?, ?> endStep = traversal.getEndStep();
+        if (!(endStep instanceof XMatchStep.XMatchEndStep)) {
+            if (endStep.getLabels().size() > 1)
+                throw new IllegalArgumentException("The end step of a match()-traversal can have at most one label: " + endStep);
+            final String label = endStep.getLabels().size() == 0 ? null : endStep.getLabels().iterator().next();
+            if (null != label) endStep.removeLabel(label);
+            final Step<?, ?> xMatchEndStep = new XMatchStep.XMatchEndStep(traversal, matchStep, label);
+            if (null != label) xMatchEndStep.addLabel(label);
+            traversal.asAdmin().addStep(xMatchEndStep);
+        }
+
+
+        for (final Step<?, ?> step : traversal.getSteps()) {
+            if (step instanceof TraversalParent && !(step instanceof XMatchStep)) {
+                ((TraversalParent) step).getGlobalChildren().forEach(t -> MatchStartEndStrategy.addSelectOneStartStep(matchStep, t));
+                ((TraversalParent) step).getLocalChildren().forEach(t -> MatchStartEndStrategy.addSelectOneStartStep(matchStep, t));
+
+            }
+        }
+    }
+
+    @Override
+    public void apply(final Traversal.Admin<?, ?> traversal) {
+        TraversalHelper.getStepsOfClass(XMatchStep.class, traversal).forEach(matchStep -> ((XMatchStep<?>) matchStep).getGlobalChildren().forEach(t -> MatchStartEndStrategy.addSelectOneStartStep(matchStep, t)));
+    }
+
+    @Override
+    public Set<Class<? extends DecorationStrategy>> applyPrior() {
+        return PRIORS;
+    }
+
+    public static MatchStartEndStrategy instance() {
+        return INSTANCE;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/a44bcaec/tinkergraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraphTest.java
----------------------------------------------------------------------
diff --git a/tinkergraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraphTest.java b/tinkergraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraphTest.java
index abb4baa..494f46d 100644
--- a/tinkergraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraphTest.java
+++ b/tinkergraph-gremlin/src/test/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraphTest.java
@@ -36,7 +36,6 @@ import java.util.List;
 import java.util.Set;
 import java.util.function.Supplier;
 
-import static org.apache.tinkerpop.gremlin.process.traversal.P.neq;
 import static org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__.*;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -159,17 +158,33 @@ public class TinkerGraphTest {
     @Ignore
     public void testPlay5() throws Exception {
         GraphTraversalSource g = TinkerFactory.createModern().traversal(GraphTraversalSource.standard());
-        g.V().as("a").xmatch(
-                as("a").out("knows").as("b"),
-                as("a").out("created").as("c"),
-                as("b").out("created").as("c"),
-                as("c").in("created").as("d"),
-                as("d").where(neq("a")).where(neq("b")),
-                as("b").out("created").has("name", "ripple"))
-                .select(Pop.head, "a", "b", "c", "d").forEachRemaining(System.out::println);
-
-        System.out.println(g.V().out().and().in().iterate());
-        System.out.println(g.V().as("a","b").where(as("a").out().and().in().as("b")).iterate());
+        final Supplier<Traversal<?, ?>> traversal = () -> g.V().as("a").xmatch(
+                as("a").out("created").as("b"),
+                or(
+                        as("a").out("knows").as("c"),
+                        or(
+                                as("a").out("created").has("name", "ripple"),
+                                as("a").out().out()
+                        )
+                ))
+                .select(Pop.head).by("name");
+        /*
+                g.V().as("a").xmatch(
+                        as("a").out("knows").as("b"),
+                        as("a").out("created").as("c"),
+                        as("b").out("created").as("c"),
+                        as("c").in("created").as("d"),
+                        as("d").where(neq("a")).where(neq("b")),
+                        as("b").out("created").has("name", "ripple"))
+                        .select(Pop.head, "a", "b", "c", "d").forEachRemaining(System.out::println);
+                        */
+
+        System.out.println(traversal.get());
+        System.out.println(traversal.get().iterate());
+        traversal.get().forEachRemaining(System.out::println);
+
+        //System.out.println(g.V().and(out("created"),or(out("knows"),out("created").has("name","ripple"))).values("name").iterate());
+        //g.V().and(out("created"),or(out("knows"),out("created").has("name","ripple"))).values("name").forEachRemaining(System.out::println);
     }
 
     @Test