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/08/17 18:04:06 UTC

tinkerpop git commit: Deprecating RemoteGraph in favor of withRemote()

Repository: tinkerpop
Updated Branches:
  refs/heads/TINKERPOP-1278 f1a0e9199 -> edc11d91e


Deprecating RemoteGraph in favor of withRemote()

withRemote() better matches how users interact with gremlin server. it binds the remote to a TraversalSource and not a Graph on the server.


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

Branch: refs/heads/TINKERPOP-1278
Commit: edc11d91ef28dbebdd907f46dac4eff569fc1321
Parents: f1a0e91
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Wed Aug 17 07:06:58 2016 -0400
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Wed Aug 17 13:25:22 2016 -0400

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |  1 +
 .../decoration/VertexProgramStrategy.java       |  2 +-
 .../gremlin/process/remote/RemoteGraph.java     |  4 ++
 .../strategy/decoration/RemoteStrategy.java     | 36 ++++++++++----
 .../process/traversal/TraversalSource.java      | 49 ++++++++++++++++++++
 .../dsl/graph/GraphTraversalSource.java         | 20 ++++++++
 .../gremlin_python/process/graph_traversal.py   |  4 ++
 .../driver/remote/RemoteGraphProvider.java      | 12 +++++
 8 files changed, 118 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/edc11d91/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index 6be1f48..79935af 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -35,6 +35,7 @@ TinkerPop 3.2.2 (NOT OFFICIALLY RELEASED YET)
 * Added `Traversal.Admin.nextTraverser()` to get the next result in bulk-form (w/ default implementation).
 * Added `TraversalSource.getAnonymousTraversalClass()` (w/ default implementation).
 * Added `GremlinScriptEngine` interface which specifies a `eval(Bytecode, Bindings)` method.
+* Deprecated `RemoteGraph` in favor of `TraversalSource.withRemote()` as it is more technically correct to tie a remote traversal to the `TraversalSource` than a `Graph` instance.
 * `GremlinGroovyScriptEngine` implements `GremlinScriptEngine`.
 * Added `GremlinJythonScriptEngine` which implements `GremlinScriptEngine`.
 * Removed support for submitting a Java serialized `Traversal` to Gremlin Server.

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/edc11d91/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/traversal/strategy/decoration/VertexProgramStrategy.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/traversal/strategy/decoration/VertexProgramStrategy.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/traversal/strategy/decoration/VertexProgramStrategy.java
index 85a2f83..0496ae3 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/traversal/strategy/decoration/VertexProgramStrategy.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/traversal/strategy/decoration/VertexProgramStrategy.java
@@ -58,7 +58,7 @@ public final class VertexProgramStrategy extends AbstractTraversalStrategy<Trave
     @Override
     public void apply(final Traversal.Admin<?, ?> traversal) {
         // VertexPrograms can only execute at the root level of a Traversal and should not be applied locally prior to RemoteStrategy
-        if (!(traversal.getParent() instanceof EmptyStep) || traversal.getStrategies().toList().contains(RemoteStrategy.instance()))
+        if (!(traversal.getParent() instanceof EmptyStep) || traversal.getStrategies().getStrategy(RemoteStrategy.class).isPresent())
             return;
 
         // back propagate as()-labels off of vertex computing steps

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/edc11d91/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/remote/RemoteGraph.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/remote/RemoteGraph.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/remote/RemoteGraph.java
index 6c2522a..320a60d 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/remote/RemoteGraph.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/remote/RemoteGraph.java
@@ -22,7 +22,9 @@ import org.apache.commons.configuration.Configuration;
 import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.tinkerpop.gremlin.process.computer.GraphComputer;
 import org.apache.tinkerpop.gremlin.process.remote.traversal.strategy.decoration.RemoteStrategy;
+import org.apache.tinkerpop.gremlin.process.traversal.TraversalSource;
 import org.apache.tinkerpop.gremlin.process.traversal.TraversalStrategies;
+import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversalSource;
 import org.apache.tinkerpop.gremlin.structure.Edge;
 import org.apache.tinkerpop.gremlin.structure.Graph;
 import org.apache.tinkerpop.gremlin.structure.Transaction;
@@ -42,6 +44,7 @@ import java.util.Iterator;
  * traversal instances.
  *
  * @author Stephen Mallette (http://stephen.genoprime.com)
+ * @deprecated As for release 3.2.2, replaced by {@link TraversalSource#withRemote(RemoteConnection)} or its overloads.
  */
 @Graph.OptIn(Graph.OptIn.SUITE_PROCESS_STANDARD)
 @Graph.OptIn(Graph.OptIn.SUITE_PROCESS_COMPUTER)
@@ -145,6 +148,7 @@ import java.util.Iterator;
         test = "org.apache.tinkerpop.gremlin.process.traversal.TraversalInterruptionComputerTest",
         method = "*",
         reason = "The interruption model in the test can't guarantee interruption at the right time with RemoteGraph.")
+@Deprecated
 public class RemoteGraph implements Graph {
 
     private final RemoteConnection connection;

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/edc11d91/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/remote/traversal/strategy/decoration/RemoteStrategy.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/remote/traversal/strategy/decoration/RemoteStrategy.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/remote/traversal/strategy/decoration/RemoteStrategy.java
index 79e700b..38ceb95 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/remote/traversal/strategy/decoration/RemoteStrategy.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/remote/traversal/strategy/decoration/RemoteStrategy.java
@@ -32,6 +32,7 @@ import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalHelper;
 import org.apache.tinkerpop.gremlin.structure.util.empty.EmptyGraph;
 
 import java.util.Collections;
+import java.util.Optional;
 import java.util.Set;
 
 /**
@@ -47,11 +48,23 @@ public final class RemoteStrategy extends AbstractTraversalStrategy<TraversalStr
 
     private static final RemoteStrategy INSTANCE = new RemoteStrategy();
 
+    private final Optional<RemoteConnection> remoteConnection;
+
     private static final Set<Class<? extends DecorationStrategy>> POSTS = Collections.singleton(VertexProgramStrategy.class);
 
     private RemoteStrategy() {
+        remoteConnection = Optional.empty();
+    }
+
+    public RemoteStrategy(final RemoteConnection remoteConnection) {
+        this.remoteConnection = Optional.ofNullable(remoteConnection);
     }
 
+    /**
+     * @deprecated As of release 3.2.2, replaced by {@link #RemoteStrategy(RemoteConnection)} where this method should
+     * only be used by {@link RemoteGraph} for backward compatibility.
+     */
+    @Deprecated
     public static RemoteStrategy instance() {
         return INSTANCE;
     }
@@ -63,6 +76,17 @@ public final class RemoteStrategy extends AbstractTraversalStrategy<TraversalStr
 
     @Override
     public void apply(final Traversal.Admin<?, ?> traversal) {
+        // this check for a remoteConnection is really only relevant for backward compatibility for RemoteGraph prior
+        // to the now preferred method of withRemote().
+        if (!remoteConnection.isPresent()) {
+            if (!(traversal.getGraph().orElse(EmptyGraph.instance()) instanceof RemoteGraph))
+                throw new IllegalStateException("RemoteStrategy expects a RemoteGraph instance attached to the Traversal");
+
+            final RemoteGraph remoteGraph = (RemoteGraph) traversal.getGraph().get();
+            if (null == remoteGraph.getConnection())
+                throw new IllegalStateException("RemoteStrategy expects the RemoteGraph instance to have a RemoteConnection");
+        }
+
         if (!(traversal.getParent() instanceof EmptyStep))
             return;
 
@@ -73,15 +97,9 @@ public final class RemoteStrategy extends AbstractTraversalStrategy<TraversalStr
             throw new VerificationException("Test suite does not support profiling nor lambdas", traversal);
         }
 
-        if (!(traversal.getGraph().orElse(EmptyGraph.instance()) instanceof RemoteGraph))
-            throw new IllegalStateException("RemoteStrategy expects a RemoteGraph instance attached to the Traversal");
-
-        final RemoteGraph remoteGraph = (RemoteGraph) traversal.getGraph().get();
-        if (null == remoteGraph.getConnection())
-            throw new IllegalStateException("RemoteStrategy expects the RemoteGraph instance to have a RemoteConnection");
-
-        // remote step wraps the traversal and emits the results from the remote connection
-        final RemoteStep<?, ?> remoteStep = new RemoteStep<>(traversal, remoteGraph.getConnection());
+        // remote step wraps the traversal and emits the results from the remote connection.
+        final RemoteStep<?, ?> remoteStep = new RemoteStep<>(traversal,
+                remoteConnection.orElseGet(() -> ((RemoteGraph) traversal.getGraph().get()).getConnection()));
         TraversalHelper.removeAllSteps(traversal);
         traversal.addStep(remoteStep);
 

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/edc11d91/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/TraversalSource.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/TraversalSource.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/TraversalSource.java
index 7f9ebe3..5e82716 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/TraversalSource.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/TraversalSource.java
@@ -18,15 +18,20 @@
  */
 package org.apache.tinkerpop.gremlin.process.traversal;
 
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.tinkerpop.gremlin.process.computer.Computer;
 import org.apache.tinkerpop.gremlin.process.computer.GraphComputer;
 import org.apache.tinkerpop.gremlin.process.computer.traversal.strategy.decoration.VertexProgramStrategy;
+import org.apache.tinkerpop.gremlin.process.remote.RemoteConnection;
+import org.apache.tinkerpop.gremlin.process.remote.traversal.strategy.decoration.RemoteStrategy;
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.decoration.SackStrategy;
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.decoration.SideEffectStrategy;
 import org.apache.tinkerpop.gremlin.structure.Graph;
 import org.apache.tinkerpop.gremlin.util.function.ConstantSupplier;
 
 import java.io.Serializable;
+import java.lang.reflect.Constructor;
 import java.util.List;
 import java.util.Optional;
 import java.util.function.BinaryOperator;
@@ -50,6 +55,9 @@ import java.util.function.UnaryOperator;
  */
 public interface TraversalSource extends Cloneable {
 
+    // TODO: this is GraphFactory naming convention, but we're not GraphFactory compliant anymore so maybe change the config option?
+    public static final String GREMLIN_REMOTE_GRAPH_REMOTE_CONNECTION_CLASS = "gremlin.remoteGraph.remoteConnectionClass";
+
     /**
      * Get the {@link TraversalStrategies} associated with this traversal source.
      *
@@ -386,6 +394,47 @@ public interface TraversalSource extends Cloneable {
         return clone;
     }
 
+    /**
+     * Configures the {@code TraversalSource} as a "remote" to issue the {@link Traversal} for execution elsewhere.
+     * Expects key for {@link #GREMLIN_REMOTE_GRAPH_REMOTE_CONNECTION_CLASS} as well as any configuration required by
+     * the underlying {@link RemoteConnection} which will be instantiated. Note that the {@code Configuration} object
+     * is passed down without change to the creation of the {@link RemoteConnection} instance.
+     */
+    public default TraversalSource withRemote(final Configuration conf) {
+        if (!conf.containsKey(GREMLIN_REMOTE_GRAPH_REMOTE_CONNECTION_CLASS))
+            throw new IllegalArgumentException("Configuration must contain the '" + GREMLIN_REMOTE_GRAPH_REMOTE_CONNECTION_CLASS + "' key");
+
+        final RemoteConnection remoteConnection;
+        try {
+            final Class<? extends RemoteConnection> clazz = Class.forName(conf.getString(GREMLIN_REMOTE_GRAPH_REMOTE_CONNECTION_CLASS)).asSubclass(RemoteConnection.class);
+            final Constructor<? extends RemoteConnection> ctor = clazz.getConstructor(Configuration.class);
+            remoteConnection = ctor.newInstance(conf);
+        } catch (Exception ex) {
+            throw new IllegalStateException(ex);
+        }
+
+        return withRemote(remoteConnection);
+    }
+
+    /**
+     * Configures the {@code TraversalSource} as a "remote" to issue the {@link Traversal} for execution elsewhere.
+     * Calls {@link #withRemote(Configuration)} after reading the properties file specified.
+     */
+    public default TraversalSource withRemote(final String configFile) throws Exception {
+        return withRemote(new PropertiesConfiguration(configFile));
+    }
+
+    /**
+     * Configures the {@code TraversalSource} as a "remote" to issue the {@link Traversal} for execution elsewhere.
+     *
+     * @param connection the {@link RemoteConnection} instance to use to submit the {@link Traversal}.
+     */
+    public default TraversalSource withRemote(final RemoteConnection connection) {
+        final TraversalSource clone = this.clone();
+        clone.getStrategies().addStrategies(new RemoteStrategy(connection));
+        return clone;
+    }
+
     public default Optional<Class> getAnonymousTraversalClass() {
         return Optional.empty();
     }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/edc11d91/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/dsl/graph/GraphTraversalSource.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/dsl/graph/GraphTraversalSource.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/dsl/graph/GraphTraversalSource.java
index a4f2378..fdafb57 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/dsl/graph/GraphTraversalSource.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/dsl/graph/GraphTraversalSource.java
@@ -18,8 +18,11 @@
  */
 package org.apache.tinkerpop.gremlin.process.traversal.dsl.graph;
 
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.tinkerpop.gremlin.process.computer.Computer;
 import org.apache.tinkerpop.gremlin.process.computer.GraphComputer;
+import org.apache.tinkerpop.gremlin.process.remote.RemoteConnection;
 import org.apache.tinkerpop.gremlin.process.traversal.Bindings;
 import org.apache.tinkerpop.gremlin.process.traversal.Bytecode;
 import org.apache.tinkerpop.gremlin.process.traversal.TraversalEngine;
@@ -38,8 +41,10 @@ import org.apache.tinkerpop.gremlin.structure.Graph;
 import org.apache.tinkerpop.gremlin.structure.T;
 import org.apache.tinkerpop.gremlin.structure.Transaction;
 import org.apache.tinkerpop.gremlin.structure.Vertex;
+import org.apache.tinkerpop.gremlin.structure.util.GraphFactory;
 import org.apache.tinkerpop.gremlin.structure.util.StringFactory;
 
+import java.lang.reflect.Constructor;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Optional;
@@ -225,6 +230,21 @@ public class GraphTraversalSource implements TraversalSource {
         return clone;
     }
 
+    @Override
+    public GraphTraversalSource withRemote(final Configuration conf) {
+        return (GraphTraversalSource) TraversalSource.super.withRemote(conf);
+    }
+
+    @Override
+    public GraphTraversalSource withRemote(final String configFile) throws Exception {
+        return (GraphTraversalSource) TraversalSource.super.withRemote(configFile);
+    }
+
+    @Override
+    public GraphTraversalSource withRemote(final RemoteConnection connection) {
+        return (GraphTraversalSource) TraversalSource.super.withRemote(connection);
+    }
+
     //// SPAWNS
 
     /**

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/edc11d91/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 4a94e24..43d9b91 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
@@ -57,6 +57,10 @@ class GraphTraversalSource(object):
     source = GraphTraversalSource(self.graph, self.traversal_strategies, Bytecode(self.bytecode))
     source.bytecode.add_source("withPath", *args)
     return source
+  def withRemote(self, *args):
+    source = GraphTraversalSource(self.graph, self.traversal_strategies, Bytecode(self.bytecode))
+    source.bytecode.add_source("withRemote", *args)
+    return source
   def withSack(self, *args):
     source = GraphTraversalSource(self.graph, self.traversal_strategies, Bytecode(self.bytecode))
     source.bytecode.add_source("withSack", *args)

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/edc11d91/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/driver/remote/RemoteGraphProvider.java
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/driver/remote/RemoteGraphProvider.java b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/driver/remote/RemoteGraphProvider.java
index 85c9bbb..35bbd3f 100644
--- a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/driver/remote/RemoteGraphProvider.java
+++ b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/driver/remote/RemoteGraphProvider.java
@@ -24,6 +24,8 @@ import org.apache.tinkerpop.gremlin.LoadGraphWith;
 import org.apache.tinkerpop.gremlin.driver.Client;
 import org.apache.tinkerpop.gremlin.driver.Cluster;
 import org.apache.tinkerpop.gremlin.process.remote.RemoteGraph;
+import org.apache.tinkerpop.gremlin.process.traversal.TraversalStrategy;
+import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversalSource;
 import org.apache.tinkerpop.gremlin.server.GremlinServer;
 import org.apache.tinkerpop.gremlin.server.ServerTestHelper;
 import org.apache.tinkerpop.gremlin.server.Settings;
@@ -97,6 +99,16 @@ public class RemoteGraphProvider extends AbstractGraphProvider {
         return IMPLEMENTATION;
     }
 
+    @Override
+    public GraphTraversalSource traversal(final Graph graph) {
+        // ensure that traversal is created using withRemote() rather than just using RemoteGraph. withRemote() is
+        // the appropriate way for users to create a remote traversal. RemoteGraph has been deprecated for users
+        // concerns and will be likely relegated to the test module so that OptOut can continue to work and we can
+        // full execute the process tests. we should be able to clean this up considerably when RemoteGraph can be
+        // moved with breaking change.
+        return super.traversal(graph).withRemote(((RemoteGraph) graph).getConnection());
+    }
+
     public static void startServer() throws Exception {
         final InputStream stream = RemoteGraphProvider.class.getResourceAsStream("gremlin-server-integration.yaml");
         final Settings settings = Settings.read(stream);