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/16 00:08:27 UTC

[26/31] incubator-tinkerpop git commit: XMatchStep OR/AND nesting works for both Computer and Standard --- code has been greatly simplified.

XMatchStep OR/AND nesting works for both Computer and Standard --- code has been greatly simplified.


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

Branch: refs/heads/master
Commit: 59d2804590a6caa5934873e94c82367d3205f924
Parents: a44bcae
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Mon Jun 15 13:35:15 2015 -0600
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Mon Jun 15 13:35:15 2015 -0600

----------------------------------------------------------------------
 .../process/traversal/TraversalStrategies.java  |   3 -
 .../traversal/step/filter/exp/XMatchStep.java   | 172 +++++++++++--------
 .../decoration/MatchStartEndStrategy.java       | 107 ------------
 .../tinkergraph/structure/TinkerGraphTest.java  |   2 +-
 4 files changed, 102 insertions(+), 182 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/59d28045/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 50dd7f3..e119ebe 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,10 +19,8 @@
 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;
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.finalization.ProfileStrategy;
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.finalization.ScopingStrategy;
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.optimization.AdjacentToIncidentStrategy;
@@ -200,7 +198,6 @@ 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/59d28045/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 8296712..a65de9c 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,14 +23,18 @@ 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.filter.AndStep;
+import org.apache.tinkerpop.gremlin.process.traversal.step.filter.ConjunctionStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.map.SelectOneStep;
-import org.apache.tinkerpop.gremlin.process.traversal.step.util.AbstractStep;
+import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.StartStep;
 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;
 
@@ -63,13 +67,50 @@ public final class XMatchStep<S> extends ComputerAwareStep<S, S> implements Trav
         super(traversal);
         this.conjunction = conjunction;
         this.conjunctionTraversals = (List) Stream.of(conjunctionTraversals).map(Traversal::asAdmin).map(this::integrateChild).collect(Collectors.toList());
+        this.conjunctionTraversals.forEach(this::addSelectOneStartStep);
+    }
+
+    private void addSelectOneStartStep(final Traversal.Admin<?, ?> conjunctionTraversal) {
+        // CONJUNCTION STEPS to XMatchSteps
+        for (final Step<?, ?> conjunction : TraversalHelper.getStepsOfAssignableClass(ConjunctionStep.class, conjunctionTraversal)) {
+            final XMatchStep xMatchStep = new XMatchStep(conjunctionTraversal, conjunction instanceof AndStep ? XMatchStep.Conjunction.AND : XMatchStep.Conjunction.OR, ((ConjunctionStep<?>) conjunction).getLocalChildren().toArray(new Traversal[((ConjunctionStep<?>) conjunction).getLocalChildren().size()]));
+            TraversalHelper.replaceStep(conjunction, xMatchStep, conjunctionTraversal);
+        }
+        // START STEP to SelectOneStep
+        final Step<?, ?> startStep = conjunctionTraversal.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(conjunctionTraversal.getStartStep(), new SelectOneStep<>(conjunctionTraversal, Scope.global, Pop.head, startStep.getLabels().iterator().next()), conjunctionTraversal);
+        }
+        // END STEP to XMatchStep
+        final Step<?, ?> endStep = conjunctionTraversal.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 XMatchEndStep(conjunctionTraversal, label);
+            if (null != label) xMatchEndStep.addLabel(label);
+            conjunctionTraversal.asAdmin().addStep(xMatchEndStep);
+        }
+        // TODO: is this needed?
+        conjunctionTraversal.getSteps().stream()
+                .filter(step -> step instanceof TraversalParent)
+                .filter(step -> !(step instanceof XMatchStep))
+                .forEach(step -> {
+                    ((TraversalParent) step).getGlobalChildren().forEach(this::addSelectOneStartStep);
+                    ((TraversalParent) step).getLocalChildren().forEach(this::addSelectOneStartStep);
+                });
     }
 
     public Set<String> getMatchStartLabels() {
         if (null == this.matchStartLabels) {
             this.matchStartLabels = new HashSet<>();
-            for (final Traversal.Admin<Object, Object> andTraversal : this.conjunctionTraversals) {
-                this.matchStartLabels.addAll(andTraversal.getStartStep() instanceof XMatchStep ? ((XMatchStep) andTraversal.getStartStep()).getMatchStartLabels() : ((SelectOneStep) andTraversal.getStartStep()).getScopeKeys());
+            for (final Traversal.Admin<Object, Object> conjunctionTraversals : this.conjunctionTraversals) {
+                this.matchStartLabels.addAll(conjunctionTraversals.getStartStep() instanceof XMatchStep ?
+                        ((XMatchStep) conjunctionTraversals.getStartStep()).getMatchStartLabels() :
+                        ((SelectOneStep) conjunctionTraversals.getStartStep()).getScopeKeys());
             }
             this.matchStartLabels = Collections.unmodifiableSet(this.matchStartLabels);
         }
@@ -102,12 +143,24 @@ public final class XMatchStep<S> extends ComputerAwareStep<S, S> implements Trav
         return clone;
     }
 
+    private boolean hasMatched(final Conjunction conjunction, final Traverser<S> traverser) {
+        final Path path = traverser.path();
+        int counter = 0;
+        for (final Traversal.Admin<Object, Object> conjunctionTraversal : this.conjunctionTraversals) {
+            if (path.hasLabel(conjunctionTraversal.getStartStep().getId())) {
+                if (conjunction == Conjunction.OR) return true;
+                counter++;
+            }
+        }
+        return this.conjunctionTraversals.size() == counter;
+    }
+
     @Override
     protected Iterator<Traverser<S>> standardAlgorithm() throws NoSuchElementException {
         while (true) {
             Traverser.Admin traverser = null;
             if (this.first) {
-                this.matchAlgorithm.initialize(this.conjunction, this.conjunctionTraversals);
+                this.matchAlgorithm.initialize(this.conjunctionTraversals);
                 this.first = false;
             } else {
                 for (final Traversal.Admin<?, ?> conjunctionTraversal : this.conjunctionTraversals) {
@@ -117,46 +170,51 @@ public final class XMatchStep<S> extends ComputerAwareStep<S, S> implements Trav
                     }
                 }
             }
-            if (null == traverser) {
+            if (null == traverser)
                 traverser = this.starts.next();
-                if (this.conjunction == Conjunction.OR) {
-                    for (final Traversal.Admin<?, ?> conjunctionTraversal : this.conjunctionTraversals) {
-                        conjunctionTraversal.addStart(traverser.split());
-                    }
+            else if (hasMatched(this.conjunction, traverser))
+                return IteratorUtils.of(traverser);
+
+            if (this.conjunction == Conjunction.AND) {
+                final Traversal.Admin<Object, Object> conjunctionTraversal = this.matchAlgorithm.apply(traverser); // determine which sub-pattern the traverser should try next
+                traverser.path().addLabel(conjunctionTraversal.getStartStep().getId()); // unique identifier for the traversal match sub-pattern
+                conjunctionTraversal.addStart(traverser);  // go down the traversal match sub-pattern
+            } else {
+                for (final Traversal.Admin<?, ?> conjunctionTraversal : this.conjunctionTraversals) {
+                    final Traverser split = traverser.split();
+                    split.path().addLabel(conjunctionTraversal.getStartStep().getId());
+                    conjunctionTraversal.addStart(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();
-                traverser.path().addLabel(traversal.getStartStep().getId()); // unique identifier for the traversal match sub-pattern
-                traversal.addStart(traverser);  // go down the traversal match sub-pattern
-            } else
-                // TODO: trim off internal traversal labels from path
-                // TODO: simply iterate through traversals.startStep.getId() and remove those labels
-                // TODO: however, they are globally unique so it might not be necessary especially if we return Map<String,Object>
-                return IteratorUtils.of(traverser); // the traverser has survived all requisite match patterns and is ready to move onto the next step
         }
     }
 
     @Override
     protected Iterator<Traverser<S>> computerAlgorithm() throws NoSuchElementException {
         if (this.first) {
-            this.matchAlgorithm.initialize(this.conjunction, this.conjunctionTraversals);
+            this.matchAlgorithm.initialize(this.conjunctionTraversals);
             this.first = false;
         }
         final Traverser.Admin traverser = this.starts.next();
-        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();
-            traverser.path().addLabel(traversal.getStartStep().getId()); // unique identifier for the traversal match sub-pattern
-            traverser.setStepId(traversal.getStartStep().getId()); // go down the traversal match sub-pattern
+        if (hasMatched(this.conjunction, traverser)) {
+            traverser.setStepId(this.getNextStep().getId());
+            return IteratorUtils.of(traverser);
+        }
+
+        if (this.conjunction == Conjunction.AND) {
+            final Traversal.Admin<Object, Object> conjunctionTraversal = this.matchAlgorithm.apply(traverser); // determine which sub-pattern the traverser should try next
+            traverser.path().addLabel(conjunctionTraversal.getStartStep().getId()); // unique identifier for the traversal match sub-pattern
+            traverser.setStepId(conjunctionTraversal.getStartStep().getId()); // go down the traversal match sub-pattern
             return IteratorUtils.of(traverser);
         } else {
-            // TODO: trim off internal traversal labels from path
-            // TODO: simply iterate through traversals.startStep.getId() and remove those labels
-            // TODO: however, they are globally unique so it might not be necessary especially if we return Map<String,Object>
-            traverser.asAdmin().setStepId(this.getNextStep().getId());
-            return IteratorUtils.of(traverser); // the traverser has survived all requisite match patterns and is ready to move onto the next step
+            final List<Traverser<S>> traversers = new ArrayList<>();
+            this.conjunctionTraversals.forEach(conjunctionTraversal -> {
+                final Traverser.Admin split = traverser.split();
+                split.path().addLabel(conjunctionTraversal.getStartStep().getId());
+                split.setStepId(conjunctionTraversal.getStartStep().getId());
+                traversers.add(split);
+            });
+            return traversers.iterator();
         }
     }
 
@@ -172,15 +230,13 @@ public final class XMatchStep<S> extends ComputerAwareStep<S, S> implements Trav
 
     //////////////////////////////
 
-    public static class XMatchEndStep<S> extends AbstractStep<S, S> {
+    public class XMatchEndStep extends EndStep {
 
         private final String matchKey;
-        private final String matchStepId;
 
-        public XMatchEndStep(final Traversal.Admin traversal, final XMatchStep matchStep, final String matchKey) {
+        public XMatchEndStep(final Traversal.Admin traversal, final String matchKey) {
             super(traversal);
             this.matchKey = matchKey;
-            this.matchStepId = matchStep.getId();
         }
 
         @Override
@@ -189,34 +245,29 @@ 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(this.matchStepId);
+                    if (this.traverserStepIdSetByChild) start.setStepId(XMatchStep.this.getId());
                     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(this.matchStepId);
+                    if (this.traverserStepIdSetByChild) start.setStepId(XMatchStep.this.getId());
                     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(this.matchStepId);
+                    if (this.traverserStepIdSetByChild) start.setStepId(XMatchStep.this.getId());
                     return start;
                 }
             }
         }
-
-        @Override
-        public String toString() {
-            return StringFactory.stepString(this, this.matchKey, this.matchStepId);
-        }
     }
 
 
     //////////////////////////////
 
-    public interface MatchAlgorithm extends Function<Traverser.Admin<Object>, Optional<Traversal.Admin<Object, Object>>> {
+    public interface MatchAlgorithm extends Function<Traverser.Admin<Object>, Traversal.Admin<Object, Object>> {
 
         public static Set<String> getStartLabels(final Traversal.Admin<Object, Object> traversal) {
             final Step<?, ?> startStep = traversal.getStartStep();
@@ -228,7 +279,7 @@ public final class XMatchStep<S> extends ComputerAwareStep<S, S> implements Trav
                 return Collections.emptySet();
         }
 
-        public void initialize(final Conjunction conjunction, final List<Traversal.Admin<Object, Object>> traversals);
+        public void initialize(final List<Traversal.Admin<Object, Object>> traversals);
     }
 
     public static class GreedyMatchAlgorithm implements MatchAlgorithm {
@@ -236,12 +287,10 @@ 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 Conjunction conjunction, final List<Traversal.Admin<Object, Object>> traversals) {
+        public void initialize(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,33 +298,14 @@ public final class XMatchStep<S> extends ComputerAwareStep<S, S> implements Trav
         }
 
         @Override
-        public Optional<Traversal.Admin<Object, Object>> apply(final Traverser.Admin<Object> traverser) {
+        public Traversal.Admin<Object, Object> apply(final Traverser.Admin<Object> traverser) {
             final Path path = traverser.path();
-            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));
-                    }
+            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 this.traversals.get(i);
                 }
-                if (count == 0)
-                    throw new IllegalStateException("The provided match or-pattern is unsolvable: " + this.traversals);
-                return Optional.empty();
             }
+            throw new IllegalStateException("The provided match pattern is unsolvable: " + this.traversals);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/59d28045/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
deleted file mode 100644
index f7fd400..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/decoration/MatchStartEndStrategy.java
+++ /dev/null
@@ -1,107 +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.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/59d28045/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 494f46d..de6d55e 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
@@ -167,7 +167,7 @@ public class TinkerGraphTest {
                                 as("a").out().out()
                         )
                 ))
-                .select(Pop.head).by("name");
+                .select(Pop.head);
         /*
                 g.V().as("a").xmatch(
                         as("a").out("knows").as("b"),