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/11 19:16:22 UTC

incubator-tinkerpop git commit: added XMatchStep which implements the new Match model.

Repository: incubator-tinkerpop
Updated Branches:
  refs/heads/xmatch [created] 925952546


added XMatchStep which implements the new Match model.


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

Branch: refs/heads/xmatch
Commit: 925952546562aae081a1357104b420922d43ba50
Parents: 1d576d8
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Thu Jun 11 11:16:12 2015 -0600
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Thu Jun 11 11:16:12 2015 -0600

----------------------------------------------------------------------
 .../traversal/dsl/graph/GraphTraversal.java     |   5 +
 .../step/filter/exp/IsOrAllowStep.java          |  52 +++++++
 .../traversal/step/filter/exp/XMatchStep.java   | 138 +++++++++++++++++++
 .../tinkergraph/structure/TinkerGraphTest.java  |  24 +---
 4 files changed, 199 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/92595254/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 28d752a..69597cb 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
@@ -55,6 +55,7 @@ import org.apache.tinkerpop.gremlin.process.traversal.step.filter.SimplePathStep
 import org.apache.tinkerpop.gremlin.process.traversal.step.filter.TailGlobalStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.filter.TimeLimitStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.filter.WhereStep;
+import org.apache.tinkerpop.gremlin.process.traversal.step.filter.exp.XMatchStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.map.AddEdgeStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.map.AddVertexStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.map.CoalesceStep;
@@ -988,6 +989,10 @@ 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));
+    }
+
     @Override
     public default GraphTraversal<S, E> iterate() {
         Traversal.super.iterate();

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/92595254/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/exp/IsOrAllowStep.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/exp/IsOrAllowStep.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/exp/IsOrAllowStep.java
new file mode 100644
index 0000000..081f747
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/exp/IsOrAllowStep.java
@@ -0,0 +1,52 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  * http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ */
+
+package org.apache.tinkerpop.gremlin.process.traversal.step.filter.exp;
+
+import org.apache.tinkerpop.gremlin.process.traversal.Pop;
+import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
+import org.apache.tinkerpop.gremlin.process.traversal.Traverser;
+import org.apache.tinkerpop.gremlin.process.traversal.step.filter.FilterStep;
+
+import java.util.Optional;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public final class IsOrAllowStep<S> extends FilterStep<S> {
+
+    private final String key;
+
+    public IsOrAllowStep(final Traversal.Admin traversal, final String key) {
+        super(traversal);
+        this.key = key;
+    }
+
+    @Override
+    protected boolean filter(final Traverser.Admin<S> traverser) {
+        final Optional<S> optional = traverser.getSideEffects().get(this.key);
+        if (optional.isPresent())
+            return traverser.get().equals(optional.get());
+        else
+            return !traverser.path().hasLabel(this.key) || traverser.get().equals(traverser.path().getSingle(Pop.head, this.key));
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/92595254/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
new file mode 100644
index 0000000..29ee69d
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/exp/XMatchStep.java
@@ -0,0 +1,138 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  * http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ */
+
+package org.apache.tinkerpop.gremlin.process.traversal.step.filter.exp;
+
+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.map.TailLocalStep;
+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.traverser.TraverserRequirement;
+import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalHelper;
+import org.apache.tinkerpop.gremlin.structure.util.StringFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public final class XMatchStep<S> extends AbstractStep<S, S> implements TraversalParent {
+
+    private final List<Traversal.Admin<?, ?>> andTraversals = new ArrayList<>();
+    private final List<String> traversalLabels = new ArrayList<>();
+    private final List<String> startLabels = new ArrayList<>();
+
+    public XMatchStep(final Traversal.Admin traversal, final Traversal... andTraversals) {
+        super(traversal);
+        int counter = 0;
+        for (final Traversal andTraversal : andTraversals) {
+            final String traversalLabel = "t" + counter++;
+            this.traversalLabels.add(traversalLabel);
+            //// 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 where()-traversal predicate can only have one label: " + startStep);
+                final String startLabel = startStep.getLabels().iterator().next();
+                final Step<?, ?> selectOneStep = new SelectOneStep<>(andTraversal.asAdmin(), Scope.global, startLabel);
+                selectOneStep.addLabel(traversalLabel);
+                this.startLabels.add(startLabel);
+                TraversalHelper.replaceStep(andTraversal.asAdmin().getStartStep(), selectOneStep, andTraversal.asAdmin());
+                TraversalHelper.insertAfterStep(new TailLocalStep<>(andTraversal.asAdmin(), 1), selectOneStep, andTraversal.asAdmin());
+            }
+            //// END STEP
+            final Step<?, ?> endStep = andTraversal.asAdmin().getEndStep();
+            if (!endStep.getLabels().isEmpty()) {
+                if (endStep.getLabels().size() > 1)
+                    throw new IllegalArgumentException("The end step of a where()-traversal predicate can only have one label: " + endStep);
+                final String label = endStep.getLabels().iterator().next();
+                endStep.removeLabel(label);
+                final Step<?, ?> isOrAllowStep = new IsOrAllowStep<>(andTraversal.asAdmin(), label);
+                isOrAllowStep.addLabel(label);
+                andTraversal.asAdmin().addStep(isOrAllowStep);
+            }
+            this.andTraversals.add(this.integrateChild(andTraversal.asAdmin()));
+        }
+    }
+
+    @Override
+    protected Traverser<S> processNextStart() throws NoSuchElementException {
+        while (true) {
+
+            for (final Traversal.Admin<?, ?> andTraversal : this.andTraversals) {
+                if (andTraversal.hasNext()) {
+                    this.starts.add((Traverser.Admin) andTraversal.getEndStep().next().asAdmin());
+                }
+            }
+            final Traverser<S> traverser = this.starts.next();
+            boolean repeated = false;
+            for (int i = 0; i < this.andTraversals.size(); i++) {
+                if (traverser.path().hasLabel(this.startLabels.get(i)) && !traverser.path().hasLabel(this.traversalLabels.get(i))) {
+                    repeated = true;
+                    this.andTraversals.get(i).addStart((Traverser) traverser);
+                    break;
+                }
+            }
+            if (!repeated)
+                return traverser;
+        }
+    }
+
+    @Override
+    public List<Traversal.Admin<?, ?>> getLocalChildren() {
+        return Collections.unmodifiableList(this.andTraversals);
+    }
+
+    @Override
+    public String toString() {
+        return StringFactory.stepString(this, this.andTraversals);
+    }
+
+
+    @Override
+    public XMatchStep<S> clone() {
+        final XMatchStep<S> clone = (XMatchStep<S>) super.clone();
+        clone.andTraversals.clear();
+        for (final Traversal.Admin<?, ?> traversal : this.andTraversals) {
+            clone.andTraversals.add(clone.integrateChild(traversal.clone()));
+        }
+        return clone;
+    }
+
+    @Override
+    public int hashCode() {
+        return super.hashCode() ^ this.andTraversals.hashCode();
+    }
+
+    @Override
+    public Set<TraverserRequirement> getRequirements() {
+        return this.getSelfAndChildRequirements(TraverserRequirement.PATH, TraverserRequirement.SIDE_EFFECTS);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/92595254/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 4221c31..54a66a3 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
@@ -155,26 +155,10 @@ public class TinkerGraphTest {
     @Ignore
     public void testPlay5() throws Exception {
         GraphTraversalSource g = TinkerFactory.createModern().traversal();
-        //g.V().has("name", "gremlin").inE("uses").order().by("skill", Order.incr).as("a").outV().as("b").path().forEachRemaining(System.out::println);
-        //g.V().has("name", "gremlin").inE("uses").order().by("skill", Order.incr).as("a").outV().as("b").select().by("skill").by("name").forEachRemaining(System.out::println);
-        //g.V().label().groupCount().as("x").select().forEachRemaining(System.out::println);
-        //g.V().choose(__.outE().count().is(0L), __.as("x"), __.as("y")).select("x", "y").forEachRemaining(System.out::println);
-        // g.V().hasLabel("person").values("age").is(P.lt(27).or(P.gt(29))).forEachRemaining(System.out::println);
-        //System.out.println(g.V().as("a").out("knows").as("b").where(as("a","b").out("created")).select().by("name"));
-        Supplier<Traversal<?,?>> supplier = () -> g.V().as("a").values("name").as("b").where(as("b").is(eq("marko"))).select();
-
-                /*() ->
-                g.V().as("a").out("created").as("b").in("created").as("c").where(
-                        or(
-                                as("a").out("knows"),
-                                and(
-                                        as("b").has("name","ripple"),
-                                        as("b").in("created").as("a"),
-                                        as("b").in().has("name","josh"))
-                        )
-                ).select().by("name");*/
-        System.out.println(supplier.get().iterate());
-        supplier.get().forEachRemaining(System.out::println);
+        g.V().as("a").xmatch(
+                as("a").out("knows").as("b"),
+                as("a").out("created").as("c"),
+                as("b").out("created").as("c")).select("a","b","c").forEachRemaining(System.out::println);
     }
 
     @Test