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

tinkerpop git commit: Traversal.nextTraverser() now exists in Gremlin-Python. Lambda serialization in Python works as expected. Removed rest_remote_connection.py as it doesn't do anything. Moved some methods in Traversal.py to GraphTraversal.py that were

Repository: tinkerpop
Updated Branches:
  refs/heads/TINKERPOP-1278 f0d0302ee -> 272dae06b


Traversal.nextTraverser() now exists in Gremlin-Python. Lambda serialization in Python works as expected. Removed rest_remote_connection.py as it doesn't do anything. Moved some methods in Traversal.py to GraphTraversal.py that were at the wrong hiearchy level. remote_connection.py uses the same semantics as Gremlin-Java -- RemoteTraversal now exists. Made websockets.py clean -- removed unneeded arguments in the protcol. Did lots of manual testing against GremlinServer/GremlinPython. Stuff is solid. We still need to get the websockets work to execute in Jython so such testing is automated. Also, we need to get sideEffects in Gremlin-Python workings.


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

Branch: refs/heads/TINKERPOP-1278
Commit: 272dae06b24e598689bfd6c5b12d2454d8434230
Parents: f0d0302
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Mon Aug 15 17:06:50 2016 -0600
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Mon Aug 15 17:06:50 2016 -0600

----------------------------------------------------------------------
 .../graphson/GraphSONTraversalSerializers.java  | 18 ++++---
 .../tinkerpop/gremlin/util/function/Lambda.java |  7 +++
 .../python/GraphTraversalSourceGenerator.groovy | 11 +++++
 .../python/TraversalSourceGenerator.groovy      | 21 ++++----
 .../jython/gremlin_python/driver/__init__.py    |  4 +-
 .../gremlin_python/driver/remote_connection.py  | 11 ++---
 .../driver/rest_remote_connection.py            | 44 -----------------
 .../driver/websocket_remote_connection.py       | 50 ++++++--------------
 .../gremlin_python/process/graph_traversal.py   | 11 +++++
 .../jython/gremlin_python/process/graphson.py   | 32 +++++--------
 .../jython/gremlin_python/process/traversal.py  | 21 ++++----
 .../src/main/jython/gremlin_python/statics.py   |  1 +
 .../gremlin_python/structure/remote_graph.py    | 10 ++--
 .../python/driver/RESTRemoteConnectionTest.java |  4 +-
 .../python/jsr223/JythonScriptEngineSetup.java  |  6 ++-
 15 files changed, 105 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/272dae06/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTraversalSerializers.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTraversalSerializers.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTraversalSerializers.java
index 89f0b42..1c02d31 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTraversalSerializers.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTraversalSerializers.java
@@ -165,7 +165,8 @@ public final class GraphSONTraversalSerializers {
             jsonGenerator.writeStringField("@type", "Lambda");
             jsonGenerator.writeStringField("value", lambda.getLambdaScript());
             jsonGenerator.writeStringField("language", lambda.getLambdaLanguage());
-            jsonGenerator.writeNumberField("arguments", lambda instanceof Lambda.ZeroArgLambda ? 0 : lambda instanceof Lambda.OneArgLambda ? 1 : 2);
+            if (!(lambda instanceof Lambda.UnknownArgLambda))
+                jsonGenerator.writeNumberField("arguments", lambda instanceof Lambda.ZeroArgLambda ? 0 : lambda instanceof Lambda.OneArgLambda ? 1 : 2);
             jsonGenerator.writeEndObject();
         }
 
@@ -346,12 +347,15 @@ public final class GraphSONTraversalSerializers {
             assert node.get("@type").textValue().equals("Lambda");
             final String lambdaScript = node.get("value").textValue();
             final String lambdaLanguage = node.get("language").textValue();
-            final int arguments = node.get("arguments").intValue();
-            return 0 == arguments ?
-                    new Lambda.ZeroArgLambda<>(lambdaScript, lambdaLanguage) :
-                    1 == arguments ?
-                            new Lambda.OneArgLambda<>(lambdaScript, lambdaLanguage) :
-                            new Lambda.TwoArgLambda<>(lambdaScript, lambdaLanguage);
+            final int arguments = node.has("argument") ? node.get("arguments").intValue() : -1;
+            if (-1 == arguments || arguments > 2)
+                return new Lambda.UnknownArgLambda(lambdaScript, lambdaLanguage);
+            else if (0 == arguments)
+                return new Lambda.ZeroArgLambda<>(lambdaScript, lambdaLanguage);
+            else if (1 == arguments)
+                return new Lambda.OneArgLambda<>(lambdaScript, lambdaLanguage);
+            else
+                return new Lambda.TwoArgLambda<>(lambdaScript, lambdaLanguage);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/272dae06/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/Lambda.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/Lambda.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/Lambda.java
index fe93525..1f6455c 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/Lambda.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/Lambda.java
@@ -73,6 +73,13 @@ public interface Lambda extends Serializable {
         }
     }
 
+    public static class UnknownArgLambda extends AbstractLambda {
+
+        public UnknownArgLambda(final String lambdaSource, final String lambdaLanguage) {
+            super(lambdaSource, lambdaLanguage);
+        }
+    }
+
     public static class ZeroArgLambda<A> extends AbstractLambda implements Supplier<A> {
 
         public ZeroArgLambda(final String lambdaSource, final String lambdaLanguage) {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/272dae06/gremlin-python/src/main/groovy/org/apache/tinkerpop/gremlin/python/GraphTraversalSourceGenerator.groovy
----------------------------------------------------------------------
diff --git a/gremlin-python/src/main/groovy/org/apache/tinkerpop/gremlin/python/GraphTraversalSourceGenerator.groovy b/gremlin-python/src/main/groovy/org/apache/tinkerpop/gremlin/python/GraphTraversalSourceGenerator.groovy
index 75f8beb..f076da9 100644
--- a/gremlin-python/src/main/groovy/org/apache/tinkerpop/gremlin/python/GraphTraversalSourceGenerator.groovy
+++ b/gremlin-python/src/main/groovy/org/apache/tinkerpop/gremlin/python/GraphTraversalSourceGenerator.groovy
@@ -112,6 +112,17 @@ under the License.
                 """class GraphTraversal(Traversal):
   def __init__(self, graph, traversal_strategies, bytecode):
     Traversal.__init__(self, graph, traversal_strategies, bytecode)
+
+  def __getitem__(self, index):
+    if isinstance(index, int):
+        return self.range(index, index + 1)
+    elif isinstance(index, slice):
+        return self.range(index.start, index.stop)
+    else:
+        raise TypeError("Index must be int or slice")
+
+  def __getattr__(self, key):
+    return self.values(key)
 """)
         GraphTraversal.getMethods()
                 .findAll { !it.name.equals("clone") }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/272dae06/gremlin-python/src/main/groovy/org/apache/tinkerpop/gremlin/python/TraversalSourceGenerator.groovy
----------------------------------------------------------------------
diff --git a/gremlin-python/src/main/groovy/org/apache/tinkerpop/gremlin/python/TraversalSourceGenerator.groovy b/gremlin-python/src/main/groovy/org/apache/tinkerpop/gremlin/python/TraversalSourceGenerator.groovy
index 12eeed5..9845595 100644
--- a/gremlin-python/src/main/groovy/org/apache/tinkerpop/gremlin/python/TraversalSourceGenerator.groovy
+++ b/gremlin-python/src/main/groovy/org/apache/tinkerpop/gremlin/python/TraversalSourceGenerator.groovy
@@ -70,17 +70,6 @@ class Traversal(object):
     def __repr__(self):
         return str(self.bytecode)
 
-    def __getitem__(self, index):
-        if isinstance(index, int):
-            return self.range(index, index + 1)
-        elif isinstance(index, slice):
-            return self.range(index.start, index.stop)
-        else:
-            raise TypeError("Index must be int or slice")
-
-    def __getattr__(self, key):
-        return self.values(key)
-
     def __iter__(self):
         return self
 
@@ -101,6 +90,16 @@ class Traversal(object):
     def toSet(self):
         return set(iter(self))
 
+    def nextTraverser(self):
+        if self.traversers is None:
+            self.traversal_strategies.apply_strategies(self)
+        if self.last_traverser is None:
+            return next(self.traversers)
+        else:
+            temp = self.last_traverser
+            self.last_traverser = None
+            return temp
+
     def next(self, amount=None):
         if amount is None:
             return self.__next__()

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/272dae06/gremlin-python/src/main/jython/gremlin_python/driver/__init__.py
----------------------------------------------------------------------
diff --git a/gremlin-python/src/main/jython/gremlin_python/driver/__init__.py b/gremlin-python/src/main/jython/gremlin_python/driver/__init__.py
index d89b59c..c9f1586 100644
--- a/gremlin-python/src/main/jython/gremlin_python/driver/__init__.py
+++ b/gremlin-python/src/main/jython/gremlin_python/driver/__init__.py
@@ -17,7 +17,7 @@ specific language governing permissions and limitations
 under the License.
 '''
 from .remote_connection import RemoteConnection
-from .remote_connection import RemoteResponse
-from .rest_remote_connection import RESTRemoteConnection
+from .remote_connection import RemoteTraversal
+from .websocket_remote_connection import WebSocketRemoteConnection
 
 __author__ = 'Marko A. Rodriguez (http://markorodriguez.com)'

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/272dae06/gremlin-python/src/main/jython/gremlin_python/driver/remote_connection.py
----------------------------------------------------------------------
diff --git a/gremlin-python/src/main/jython/gremlin_python/driver/remote_connection.py b/gremlin-python/src/main/jython/gremlin_python/driver/remote_connection.py
index 89a7898..fffa677 100644
--- a/gremlin-python/src/main/jython/gremlin_python/driver/remote_connection.py
+++ b/gremlin-python/src/main/jython/gremlin_python/driver/remote_connection.py
@@ -17,16 +17,15 @@ specific language governing permissions and limitations
 under the License.
 '''
 import abc
-
 import six
 
+from ..process.traversal import Traversal
 
 __author__ = 'Marko A. Rodriguez (http://markorodriguez.com)'
 
 
 @six.add_metaclass(abc.ABCMeta)
 class RemoteConnection(object):
-
     def __init__(self, url, traversal_source):
         self._url = url
         self._traversal_source = traversal_source
@@ -40,13 +39,13 @@ class RemoteConnection(object):
         return self._traversal_source
 
     @abc.abstractmethod
-    def submit(self, target_language, bytecode):
+    def submit(self, bytecode):
         print "sending " + bytecode + " to GremlinServer..."
-        return RemoteResponse(iter([]), {})
-
+        return RemoteTraversal(iter([]), {})
 
-class RemoteResponse(object):
 
+class RemoteTraversal(Traversal):
     def __init__(self, traversers, side_effects):
+        Traversal.__init__(self, None, None, None)
         self.traversers = traversers
         self.side_effects = side_effects

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/272dae06/gremlin-python/src/main/jython/gremlin_python/driver/rest_remote_connection.py
----------------------------------------------------------------------
diff --git a/gremlin-python/src/main/jython/gremlin_python/driver/rest_remote_connection.py b/gremlin-python/src/main/jython/gremlin_python/driver/rest_remote_connection.py
deleted file mode 100644
index aedbe82..0000000
--- a/gremlin-python/src/main/jython/gremlin_python/driver/rest_remote_connection.py
+++ /dev/null
@@ -1,44 +0,0 @@
-'''
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
-'''
-import json
-import requests
-
-from ..process.traversal import Traverser
-from .remote_connection import RemoteConnection
-from .remote_connection import RemoteResponse
-
-__author__ = 'Marko A. Rodriguez (http://markorodriguez.com)'
-
-
-class RESTRemoteConnection(RemoteConnection):
-    def __init__(self, url, traversal_source):
-        RemoteConnection.__init__(self, url, traversal_source)
-
-    def __repr__(self):
-        return "RESTRemoteConnection[" + self.url + "]"
-
-    def submit(self, target_language, bytecode):
-        response = requests.post(self.url, data=json.dumps(
-            {"gremlin": bytecode, "source": self.traversal_source, "language": target_language, "bindings": bytecode.bindings}))
-        if response.status_code != requests.codes.ok:
-            raise BaseException(response.text)
-        traversers = []
-        for x in response.json()['result']['data']:
-            traversers.append(Traverser(x, 1))
-        return RemoteResponse(iter(traversers), {})

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/272dae06/gremlin-python/src/main/jython/gremlin_python/driver/websocket_remote_connection.py
----------------------------------------------------------------------
diff --git a/gremlin-python/src/main/jython/gremlin_python/driver/websocket_remote_connection.py b/gremlin-python/src/main/jython/gremlin_python/driver/websocket_remote_connection.py
index b7eb964..926f867 100644
--- a/gremlin-python/src/main/jython/gremlin_python/driver/websocket_remote_connection.py
+++ b/gremlin-python/src/main/jython/gremlin_python/driver/websocket_remote_connection.py
@@ -18,15 +18,14 @@ under the License.
 '''
 import json
 import uuid
-
 from tornado import gen
 from tornado import ioloop
 from tornado import websocket
 
-from ..process.traversal import Traverser
 from .remote_connection import RemoteConnection
-from .remote_connection import RemoteResponse
+from .remote_connection import RemoteTraversal
 from ..process.graphson import GraphSONWriter
+from ..process.traversal import Traverser
 
 
 class GremlinServerError(Exception):
@@ -35,39 +34,29 @@ class GremlinServerError(Exception):
 
 class WebSocketRemoteConnection(RemoteConnection):
     """Remote connection to the Gremlin Server"""
-    def __init__(self, url, traversal_source, loop=None, username='',
-                 password=''):
-        super(WebSocketRemoteConnection, self).__init__(
-            url, traversal_source)
+
+    def __init__(self, url, traversal_source, loop=None, username='', password=''):
+        super(WebSocketRemoteConnection, self).__init__(url, traversal_source)
         if loop is None:
             self._loop = ioloop.IOLoop.current()
-        self._ws = self._loop.run_sync(
-            lambda: websocket.websocket_connect(self.url))
+        self._ws = self._loop.run_sync(lambda: websocket.websocket_connect(self.url))
         self._username = username
         self._password = password
 
     def submit(self,
-               target_language,
                bytecode,
                op="bytecode",
                processor="traversal",
                session=None):
-        traversers = self._loop.run_sync(lambda: self._submit(
-            target_language, bytecode, op, processor, session))
-        return RemoteResponse(iter(traversers), {})
+        traversers = self._loop.run_sync(lambda: self._submit(bytecode, op, processor, session))
+        return RemoteTraversal(iter(traversers), {})
 
     @gen.coroutine
-    def _submit(self,
-                target_language,
-                bytecode,
-                op,
-                processor,
-                session):
+    def _submit(self, bytecode, op, processor, session):
         """
-        Submit a script and bindings to the Gremlin Server
+        Submit bytecode to Gremlin Server
 
         :param str gremlin: Gremlin script to submit to server.
-        :param dict bindings: A mapping of bindings for Gremlin script.
         :param str lang: Language of scripts submitted to the server.
             "gremlin-groovy" by default
         :param str op: Gremlin Server op argument. "eval" by default.
@@ -77,18 +66,11 @@ class WebSocketRemoteConnection(RemoteConnection):
         :returns: :py:class:`Response` object
         """
         request_id = str(uuid.uuid4())
-        message = self._prepare_message(bytecode,
-                                        bytecode.bindings,
-                                        target_language,
-                                        op,
-                                        processor,
-                                        session,
-                                        request_id)
+        message = self._prepare_message(bytecode, op, processor, session, request_id)
         if self._ws.protocol is None:
             self._ws = yield websocket.websocket_connect(self.url)
         self._ws.write_message(message, binary=True)
-        resp = Response(self._ws, processor, session, self._username,
-                        self._password)
+        resp = Response(self._ws, processor, session, self._username, self._password)
         traversers = []
         while True:
             msg = yield resp.receive()
@@ -101,16 +83,13 @@ class WebSocketRemoteConnection(RemoteConnection):
         """Close underlying connection and mark as closed."""
         self._ws.close()
 
-    def _prepare_message(self, gremlin, bindings, lang, op,
-                         processor, session, request_id):
+    def _prepare_message(self, bytecode, op, processor, session, request_id):
         message = {
             "requestId": request_id,
             "op": op,
             "processor": processor,
             "args": {
-                "gremlin": GraphSONWriter.writeObject(gremlin),
-                "bindings": bindings,
-                "language":  lang,
+                "gremlin": GraphSONWriter.writeObject(bytecode),
                 "aliases": {'g': self.traversal_source}
             }
         }
@@ -151,7 +130,6 @@ class WebSocketRemoteConnection(RemoteConnection):
 
 
 class Response:
-
     def __init__(self, ws, processor, session, username, password):
         self._ws = ws
         self._closed = False

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/272dae06/gremlin-python/src/main/jython/gremlin_python/process/graph_traversal.py
----------------------------------------------------------------------
diff --git a/gremlin-python/src/main/jython/gremlin_python/process/graph_traversal.py b/gremlin-python/src/main/jython/gremlin_python/process/graph_traversal.py
index 6290065..4a94e24 100644
--- a/gremlin-python/src/main/jython/gremlin_python/process/graph_traversal.py
+++ b/gremlin-python/src/main/jython/gremlin_python/process/graph_traversal.py
@@ -78,6 +78,17 @@ class GraphTraversalSource(object):
 class GraphTraversal(Traversal):
   def __init__(self, graph, traversal_strategies, bytecode):
     Traversal.__init__(self, graph, traversal_strategies, bytecode)
+
+  def __getitem__(self, index):
+    if isinstance(index, int):
+        return self.range(index, index + 1)
+    elif isinstance(index, slice):
+        return self.range(index.start, index.stop)
+    else:
+        raise TypeError("Index must be int or slice")
+
+  def __getattr__(self, key):
+    return self.values(key)
   def V(self, *args):
     self.bytecode.add_step("V", *args)
     return self

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/272dae06/gremlin-python/src/main/jython/gremlin_python/process/graphson.py
----------------------------------------------------------------------
diff --git a/gremlin-python/src/main/jython/gremlin_python/process/graphson.py b/gremlin-python/src/main/jython/gremlin_python/process/graphson.py
index b3a1ea1..571499f 100644
--- a/gremlin-python/src/main/jython/gremlin_python/process/graphson.py
+++ b/gremlin-python/src/main/jython/gremlin_python/process/graphson.py
@@ -26,6 +26,7 @@ from .traversal import Binding
 from .traversal import Bytecode
 from .traversal import P
 from .traversal import Traversal
+from .. import statics
 
 
 class GraphSONWriter(object):
@@ -59,14 +60,14 @@ class BytecodeSerializer(GraphSONSerializer):
         sources = []
         for instruction in bytecode.source_instructions:
             inst = []
-            inst.append(_SymbolHelper.toJava(instruction[0]))
+            inst.append(_SymbolHelper.toGremlin(instruction[0]))
             for arg in instruction[1]:
                 inst.append(GraphSONWriter._dictify(arg))
             sources.append(inst)
         steps = []
         for instruction in bytecode.step_instructions:
             inst = []
-            inst.append(_SymbolHelper.toJava(instruction[0]))
+            inst.append(_SymbolHelper.toGremlin(instruction[0]))
             for arg in instruction[1]:
                 inst.append(GraphSONWriter._dictify(arg))
             steps.append(inst)
@@ -80,8 +81,8 @@ class BytecodeSerializer(GraphSONSerializer):
 class EnumSerializer(GraphSONSerializer):
     def _dictify(self, enum):
         dict = {}
-        dict["@type"] = _SymbolHelper.toJava(type(enum).__name__)
-        dict["value"] = _SymbolHelper.toJava(str(enum.name))
+        dict["@type"] = _SymbolHelper.toGremlin(type(enum).__name__)
+        dict["value"] = _SymbolHelper.toGremlin(str(enum.name))
         return dict
 
 
@@ -89,7 +90,7 @@ class PSerializer(GraphSONSerializer):
     def _dictify(self, p):
         dict = {}
         dict["@type"] = "P"
-        dict["predicate"] = _SymbolHelper.toJava(p.operator)
+        dict["predicate"] = _SymbolHelper.toGremlin(p.operator)
         if p.other is None:
             dict["value"] = GraphSONWriter._dictify(p.value)
         else:
@@ -111,9 +112,10 @@ class LambdaSerializer(GraphSONSerializer):
         lambdaString = lambdaObject()
         dict = {}
         dict["@type"] = "Lambda"
-        dict["value"] = lambdaString
-        dict["language"] = "gremlin-python"
-        dict["arguments"] = eval(lambdaString).func_code.co_argcount
+        dict["value"] = lambdaString if isinstance(lambdaString, str) else lambdaString[0]
+        dict["language"] = statics.lambdaLanguage if isinstance(lambdaString, str) else lambdaString[1]
+        if dict["language"] is "gremlin-jython":
+            dict["arguments"] = eval(dict["value"]).func_code.co_argcount
         return dict
 
 
@@ -127,18 +129,8 @@ class _SymbolHelper(object):
                  "_or": "or", "_is": "is", "_not": "not", "_from": "from"}
 
     @staticmethod
-    def toJava(symbol):
-        if (symbol in _SymbolHelper.symbolMap):
-            return _SymbolHelper.symbolMap[symbol]
-        else:
-            return symbol
-
-    @staticmethod
-    def mapEnum(enum):
-        if (enum in enumMap):
-            return enumMap[enum]
-        else:
-            return enum
+    def toGremlin(symbol):
+        return _SymbolHelper.symbolMap[symbol] if symbol in _SymbolHelper.symbolMap else symbol
 
 
 serializers = {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/272dae06/gremlin-python/src/main/jython/gremlin_python/process/traversal.py
----------------------------------------------------------------------
diff --git a/gremlin-python/src/main/jython/gremlin_python/process/traversal.py b/gremlin-python/src/main/jython/gremlin_python/process/traversal.py
index 9ae5afe..46b7faa 100644
--- a/gremlin-python/src/main/jython/gremlin_python/process/traversal.py
+++ b/gremlin-python/src/main/jython/gremlin_python/process/traversal.py
@@ -32,17 +32,6 @@ class Traversal(object):
     def __repr__(self):
         return str(self.bytecode)
 
-    def __getitem__(self, index):
-        if isinstance(index, int):
-            return self.range(index, index + 1)
-        elif isinstance(index, slice):
-            return self.range(index.start, index.stop)
-        else:
-            raise TypeError("Index must be int or slice")
-
-    def __getattr__(self, key):
-        return self.values(key)
-
     def __iter__(self):
         return self
 
@@ -63,6 +52,16 @@ class Traversal(object):
     def toSet(self):
         return set(iter(self))
 
+    def nextTraverser(self):
+        if self.traversers is None:
+            self.traversal_strategies.apply_strategies(self)
+        if self.last_traverser is None:
+            return next(self.traversers)
+        else:
+            temp = self.last_traverser
+            self.last_traverser = None
+            return temp
+
     def next(self, amount=None):
         if amount is None:
             return self.__next__()

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/272dae06/gremlin-python/src/main/jython/gremlin_python/statics.py
----------------------------------------------------------------------
diff --git a/gremlin-python/src/main/jython/gremlin_python/statics.py b/gremlin-python/src/main/jython/gremlin_python/statics.py
index 12cbb70..70e3acc 100644
--- a/gremlin-python/src/main/jython/gremlin_python/statics.py
+++ b/gremlin-python/src/main/jython/gremlin_python/statics.py
@@ -20,6 +20,7 @@ from aenum import Enum
 
 staticMethods = {}
 staticEnums = {}
+lambdaLanguage = "gremlin-jython"
 
 
 def add_static(key, value):

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/272dae06/gremlin-python/src/main/jython/gremlin_python/structure/remote_graph.py
----------------------------------------------------------------------
diff --git a/gremlin-python/src/main/jython/gremlin_python/structure/remote_graph.py b/gremlin-python/src/main/jython/gremlin_python/structure/remote_graph.py
index 72be076..fa2fdb1 100644
--- a/gremlin-python/src/main/jython/gremlin_python/structure/remote_graph.py
+++ b/gremlin-python/src/main/jython/gremlin_python/structure/remote_graph.py
@@ -19,9 +19,9 @@ under the License.
 
 __author__ = 'Marko A. Rodriguez (http://markorodriguez.com)'
 
-from .graph import Graph
 from gremlin_python.process.traversal import TraversalStrategies
 from gremlin_python.process.traversal import TraversalStrategy
+from .graph import Graph
 
 
 class RemoteGraph(Graph):
@@ -39,9 +39,7 @@ class RemoteStrategy(TraversalStrategy):
             raise BaseException(
                 "RemoteStrategy can only be used with a RemoteGraph: " + traversal.graph.__class__.__name__)
         if traversal.traversers is None:
-            remote_response = traversal.graph.remote_connection.submit(
-                'gremlin-groovy',  # script engine
-                traversal.bytecode)  # script
-            traversal.side_effects = remote_response.side_effects
-            traversal.traversers = remote_response.traversers
+            remote_traversal = traversal.graph.remote_connection.submit(traversal.bytecode)
+            traversal.side_effects = remote_traversal.side_effects
+            traversal.traversers = remote_traversal.traversers
         return

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/272dae06/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/driver/RESTRemoteConnectionTest.java
----------------------------------------------------------------------
diff --git a/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/driver/RESTRemoteConnectionTest.java b/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/driver/RESTRemoteConnectionTest.java
index 0796401..a2275e5 100644
--- a/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/driver/RESTRemoteConnectionTest.java
+++ b/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/driver/RESTRemoteConnectionTest.java
@@ -49,9 +49,9 @@ public class RESTRemoteConnectionTest {
         try {
             JythonScriptEngineSetup.setup();
             jython.getContext().getBindings(ScriptContext.ENGINE_SCOPE)
-                    .put("g", jython.eval("RemoteGraph(WebSocketRemoteConnection('http://localhost:8182','g')).traversal()"));
+                    .put("g", jython.eval("RemoteGraph(WebSocketRemoteConnection('ws://localhost:8182','g')).traversal()"));
             jython.getContext().getBindings(ScriptContext.ENGINE_SCOPE)
-                    .put("j", jython.eval("RemoteGraph(WebSocketRemoteConnection('http://localhost:8182','g')).traversal()"));
+                    .put("j", jython.eval("RemoteGraph(WebSocketRemoteConnection('ws://localhost:8182','g')).traversal()"));
             new GremlinServer(Settings.read(RESTRemoteConnectionTest.class.getResourceAsStream("gremlin-server-rest-modern.yaml"))).start().join();
         } catch (final Exception ex) {
             ex.printStackTrace();

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/272dae06/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/jsr223/JythonScriptEngineSetup.java
----------------------------------------------------------------------
diff --git a/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/jsr223/JythonScriptEngineSetup.java b/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/jsr223/JythonScriptEngineSetup.java
index 2a61666..618c488 100644
--- a/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/jsr223/JythonScriptEngineSetup.java
+++ b/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/jsr223/JythonScriptEngineSetup.java
@@ -35,11 +35,15 @@ public class JythonScriptEngineSetup {
     public static void setup() {
         try {
             final ScriptEngine jythonEngine = ScriptEngineCache.get("jython");
+            if (null != System.getenv("PYTHONPATH")) {
+                jythonEngine.eval("import sys");
+                jythonEngine.eval("sys.path.append('" + System.getenv("PYTHONPATH") + "')");
+            }
             jythonEngine.eval("import gremlin_python.statics");
             jythonEngine.eval("from gremlin_python.process.traversal import *");
             jythonEngine.eval("from gremlin_python.process.graph_traversal import *");
             jythonEngine.eval("from gremlin_python.process.graph_traversal import __");
-            jythonEngine.eval("from gremlin_python.driver.websocket_remote_connection import WebSocketRemoteConnection");
+            // jythonEngine.eval("from gremlin_python.driver.websocket_remote_connection import WebSocketRemoteConnection");
             jythonEngine.eval("from gremlin_python.process.traversal import Bytecode");
             jythonEngine.eval("from gremlin_python.structure.remote_graph import RemoteGraph");
             jythonEngine.eval("from gremlin_python.process.graphson import GraphSONWriter");