You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@tinkerpop.apache.org by GCHQResearcher1337 <gi...@git.apache.org> on 2018/06/11 09:29:23 UTC

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

GitHub user GCHQResearcher1337 opened a pull request:

    https://github.com/apache/tinkerpop/pull/876

    TINKERPOP-967 Support nested-repeat() structures

    https://issues.apache.org/jira/browse/TINKERPOP-967
    
    This allows nested loops to be used in traversals e.g.
    ```gremlin
    g.V().repeat(out("knows")).until(repeat(out("created")).emit(hasId(3)))
    ```
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/GCHQResearcher1337/tinkerpop TINKERPOP-967

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/tinkerpop/pull/876.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #876
    
----
commit 320c14365f9a77a6942d8c5a6e997560634cf855
Author: GCHQResearcher1337 <39...@...>
Date:   2018-06-04T09:35:29Z

    TINKERPOP-967 Support nested-repeat() structures
    
    Implementation of a loop counter stack and set of nested loop traversers as described in TINKERPOP-967.
    
    Added NESTED_LOOP (NL) supporting traversers alongside each SINGLE_LOOP supporting traverser and
    included these in the DefaultTraverserGeneratorFactory.
    
    Added new Traversers and dependencies into GryoVersion.
    
    Added extra functionality tests into RepeatTest and DefaultTraverserGeneratorFactoryTest to check that
    the nested loops work as exected and that the Traversers are correctly selected respectively.

----


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by dkuppitz <gi...@git.apache.org>.
Github user dkuppitz commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202453831
  
    --- Diff: gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/step/branch/RepeatStepTest.java ---
    @@ -39,7 +39,9 @@
                     __.repeat(out()).times(3),
                     __.repeat(out().as("x")).times(3),
                     __.out().emit().repeat(out()).times(3),
    -                __.repeat(out()).until(hasLabel("x"))
    +                __.repeat(out()).until(hasLabel("x")),
    +                __.repeat("a", __.out()).times(3),
    --- End diff --
    
    Ah, alright, I totally missed the first entry, all good then.
    
    We would have to shift or rotate the `loopName`'s hashCode value by the depth of the repeat traversal. Unfortunately, we don't have this information and I don't see how your `hashCode` implementation above would solve the problem. Anyway, it's an unlikely corner case and I don't think we need to care about it too much, so no big deal if we don't have a solution for now.


---

[GitHub] tinkerpop issue #876: TINKERPOP-967 Support nested-repeat() structures

Posted by robertdale <gi...@git.apache.org>.
Github user robertdale commented on the issue:

    https://github.com/apache/tinkerpop/pull/876
  
    VOTE +1


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/tinkerpop/pull/876


---

[GitHub] tinkerpop issue #876: TINKERPOP-967 Support nested-repeat() structures

Posted by spmallette <gi...@git.apache.org>.
Github user spmallette commented on the issue:

    https://github.com/apache/tinkerpop/pull/876
  
    @GCHQResearcher1337 any reason you didn't implement the labelling for `repeat()` as described in the JIRA issue:
    
    ```text
    repeat('a',out('knows').repeat('b',out('parent')))
    ```
    
    As the issue states, it's usage would probably be rare, but marko had that intention when he created the ticket so I thought I'd ask about it (also marko has been lurking about and mentioned it to me last night :smile: ).


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by GCHQResearcher1337 <gi...@git.apache.org>.
Github user GCHQResearcher1337 commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r195046349
  
    --- Diff: gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/branch/RepeatTest.java ---
    @@ -386,5 +456,26 @@ public void g_V_hasXname_markoX_repeatXoutE_inV_simplePathX_untilXhasXname_rippl
             public Traversal<Vertex, Path> get_g_V_hasXloop_name_loopX_repeatXinX_timesX5X_path_by_name() {
                 return g.V().has("loops","name","loop").repeat(__.in()).times(5).path().by("name");
             }
    +
    +        @Override
    +        public Traversal<Vertex, Path> get_g_V_repeatXout_repeatXoutX_timesX1XX_timesX1X_limitX1X_path_by_name() {
    +            // NB We need to prevent the RepeatUnrollStrategy from applying to properly exercise this test as this traversal can be simplified
    --- End diff --
    
    Thanks @spmallette I wanted to do something like this but hadn't spotted these tests. While writing some tests I've noticed that when RepeatUnrollStrategy is applied from the inside out, adding a barrier step onto the end: 
    repeat(a.repeat(b).times(1)).times(1)  -> repeat(a.b.barrier).times(1)
    repeat(a.b.barrier).times(1) -> a.b.barrier.barrier
    
    eg
    ```
    gremlin> g.V().repeat(out().repeat(out()).times(1)).times(1).explain()
    ==>Traversal Explanation
    ===========================================================================================================================================================================================================================
    Original Traversal                 [GraphStep(vertex,[]), RepeatStep([VertexStep(OUT,vertex), RepeatStep([VertexStep(OUT,vertex), RepeatEndStep],until(loops(1)),emit(false)), RepeatEndStep],until(loops(1)),emit(false))]
    
    ConnectiveStrategy           [D]   [GraphStep(vertex,[]), RepeatStep([VertexStep(OUT,vertex), RepeatStep([VertexStep(OUT,vertex), RepeatEndStep],until(loops(1)),emit(false)), RepeatEndStep],until(loops(1)),emit(false))]
    MatchPredicateStrategy       [O]   [GraphStep(vertex,[]), RepeatStep([VertexStep(OUT,vertex), RepeatStep([VertexStep(OUT,vertex), RepeatEndStep],until(loops(1)),emit(false)), RepeatEndStep],until(loops(1)),emit(false))]
    FilterRankingStrategy        [O]   [GraphStep(vertex,[]), RepeatStep([VertexStep(OUT,vertex), RepeatStep([VertexStep(OUT,vertex), RepeatEndStep],until(loops(1)),emit(false)), RepeatEndStep],until(loops(1)),emit(false))]
    InlineFilterStrategy         [O]   [GraphStep(vertex,[]), RepeatStep([VertexStep(OUT,vertex), RepeatStep([VertexStep(OUT,vertex), RepeatEndStep],until(loops(1)),emit(false)), RepeatEndStep],until(loops(1)),emit(false))]
    IncidentToAdjacentStrategy   [O]   [GraphStep(vertex,[]), RepeatStep([VertexStep(OUT,vertex), RepeatStep([VertexStep(OUT,vertex), RepeatEndStep],until(loops(1)),emit(false)), RepeatEndStep],until(loops(1)),emit(false))]
    AdjacentToIncidentStrategy   [O]   [GraphStep(vertex,[]), RepeatStep([VertexStep(OUT,vertex), RepeatStep([VertexStep(OUT,vertex), RepeatEndStep],until(loops(1)),emit(false)), RepeatEndStep],until(loops(1)),emit(false))]
    RepeatUnrollStrategy         [O]   [GraphStep(vertex,[]), VertexStep(OUT,vertex), VertexStep(OUT,vertex), NoOpBarrierStep(2500), NoOpBarrierStep(2500)]
    CountStrategy                [O]   [GraphStep(vertex,[]), VertexStep(OUT,vertex), VertexStep(OUT,vertex), NoOpBarrierStep(2500), NoOpBarrierStep(2500)]
    PathRetractionStrategy       [O]   [GraphStep(vertex,[]), VertexStep(OUT,vertex), VertexStep(OUT,vertex), NoOpBarrierStep(2500), NoOpBarrierStep(2500)]
    LazyBarrierStrategy          [O]   [GraphStep(vertex,[]), VertexStep(OUT,vertex), NoOpBarrierStep(2500), VertexStep(OUT,vertex), NoOpBarrierStep(2500), NoOpBarrierStep(2500)]
    TinkerGraphCountStrategy     [P]   [GraphStep(vertex,[]), VertexStep(OUT,vertex), NoOpBarrierStep(2500), VertexStep(OUT,vertex), NoOpBarrierStep(2500), NoOpBarrierStep(2500)]
    TinkerGraphStepStrategy      [P]   [TinkerGraphStep(vertex,[]), VertexStep(OUT,vertex), NoOpBarrierStep(2500), VertexStep(OUT,vertex), NoOpBarrierStep(2500), NoOpBarrierStep(2500)]
    ProfileStrategy              [F]   [TinkerGraphStep(vertex,[]), VertexStep(OUT,vertex), NoOpBarrierStep(2500), VertexStep(OUT,vertex), NoOpBarrierStep(2500), NoOpBarrierStep(2500)]
    StandardVerificationStrategy [V]   [TinkerGraphStep(vertex,[]), VertexStep(OUT,vertex), NoOpBarrierStep(2500), VertexStep(OUT,vertex), NoOpBarrierStep(2500), NoOpBarrierStep(2500)]
    
    Final Traversal                    [TinkerGraphStep(vertex,[]), VertexStep(OUT,vertex), NoOpBarrierStep(2500), VertexStep(OUT,vertex), NoOpBarrierStep(2500), NoOpBarrierStep(2500)]
    ```
    
    Do you think it's ok to leave multiple barrier steps at the end? I could add an extra check that we aren't inserting a barrier after a barrier here - but this could interfere with a barrier added by a user?
    https://github.com/apache/tinkerpop/blob/d3b6a340dbb9c89355c40089a5c083dc0dbd4c5b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/optimization/RepeatUnrollStrategy.java#L72


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by spmallette <gi...@git.apache.org>.
Github user spmallette commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r195708305
  
    --- Diff: gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/branch/RepeatTest.java ---
    @@ -386,5 +456,26 @@ public void g_V_hasXname_markoX_repeatXoutE_inV_simplePathX_untilXhasXname_rippl
             public Traversal<Vertex, Path> get_g_V_hasXloop_name_loopX_repeatXinX_timesX5X_path_by_name() {
                 return g.V().has("loops","name","loop").repeat(__.in()).times(5).path().by("name");
             }
    +
    +        @Override
    +        public Traversal<Vertex, Path> get_g_V_repeatXout_repeatXoutX_timesX1XX_timesX1X_limitX1X_path_by_name() {
    +            // NB We need to prevent the RepeatUnrollStrategy from applying to properly exercise this test as this traversal can be simplified
    --- End diff --
    
    hmm - seems like we shouldn't have extra barriers. @dkuppitz any thoughts on how this should be handled?


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by GCHQResearcher1337 <gi...@git.apache.org>.
Github user GCHQResearcher1337 commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202452106
  
    --- Diff: gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/B_LP_NL_O_P_S_SE_SL_Traverser.java ---
    @@ -0,0 +1,156 @@
    +/*
    + * 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.traverser;
    +
    +import org.apache.commons.collections.map.ReferenceMap;
    +import org.apache.tinkerpop.gremlin.process.traversal.Step;
    +import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.LabelledCounter;
    +
    +
    +import java.util.Iterator;
    +import java.util.Stack;
    +
    +public class B_LP_NL_O_P_S_SE_SL_Traverser<T> extends B_LP_O_P_S_SE_SL_Traverser<T> {
    +
    +    protected Stack<LabelledCounter> nestedLoops;
    +    protected ReferenceMap loopNames = null;
    --- End diff --
    
    Sure. So the loop stack is a stack of LabelledCounters which store the stepId and the loop counter. As per the second part of https://issues.apache.org/jira/browse/TINKERPOP-967 we want to be able to 'name' a loop counter. I could store this in the LabelledCounter alongside the stepId and the counter, but to get more constant time lookup by name I use a separate Map from loop name -> LabelledCounter.
    
    When `resetLoops()` is called the counter is removed from the Stack and also needs to be removed from this lookup Map. Because the map is keyed on 'name' not stepId, I don't know which entry to remove without either:
    * Checking every entry on the map to see if the LabelledCounter matches the one I'm removing from the stack.
    * Storing the 'name' in the stack of loop counter
    
    A Map with 'Weak values' only holds onto entries while the value (the reference to the LabelledCounter) is valid. When it gets released the entry (the 'name' and the Reference to the LabelledCounter) get removed. This avoids needing a second Map/second index/extra to store lookups from  
    
    I may have overcomplicated this. I was trying to avoid looping over the loopStack (which I expect to be small) and to keep storage down (which is hard to do) - especially considering the existing code seems to want to shave as many bytes as possible off the looping storage (using a short under the hood when the interface exposes an int):
    https://github.com/apache/tinkerpop/blob/a80eb84169048ed74c5ad27ebc4d12944fd0136a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/B_O_S_SE_SL_Traverser.java#L34



---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by dkuppitz <gi...@git.apache.org>.
Github user dkuppitz commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202437245
  
    --- Diff: gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/step/branch/RepeatStepTest.java ---
    @@ -39,7 +39,9 @@
                     __.repeat(out()).times(3),
                     __.repeat(out().as("x")).times(3),
                     __.out().emit().repeat(out()).times(3),
    -                __.repeat(out()).until(hasLabel("x"))
    +                __.repeat(out()).until(hasLabel("x")),
    +                __.repeat("a", __.out()).times(3),
    --- End diff --
    
    Sorry, perhaps I should have been more clear on what kind of traversals to add here.
    
    Can you please also add `__.repeat(out()).times(3)`? Without the changes in `hashCode()` this traversal would have had the same hashCode as `__.repeat("a", out()).times(3)` and thus this test would have failed.
    
    I guess these two will still collide:
    
    ```
    __.repeat("a", __.repeat("b", out()))
    __.repeat("b", __.repeat("a", out()))
    ```
    
    ...but I can't think of a solution that doesn't involve any expensive method calls.


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by dkuppitz <gi...@git.apache.org>.
Github user dkuppitz commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202456800
  
    --- Diff: gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/B_LP_NL_O_P_S_SE_SL_Traverser.java ---
    @@ -0,0 +1,156 @@
    +/*
    + * 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.traverser;
    +
    +import org.apache.commons.collections.map.ReferenceMap;
    +import org.apache.tinkerpop.gremlin.process.traversal.Step;
    +import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.LabelledCounter;
    +
    +
    +import java.util.Iterator;
    +import java.util.Stack;
    +
    +public class B_LP_NL_O_P_S_SE_SL_Traverser<T> extends B_LP_O_P_S_SE_SL_Traverser<T> {
    +
    +    protected Stack<LabelledCounter> nestedLoops;
    +    protected ReferenceMap loopNames = null;
    --- End diff --
    
    Hmm, to be realistic, I don't expect more than 2 or 3 named repeat steps in a single traversal, so an index for the names might really not be necessary (IMO). 


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by GCHQResearcher1337 <gi...@git.apache.org>.
Github user GCHQResearcher1337 commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202442057
  
    --- Diff: gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/step/branch/RepeatStepTest.java ---
    @@ -39,7 +39,9 @@
                     __.repeat(out()).times(3),
                     __.repeat(out().as("x")).times(3),
                     __.out().emit().repeat(out()).times(3),
    -                __.repeat(out()).until(hasLabel("x"))
    +                __.repeat(out()).until(hasLabel("x")),
    +                __.repeat("a", __.out()).times(3),
    --- End diff --
    
    I don't quite understand your first point - `__.repeat(out()).times(3)` is the first entry in the array and I believe this did fail before the `hashCode()` fix.
    
    Good point about the collision. 
    Maybe I could use the same method the emit first and until first use? ie:
    ```
    @Override
        public int hashCode() {
            int result = super.hashCode() ^ (this.repeatTraversal.hashCode() << 1);
            result ^= Boolean.hashCode(this.untilFirst);
            result ^= Boolean.hashCode(this.emitFirst) << 1;
            if (this.loopName != null)
                result ^= this.loopName.hashCode();
            if (this.untilTraversal != null)
                result ^= this.untilTraversal.hashCode();
            if (this.emitTraversal != null)
                result ^= this.emitTraversal.hashCode();
            return result;
        }
    ```


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by dkuppitz <gi...@git.apache.org>.
Github user dkuppitz commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202113071
  
    --- Diff: gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/util/LabelledCounter.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * 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.traverser.util;
    +
    +import org.apache.commons.lang.mutable.MutableShort;
    +
    +import java.io.Serializable;
    +
    +/**
    + * Class to track a count associated with a Label
    + */
    +public class LabelledCounter implements Serializable, Cloneable {
    +
    +    private final String label;
    +    private final MutableShort count = new MutableShort();
    --- End diff --
    
    Just asking for a comment here: Why can this not be a simple `int`?


---

[GitHub] tinkerpop issue #876: TINKERPOP-967 Support nested-repeat() structures

Posted by GCHQResearcher1337 <gi...@git.apache.org>.
Github user GCHQResearcher1337 commented on the issue:

    https://github.com/apache/tinkerpop/pull/876
  
    @dkuppitz Thanks for your comments. I think I've fixed the issues you spotted.


---

[GitHub] tinkerpop issue #876: TINKERPOP-967 Support nested-repeat() structures

Posted by spmallette <gi...@git.apache.org>.
Github user spmallette commented on the issue:

    https://github.com/apache/tinkerpop/pull/876
  
    I will take care of merging this now that we have reviews done. Thanks for patiently working through this process @GCHQResearcher1337 - is that how you will forever be known to us by the way? :smile: 


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by dkuppitz <gi...@git.apache.org>.
Github user dkuppitz commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r196096436
  
    --- Diff: gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/branch/RepeatTest.java ---
    @@ -386,5 +456,26 @@ public void g_V_hasXname_markoX_repeatXoutE_inV_simplePathX_untilXhasXname_rippl
             public Traversal<Vertex, Path> get_g_V_hasXloop_name_loopX_repeatXinX_timesX5X_path_by_name() {
                 return g.V().has("loops","name","loop").repeat(__.in()).times(5).path().by("name");
             }
    +
    +        @Override
    +        public Traversal<Vertex, Path> get_g_V_repeatXout_repeatXoutX_timesX1XX_timesX1X_limitX1X_path_by_name() {
    +            // NB We need to prevent the RepeatUnrollStrategy from applying to properly exercise this test as this traversal can be simplified
    --- End diff --
    
    For now, an extra check seems good to me. In the long run, we should get rid of these NoOpBarrierStep injections altogether as it should only be done by `LazyBarrierStrategy` (that goes back to the whole DFS vs BFS discussion).


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by spmallette <gi...@git.apache.org>.
Github user spmallette commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r194776167
  
    --- Diff: gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/branch/RepeatTest.java ---
    @@ -386,5 +456,26 @@ public void g_V_hasXname_markoX_repeatXoutE_inV_simplePathX_untilXhasXname_rippl
             public Traversal<Vertex, Path> get_g_V_hasXloop_name_loopX_repeatXinX_timesX5X_path_by_name() {
                 return g.V().has("loops","name","loop").repeat(__.in()).times(5).path().by("name");
             }
    +
    +        @Override
    +        public Traversal<Vertex, Path> get_g_V_repeatXout_repeatXoutX_timesX1XX_timesX1X_limitX1X_path_by_name() {
    +            // NB We need to prevent the RepeatUnrollStrategy from applying to properly exercise this test as this traversal can be simplified
    --- End diff --
    
    Nice to know that `RepeatUnrollStrategy` works in this way. @GCHQResearcher1337 could you add nested `repeat()` traversals (as necessary) to this parameterized unit test:
    
    https://github.com/apache/tinkerpop/blob/d3b6a340dbb9c89355c40089a5c083dc0dbd4c5b/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/optimization/RepeatUnrollStrategyTest.java#L74
    
    to validate that `RepeatUnrollStrategy` continues to behave properly?


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by dkuppitz <gi...@git.apache.org>.
Github user dkuppitz commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202105366
  
    --- Diff: gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/LoopsStep.java ---
    @@ -26,12 +26,15 @@
      */
     public final class LoopsStep<S> extends MapStep<S, Integer> {
     
    -    public LoopsStep(final Traversal.Admin traversal) {
    +    private String loopName;
    --- End diff --
    
    `LoopsStep` should now implement `hashCode()`.


---

[GitHub] tinkerpop issue #876: TINKERPOP-967 Support nested-repeat() structures

Posted by GCHQResearcher1337 <gi...@git.apache.org>.
Github user GCHQResearcher1337 commented on the issue:

    https://github.com/apache/tinkerpop/pull/876
  
    @spmallette Yep. Awesome - thanks for all your input + reviews!


---

[GitHub] tinkerpop issue #876: TINKERPOP-967 Support nested-repeat() structures

Posted by spmallette <gi...@git.apache.org>.
Github user spmallette commented on the issue:

    https://github.com/apache/tinkerpop/pull/876
  
    nice catch @dkuppitz - `hashCode()` == crucial


---

[GitHub] tinkerpop issue #876: TINKERPOP-967 Support nested-repeat() structures

Posted by spmallette <gi...@git.apache.org>.
Github user spmallette commented on the issue:

    https://github.com/apache/tinkerpop/pull/876
  
    > perhaps there is a reason to unite the two concepts.
    
    can't really think of a reason to "unite the two concepts" - I think that we want to keep it as `repeat(‘a’,…) != repeat(…).as(‘a’)`. @dkuppitz any thoughts on that?


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by dkuppitz <gi...@git.apache.org>.
Github user dkuppitz commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202114064
  
    --- Diff: gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/B_LP_NL_O_P_S_SE_SL_Traverser.java ---
    @@ -0,0 +1,156 @@
    +/*
    + * 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.traverser;
    +
    +import org.apache.commons.collections.map.ReferenceMap;
    +import org.apache.tinkerpop.gremlin.process.traversal.Step;
    +import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.LabelledCounter;
    +
    +
    +import java.util.Iterator;
    +import java.util.Stack;
    +
    +public class B_LP_NL_O_P_S_SE_SL_Traverser<T> extends B_LP_O_P_S_SE_SL_Traverser<T> {
    +
    +    protected Stack<LabelledCounter> nestedLoops;
    +    protected ReferenceMap loopNames = null;
    --- End diff --
    
    Just asking for a comment here: Why can this not be a simple `HashMap<String, LabelledCounter>`? This would save you the class casting later on and you wouldn't have to register a new class for the Gryo serializer.


---

[GitHub] tinkerpop issue #876: TINKERPOP-967 Support nested-repeat() structures

Posted by GCHQResearcher1337 <gi...@git.apache.org>.
Github user GCHQResearcher1337 commented on the issue:

    https://github.com/apache/tinkerpop/pull/876
  
    I've now added support for named loops - described in https://issues.apache.org/jira/browse/TINKERPOP-967, however I have not introduced a new `times()` step. 
    
    I think a new `times()` step would be of marginal benefit and could be confusing - currently `times()` is a step modulator on `repeat()`. 
    
    I've also changed to explicit loop stack initialisation rather than setting it up on the increment.  Setting up the stack/counter on the increment creates a problem in the nested context - when the inner `loops()` is called it incorrectly returns the top counter from the stack. 


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by dkuppitz <gi...@git.apache.org>.
Github user dkuppitz commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202110691
  
    --- Diff: gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/B_O_S_SE_SL_Traverser.java ---
    @@ -32,6 +32,7 @@
     
         protected Object sack = null;
         protected short loops = 0;  // an optimization hack to use a short internally to save bits :)
    +    protected String loopName = null;
    --- End diff --
    
    Again, this should be included in `hashCode()`.


---

[GitHub] tinkerpop issue #876: TINKERPOP-967 Support nested-repeat() structures

Posted by dkuppitz <gi...@git.apache.org>.
Github user dkuppitz commented on the issue:

    https://github.com/apache/tinkerpop/pull/876
  
    I agree it should be treated differently. Since `as()` labeling has an impact on the path history, it really shouldn't be used for this case.


---

[GitHub] tinkerpop issue #876: TINKERPOP-967 Support nested-repeat() structures

Posted by dkuppitz <gi...@git.apache.org>.
Github user dkuppitz commented on the issue:

    https://github.com/apache/tinkerpop/pull/876
  
    VOTE: +1


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by dkuppitz <gi...@git.apache.org>.
Github user dkuppitz commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202104064
  
    --- Diff: gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/dsl/graph/GraphTraversal.java ---
    @@ -2289,6 +2302,21 @@ else if (value instanceof Traversal)
             return RepeatStep.addRepeatToTraversal(this, (Traversal.Admin<E, E>) repeatTraversal);
         }
     
    +    /**
    +     * This step is used for looping over a some traversal given some break predicate.
    --- End diff --
    
    ```
    over a some traversal
         ^^^^^^
    ```
    
    Minor thing.


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by dkuppitz <gi...@git.apache.org>.
Github user dkuppitz commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202104896
  
    --- Diff: gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/branch/RepeatStep.java ---
    @@ -43,6 +43,7 @@
         private Traversal.Admin<S, S> repeatTraversal = null;
         private Traversal.Admin<S, ?> untilTraversal = null;
         private Traversal.Admin<S, ?> emitTraversal = null;
    +    private String loopName = null;
    --- End diff --
    
    This field should be included in `RepeatStep::hashCode()`.


---

[GitHub] tinkerpop issue #876: TINKERPOP-967 Support nested-repeat() structures

Posted by spmallette <gi...@git.apache.org>.
Github user spmallette commented on the issue:

    https://github.com/apache/tinkerpop/pull/876
  
    Nice @GCHQResearcher1337 - VOTE +1


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by dkuppitz <gi...@git.apache.org>.
Github user dkuppitz commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202102880
  
    --- Diff: docs/src/reference/the-traversal.asciidoc ---
    @@ -2125,6 +2125,17 @@ traverser repeats. However, because the emit-predicate is declared true, those v
       Given that `loops==2`, the until-predicate fails and ripple and lop are emitted.
     Therefore, the traverser has seen the vertices: lop, vadas, josh, ripple, and lop.
     
    +`repeat()`-steps may be nested inside each other or inside the `emit()` or `until()` predicates and they can also be 'named' by passing a string as the first parameter to `repeat()`. The loop counter of a named repeat step can be accessed within the looped context with `loops(loopName)` where `loopName` is the name set whe creating the `repeat()`-step.
    +
    +[gremlin-groovy,modern]
    +----
    +g.V(1).repeat(out("knows")).until(__.repeat(out("created")).emit(__.has("name", "lop"))) <1>
    +g.V(6).repeat('a', both('created')).emit(repeat('b', __.both('knows')).until(or(loops().is(2), loops('b').is(loops('a')))).hasId(2)).dedup() <2>
    --- End diff --
    
    I haven't tried it yet, but I don't think this query does what the description says. It only works because it has the early break condition `loops().is(2)`. `.is(loops('a'))`, on the other hand, should always return `false`, since it's comparing the incoming `Long` value against a `Traversal`. To match the description, the query should look more like this (IMO):
    
    ```
    g.V(6).
      repeat('a', both('created').simplePath()).
        emit(repeat('b', __.both('knows')).
               until(loops('b').as('b').where(loops('a').as('b')))).
      hasId(2)).dedup()
    ```


---

[GitHub] tinkerpop issue #876: TINKERPOP-967 Support nested-repeat() structures

Posted by spmallette <gi...@git.apache.org>.
Github user spmallette commented on the issue:

    https://github.com/apache/tinkerpop/pull/876
  
    Some more notes from Marko (i haven't had a chance to let them sink in myself, just pasting them here): 
    
    ```text
    times(a,2) == loops(a,eq(2))
    ```
    
    the idea being, you might want to check on the loop count of an outer loop within an inner loop and vice versa. Basically, you are trying to solve:
    
    ```text
    for(i=0;i<10) { for(j=0;j<10) if(i == x) …. } ….
    ```
    
    the point being, you want to be able to check for `i` and `j` counters not just at the loop check point but anywhere in the respective body.
    
    And note that naming `repeat()`s is not the same as naming the `repeat()`-step. It isn’t a step label……… though, some thinking on that might be worth it. In essence `repeat(‘a’,…) != repeat(…).as(‘a’)`
    the ‘a’ in the first is for the loop stack, not for the step label….but again, perhaps there is a reason to unite the two concepts. Though if done so, a new syntax would be introduced which would add an “irregularity” to Gremlin


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by dkuppitz <gi...@git.apache.org>.
Github user dkuppitz commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202111311
  
    --- Diff: gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/O_OB_S_SE_SL_Traverser.java ---
    @@ -30,6 +30,7 @@
     
         protected Object sack = null;
         protected short loops = 0;  // an optimization hack to use a short internally to save bits :)
    +    protected String loopName = null;
    --- End diff --
    
    `hashCode()`


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by spmallette <gi...@git.apache.org>.
Github user spmallette commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202446882
  
    --- Diff: gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/B_LP_NL_O_P_S_SE_SL_Traverser.java ---
    @@ -0,0 +1,156 @@
    +/*
    + * 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.traverser;
    +
    +import org.apache.commons.collections.map.ReferenceMap;
    +import org.apache.tinkerpop.gremlin.process.traversal.Step;
    +import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.LabelledCounter;
    +
    +
    +import java.util.Iterator;
    +import java.util.Stack;
    +
    +public class B_LP_NL_O_P_S_SE_SL_Traverser<T> extends B_LP_O_P_S_SE_SL_Traverser<T> {
    +
    +    protected Stack<LabelledCounter> nestedLoops;
    +    protected ReferenceMap loopNames = null;
    --- End diff --
    
    Could you explain a bit further why a weak reference is desirable here? You mentioned that it helps simplify `resetLoops()` but I'm unfortunately not making the connection as to why even with the phraseology you followed with about not needing "to walk the Map or store the 'loop name' multiple times." 


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by GCHQResearcher1337 <gi...@git.apache.org>.
Github user GCHQResearcher1337 commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202445268
  
    --- Diff: gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/B_LP_NL_O_P_S_SE_SL_Traverser.java ---
    @@ -0,0 +1,156 @@
    +/*
    + * 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.traverser;
    +
    +import org.apache.commons.collections.map.ReferenceMap;
    +import org.apache.tinkerpop.gremlin.process.traversal.Step;
    +import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.LabelledCounter;
    +
    +
    +import java.util.Iterator;
    +import java.util.Stack;
    +
    +public class B_LP_NL_O_P_S_SE_SL_Traverser<T> extends B_LP_O_P_S_SE_SL_Traverser<T> {
    +
    +    protected Stack<LabelledCounter> nestedLoops;
    +    protected ReferenceMap loopNames = null;
    --- End diff --
    
    Yep :) magic. I think at garbage collection the entry will get removed, or there will be on access checks of the validity of the reference. I've used something similar before (can't quite remember what) but didn't want to introduce an extra dependency, but luckily I found ReferenceMap in org.apache.commons.collections which was already included and seems to do this. 
    
    I wasn't quite sure if these references would be safe to 'attaching' as I don't fully understand this, but I think attaching only moves a traverser to a different part of the graph so the References/counters should still be safe.
    https://github.com/apache/tinkerpop/blob/a80eb84169048ed74c5ad27ebc4d12944fd0136a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/Attachable.java#L62


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by dkuppitz <gi...@git.apache.org>.
Github user dkuppitz commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202114725
  
    --- Diff: gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/branch/RepeatTest.java ---
    @@ -386,5 +531,51 @@ public void g_V_hasXname_markoX_repeatXoutE_inV_simplePathX_untilXhasXname_rippl
             public Traversal<Vertex, Path> get_g_V_hasXloop_name_loopX_repeatXinX_timesX5X_path_by_name() {
                 return g.V().has("loops","name","loop").repeat(__.in()).times(5).path().by("name");
             }
    +
    +        @Override
    +        public Traversal<Vertex, Path> get_g_V_repeatXout_repeatXoutX_timesX1XX_timesX1X_limitX1X_path_by_name() {
    +            // NB We need to prevent the RepeatUnrollStrategy from applying to properly exercise this test as this traversal can be simplified
    +            return g.V().repeat(out().repeat(out()).times(1)).times(1).limit(1).path().by("name");
    +        }
    +
    +        @Override
    +        public Traversal<Vertex, Path> get_g_V_repeatXoutXknowsXX_untilXrepeatXoutXcreatedXX_emitXhasXname_lopXXX_path_byXnameX() {
    +            return g.V().repeat(out("knows")).until(__.repeat(out("created")).emit(__.has("name", "lop"))).path().by("name");
    +        }
    +
    +        @Override
    +        public Traversal<Vertex, String> get_g_V_repeatXrepeatXout_createdXX_untilXhasXname_rippleXXXemit_lang() {
    +            return g.V().repeat(__.repeat(out("created")).until(__.has("name", "ripple"))).emit().values("lang");
    +        }
    +
    +        @Override
    +        public Traversal<Vertex, String> get_g_V_untilXconstantXtrueXX_repeatXrepeatXout_createdXX_untilXhasXname_rippleXXXemit_lang() {
    +            return g.V().until(__.constant(true)).repeat(__.repeat(out("created")).until(__.has("name", "ripple"))).emit().values("lang");
    +        }
    +
    +        @Override
    +        public Traversal<Vertex, String> get_g_VX3X_repeatXbothX_createdXX_untilXloops_is_40XXemit_repeatXin_knowsXX_emit_loopsXisX1Xdedup_values(final Object v3Id) {
    +            return g.V(v3Id).repeat(__.both("created")).until(loops().is(40)).emit(__.repeat(__.in("knows")).emit(loops().is(1))).dedup().values("name");
    +        }
    +
    +        @Override
    +        public Traversal<Vertex, String> get_g_V_repeatXa_outXknows_repeatXb_outXcreatedX_filterXloops_isX0XX_emit_lang() {
    +            return g.V().repeat("a", out("knows").repeat("b", out("created").filter(loops("a").is(0))).emit()).emit().values("lang");
    +        }
    +
    +        @Override
    +        public Traversal<Vertex, String> get_g_V_emit_repeatXa_outXknows_filterXloops_isX0XX_lang() {
    +            return g.V().emit().repeat("a", out("knows").filter(loops("a").is(0))).values("lang");
    +        }
    +
    +        @Override
    +        public Traversal<Vertex, String> get_g_VX6X_repeatXa_bothXcreatedXX_emitXrepeatXb_bothXknowsXX_untilXorXloops_isX2X_loopsXbX_isXloopsXaXXXX_hasXname_vadasXX_dedup_name(final Object v6Id) {
    +            return g.V(v6Id).repeat("a", both("created")).emit(__.repeat("b", __.both("knows")).until(__.or(loops().is(2), loops("b").is(loops("a")))).has("name", "vadas")).dedup().values("name");
    --- End diff --
    
    See my comment in the asciidoc file (same query).


---

[GitHub] tinkerpop issue #876: TINKERPOP-967 Support nested-repeat() structures

Posted by spmallette <gi...@git.apache.org>.
Github user spmallette commented on the issue:

    https://github.com/apache/tinkerpop/pull/876
  
    This PR shows a pretty nice level of understanding of the intricacies of the core of how Gremlin works.  We rarely get PRs that touch this area of TinkerPop to any depth. I found it interesting that most of the plumbing for this feature was already there. It mostly just needed a new `Traverser` species in place to make it all come together. 
    
    I made a separate comment on this PR to expand a unit test case, but other than that, it looks pretty good. As a minor nit, @GCHQResearcher1337 if you could please make a second pass through your changes and `final` any variables you see that can be marked as such (as that is our style). 
    
    @dkuppitz could you take a look at the Gremlin tests added here in detail and see if you would recommend any others?
    
    Anyway, that's my initial pass at this. I expect to give it a second look before I provide my VOTE. I would also like to see feedback from @dkuppitz who is on holiday until next week, so I probably won't have any other feedback until then.


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by dkuppitz <gi...@git.apache.org>.
Github user dkuppitz commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202438650
  
    --- Diff: gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/B_LP_NL_O_P_S_SE_SL_Traverser.java ---
    @@ -0,0 +1,156 @@
    +/*
    + * 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.traverser;
    +
    +import org.apache.commons.collections.map.ReferenceMap;
    +import org.apache.tinkerpop.gremlin.process.traversal.Step;
    +import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.LabelledCounter;
    +
    +
    +import java.util.Iterator;
    +import java.util.Stack;
    +
    +public class B_LP_NL_O_P_S_SE_SL_Traverser<T> extends B_LP_O_P_S_SE_SL_Traverser<T> {
    +
    +    protected Stack<LabelledCounter> nestedLoops;
    +    protected ReferenceMap loopNames = null;
    --- End diff --
    
    So, for my own education, does that mean that `ReferenceMap` automagically removes entries from the map, when there's no more reference to the value (e.g. when you pop a labeled counter from the stack)?


---

[GitHub] tinkerpop issue #876: TINKERPOP-967 Support nested-repeat() structures

Posted by spmallette <gi...@git.apache.org>.
Github user spmallette commented on the issue:

    https://github.com/apache/tinkerpop/pull/876
  
    @GCHQResearcher1337  I should have also added in my last comment:
    
    1. Please add [CHANGELOG ](https://github.com/apache/tinkerpop/blob/master/CHANGELOG.asciidoc) entries as necessary to 3.4.0
    2. This feature is sufficiently interesting that it probably deserves some special attention. Do you mind writing up something to introduce this change to users in the [Upgrade Documentation](https://github.com/apache/tinkerpop/blob/master/docs/src/upgrade/release-3.4.x.asciidoc) for 3.4.0. 
    
    If you like, I can handle the upgrade docs for you when we go to merge. Just let me know how you'd like to proceed.


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by GCHQResearcher1337 <gi...@git.apache.org>.
Github user GCHQResearcher1337 commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202418371
  
    --- Diff: gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/util/LabelledCounter.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * 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.traverser.util;
    +
    +import org.apache.commons.lang.mutable.MutableShort;
    +
    +import java.io.Serializable;
    +
    +/**
    + * Class to track a count associated with a Label
    + */
    +public class LabelledCounter implements Serializable, Cloneable {
    +
    +    private final String label;
    +    private final MutableShort count = new MutableShort();
    --- End diff --
    
    Whoops that my mistake. I'll change it to a primitive short. 
    
    It's a short to match the loop counters used elsewhere e.g.
    https://github.com/GCHQResearcher1337/tinkerpop/blob/a80eb84169048ed74c5ad27ebc4d12944fd0136a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/B_O_S_SE_SL_Traverser.java#L34
    
    But I originally used a Pair<> before switching to a class, which prevented me from using primitive types.


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by GCHQResearcher1337 <gi...@git.apache.org>.
Github user GCHQResearcher1337 commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r196370588
  
    --- Diff: gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/branch/RepeatTest.java ---
    @@ -386,5 +456,26 @@ public void g_V_hasXname_markoX_repeatXoutE_inV_simplePathX_untilXhasXname_rippl
             public Traversal<Vertex, Path> get_g_V_hasXloop_name_loopX_repeatXinX_timesX5X_path_by_name() {
                 return g.V().has("loops","name","loop").repeat(__.in()).times(5).path().by("name");
             }
    +
    +        @Override
    +        public Traversal<Vertex, Path> get_g_V_repeatXout_repeatXoutX_timesX1XX_timesX1X_limitX1X_path_by_name() {
    +            // NB We need to prevent the RepeatUnrollStrategy from applying to properly exercise this test as this traversal can be simplified
    --- End diff --
    
    Ok, I've added this as ad35a6a to prevent RepeatUnrollStrategy from inserting a NoOpBarrierStep immediately after another NoOpBarrierStep.


---

[GitHub] tinkerpop issue #876: TINKERPOP-967 Support nested-repeat() structures

Posted by GCHQResearcher1337 <gi...@git.apache.org>.
Github user GCHQResearcher1337 commented on the issue:

    https://github.com/apache/tinkerpop/pull/876
  
    @spamallete I think I steered away from that because I initially misunderstood what marko meant - as you pointed out the plumbing (`incrLoops` taking a `stepLabel`) for nested repeats is already there, but I didn't find any plumbing had been done for this (neither `times()` nor `loop()` can take a `stepLabel`) so I thought that the 'work on `LoopStep`' had been abandoned. 
    
    Re-reading this I think I see what is involved and can implement this - both `TimesModulating` and `LoopStep` as well as the `loops()` method of a traverser need to be able to take a stepLabel. We would need to maintain a separate user-defined stepLabel that could be accessed.
    
    I can see why `loops('a')` could be helpful, but I don't know why you would want to do `times('a',2)`. Could you please provide an example of this?
    
    PS Hi Marko! Thanks for Gremlin!


---

[GitHub] tinkerpop issue #876: TINKERPOP-967 Support nested-repeat() structures

Posted by spmallette <gi...@git.apache.org>.
Github user spmallette commented on the issue:

    https://github.com/apache/tinkerpop/pull/876
  
    wow - this is a cool pull request. it may take some time to go through review as it is a non-trivial change. thanks.


---

[GitHub] tinkerpop pull request #876: TINKERPOP-967 Support nested-repeat() structure...

Posted by GCHQResearcher1337 <gi...@git.apache.org>.
Github user GCHQResearcher1337 commented on a diff in the pull request:

    https://github.com/apache/tinkerpop/pull/876#discussion_r202415964
  
    --- Diff: gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/B_LP_NL_O_P_S_SE_SL_Traverser.java ---
    @@ -0,0 +1,156 @@
    +/*
    + * 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.traverser;
    +
    +import org.apache.commons.collections.map.ReferenceMap;
    +import org.apache.tinkerpop.gremlin.process.traversal.Step;
    +import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.LabelledCounter;
    +
    +
    +import java.util.Iterator;
    +import java.util.Stack;
    +
    +public class B_LP_NL_O_P_S_SE_SL_Traverser<T> extends B_LP_O_P_S_SE_SL_Traverser<T> {
    +
    +    protected Stack<LabelledCounter> nestedLoops;
    +    protected ReferenceMap loopNames = null;
    --- End diff --
    
    I'm using a map with weak values to simplify the `resetLoops()` call so that I don't need to walk the Map or store the 'loop name' multiple times.


---