You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@tinkerpop.apache.org by spmallette <gi...@git.apache.org> on 2017/11/12 23:22:28 UTC

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

GitHub user spmallette opened a pull request:

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

    TINKERPOP-1784 GLV Test Framework

    https://issues.apache.org/jira/browse/TINKERPOP-1784
    
    This PR is for the GLV Test Framework. It contains an implementation for gremlin-python. It is not a complete porting of all the process test suite, but does provide coverage for almost all steps. I think I've built enough here to provide enough for evaluation of the framework itself - my intention is to backfill tests after this merges. I do think I have enough tests here to yield confidence in GLVs that implement it, thus allowing baseline for official release.
    
    All tests pass with `docker/build.sh -t -n -i`
    
    VOTE +1

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

    $ git pull https://github.com/apache/tinkerpop TINKERPOP-1784

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

    https://github.com/apache/tinkerpop/pull/747.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 #747
    
----
commit 61e667ddb91eb96560793843054678d4c0a925de
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-14T19:44:32Z

    TINKERPOP-1784 Initial implementation of a new language agnostic test suite
    
    Uses Gherkin to write test specifications that will be implemented by the various GLVs. Provided a basic implementation for gremlin-python.

commit ce16228ea8780b0ff60f0ff1a11a4a127dde6744
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-20T17:32:54Z

    TINKERPOP-1784 Use python eval() to setup test traversals

commit 2b92f5b20ba5e8c9f7e49f1e5ac7428c284d6603
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-20T17:39:00Z

    TINKERPOP-1784 Categorize feature by step type
    
    This matches he pattern of the java test suite.

commit eb595a6c737fa0d49df3c0062c7c463088fecda6
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-20T22:38:49Z

    TINKERPOP-1784 Added coin() gherkin tests and refactored python

commit 7aa1adcb3d1a719cb75b218bb9fa3a83112213de
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-22T14:52:20Z

    TINKERPOP-1784 Expanded GLV test framework a bit further
    
    Developed methods for vertices/maps and a way to assert unordered results.

commit b129ed30b02388710e9c39338026462d4cf88955
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-22T16:56:08Z

    TINKERPOP-1784 Added some basic support to convert vertex string to a vertex object
    
    This might be the pattern to use across the board. We'll see if there is a better idea floating about though so may not be final.

commit de32b78efe432e6b57399ca9db673ddeead313ad
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-22T17:49:50Z

    TINKERPOP-1784 Minor refactoring of python gherkin steps

commit 35aca028cbc0ccd96823e842626169b7d7b7f1e7
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-22T18:43:13Z

    TINKERPOP-1784 Added support for numeric keys in glv tests

commit 14b8d3bbeacf0ee1ec500174009c8487aa1238a6
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-23T11:12:07Z

    TINKERPOP-1784 Added test for select in GLV tests
    
    Included infrastructure for validating maps and refactored other related code.

commit 17fd3b027effc0f7b055e1934307b2c4b90ed84b
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-27T16:16:08Z

    TINKERPOP-1784 Provided translation for python specific syntax
    
    Included a way to specify element identifiers in feature files.

commit 655377380654755fb031b9d92e10fd9376665b68
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-27T17:26:17Z

    TINKERPOP-1784 Added test to enforce implementation of process tests as features

commit db6c8560d883891d0c3c53ab0fc7a9b40adca9d3
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-27T20:18:14Z

    TINKERPOP-1784 Added edge assertion support

commit f33b2b8d588d68e0374958b5aa37b32c9dc59211
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-28T14:29:47Z

    TINKERPOP-1784 Added some more vertex tests

commit 55145980b99ed78a1de82ecefe199062439ea573
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-28T18:54:40Z

    TINKERPOP-1784 Changed assertion logic and table formats in feature files

commit bed1d7c5e7aa0d1cd7fbed9914c541c48c79df86
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-28T19:17:14Z

    TINKERPOP-1784 Refactored the assertion logic for ordered/unordered

commit a9f273f603c08d958a4a25cdc7504a9a1a91bd30
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-28T19:30:54Z

    TINKERPOP-1784 Deleted some dead code

commit 80a5a0121e36c6380619de59361ebffdd40e9287
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-29T14:29:38Z

    TINKERPOP-1784 Add support for the various toy graphs

commit 9975980bca2fe1f50f05d53b4fd2f6ab6b6bb4a7
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-29T15:54:33Z

    TINKERPOP-1784 Included grateful graph and cached remotes/data
    
    Tests should be faster now that remotes and data are cached for the toy graphs.

commit e46e3a2f1828f78ba6bdd84e1e3e6c440a42d043
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-29T17:26:27Z

    TINKERPOP-1784 Added some more vertex features

commit ea41b581cdb127e09fee7819c02b0bbf43532bf9
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-09-29T19:44:50Z

    TINKERPOP-1784 Completed the VertexTest cases

commit c54e3bdffaab511b544286ceda06a3c1088d62be
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-10-01T11:28:31Z

    TINKERPOP-1784 Added do nothings for unsupported tests

commit 2a5298ae80a2416ccdb83e8091f7959f8a9ed616
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-10-02T18:59:59Z

    TINKERPOP-1784 Get all count() tests working
    
    Needed to add support for cleaning up traversal strings to more properly handle reserved python words.

commit 933b89f28addc2de200e02cf5cdb164b60605c7a
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-10-03T17:06:19Z

    TINKERPOP-1784 Added another feature test for groupCount()

commit e718176486f4406d3cfcf90aae0e7528ee156523
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-10-03T19:48:49Z

    TINKERPOP-1784 Add all tests for groupCount()

commit 8fddf699995c56b9e075188b6c83403a37a02b12
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-10-05T13:23:09Z

    TINKERPOP-1784 Added more select() tests

commit 47e6c531653cfb29e3b52ffb12792a48be1ebf08
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-10-05T15:49:19Z

    TINKERPOP-1784 Added more select() tests

commit 918a56ec6877d75fe348d8e7a5ce4dc66706ca7a
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-10-10T17:17:55Z

    TINKERPOP-1784 Added support for testing branch() with gherkin

commit 39b8417185292e6f7fb0e3c099439a6d7da94ae8
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-10-10T19:18:55Z

    TINKERPOP-1784 Define traversals without shorthand
    
    Sorta made the decision to be explicit in how traversals are defined in .feature files. Shying away from using shorthand static imports which will make it more explicit for those trying to implement the tests.

commit 4a6772c6755a791992747bd4ff9e1c2b9ac80fac
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-10-10T19:39:40Z

    TINKERPOP-1784 Added multi-line support in the .feature files

commit 6d2d835542a022baca0e6042a48e06811e73dfc2
Author: Stephen Mallette <sp...@genoprime.com>
Date:   2017-10-11T14:52:15Z

    TINKERPOP-1784 Added Choose tests to .feature files

----


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r150517147
  
    --- Diff: gremlin-dotnet/src/Gremlin.Net/Driver/Remote/DriverRemoteConnection.cs ---
    @@ -36,15 +36,27 @@ namespace Gremlin.Net.Driver.Remote
         public class DriverRemoteConnection : IRemoteConnection, IDisposable
         {
             private readonly IGremlinClient _client;
    +        private readonly string _traversalSource;
     
             /// <summary>
             ///     Initializes a new <see cref="IRemoteConnection" />.
             /// </summary>
             /// <param name="client">The <see cref="IGremlinClient" /> that will be used for the connection.</param>
             /// <exception cref="ArgumentNullException">Thrown when client is null.</exception>
    -        public DriverRemoteConnection(IGremlinClient client)
    +        public DriverRemoteConnection(IGremlinClient client):this(client, "g")
    +        {
    +        }
    +
    +        /// <summary>
    +        ///     Initializes a new <see cref="IRemoteConnection" />.
    +        /// </summary>
    +        /// <param name="client">The <see cref="IGremlinClient" /> that will be used for the connection.</param>
    +        /// <param name="traversalSource">The name of the traversal source on the server to bind to.</param>
    +        /// <exception cref="ArgumentNullException">Thrown when client is null.</exception>
    +        public DriverRemoteConnection(IGremlinClient client, string traversalSource)
             {
                 _client = client ?? throw new ArgumentNullException(nameof(client));
    +            _traversalSource = traversalSource;
    --- End diff --
    
    We could add null validation in the same form: 
    ```csharp
    _traversalSource = traversalSource ?? throw new ArgumentNullException(nameof(traversalSource));
    ```


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r150531652
  
    --- Diff: gremlin-test/src/test/java/org/apache/tinkerpop/gremlin/process/FeatureCoverageTest.java ---
    @@ -0,0 +1,131 @@
    +/*
    + * 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;
    +
    +import org.apache.tinkerpop.gremlin.process.traversal.step.branch.BranchTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.branch.ChooseTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.branch.OptionalTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.CoinTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.DropTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.FilterTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.IsTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.OrTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.map.CountTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.map.PathTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.map.ProjectTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.map.ValueMapTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.map.VertexTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.AggregateTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.GroupCountTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.InjectTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.StoreTest;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +import java.lang.reflect.Field;
    +import java.lang.reflect.Method;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +import java.util.stream.Stream;
    +
    +import static org.hamcrest.core.Is.is;
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertThat;
    +
    +/**
    + * @author Stephen Mallette (http://stephen.genoprime.com)
    + */
    +public class FeatureCoverageTest {
    +
    +    private static Pattern scenarioName = Pattern.compile("^\\s*Scenario:\\s*(.*)$");
    +
    +    @Test
    +    @Ignore("As it stands we won't have all of these tests migrated initially so there is no point to running this in full - it can be flipped on later")
    +    public void shouldImplementAllProcessTestsAsFeatures() throws Exception {
    --- End diff --
    
    As long as we continue to rely on the java process test suite I think it should run on every build. I can see people adding tests to the process suite but not to the GLV suite. Better to just catch it on `mvn clean install` than on a periodic test run to ensure the suites stay in sync. It's not a lengthy test I don't think....pretty fast to do the comparisons. 


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r150530902
  
    --- Diff: gremlin-python/src/main/jython/radish/feature_steps.py ---
    @@ -0,0 +1,231 @@
    +'''
    +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.
    +'''
    +
    +import json
    +import re
    +from gremlin_python.structure.graph import Graph, Path
    +from gremlin_python.process.graph_traversal import __
    +from gremlin_python.process.traversal import Cardinality, P, Scope, Column, Order, Direction, T, Pick, Operator
    +from radish import given, when, then
    +from hamcrest import *
    +
    +regex_and = re.compile(r"([(.,\s])and\(")
    +regex_as = re.compile(r"([(.,\s])as\(")
    +regex_from = re.compile(r"([(.,\s])from\(")
    +regex_global = re.compile(r"([(.,\s])global")
    +regex_in = re.compile(r"([(.,\s])in\(")
    +regex_is = re.compile(r"([(.,\s])is\(")
    +regex_not = re.compile(r"([(.,\s])not\(")
    +regex_or = re.compile(r"([(.,\s])or\(")
    +
    +
    +ignores = [
    +    "g.V(v1Id).out().inject(v2).values(\"name\")"  # bug in attachment won't connect v2
    +           ]
    +
    +
    +@given("the {graph_name:w} graph")
    +def choose_graph(step, graph_name):
    +    step.context.g = Graph().traversal().withRemote(step.context.remote_conn[graph_name])
    +
    +
    +@given("the graph initializer of")
    +def initialize_graph(step):
    +    traversal = _make_traversal(step.context.g, step.text, {})
    +
    +    # just be sure that the traversal returns something to prove that it worked to some degree. probably
    +    # is overkill to try to assert the complete success of this init operation. presumably the test
    +    # suite would fail elsewhere if this didn't work which would help identify a problem.
    +    assert len(traversal.toList()) > 0
    +
    +
    +@given("an unsupported test")
    +def unsupported_scenario(step):
    +    # this is a do nothing step as the test can't be supported for whatever reason
    +    return
    +
    +
    +@given("using the parameter {param_name:w} defined as {param:QuotedString}")
    +def add_parameter(step, param_name, param):
    +    if not hasattr(step.context, "traversal_params"):
    +        step.context.traversal_params = {}
    +
    +    step.context.traversal_params[param_name] = _convert(param, step.context)
    +
    +
    +@given("the traversal of")
    +def translate_traversal(step):
    +    step.context.ignore = ignores.__contains__(step.text)
    +    step.context.traversal = _make_traversal(
    +        step.context.g, step.text,
    +        step.context.traversal_params if hasattr(step.context, "traversal_params") else {})
    +
    +
    +@when("iterated to list")
    +def iterate_the_traversal(step):
    +    step.context.result = map(lambda x: _convert_results(x), step.context.traversal.toList())
    +
    +
    +@when("iterated next")
    +def next_the_traversal(step):
    +    step.context.result = map(lambda x: _convert_results(x), step.context.traversal.next())
    +
    +
    +@then("the result should be {characterized_as:w}")
    +def assert_result(step, characterized_as):
    +    if step.context.ignore:
    +        return
    +
    +    if characterized_as == "empty":        # no results
    +        assert_that(len(step.context.result), equal_to(0))
    +    elif characterized_as == "ordered":    # results asserted in the order of the data table
    +        _table_assertion(step.table, step.context.result, step.context, True)
    +    elif characterized_as == "unordered":  # results asserted in any order
    +        _table_assertion(step.table, step.context.result, step.context, False)
    +    elif characterized_as == "of":         # results may be of any of the specified items in the data table
    +        _any_assertion(step.table, step.context.result, step.context)
    +    else:
    +        raise ValueError("unknown data characterization of " + characterized_as)
    +
    +
    +@then("the graph should return {count:d} for count of {traversal_string:QuotedString}")
    +def assert_side_effects(step, count, traversal_string):
    +    if step.context.ignore:
    +        return
    +
    +    t = _make_traversal(step.context.g, traversal_string.replace('\\"', '"'),
    +                        step.context.traversal_params if hasattr(step.context, "traversal_params") else {})
    +    assert_that(count, equal_to(t.count().next()))
    +
    +
    +@then("the result should have a count of {count:d}")
    +def assert_count(step, count):
    +    assert_that(len(step.context.result), equal_to(count))
    +
    +
    +@then("nothing should happen because")
    +def nothing_happening(step):
    +    return
    --- End diff --
    
    I'm not terribly pleased with how I did that actually. I just wanted a way to mark the test as "migrated" with some comment as to why it couldn't be fully implemented. Perhaps that can be improved in some way....do you think it is important for this PR?


---

[GitHub] tinkerpop issue #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747
  
    yeah - i'm just realizing how i flubbed a bunch of things on the last commit. i was so focused on your .net problems i only ran tests for .net :disappointed: 



---

[GitHub] tinkerpop issue #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747
  
    This is strange. That command works for me. I'm not sure what could be wrong.....


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r151456526
  
    --- Diff: gremlin-test/features/branch/Choose.feature ---
    @@ -0,0 +1,124 @@
    +# 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.
    +
    +Feature: Step - choose()
    +
    +  Scenario: g_V_chooseXout_countX_optionX2L__nameX_optionX3L__valueMapX
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.V().choose(__.out().count()).
    +        option(2L, __.values("name")).
    +        option(3L, __.valueMap())
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | m[{"name":["marko"], "age":[29]}] |
    --- End diff --
    
    We should specify the expected numeric type when describing map results because it has different meaning across different scenarios, here is an `int32`, below it's also a `int64` (ie: `groupCount()`) and also a `double`.
    The json literal for numbers could be reserved for `double` (JS `Number` underlying representation) and the rest we could use something like:
    - `m[{"age":"d[29]"}]`,  value is an `int32`.
    - `m[{"age":"d[29L]"}]`,  value is an `int64`.
    - `m[{"age":29}]`,  value is an `double`.


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r150542730
  
    --- Diff: gremlin-python/src/main/jython/radish/feature_steps.py ---
    @@ -0,0 +1,231 @@
    +'''
    +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.
    +'''
    +
    +import json
    +import re
    +from gremlin_python.structure.graph import Graph, Path
    +from gremlin_python.process.graph_traversal import __
    +from gremlin_python.process.traversal import Cardinality, P, Scope, Column, Order, Direction, T, Pick, Operator
    +from radish import given, when, then
    +from hamcrest import *
    +
    +regex_and = re.compile(r"([(.,\s])and\(")
    +regex_as = re.compile(r"([(.,\s])as\(")
    +regex_from = re.compile(r"([(.,\s])from\(")
    +regex_global = re.compile(r"([(.,\s])global")
    +regex_in = re.compile(r"([(.,\s])in\(")
    +regex_is = re.compile(r"([(.,\s])is\(")
    +regex_not = re.compile(r"([(.,\s])not\(")
    +regex_or = re.compile(r"([(.,\s])or\(")
    +
    +
    +ignores = [
    +    "g.V(v1Id).out().inject(v2).values(\"name\")"  # bug in attachment won't connect v2
    +           ]
    +
    +
    +@given("the {graph_name:w} graph")
    +def choose_graph(step, graph_name):
    +    step.context.g = Graph().traversal().withRemote(step.context.remote_conn[graph_name])
    +
    +
    +@given("the graph initializer of")
    +def initialize_graph(step):
    +    traversal = _make_traversal(step.context.g, step.text, {})
    +
    +    # just be sure that the traversal returns something to prove that it worked to some degree. probably
    +    # is overkill to try to assert the complete success of this init operation. presumably the test
    +    # suite would fail elsewhere if this didn't work which would help identify a problem.
    +    assert len(traversal.toList()) > 0
    +
    +
    +@given("an unsupported test")
    +def unsupported_scenario(step):
    +    # this is a do nothing step as the test can't be supported for whatever reason
    +    return
    +
    +
    +@given("using the parameter {param_name:w} defined as {param:QuotedString}")
    +def add_parameter(step, param_name, param):
    +    if not hasattr(step.context, "traversal_params"):
    +        step.context.traversal_params = {}
    +
    +    step.context.traversal_params[param_name] = _convert(param, step.context)
    +
    +
    +@given("the traversal of")
    +def translate_traversal(step):
    +    step.context.ignore = ignores.__contains__(step.text)
    +    step.context.traversal = _make_traversal(
    +        step.context.g, step.text,
    +        step.context.traversal_params if hasattr(step.context, "traversal_params") else {})
    +
    +
    +@when("iterated to list")
    +def iterate_the_traversal(step):
    +    step.context.result = map(lambda x: _convert_results(x), step.context.traversal.toList())
    +
    +
    +@when("iterated next")
    +def next_the_traversal(step):
    +    step.context.result = map(lambda x: _convert_results(x), step.context.traversal.next())
    +
    +
    +@then("the result should be {characterized_as:w}")
    +def assert_result(step, characterized_as):
    +    if step.context.ignore:
    +        return
    +
    +    if characterized_as == "empty":        # no results
    +        assert_that(len(step.context.result), equal_to(0))
    +    elif characterized_as == "ordered":    # results asserted in the order of the data table
    +        _table_assertion(step.table, step.context.result, step.context, True)
    +    elif characterized_as == "unordered":  # results asserted in any order
    +        _table_assertion(step.table, step.context.result, step.context, False)
    +    elif characterized_as == "of":         # results may be of any of the specified items in the data table
    +        _any_assertion(step.table, step.context.result, step.context)
    +    else:
    +        raise ValueError("unknown data characterization of " + characterized_as)
    +
    +
    +@then("the graph should return {count:d} for count of {traversal_string:QuotedString}")
    +def assert_side_effects(step, count, traversal_string):
    +    if step.context.ignore:
    +        return
    +
    +    t = _make_traversal(step.context.g, traversal_string.replace('\\"', '"'),
    +                        step.context.traversal_params if hasattr(step.context, "traversal_params") else {})
    +    assert_that(count, equal_to(t.count().next()))
    +
    +
    +@then("the result should have a count of {count:d}")
    +def assert_count(step, count):
    +    assert_that(len(step.context.result), equal_to(count))
    +
    +
    +@then("nothing should happen because")
    +def nothing_happening(step):
    +    return
    --- End diff --
    
    I think including steps on the feature to denote that can't be migrated is OK and the format is OK. I was commenting on the way python feature handles it, but it's not directly related to the test suite.
    I agree that its not important for this patch.


---

[GitHub] tinkerpop issue #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747
  
    I think that there are a lot of path issues when building from outside of the root directory - gremlin-dotnet is not the only one. i typically just do this:
    
    ```text
    mvn clean install -pl :gremlin-dotnet-source,:gremlin-dotnet-tests -DskipIntegrationTests=false
    ```
    
    If you think it's important to fix the path problems you get when running from a sub-directory, I think it's best to create an issue in JIRA and I can come back to it later.


---

[GitHub] tinkerpop issue #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747
  
    @jorgebay fixed all the nits - thanks.


---

[GitHub] tinkerpop issue #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747
  
    Once I realized that the failure was not preventing the build it was easy to spot the problem. Just pushed a fix. 


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r150497558
  
    --- Diff: gremlin-python/src/main/jython/radish/feature_steps.py ---
    @@ -0,0 +1,231 @@
    +'''
    +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.
    +'''
    +
    +import json
    +import re
    +from gremlin_python.structure.graph import Graph, Path
    +from gremlin_python.process.graph_traversal import __
    +from gremlin_python.process.traversal import Cardinality, P, Scope, Column, Order, Direction, T, Pick, Operator
    +from radish import given, when, then
    +from hamcrest import *
    +
    +regex_and = re.compile(r"([(.,\s])and\(")
    +regex_as = re.compile(r"([(.,\s])as\(")
    +regex_from = re.compile(r"([(.,\s])from\(")
    +regex_global = re.compile(r"([(.,\s])global")
    +regex_in = re.compile(r"([(.,\s])in\(")
    +regex_is = re.compile(r"([(.,\s])is\(")
    +regex_not = re.compile(r"([(.,\s])not\(")
    +regex_or = re.compile(r"([(.,\s])or\(")
    +
    +
    +ignores = [
    +    "g.V(v1Id).out().inject(v2).values(\"name\")"  # bug in attachment won't connect v2
    +           ]
    +
    +
    +@given("the {graph_name:w} graph")
    +def choose_graph(step, graph_name):
    +    step.context.g = Graph().traversal().withRemote(step.context.remote_conn[graph_name])
    +
    +
    +@given("the graph initializer of")
    +def initialize_graph(step):
    +    traversal = _make_traversal(step.context.g, step.text, {})
    +
    +    # just be sure that the traversal returns something to prove that it worked to some degree. probably
    +    # is overkill to try to assert the complete success of this init operation. presumably the test
    +    # suite would fail elsewhere if this didn't work which would help identify a problem.
    +    assert len(traversal.toList()) > 0
    +
    +
    +@given("an unsupported test")
    +def unsupported_scenario(step):
    +    # this is a do nothing step as the test can't be supported for whatever reason
    +    return
    +
    +
    +@given("using the parameter {param_name:w} defined as {param:QuotedString}")
    +def add_parameter(step, param_name, param):
    +    if not hasattr(step.context, "traversal_params"):
    +        step.context.traversal_params = {}
    +
    +    step.context.traversal_params[param_name] = _convert(param, step.context)
    +
    +
    +@given("the traversal of")
    +def translate_traversal(step):
    +    step.context.ignore = ignores.__contains__(step.text)
    +    step.context.traversal = _make_traversal(
    +        step.context.g, step.text,
    +        step.context.traversal_params if hasattr(step.context, "traversal_params") else {})
    +
    +
    +@when("iterated to list")
    +def iterate_the_traversal(step):
    +    step.context.result = map(lambda x: _convert_results(x), step.context.traversal.toList())
    +
    +
    +@when("iterated next")
    +def next_the_traversal(step):
    +    step.context.result = map(lambda x: _convert_results(x), step.context.traversal.next())
    +
    +
    +@then("the result should be {characterized_as:w}")
    +def assert_result(step, characterized_as):
    +    if step.context.ignore:
    +        return
    +
    +    if characterized_as == "empty":        # no results
    +        assert_that(len(step.context.result), equal_to(0))
    +    elif characterized_as == "ordered":    # results asserted in the order of the data table
    +        _table_assertion(step.table, step.context.result, step.context, True)
    +    elif characterized_as == "unordered":  # results asserted in any order
    +        _table_assertion(step.table, step.context.result, step.context, False)
    +    elif characterized_as == "of":         # results may be of any of the specified items in the data table
    +        _any_assertion(step.table, step.context.result, step.context)
    +    else:
    +        raise ValueError("unknown data characterization of " + characterized_as)
    +
    +
    +@then("the graph should return {count:d} for count of {traversal_string:QuotedString}")
    +def assert_side_effects(step, count, traversal_string):
    +    if step.context.ignore:
    +        return
    +
    +    t = _make_traversal(step.context.g, traversal_string.replace('\\"', '"'),
    +                        step.context.traversal_params if hasattr(step.context, "traversal_params") else {})
    +    assert_that(count, equal_to(t.count().next()))
    +
    +
    +@then("the result should have a count of {count:d}")
    +def assert_count(step, count):
    +    assert_that(len(step.context.result), equal_to(count))
    +
    +
    +@then("nothing should happen because")
    +def nothing_happening(step):
    +    return
    --- End diff --
    
    Shouldn't this be marked as `ignore` instead of `success`?


---

[GitHub] tinkerpop issue #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747
  
    Nice! lgtm
    
    VOTE +1


---

[GitHub] tinkerpop issue #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747
  
    I've rebased this on top of `tp32` (after #752 was merged) on my private fork and the issue I've mentioned earlier persists: https://travis-ci.org/jorgebay/tinkerpop/jobs/305703717#L7380 (Ubuntu).
    
    I'm changing my VOTE to -1 until this is resolved.


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r151676302
  
    --- Diff: gremlin-test/features/branch/Choose.feature ---
    @@ -0,0 +1,124 @@
    +# 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.
    +
    +Feature: Step - choose()
    +
    +  Scenario: g_V_chooseXout_countX_optionX2L__nameX_optionX3L__valueMapX
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.V().choose(__.out().count()).
    +        option(2L, __.values("name")).
    +        option(3L, __.valueMap())
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | m[{"name":["marko"], "age":[29]}] |
    --- End diff --
    
    *sigh*  - stupid numbers. i wondered when that would be a problem. obviously the problem is sooner than later. I think will implement in a manner similar to how i designated identifiers on vertices, by adding a "modulator" after the "d" specification. More specifically:
    
    * `d[29].i` = int32
    * ` d[29].l` = int64
    * `d[29].f` = float
    * `d[29].d` = double


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r152009040
  
    --- Diff: gremlin-test/features/map/Fold.feature ---
    @@ -0,0 +1,57 @@
    +# 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.
    +
    +Feature: Step - fold()
    +
    +  Scenario: g_V_fold
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.V().fold()
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | l[v[marko],v[vadas],v[lop],v[josh],v[ripple],v[peter]] |
    +
    +  Scenario: g_V_fold_unfold
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.V().fold().unfold()
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | v[marko] |
    +      | v[vadas] |
    +      | v[lop] |
    +      | v[josh] |
    +      | v[ripple] |
    +      | v[peter]  |
    +
    +  Scenario: g_V_age_foldX0_plusX
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.V().values("age").fold(0, Operator.sum)
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | d[123].l |
    --- End diff --
    
    The expected value is an `int`.


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r151703665
  
    --- Diff: gremlin-test/features/map/Min.feature ---
    @@ -0,0 +1,51 @@
    +# Licensed to the Apache Software Foundation (ASF) under one
    +# or more contributor license agreements.  See the NOTICE file
    +# distributed with this work for additional information
    +# regarding copyright ownership.  The ASF licenses this file
    +# to you under the Apache License, Version 2.0 (the
    +# "License"); you may not use this file except in compliance
    +# with the License.  You may obtain a copy of the License at
    +#
    +# http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing,
    +# software distributed under the License is distributed on an
    +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    +# KIND, either express or implied.  See the License for the
    +# specific language governing permissions and limitations
    +# under the License.
    +
    +Feature: Step - min()
    +
    +  Scenario: g_V_age_min
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.V().values("age").min()
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | d[27].i |
    +
    +  Scenario: g_V_repeatXbothX_timesX5X_age_min
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.V().repeat(__.both()).times(5).values("age").min()
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | d[27].i |
    +
    +  Scenario: g_V_hasLabelXsoftwareX_group_byXnameX_byXbothE_weight_minX
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.V().hasLabel("software").group().by("name").by(__.bothE().values("weight").min())
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | m[{"ripple":"d[1.0].d","lop":"d[0.2]d"}] |
    --- End diff --
    
    Typo: should be `"d[0.2].d"`


---

[GitHub] tinkerpop issue #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747
  
    @jorgebay On which OS do you get the test errors?


---

[GitHub] tinkerpop issue #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747
  
    Ah, I see its possible to run integration test using:
    
    ```
    mvn clean install -pl :gremlin-dotnet-source,:gremlin-dotnet-tests -P gremlin-dotnet -DskipIntegrationTests=false
    ```
    
    I don't think its necessary to allow builds from a subdir, so don't mind... as there is a way to build the projects themselves with a profile that includes integration tests
    
    Thanks, my maven-fu is not strong at all.


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r151697112
  
    --- Diff: gremlin-test/features/map/Mean.feature ---
    @@ -0,0 +1,40 @@
    +# 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.
    +
    +Feature: Step - mean()
    +
    +  Scenario: g_V_age_mean
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.V().values("age").mean()
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | d[30.75].d |
    +
    +  Scenario: g_V_hasLabelXsoftwareX_group_byXnameX_byXbothE_weight_meanX
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.V().hasLabel("software").group().by("name").by(__.bothE().values("weight").mean())
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | m[{"ripple":"d[1.0]".d,"lop":"d[0.3333333333333333].d"}] |
    --- End diff --
    
    Typo: `"d[1.0]".d`, use `"d[1.0].d"` instead.


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r150517817
  
    --- Diff: gremlin-python/src/main/jython/radish/feature_steps.py ---
    @@ -0,0 +1,231 @@
    +'''
    +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.
    +'''
    +
    +import json
    +import re
    +from gremlin_python.structure.graph import Graph, Path
    +from gremlin_python.process.graph_traversal import __
    +from gremlin_python.process.traversal import Cardinality, P, Scope, Column, Order, Direction, T, Pick, Operator
    +from radish import given, when, then
    +from hamcrest import *
    +
    +regex_and = re.compile(r"([(.,\s])and\(")
    +regex_as = re.compile(r"([(.,\s])as\(")
    +regex_from = re.compile(r"([(.,\s])from\(")
    +regex_global = re.compile(r"([(.,\s])global")
    +regex_in = re.compile(r"([(.,\s])in\(")
    +regex_is = re.compile(r"([(.,\s])is\(")
    +regex_not = re.compile(r"([(.,\s])not\(")
    +regex_or = re.compile(r"([(.,\s])or\(")
    +
    +
    +ignores = [
    +    "g.V(v1Id).out().inject(v2).values(\"name\")"  # bug in attachment won't connect v2
    +           ]
    +
    +
    +@given("the {graph_name:w} graph")
    +def choose_graph(step, graph_name):
    +    step.context.g = Graph().traversal().withRemote(step.context.remote_conn[graph_name])
    +
    +
    +@given("the graph initializer of")
    +def initialize_graph(step):
    +    traversal = _make_traversal(step.context.g, step.text, {})
    +
    +    # just be sure that the traversal returns something to prove that it worked to some degree. probably
    +    # is overkill to try to assert the complete success of this init operation. presumably the test
    +    # suite would fail elsewhere if this didn't work which would help identify a problem.
    +    assert len(traversal.toList()) > 0
    +
    +
    +@given("an unsupported test")
    +def unsupported_scenario(step):
    +    # this is a do nothing step as the test can't be supported for whatever reason
    +    return
    +
    +
    +@given("using the parameter {param_name:w} defined as {param:QuotedString}")
    +def add_parameter(step, param_name, param):
    +    if not hasattr(step.context, "traversal_params"):
    +        step.context.traversal_params = {}
    +
    +    step.context.traversal_params[param_name] = _convert(param, step.context)
    +
    +
    +@given("the traversal of")
    +def translate_traversal(step):
    +    step.context.ignore = ignores.__contains__(step.text)
    +    step.context.traversal = _make_traversal(
    +        step.context.g, step.text,
    +        step.context.traversal_params if hasattr(step.context, "traversal_params") else {})
    +
    +
    +@when("iterated to list")
    +def iterate_the_traversal(step):
    +    step.context.result = map(lambda x: _convert_results(x), step.context.traversal.toList())
    +
    +
    +@when("iterated next")
    +def next_the_traversal(step):
    +    step.context.result = map(lambda x: _convert_results(x), step.context.traversal.next())
    +
    +
    +@then("the result should be {characterized_as:w}")
    +def assert_result(step, characterized_as):
    +    if step.context.ignore:
    +        return
    +
    +    if characterized_as == "empty":        # no results
    +        assert_that(len(step.context.result), equal_to(0))
    +    elif characterized_as == "ordered":    # results asserted in the order of the data table
    +        _table_assertion(step.table, step.context.result, step.context, True)
    +    elif characterized_as == "unordered":  # results asserted in any order
    +        _table_assertion(step.table, step.context.result, step.context, False)
    +    elif characterized_as == "of":         # results may be of any of the specified items in the data table
    +        _any_assertion(step.table, step.context.result, step.context)
    +    else:
    +        raise ValueError("unknown data characterization of " + characterized_as)
    +
    +
    +@then("the graph should return {count:d} for count of {traversal_string:QuotedString}")
    +def assert_side_effects(step, count, traversal_string):
    +    if step.context.ignore:
    +        return
    +
    +    t = _make_traversal(step.context.g, traversal_string.replace('\\"', '"'),
    +                        step.context.traversal_params if hasattr(step.context, "traversal_params") else {})
    +    assert_that(count, equal_to(t.count().next()))
    +
    +
    +@then("the result should have a count of {count:d}")
    +def assert_count(step, count):
    +    assert_that(len(step.context.result), equal_to(count))
    +
    +
    +@then("nothing should happen because")
    +def nothing_happening(step):
    +    return
    +
    +
    +def _convert(val, ctx):
    +    if isinstance(val, dict):                                         # convert dictionary keys/values
    +        n = {}
    +        for key, value in val.items():
    +            n[_convert(key, ctx)] = _convert(value, ctx)
    +        return n
    +    elif isinstance(val, unicode):                                    # convert annoying python 2.x unicode nonsense
    +        return _convert(val.encode('utf-8'), ctx)
    +    elif isinstance(val, str) and re.match("^l\[.*\]$", val):         # parse list
    +        return list(map((lambda x: _convert(x, ctx)), val[2:-1].split(",")))
    +    elif isinstance(val, str) and re.match("^s\[.*\]$", val):         # parse set
    +        return set(map((lambda x: _convert(x, ctx)), val[2:-1].split(",")))
    +    elif isinstance(val, str) and re.match("^d\[.*\]$", val):         # parse numeric
    +        return float(val[2:-1]) if val[2:-1].__contains__(".") else long(val[2:-1])
    +    elif isinstance(val, str) and re.match("^v\[.*\]\.id$", val):     # parse vertex id
    +        return ctx.lookup_v["modern"][val[2:-4]].id
    +    elif isinstance(val, str) and re.match("^v\[.*\]\.sid$", val):    # parse vertex id as string
    +        return ctx.lookup_v["modern"][val[2:-5]].id
    +    elif isinstance(val, str) and re.match("^v\[.*\]$", val):         # parse vertex
    +        return ctx.lookup_v["modern"][val[2:-1]]
    +    elif isinstance(val, str) and re.match("^e\[.*\]\.id$", val):     # parse edge id
    +        return ctx.lookup_e["modern"][val[2:-4]].id
    +    elif isinstance(val, str) and re.match("^e\[.*\]\.sid$", val):    # parse edge id as string
    +        return ctx.lookup_e["modern"][val[2:-5]].id
    +    elif isinstance(val, str) and re.match("^e\[.*\]$", val):         # parse edge
    +        return ctx.lookup_e["modern"][val[2:-1]]
    +    elif isinstance(val, str) and re.match("^m\[.*\]$", val):         # parse json as a map
    +        return _convert(json.loads(val[2:-1].replace('\\"', '"')), ctx)
    +    elif isinstance(val, str) and re.match("^p\[.*\]$", val):         # parse path
    +        path_objects = list(map((lambda x: _convert(x, ctx)), val[2:-1].split(",")))
    +        return Path([set([])], path_objects)
    +    elif isinstance(val, str) and re.match("^c\[.*\]$", val):         # parse lambda/closure
    +        return lambda: (val[2:-1], "gremlin-groovy")
    +    else:
    +        return val
    +
    +
    +def _convert_results(val):
    +    if isinstance(val, Path):
    +        # kill out labels as they aren't in the assertion logic
    +        return Path([set([])], map(lambda p: p.encode("utf-8") if isinstance(p, unicode) else p, val.objects))
    +    else:
    +        return val
    +
    +
    +def _any_assertion(data, result, ctx):
    +    converted = [_convert(line['result'], ctx) for line in data]
    +    for r in result:
    +        assert_that(r, is_in(converted))
    +
    +
    +def _table_assertion(data, result, ctx, ordered):
    +    # results from traversal should have the same number of entries as the feature data table
    +    assert_that(len(result), equal_to(len(data)))
    +
    +    results_to_test = list(result)
    +
    +    # finds a match in the results for each line of data to assert and then removes that item
    +    # from the list - in the end there should be no items left over and each will have been asserted
    +    for ix, line in enumerate(data):
    +        val = _convert(line['result'], ctx)
    +
    +        # clear the labels since we don't define them in .feature files
    +        if isinstance(val, Path):
    +            val.labels = [set([])]
    +
    +        if ordered:
    +            assert_that(results_to_test[ix], equal_to(val))
    +        else:
    +            assert_that(val, is_in(results_to_test))
    +            results_to_test.remove(val)
    +
    +    if not ordered:
    +        assert_that(len(results_to_test), is_(0))
    +
    +
    +def _translate(traversal):
    +    replaced = traversal.replace("\n", "")
    +    replaced = regex_and.sub(r"\1and_(", replaced)
    +    replaced = regex_from.sub(r"\1from_(", replaced)
    +    replaced = regex_global.sub(r"\1global_", replaced)
    +    replaced = regex_as.sub(r"\1as_(", replaced)
    +    replaced = regex_is.sub(r"\1is_(", replaced)
    +    replaced = regex_not.sub(r"\1not_(", replaced)
    +    replaced = regex_or.sub(r"\1or_(", replaced)
    +    return regex_in.sub(r"\1in_(", replaced)
    +
    +
    +def _make_traversal(g, traversal_string, params):
    +    b = {"g": g,
    +         "__": __,
    +         "Cardinality": Cardinality,
    +         "Column": Column,
    +         "Direction": Direction,
    +         "Order": Order,
    +         "P": P,
    +         "Pick": Pick,
    +         "Scope": Scope,
    +         "Operator": Operator,
    +         "T": T}
    +
    +    b.update(params)
    +
    +    # print _translate(step.text + " - " + str(b))
    --- End diff --
    
    Nit picking: remove commented line


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r151698077
  
    --- Diff: gremlin-test/features/sideEffect/Group.feature ---
    @@ -0,0 +1,244 @@
    +# 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.
    +
    +Feature: Step - group()
    +
    +  Scenario: g_V_group_byXnameX
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.V().group().by("name")
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | m[{"ripple":"l[v[ripple]]", "peter":"l[v[peter]]", "vadas":"l[v[vadas]]", "josh": "l[v[josh]]", "lop":"l[v[lop]]", "marko":"l[v[marko]]"}] |
    +
    +  Scenario: g_V_group_byXnameX_by
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.V().group().by("name").by()
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | m[{"ripple":"l[v[ripple]]", "peter":"l[v[peter]]", "vadas":"l[v[vadas]]", "josh": "l[v[josh]]", "lop":"l[v[lop]]", "marko":"l[v[marko]]"}] |
    +
    +  Scenario: g_V_groupXaX_byXnameX_capXaX
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.V().group("a").by("name").cap("a")
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | m[{"ripple":"l[v[ripple]]", "peter":"l[v[peter]]", "vadas":"l[v[vadas]]", "josh": "l[v[josh]]", "lop":"l[v[lop]]", "marko":"l[v[marko]]"}] |
    +
    +  Scenario: g_V_hasXlangX_groupXaX_byXlangX_byXnameX_out_capXaX
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.V().has("lang").group("a").by("lang").by("name").out().cap("a")
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | m[{"java":["lop","ripple"]}] |
    +
    +  Scenario: g_V_hasXlangX_group_byXlangX_byXcountX
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.V().has("lang").group().by("lang").by(__.count())
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | m[{"java":"d[2].l"}] |
    +
    +  Scenario: g_V_repeatXout_groupXaX_byXnameX_byXcountX_timesX2X_capXaX
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.V().repeat(__.out().group("a").by("name").by(__.count())).times(2).cap("a")
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | m[{"ripple":"d[2].l", "vadas":"d[1].l", "josh":"d[1].l", "lop":"d[4].l"}] |
    +
    +  Scenario: g_V_group_byXoutE_countX_byXnameX
    +    Given an unsupported test
    +    Then nothing should happen because
    +      """
    +      The result returned is not supported under GraphSON 2.x and therefore cannot be properly asserted. More
    +      specifically it has numeric keys which basically get toString()'d under GraphSON 2.x. This test can be supported
    +      with GraphSON 3.x.
    +      """
    +
    +  Scenario: g_V_groupXaX_byXlabelX_byXoutE_weight_sumX_capXaX
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.V().group("a").by(T.label).by(__.outE().values("weight").sum()).cap("a")
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | m[{"software":"d[0].i", "person":"d[3.5].d}] |
    --- End diff --
    
    Typo `"d[3.5].d"`


---

[GitHub] tinkerpop issue #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747
  
    `docker/build.sh -t -n -i` succeeds and tests pass.
    
    VOTE: +1


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r150502204
  
    --- Diff: gremlin-test/src/test/java/org/apache/tinkerpop/gremlin/process/FeatureCoverageTest.java ---
    @@ -0,0 +1,131 @@
    +/*
    + * 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;
    +
    +import org.apache.tinkerpop.gremlin.process.traversal.step.branch.BranchTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.branch.ChooseTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.branch.OptionalTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.CoinTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.DropTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.FilterTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.IsTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.OrTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.map.CountTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.map.PathTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.map.ProjectTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.map.ValueMapTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.map.VertexTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.AggregateTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.GroupCountTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.InjectTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.StoreTest;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +import java.lang.reflect.Field;
    +import java.lang.reflect.Method;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +import java.util.stream.Stream;
    +
    +import static org.hamcrest.core.Is.is;
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertThat;
    +
    +/**
    + * @author Stephen Mallette (http://stephen.genoprime.com)
    + */
    +public class FeatureCoverageTest {
    +
    +    private static Pattern scenarioName = Pattern.compile("^\\s*Scenario:\\s*(.*)$");
    +
    +    @Test
    +    @Ignore("As it stands we won't have all of these tests migrated initially so there is no point to running this in full - it can be flipped on later")
    +    public void shouldImplementAllProcessTestsAsFeatures() throws Exception {
    --- End diff --
    
    This is a good way to verify how much of the tests have been migrated, but as it should be run once / few times, maybe a script / gist outside of the repo would be better, to avoid maintaining this source code in the future.


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r151696373
  
    --- Diff: gremlin-test/features/sideEffect/Sack.feature ---
    @@ -0,0 +1,71 @@
    +# 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.
    +
    +Feature: Step - sack()
    +
    +  Scenario: g_withSackXhelloX_V_outE_sackXassignX_byXlabelX_inV_sack
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.withSack("hello").V().outE().sack(Operator.assign).by(T.label).inV().sack()
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | created |
    +      | knows |
    +      | knows |
    +      | created |
    +      | created |
    +      | created |
    +
    +  Scenario: g_withSackX0X_V_outE_sackXsumX_byXweightX_inV_sack_sum
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.withSack(0.0).V().outE().sack(Operator.sum).by("weight").inV().sack().sum()
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | d[3.5].d |
    --- End diff --
    
    The type of the result is `BigDecimal`
    ```bash
    gremlin> g.withSack(0.0).V().outE().sack(Operator.sum).by("weight").inV().sack().sum().next().getClass()
    ==>class java.math.BigDecimal
    ```
    We should introduce another suffix for decimals.


---

[GitHub] tinkerpop issue #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747
  
    I'm getting a path related issue on Gremlin.Net - Test module, while executing:
    ```
    mvn clean install -pl :gremlin-dotnet-tests -P gremlin-dotnet -DskipIntegrationTests=false
    ```
    From the root directory, I get:
    ```
    [INFO] Starting Gremlin Server instances for native testing of Apache TinkerPop :: Gremlin.Net - Tests
    [INFO] Configured Metrics Slf4jReporter configured with interval=180000ms and loggerName=org.apache.tinkerpop.gremlin.server.Settings$Slf4jReporterMetrics
    [INFO] Graph [graph] was successfully configured via [/Users/jorge/workspace/tinkerpop/gremlin-server/src/test/scripts/tinkergraph-empty.properties].
    [INFO] Graph [classic] was successfully configured via [/Users/jorge/workspace/tinkerpop/gremlin-server/src/test/scripts/tinkergraph-empty.properties].
    [INFO] Graph [modern] was successfully configured via [/Users/jorge/workspace/tinkerpop/gremlin-server/src/test/scripts/tinkergraph-empty.properties].
    [INFO] Graph [crew] was successfully configured via [/Users/jorge/workspace/tinkerpop/gremlin-server/src/test/scripts/tinkergraph-empty.properties].
    [INFO] Graph [grateful] was successfully configured via [/Users/jorge/workspace/tinkerpop/gremlin-server/src/test/scripts/tinkergraph-empty.properties].
    [INFO] Initialized Gremlin thread pool.  Threads in pool named with pattern gremlin-*
    [INFO] Loaded gremlin-groovy ScriptEngine
    [INFO] Initialized gremlin-groovy ScriptEngine with /Users/jorge/workspace/tinkerpop/gremlin-server/src/test/scripts/generate-all.groovy
    [INFO] Initialized GremlinExecutor and preparing GremlinScriptEngines instances.
    [INFO] Initialized gremlin-groovy GremlinScriptEngine and registered metrics
    [INFO] A GraphTraversalSource is now bound to [ggraph] with graphtraversalsource[tinkergraph[vertices:0 edges:0], standard]
    [INFO] A GraphTraversalSource is now bound to [g] with graphtraversalsource[tinkergraph[vertices:0 edges:0], standard]
    [INFO] A GraphTraversalSource is now bound to [gmodern] with graphtraversalsource[tinkergraph[vertices:0 edges:0], standard]
    [INFO] A GraphTraversalSource is now bound to [gclassic] with graphtraversalsource[tinkergraph[vertices:0 edges:0], standard]
    [INFO] A GraphTraversalSource is now bound to [gcrew] with graphtraversalsource[tinkergraph[vertices:0 edges:0], standard]
    [INFO] A GraphTraversalSource is now bound to [ggrateful] with graphtraversalsource[tinkergraph[vertices:0 edges:0], standard]
    [INFO] Adding the standard OpProcessor.
    [INFO] Adding the control OpProcessor.
    [INFO] Adding the session OpProcessor.
    [INFO] Adding the traversal OpProcessor.
    [INFO] Initialized cache for TraversalOpProcessor with size 1000 and expiration time of 600000 ms
    [INFO] Executing start up LifeCycleHook
    [INFO] Configured application/vnd.gremlin-v1.0+gryo with org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0
    [WARNING] The org.apache.tinkerpop.gremlin.driver.ser.GryoLiteMessageSerializerV1d0 serialization class is deprecated.
    [INFO] Configured application/vnd.gremlin-v1.0+gryo-lite with org.apache.tinkerpop.gremlin.driver.ser.GryoLiteMessageSerializerV1d0
    [INFO] Configured application/vnd.gremlin-v1.0+gryo-stringd with org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0
    [INFO] Configured application/vnd.gremlin-v1.0+json with org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0
    [INFO] Configured application/vnd.gremlin-v2.0+json with org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0
    [INFO] Configured application/json with org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0
    [INFO] Gremlin Server configured with worker thread pool of 1, gremlin pool of 8 and boss thread pool of 1.
    [INFO] Channel started at port 45940.
    [INFO] Gremlin Server with no authentication started on port 45940
    [WARNING] Graph [graph] configured at [conf/tinkergraph-empty.properties] could not be instantiated and will not be available in Gremlin Server.  GraphFactory message: The location configuration must resolve to a file and [conf/tinkergraph-empty.properties] does not
    java.lang.IllegalArgumentException: The location configuration must resolve to a file and [conf/tinkergraph-empty.properties] does not
    	at org.apache.tinkerpop.gremlin.structure.util.GraphFactory.getConfiguration(GraphFactory.java:121)
    ```
    
    The same issue appears for `classic`, `modern`, `crew`, ... graphs.


---

[GitHub] tinkerpop issue #747: TINKERPOP-1784 GLV Test Framework

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

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


---

[GitHub] tinkerpop issue #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747
  
    @robertdale thanks for looking into it.
    
    I'm running `mvn clean install -DskipTests` followed by a `mvn clean install -pl :gremlin-dotnet-tests -P gremlin-dotnet -DskipIntegrationTests=false` on a macOS Sierra.
    
    This is the output for `mvn clean install -pl :gremlin-dotnet-tests -P gremlin-dotnet -DskipIntegrationTests=false`: https://gist.github.com/jorgebay/834b28c49290038bf59df11a32ec61ad


---

[GitHub] tinkerpop issue #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747
  
    hm... with the new server start scripts, when running from within a directory like:
    
    ```
    cd gremlin-dotnet
    mvn clean install -P gremlin-dotnet
    ```
    
    I get the following issues:
    ```
    [INFO] Using Groovy 2.4.11 to perform execute.
    [INFO] Starting Gremlin Server instances for native testing of Apache TinkerPop :: Gremlin.Net - Tests
    [INFO] Configured Metrics Slf4jReporter configured with interval=180000ms and loggerName=org.apache.tinkerpop.gremlin.server.Settings$Slf4jReporterMetrics
    [INFO] Graph [graph] was successfully configured via [/Users/jorge/workspace/tinkerpop/gremlin-server/src/test/scripts/tinkergraph-empty.properties].
    [INFO] Graph [classic] was successfully configured via [/Users/jorge/workspace/tinkerpop/gremlin-server/src/test/scripts/tinkergraph-empty.properties].
    [INFO] Graph [modern] was successfully configured via [/Users/jorge/workspace/tinkerpop/gremlin-server/src/test/scripts/tinkergraph-empty.properties].
    [INFO] Graph [crew] was successfully configured via [/Users/jorge/workspace/tinkerpop/gremlin-server/src/test/scripts/tinkergraph-empty.properties].
    [INFO] Graph [grateful] was successfully configured via [/Users/jorge/workspace/tinkerpop/gremlin-server/src/test/scripts/tinkergraph-empty.properties].
    [INFO] Initialized Gremlin thread pool.  Threads in pool named with pattern gremlin-*
    [INFO] Loaded gremlin-groovy ScriptEngine
    [INFO] Initialized gremlin-groovy ScriptEngine with /Users/jorge/workspace/tinkerpop/gremlin-server/src/test/scripts/generate-all.groovy
    [INFO] Initialized GremlinExecutor and preparing GremlinScriptEngines instances.
    [INFO] Initialized gremlin-groovy GremlinScriptEngine and registered metrics
    [INFO] A GraphTraversalSource is now bound to [gclassic] with graphtraversalsource[tinkergraph[vertices:0 edges:0], standard]
    [INFO] A GraphTraversalSource is now bound to [gcrew] with graphtraversalsource[tinkergraph[vertices:0 edges:0], standard]
    [INFO] A GraphTraversalSource is now bound to [ggraph] with graphtraversalsource[tinkergraph[vertices:0 edges:0], standard]
    [INFO] A GraphTraversalSource is now bound to [g] with graphtraversalsource[tinkergraph[vertices:0 edges:0], standard]
    [INFO] A GraphTraversalSource is now bound to [gmodern] with graphtraversalsource[tinkergraph[vertices:0 edges:0], standard]
    [INFO] Adding the standard OpProcessor.
    [INFO] Adding the control OpProcessor.
    [INFO] Adding the session OpProcessor.
    [INFO] Adding the traversal OpProcessor.
    [INFO] Initialized cache for TraversalOpProcessor with size 1000 and expiration time of 600000 ms
    [INFO] Executing start up LifeCycleHook
    [INFO] Configured application/vnd.gremlin-v1.0+gryo with org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0
    [WARNING] The org.apache.tinkerpop.gremlin.driver.ser.GryoLiteMessageSerializerV1d0 serialization class is deprecated.
    [INFO] Configured application/vnd.gremlin-v1.0+gryo-lite with org.apache.tinkerpop.gremlin.driver.ser.GryoLiteMessageSerializerV1d0
    [INFO] Configured application/vnd.gremlin-v1.0+gryo-stringd with org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0
    [INFO] Configured application/vnd.gremlin-v1.0+json with org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0
    [INFO] Configured application/vnd.gremlin-v2.0+json with org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0
    [INFO] Configured application/json with org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0
    [INFO] Gremlin Server configured with worker thread pool of 1, gremlin pool of 8 and boss thread pool of 1.
    [INFO] Channel started at port 45940.
    [INFO] Gremlin Server with no authentication started on port 45940
    [WARNING] Graph [graph] configured at [conf/tinkergraph-empty.properties] could not be instantiated and will not be available in Gremlin Server.  GraphFactory message: The location configuration must resolve to a file and [conf/tinkergraph-empty.properties] does not
    java.lang.IllegalArgumentException: The location configuration must resolve to a file and [conf/tinkergraph-empty.properties] does not
    	at org.apache.tinkerpop.gremlin.structure.util.GraphFactory.getConfiguration(GraphFactory.java:121)
    	at org.apache.tinkerpop.gremlin.structure.util.GraphFactory.open(GraphFactory.java:104)
    	at  ...
    [WARNING] Graph [classic] configured at [conf/tinkergraph-empty.properties] could not be instantiated and will not be available in Gremlin Server.  GraphFactory message: The location configuration must resolve to a file and [conf/tinkergraph-empty.properties] does not
    java.lang.IllegalArgumentException: The location configuration must resolve to a file and [conf/tinkergraph-empty.properties] does not
    	at org.apache.tinkerpop.gremlin.structure.util.GraphFactory.getConfiguration(GraphFactory.java:121)
    	at org.apache.tinkerpop.gremlin.structure.util.GraphFactory.open(GraphFactory.java:104)
    	at ...
    ``` 
    
    Also, a different failure for `grateful.io(gryo()).readGraph('data/grateful-dead.kryo')`:
    
    ```
    ...
    [INFO] A GraphTraversalSource is now bound to [gmodern] with graphtraversalsource[tinkergraph[vertices:0 edges:0], standard]
    [INFO] Adding the standard OpProcessor.
    [INFO] Adding the control OpProcessor.
    [INFO] Adding the session OpProcessor.
    [INFO] Adding the traversal OpProcessor.
    [INFO] Initialized cache for TraversalOpProcessor with size 1000 and expiration time of 600000 ms
    [INFO] Executing start up LifeCycleHook
    [ERROR] Gremlin Server Error
    java.lang.reflect.UndeclaredThrowableException
    	at com.sun.proxy.$Proxy32.onStartUp(Unknown Source)
    	at org.apache.tinkerpop.gremlin.server.GremlinServer.lambda$start$52(GremlinServer.java:172)
    	at java.util.ArrayList.forEach(ArrayList.java:1249)
    	at org.apache.tinkerpop.gremlin.server.GremlinServer.start(GremlinServer.java:169)
    	at org.apache.tinkerpop.gremlin.server.GremlinServer$start.call(Unknown Source)
    	at org.codehaus.groovy.runtime.callsite.CallSiteArray.defaultCall(CallSiteArray.java:48)
    	at org.codehaus.groovy.runtime.callsite.AbstractCallSite.call(AbstractCallSite.java:113)
    	at org.codehaus.groovy.runtime.callsite.AbstractCallSite.call(AbstractCallSite.java:117)
    	at test-server-start.run(test-server-start.groovy:41)
    	at groovy.lang.GroovyShell.evaluate(GroovyShell.java:585)
    	at groovy.lang.GroovyShell.evaluate(GroovyShell.java:632)
    	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
    	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
    	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
    	at java.lang.reflect.Method.invoke(Method.java:497)
    	at org.codehaus.gmavenplus.util.ReflectionUtils.invokeMethod(ReflectionUtils.java:275)
    	at org.codehaus.gmavenplus.mojo.ExecuteMojo.executeScripts(ExecuteMojo.java:213)
    	at org.codehaus.gmavenplus.mojo.ExecuteMojo.doExecute(ExecuteMojo.java:126)
    	at org.codehaus.gmavenplus.mojo.ExecuteMojo.execute(ExecuteMojo.java:86)
    	at org.apache.maven.plugin.DefaultBuildPluginManager.executeMojo(DefaultBuildPluginManager.java:134)
    	at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:208)
    	at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:153)
    	at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:145)
    	at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:116)
    	at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:80)
    	at org.apache.maven.lifecycle.internal.builder.singlethreaded.SingleThreadedBuilder.build(SingleThreadedBuilder.java:51)
    	at org.apache.maven.lifecycle.internal.LifecycleStarter.execute(LifecycleStarter.java:128)
    	at org.apache.maven.DefaultMaven.doExecute(DefaultMaven.java:307)
    	at org.apache.maven.DefaultMaven.doExecute(DefaultMaven.java:193)
    	at org.apache.maven.DefaultMaven.execute(DefaultMaven.java:106)
    	at org.apache.maven.cli.MavenCli.execute(MavenCli.java:862)
    	at org.apache.maven.cli.MavenCli.doMain(MavenCli.java:286)
    	at org.apache.maven.cli.MavenCli.main(MavenCli.java:197)
    	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
    	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
    	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
    	at java.lang.reflect.Method.invoke(Method.java:497)
    	at org.codehaus.plexus.classworlds.launcher.Launcher.launchEnhanced(Launcher.java:289)
    	at org.codehaus.plexus.classworlds.launcher.Launcher.launch(Launcher.java:229)
    	at org.codehaus.plexus.classworlds.launcher.Launcher.mainWithExitCode(Launcher.java:415)
    	at org.codehaus.plexus.classworlds.launcher.Launcher.main(Launcher.java:356)
    Caused by: java.io.FileNotFoundException: data/grateful-dead.kryo (No such file or directory)
    	at java.io.FileInputStream.open0(Native Method)
    	at java.io.FileInputStream.open(FileInputStream.java:195)
    
    ```
    
    Running integration tests within a directory used to work and allowed fast builds... can we look into making it work?


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

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


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r152230125
  
    --- Diff: gremlin-test/features/sideEffect/Sack.feature ---
    @@ -0,0 +1,71 @@
    +# 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.
    +
    +Feature: Step - sack()
    +
    +  Scenario: g_withSackXhelloX_V_outE_sackXassignX_byXlabelX_inV_sack
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.withSack("hello").V().outE().sack(Operator.assign).by(T.label).inV().sack()
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | created |
    +      | knows |
    +      | knows |
    +      | created |
    +      | created |
    +      | created |
    +
    +  Scenario: g_withSackX0X_V_outE_sackXsumX_byXweightX_inV_sack_sum
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.withSack(0.0).V().outE().sack(Operator.sum).by("weight").inV().sack().sum()
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | d[3.5].m |
    +
    +  Scenario: g_withSackX0X_V_repeatXoutE_sackXsumX_byXweightX_inVX_timesX2X_sack
    +    Given the modern graph
    +    And the traversal of
    +      """
    +      g.withSack(0.0).V().repeat(__.outE().sack(Operator.sum).by("weight").inV()).times(2).sack()
    +      """
    +    When iterated to list
    +    Then the result should be unordered
    +      | result |
    +      | d[2.0].d |
    --- End diff --
    
    These two are also big decimal...


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r150542775
  
    --- Diff: gremlin-test/src/test/java/org/apache/tinkerpop/gremlin/process/FeatureCoverageTest.java ---
    @@ -0,0 +1,131 @@
    +/*
    + * 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;
    +
    +import org.apache.tinkerpop.gremlin.process.traversal.step.branch.BranchTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.branch.ChooseTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.branch.OptionalTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.CoinTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.DropTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.FilterTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.IsTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.filter.OrTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.map.CountTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.map.PathTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.map.ProjectTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.map.ValueMapTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.map.VertexTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.AggregateTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.GroupCountTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.InjectTest;
    +import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.StoreTest;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +
    +import java.io.BufferedReader;
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +import java.lang.reflect.Field;
    +import java.lang.reflect.Method;
    +import java.util.Arrays;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +import java.util.stream.Stream;
    +
    +import static org.hamcrest.core.Is.is;
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertThat;
    +
    +/**
    + * @author Stephen Mallette (http://stephen.genoprime.com)
    + */
    +public class FeatureCoverageTest {
    +
    +    private static Pattern scenarioName = Pattern.compile("^\\s*Scenario:\\s*(.*)$");
    +
    +    @Test
    +    @Ignore("As it stands we won't have all of these tests migrated initially so there is no point to running this in full - it can be flipped on later")
    +    public void shouldImplementAllProcessTestsAsFeatures() throws Exception {
    --- End diff --
    
    sgtm


---

[GitHub] tinkerpop issue #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747
  
    Fixes look good to me!
    I'm running integration tests locally on docker and I'll come back to give my plus one.


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r150476717
  
    --- Diff: gremlin-dotnet/src/Gremlin.Net/Driver/Remote/DriverRemoteConnection.cs ---
    @@ -36,15 +36,27 @@ namespace Gremlin.Net.Driver.Remote
         public class DriverRemoteConnection : IRemoteConnection, IDisposable
         {
             private readonly IGremlinClient _client;
    +        private readonly string _traversalSource;
    --- End diff --
    
    C# skills :)


---

[GitHub] tinkerpop pull request #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747#discussion_r150477310
  
    --- Diff: gremlin-python/pom.xml ---
    @@ -440,6 +440,17 @@ limitations under the License.
                                                 <env key="PYTHONPATH" value=""/>
                                                 <arg line="setup.py test"/>
                                             </exec>
    +                                        <!-- radish seems to like all dependencies in place -->
    +                                        <exec executable="env/bin/python" dir="${project.build.directory}/python2"
    +                                              failonerror="true">
    +                                            <env key="PYTHONPATH" value=""/>
    +                                            <arg line="setup.py install"/>
    +                                        </exec>
    +                                        <exec executable="env/bin/radish" dir="${project.build.directory}/python2"
    +                                              failonerror="true">
    +                                            <env key="PYTHONPATH" value=""/>
    +                                            <arg line="-e -t -b ${project.build.directory}/python2/radish ${project.basedir}/../gremlin-test/features/"/> <!-- -no-line-jump -->
    +                                        </exec>                                                                  
    --- End diff --
    
    Nit: undesired additional spaces after `exec` close tag.


---

[GitHub] tinkerpop issue #747: TINKERPOP-1784 GLV Test Framework

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

    https://github.com/apache/tinkerpop/pull/747
  
    Interesting. Those errors don't seem to affect the build though - it still passes. I guess it's because we don't run any tests against those particular graphs on the "secure" server? or is it failing and i'm misinterpreting the build output?


---