You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tinkerpop.apache.org by sp...@apache.org on 2016/02/18 16:49:53 UTC

incubator-tinkerpop git commit: Allow ScriptEngine to accept scripts in interpreter mode.

Repository: incubator-tinkerpop
Updated Branches:
  refs/heads/TINKERPOP-1107 [created] c05b1337b


Allow ScriptEngine to accept scripts in interpreter mode.

Scripts sent with variables normally considered "local" by use of the def keyword or the inclusion of a type will be treated as global variables when the InterpreterModeCustomizerProvider configuration is included in the configuration of the ScriptEngine.


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

Branch: refs/heads/TINKERPOP-1107
Commit: c05b1337bc63f1a2d3a7bd2af669e594f2a19e45
Parents: 70a3065
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Thu Feb 18 10:47:28 2016 -0500
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Feb 18 10:47:28 2016 -0500

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |   1 +
 .../jsr223/GremlinVariableAnalyzer.groovy       | 104 +++++++++++++++++
 .../groovy/jsr223/ast/InterpreterMode.groovy    |  30 +++++
 .../ast/InterpreterModeASTTransformation.groovy | 115 +++++++++++++++++++
 .../jsr223/GremlinGroovyScriptEngine.java       |  41 ++++++-
 .../InterpreterModeCustomizerProvider.java      |  37 ++++++
 .../jsr223/GremlinGroovyScriptEngineTest.java   |  27 +++++
 .../server/GremlinServerIntegrateTest.java      |  57 +++++++++
 8 files changed, 408 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/c05b1337/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index b35efcb..5a62d70 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -26,6 +26,7 @@ image::https://raw.githubusercontent.com/apache/incubator-tinkerpop/master/docs/
 TinkerPop 3.1.2 (NOT OFFICIALLY RELEASED YET)
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
+* Added "interpreter mode" for the `ScriptEngine` and Gremlin Server which allows variables defined with `def` or a type to be recognized as "global".
 * Fixed a bug where multiple "close" requests were being sent by the driver on `Client.close()`.
 * Fixed an `Property` attach bug that shows up in serialization-based `GraphComputer` implementations.
 * Fixed a pom.xml bug where Gremlin Console/Server were not pulling the latest Neo4j 2.3.2.

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/c05b1337/gremlin-groovy/src/main/groovy/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinVariableAnalyzer.groovy
----------------------------------------------------------------------
diff --git a/gremlin-groovy/src/main/groovy/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinVariableAnalyzer.groovy b/gremlin-groovy/src/main/groovy/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinVariableAnalyzer.groovy
new file mode 100644
index 0000000..1d1bcbe
--- /dev/null
+++ b/gremlin-groovy/src/main/groovy/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinVariableAnalyzer.groovy
@@ -0,0 +1,104 @@
+/*
+ * 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.groovy.jsr223
+
+import org.codehaus.groovy.ast.DynamicVariable
+import org.codehaus.groovy.ast.GroovyClassVisitor
+import org.codehaus.groovy.ast.expr.VariableExpression
+import org.codehaus.groovy.control.CompilationUnit
+import org.codehaus.groovy.control.CompilerConfiguration
+import org.codehaus.groovy.control.Phases
+import org.codehaus.groovy.tools.shell.util.ScriptVariableAnalyzer
+
+import java.security.CodeSource
+
+/**
+ * An extension of Groovy's {@code VariableVisitor} that exposes the bound and unbound variables publicly. This
+ * class can likely be removed with the next update of Groovy (after 2.4.5) as the {code ScriptVariableAnalyzer} ends
+ * up exposing {@code getBoundVars() in such a way as to allow for the {@code ClassLoader} to be supplied.
+ */
+class GremlinVariableAnalyzer {
+    public static class GremlinVariableVisitor extends ScriptVariableAnalyzer.VariableVisitor {
+        String lastBound
+
+        @Override
+        void visitVariableExpression(VariableExpression expression) {
+            if (!(expression.variable in ['args', 'context', 'this', 'super'])) {
+                if (expression.accessedVariable instanceof DynamicVariable) {
+                    unbound << expression.variable
+                } else {
+                    bound << expression.variable
+                    lastBound = bound
+                }
+            }
+            super.visitVariableExpression(expression)
+        }
+
+        @Override
+        public Set<String> getBound() {
+            return super.getBound()
+        }
+
+        @Override
+        public Set<String> getUnbound() {
+            return super.getUnbound()
+        }
+    }
+
+    public static class GremlinVisitorClassLoader extends GroovyClassLoader {
+        private final GroovyClassVisitor visitor
+
+        public GremlinVisitorClassLoader(final GroovyClassVisitor visitor, ClassLoader parent) {
+            super(parent == null ?  Thread.currentThread().getContextClassLoader() : parent)
+            this.visitor = visitor
+        }
+
+        @Override
+        protected CompilationUnit createCompilationUnit(final CompilerConfiguration config, final CodeSource source) {
+            CompilationUnit cu = super.createCompilationUnit(config, source)
+            cu.addPhaseOperation(new ScriptVariableAnalyzer.VisitorSourceOperation(visitor), Phases.CLASS_GENERATION)
+            return cu
+        }
+    }
+
+    public static BoundVars getBoundVars(final String scriptText, ClassLoader parent) {
+        assert scriptText != null
+        final GroovyClassVisitor visitor = new GremlinVariableVisitor()
+        new GremlinVisitorClassLoader(visitor, parent).parseClass(scriptText)
+        return new BoundVars(visitor.getLastBound(), visitor.getBound())
+    }
+
+    public static class BoundVars {
+        private String lastBound;
+        private Set<String> bound;
+
+        BoundVars(String lastBound, Set<String> bound) {
+            this.lastBound = lastBound
+            this.bound = bound
+        }
+
+        String getLastBound() {
+            return lastBound
+        }
+
+        Set<String> getBound() {
+            return bound
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/c05b1337/gremlin-groovy/src/main/groovy/org/apache/tinkerpop/gremlin/groovy/jsr223/ast/InterpreterMode.groovy
----------------------------------------------------------------------
diff --git a/gremlin-groovy/src/main/groovy/org/apache/tinkerpop/gremlin/groovy/jsr223/ast/InterpreterMode.groovy b/gremlin-groovy/src/main/groovy/org/apache/tinkerpop/gremlin/groovy/jsr223/ast/InterpreterMode.groovy
new file mode 100644
index 0000000..a8da72f
--- /dev/null
+++ b/gremlin-groovy/src/main/groovy/org/apache/tinkerpop/gremlin/groovy/jsr223/ast/InterpreterMode.groovy
@@ -0,0 +1,30 @@
+/*
+ * 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.groovy.jsr223.ast
+
+import org.codehaus.groovy.transform.GroovyASTTransformationClass
+
+import java.lang.annotation.Retention
+import java.lang.annotation.RetentionPolicy
+
+@Retention(RetentionPolicy.SOURCE)
+@GroovyASTTransformationClass(["org.apache.tinkerpop.gremlin.groovy.jsr223.ast.InterpreterModeASTTransformation"])
+public @interface InterpreterMode {
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/c05b1337/gremlin-groovy/src/main/groovy/org/apache/tinkerpop/gremlin/groovy/jsr223/ast/InterpreterModeASTTransformation.groovy
----------------------------------------------------------------------
diff --git a/gremlin-groovy/src/main/groovy/org/apache/tinkerpop/gremlin/groovy/jsr223/ast/InterpreterModeASTTransformation.groovy b/gremlin-groovy/src/main/groovy/org/apache/tinkerpop/gremlin/groovy/jsr223/ast/InterpreterModeASTTransformation.groovy
new file mode 100644
index 0000000..66638b8
--- /dev/null
+++ b/gremlin-groovy/src/main/groovy/org/apache/tinkerpop/gremlin/groovy/jsr223/ast/InterpreterModeASTTransformation.groovy
@@ -0,0 +1,115 @@
+/*
+ * 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.groovy.jsr223.ast
+
+import groovy.transform.CompileStatic
+import org.apache.tinkerpop.gremlin.groovy.jsr223.GremlinGroovyScriptEngine
+import org.codehaus.groovy.ast.ASTNode
+import org.codehaus.groovy.ast.ClassHelper
+import org.codehaus.groovy.ast.ClassNode
+import org.codehaus.groovy.ast.MethodNode
+import org.codehaus.groovy.ast.Parameter
+import org.codehaus.groovy.ast.expr.ArgumentListExpression
+import org.codehaus.groovy.ast.expr.BinaryExpression
+import org.codehaus.groovy.ast.expr.ConstantExpression
+import org.codehaus.groovy.ast.expr.DeclarationExpression
+import org.codehaus.groovy.ast.expr.MapExpression
+import org.codehaus.groovy.ast.expr.MethodCallExpression
+import org.codehaus.groovy.ast.expr.VariableExpression
+import org.codehaus.groovy.ast.stmt.BlockStatement
+import org.codehaus.groovy.ast.stmt.CatchStatement
+import org.codehaus.groovy.ast.stmt.EmptyStatement
+import org.codehaus.groovy.ast.stmt.ExpressionStatement
+import org.codehaus.groovy.ast.stmt.Statement
+import org.codehaus.groovy.ast.stmt.TryCatchStatement
+import org.codehaus.groovy.control.CompilePhase
+import org.codehaus.groovy.control.SourceUnit
+import org.codehaus.groovy.syntax.Token
+import org.codehaus.groovy.syntax.Types
+import org.codehaus.groovy.transform.ASTTransformation
+import org.codehaus.groovy.transform.GroovyASTTransformation
+
+/**
+ * An {@code ASTTransformation} that promotes "local" variables to global ones.  In this case, "local" refers to those
+ * variables that are defined in a script with "def" at the root of the script.  These would typically be interpreted
+ * as local to the script, but this transform changes that, by wrapping the entire script in a try/catch where such
+ * variables are written to a "hidden" {@link Map} so that the {@code ScriptEngine} can later access them to place
+ * them into the global context.
+ */
+@CompileStatic
+@GroovyASTTransformation(phase=CompilePhase.SEMANTIC_ANALYSIS)
+class InterpreterModeASTTransformation implements ASTTransformation {
+
+    @Override
+    void visit(ASTNode[] astNodes, SourceUnit sourceUnit) {
+        ClassNode scriptNode = (ClassNode) astNodes[1]
+        def runMethodOfScript = scriptNode.declaredMethodsMap["java.lang.Object run()"]
+        runMethodOfScript.code = wrap(runMethodOfScript)
+    }
+
+    private static BlockStatement wrap(MethodNode method) {
+        BlockStatement wrappedBlock = new BlockStatement()
+        BlockStatement existingBlock = ((BlockStatement) method.code)
+
+        // the variable names that will be written back to the global context
+        def variableNames = [] as Set<String>
+        variableNames.addAll(findTopLevelVariableDeclarations(existingBlock.statements))
+        method.variableScope.referencedClassVariablesIterator.each{variableNames << it.name}
+
+        // the map to hold the variables and values
+        wrappedBlock.addStatement(createGlobalMapAST())
+
+        // the finally block will capture all the vars in the "globals" map
+        BlockStatement finallyBlock = new BlockStatement()
+        variableNames.each {
+            finallyBlock.addStatement(createAssignToGlobalMapAST(it))
+        }
+
+        wrappedBlock.addStatement(new TryCatchStatement(existingBlock, finallyBlock))
+
+        return wrappedBlock
+    }
+
+    private static List<String> findTopLevelVariableDeclarations(def existingStatements) {
+        existingStatements.findAll{ it instanceof ExpressionStatement }
+                .collect{ ((ExpressionStatement) it).expression }
+                .findAll{ it instanceof DeclarationExpression}
+                .collect{ ((DeclarationExpression) it).leftExpression }
+                .collect{ ((VariableExpression) it).name }
+    }
+
+    private static Statement createAssignToGlobalMapAST(String varName) {
+        def tryCatch = new TryCatchStatement(new ExpressionStatement(
+                new MethodCallExpression(
+                        new VariableExpression(GremlinGroovyScriptEngine.COLLECTED_BOUND_VARS_MAP_VARNAME),
+                        "put",
+                        new ArgumentListExpression(new ConstantExpression(varName), new VariableExpression(varName)))), EmptyStatement.INSTANCE)
+
+        tryCatch.addCatch(new CatchStatement(new Parameter(ClassHelper.make(MissingPropertyException), "ex"), EmptyStatement.INSTANCE))
+        return tryCatch
+    }
+
+    private static Statement createGlobalMapAST() {
+        new ExpressionStatement(
+                new BinaryExpression(
+                        new VariableExpression(GremlinGroovyScriptEngine.COLLECTED_BOUND_VARS_MAP_VARNAME),
+                        Token.newSymbol(Types.EQUAL, 0, 0),
+                        new MapExpression()))
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/c05b1337/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinGroovyScriptEngine.java
----------------------------------------------------------------------
diff --git a/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinGroovyScriptEngine.java b/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinGroovyScriptEngine.java
index 72e38c3..23240cb 100644
--- a/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinGroovyScriptEngine.java
+++ b/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinGroovyScriptEngine.java
@@ -23,6 +23,7 @@ import org.apache.tinkerpop.gremlin.groovy.DefaultImportCustomizerProvider;
 import org.apache.tinkerpop.gremlin.groovy.EmptyImportCustomizerProvider;
 import org.apache.tinkerpop.gremlin.groovy.ImportCustomizerProvider;
 import org.apache.tinkerpop.gremlin.groovy.NoImportCustomizerProvider;
+import org.apache.tinkerpop.gremlin.groovy.jsr223.customizer.InterpreterModeCustomizerProvider;
 import org.apache.tinkerpop.gremlin.groovy.loaders.GremlinLoader;
 import org.apache.tinkerpop.gremlin.groovy.plugin.Artifact;
 import org.apache.tinkerpop.gremlin.groovy.plugin.GremlinPlugin;
@@ -125,6 +126,12 @@ public class GremlinGroovyScriptEngine extends GroovyScriptEngineImpl implements
      */
     public static final String REFERENCE_TYPE_HARD = "hard";
 
+    /**
+     * Name of variable that holds local variables to be globally bound if "interpreter mode" is enabled with
+     * {@link InterpreterModeCustomizerProvider}.
+     */
+    public static final String COLLECTED_BOUND_VARS_MAP_VARNAME = "gremlin_script_engine_collected_boundvars";
+
     private static final Pattern patternImportStatic = Pattern.compile("\\Aimport\\sstatic.*");
 
     public static final ThreadLocal<Map<String, Object>> COMPILE_OPTIONS = new ThreadLocal<Map<String, Object>>(){
@@ -146,7 +153,7 @@ public class GremlinGroovyScriptEngine extends GroovyScriptEngineImpl implements
 
     private GremlinGroovyClassLoader loader;
 
-    private AtomicLong counter = new AtomicLong(0l);
+    private AtomicLong counter = new AtomicLong(0L);
 
     /**
      * The list of loaded plugins for the console.
@@ -164,6 +171,7 @@ public class GremlinGroovyScriptEngine extends GroovyScriptEngineImpl implements
     private final List<CompilerCustomizerProvider> customizerProviders;
 
     private final Set<Artifact> artifactsToUse = new HashSet<>();
+    private final boolean interpreterModeEnabled;
 
     /**
      * Creates a new instance using the {@link DefaultImportCustomizerProvider}.
@@ -193,9 +201,14 @@ public class GremlinGroovyScriptEngine extends GroovyScriptEngineImpl implements
                 .map(p -> (ImportCustomizerProvider) p)
                 .findFirst().orElse(NoImportCustomizerProvider.INSTANCE);
 
+        // determine if interpreter mode should be enabled
+        interpreterModeEnabled = providers.stream()
+                .anyMatch(p -> p.getClass().equals(InterpreterModeCustomizerProvider.class));
+
         // remove used providers as the rest will be applied directly
         customizerProviders = providers.stream()
-                .filter(p -> p != null && !(p instanceof ImportCustomizerProvider))
+                .filter(p -> p != null &&
+                             !((p instanceof ImportCustomizerProvider)))
                 .collect(Collectors.toList());
 
         createClassLoader();
@@ -531,7 +544,28 @@ public class GremlinGroovyScriptEngine extends GroovyScriptEngineImpl implements
                     }
                 }
             });
-            return scriptObject.run();
+
+            final Object o = scriptObject.run();
+
+            // if interpreter mode is enable then local vars of the script are promoted to engine scope bindings.
+            if (interpreterModeEnabled) {
+                final Map<String, Object> localVars = (Map<String, Object>) context.getAttribute(COLLECTED_BOUND_VARS_MAP_VARNAME);
+                if (localVars != null) {
+                    localVars.entrySet().forEach(e -> {
+                        // closures need to be cached for later use
+                        if (e.getValue() instanceof Closure)
+                            globalClosures.put(e.getKey(), (Closure) e.getValue());
+
+                        context.setAttribute(e.getKey(), e.getValue(), ScriptContext.ENGINE_SCOPE);
+                    });
+
+                    // get rid of the temporary collected vars
+                    context.removeAttribute(COLLECTED_BOUND_VARS_MAP_VARNAME, ScriptContext.ENGINE_SCOPE);
+                    localVars.clear();
+                }
+            }
+
+            return o;
         } catch (Exception e) {
             throw new ScriptException(e);
         }
@@ -642,5 +676,4 @@ public class GremlinGroovyScriptEngine extends GroovyScriptEngineImpl implements
         }
         return buf.toString();
     }
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/c05b1337/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/jsr223/customizer/InterpreterModeCustomizerProvider.java
----------------------------------------------------------------------
diff --git a/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/jsr223/customizer/InterpreterModeCustomizerProvider.java b/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/jsr223/customizer/InterpreterModeCustomizerProvider.java
new file mode 100644
index 0000000..e3f95f4
--- /dev/null
+++ b/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/jsr223/customizer/InterpreterModeCustomizerProvider.java
@@ -0,0 +1,37 @@
+/*
+ * 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.groovy.jsr223.customizer;
+
+import org.apache.tinkerpop.gremlin.groovy.CompilerCustomizerProvider;
+import org.apache.tinkerpop.gremlin.groovy.jsr223.ast.InterpreterMode;
+import org.codehaus.groovy.control.customizers.ASTTransformationCustomizer;
+import org.codehaus.groovy.control.customizers.CompilationCustomizer;
+
+/**
+ * Places the {@code ScriptEngine} in "interpreter mode" where local variables of a script are treated as global
+ * bindings. This implementation is technically not a true {@link CompilerCustomizerProvider} instance as the
+ * "interpreter mode" feature does not require a {@code CompilerCustomizer}. This class merely acts as a flag that
+ * tells the {@link org.apache.tinkerpop.gremlin.groovy.jsr223.GremlinGroovyScriptEngine} to turn this feature on.
+ */
+public class InterpreterModeCustomizerProvider implements CompilerCustomizerProvider {
+    @Override
+    public CompilationCustomizer create() {
+        return new ASTTransformationCustomizer(InterpreterMode.class);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/c05b1337/gremlin-groovy/src/test/java/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinGroovyScriptEngineTest.java
----------------------------------------------------------------------
diff --git a/gremlin-groovy/src/test/java/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinGroovyScriptEngineTest.java b/gremlin-groovy/src/test/java/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinGroovyScriptEngineTest.java
index 4740cd2..17cd809 100644
--- a/gremlin-groovy/src/test/java/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinGroovyScriptEngineTest.java
+++ b/gremlin-groovy/src/test/java/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinGroovyScriptEngineTest.java
@@ -22,6 +22,7 @@ import groovy.lang.Closure;
 import org.apache.commons.lang3.concurrent.BasicThreadFactory;
 import org.apache.tinkerpop.gremlin.groovy.CompilerCustomizerProvider;
 import org.apache.tinkerpop.gremlin.groovy.NoImportCustomizerProvider;
+import org.apache.tinkerpop.gremlin.groovy.jsr223.customizer.InterpreterModeCustomizerProvider;
 import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 import org.javatuples.Pair;
@@ -85,7 +86,33 @@ public class GremlinGroovyScriptEngineTest {
     @Test
     public void shouldEvalWithNoBindings() throws Exception {
         final GremlinGroovyScriptEngine engine = new GremlinGroovyScriptEngine();
+        engine.eval("def addItUp(x,y){x+y}");
         assertEquals(3, engine.eval("1+2"));
+        assertEquals(3, engine.eval("addItUp(1,2)"));
+    }
+
+    @Test
+    public void shouldPromoteDefinedVarsInInterpreterModeWithNoBindings() throws Exception {
+        final GremlinGroovyScriptEngine engine = new GremlinGroovyScriptEngine(new InterpreterModeCustomizerProvider());
+        engine.eval("def addItUp = { x, y -> x + y }");
+        assertEquals(3, engine.eval("int xxx = 1 + 2"));
+        assertEquals(4, engine.eval("yyy = xxx + 1"));
+        assertEquals(7, engine.eval("def zzz = yyy + xxx"));
+        assertEquals(4, engine.eval("zzz - xxx"));
+        assertEquals(10, engine.eval("addItUp(zzz,xxx)"));
+    }
+
+    @Test
+    public void shouldPromoteDefinedVarsInInterpreterModeWithBindings() throws Exception {
+        final GremlinGroovyScriptEngine engine = new GremlinGroovyScriptEngine(new InterpreterModeCustomizerProvider());
+        final Bindings b = new SimpleBindings();
+        b.put("x", 2);
+        engine.eval("def addItUp = { x, y -> x + y }", b);
+        assertEquals(3, engine.eval("int xxx = 1 + x", b));
+        assertEquals(4, engine.eval("yyy = xxx + 1", b));
+        assertEquals(7, engine.eval("def zzz = yyy + xxx", b));
+        assertEquals(4, engine.eval("zzz - xxx", b));
+        assertEquals(10, engine.eval("addItUp(zzz,xxx)", b));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/c05b1337/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerIntegrateTest.java
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerIntegrateTest.java b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerIntegrateTest.java
index 6ca7dff..fda330a 100644
--- a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerIntegrateTest.java
+++ b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerIntegrateTest.java
@@ -32,6 +32,7 @@ import org.apache.tinkerpop.gremlin.driver.simple.SimpleClient;
 import org.apache.tinkerpop.gremlin.driver.simple.WebSocketClient;
 import org.apache.tinkerpop.gremlin.groovy.jsr223.GremlinGroovyScriptEngine;
 import org.apache.tinkerpop.gremlin.groovy.jsr223.customizer.CompileStaticCustomizerProvider;
+import org.apache.tinkerpop.gremlin.groovy.jsr223.customizer.InterpreterModeCustomizerProvider;
 import org.apache.tinkerpop.gremlin.groovy.jsr223.customizer.SimpleSandboxExtension;
 import org.apache.tinkerpop.gremlin.groovy.jsr223.customizer.TimedInterruptCustomizerProvider;
 import org.apache.tinkerpop.gremlin.structure.T;
@@ -45,6 +46,7 @@ import org.junit.Test;
 
 import java.nio.channels.ClosedChannelException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -136,6 +138,9 @@ public class GremlinServerIntegrateTest extends AbstractGremlinServerIntegration
             case "shouldUseSimpleSandbox":
                 settings.scriptEngines.get("gremlin-groovy").config = getScriptEngineConfForSimpleSandbox();
                 break;
+            case "shouldUseInterpreterMode":
+                settings.scriptEngines.get("gremlin-groovy").config = getScriptEngineConfForInterpreterMode();
+                break;
             case "shouldReceiveFailureTimeOutOnScriptEvalOfOutOfControlLoop":
                 settings.scriptEngines.get("gremlin-groovy").config = getScriptEngineConfForTimedInterrupt();
                 break;
@@ -164,6 +169,58 @@ public class GremlinServerIntegrateTest extends AbstractGremlinServerIntegration
         return scriptEngineConf;
     }
 
+    private static Map<String, Object> getScriptEngineConfForInterpreterMode() {
+        final Map<String,Object> scriptEngineConf = new HashMap<>();
+        final Map<String,Object> interpreterProviderConf = new HashMap<>();
+        interpreterProviderConf.put(InterpreterModeCustomizerProvider.class.getName(), Collections.EMPTY_LIST);
+        scriptEngineConf.put("compilerCustomizerProviders", interpreterProviderConf);
+        return scriptEngineConf;
+    }
+
+    @Test
+    public void shouldUseInterpreterMode() throws Exception {
+        final Cluster cluster = Cluster.open();
+        final Client client = cluster.connect(name.getMethodName());
+
+        client.submit("def subtractAway(x,y){x-y};[]").all().get();
+        client.submit("multiplyIt = { x,y -> x * y};[]").all().get();
+
+        assertEquals(2, client.submit("x = 1 + 1").all().get().get(0).getInt());
+        assertEquals(3, client.submit("int y = x + 1").all().get().get(0).getInt());
+        assertEquals(5, client.submit("def z = x + y").all().get().get(0).getInt());
+
+        final Map<String,Object> m = new HashMap<>();
+        m.put("x", 10);
+        assertEquals(-5, client.submit("z - x", m).all().get().get(0).getInt());
+        assertEquals(15, client.submit("addItUp(x,z)", m).all().get().get(0).getInt());
+        assertEquals(5, client.submit("subtractAway(x,z)", m).all().get().get(0).getInt());
+        assertEquals(50, client.submit("multiplyIt(x,z)", m).all().get().get(0).getInt());
+
+        cluster.close();
+    }
+
+    @Test
+    public void shouldNotUseInterpreterMode() throws Exception {
+        final Cluster cluster = Cluster.open();
+        final Client client = cluster.connect(name.getMethodName());
+
+        client.submit("def subtractAway(x,y){x-y};[]").all().get();
+        client.submit("multiplyIt = { x,y -> x * y};[]").all().get();
+
+        assertEquals(2, client.submit("x = 1 + 1").all().get().get(0).getInt());
+        assertEquals(3, client.submit("y = x + 1").all().get().get(0).getInt());
+        assertEquals(5, client.submit("z = x + y").all().get().get(0).getInt());
+
+        final Map<String,Object> m = new HashMap<>();
+        m.put("x", 10);
+        assertEquals(-5, client.submit("z - x", m).all().get().get(0).getInt());
+        assertEquals(15, client.submit("addItUp(x,z)", m).all().get().get(0).getInt());
+        assertEquals(5, client.submit("subtractAway(x,z)", m).all().get().get(0).getInt());
+        assertEquals(50, client.submit("multiplyIt(x,z)", m).all().get().get(0).getInt());
+
+        cluster.close();
+    }
+
     @Test
     public void shouldUseSimpleSandbox() throws Exception {
         final Cluster cluster = Cluster.open();