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/12/20 22:55:42 UTC

tinkerpop git commit: added a GraphComputerTest that ensures that the GraphComputer configuration evolves as expected and that it does not alter the Graph configuration.

Repository: tinkerpop
Updated Branches:
  refs/heads/TINKERPOP-1564 2cb9258d4 -> 6639e4cd7


added a GraphComputerTest that ensures that the GraphComputer configuration evolves as expected and that it does not alter the Graph configuration.


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

Branch: refs/heads/TINKERPOP-1564
Commit: 6639e4cd70fcc4ab9029b124ff4040f5c067e57f
Parents: 2cb9258
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Tue Dec 20 15:55:36 2016 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Tue Dec 20 15:55:36 2016 -0700

----------------------------------------------------------------------
 .../process/computer/GraphComputerTest.java     | 33 ++++++++++++++++++++
 .../computer/AbstractHadoopGraphComputer.java   |  3 +-
 ...PartitionerComputerProcessIntegrateTest.java | 33 ++++++++++++++++++++
 ...parkGraphPartitionerComputerProcessTest.java | 33 --------------------
 .../process/computer/TinkerGraphComputer.java   |  2 +-
 5 files changed, 69 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/6639e4cd/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/computer/GraphComputerTest.java
----------------------------------------------------------------------
diff --git a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/computer/GraphComputerTest.java b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/computer/GraphComputerTest.java
index 9cedf7f..a166423 100644
--- a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/computer/GraphComputerTest.java
+++ b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/computer/GraphComputerTest.java
@@ -20,6 +20,7 @@ package org.apache.tinkerpop.gremlin.process.computer;
 
 import org.apache.commons.configuration.BaseConfiguration;
 import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.ConfigurationConverter;
 import org.apache.commons.configuration.ConfigurationUtils;
 import org.apache.tinkerpop.gremlin.ExceptionCoverage;
 import org.apache.tinkerpop.gremlin.LoadGraphWith;
@@ -131,6 +132,38 @@ public class GraphComputerTest extends AbstractGremlinProcessTest {
         }
     }
 
+    @Test
+    @LoadGraphWith(MODERN)
+    public void shouldMaintainConfigurationAndShouldNotAlterGraphConfiguration() throws Exception {
+        final Configuration graphConfiguration = new BaseConfiguration();
+        ConfigurationUtils.copy(graph.configuration(), graphConfiguration);
+        final GraphComputer graphComputer = graphProvider.getGraphComputer(graph);
+        final Configuration graphComputerConfiguration = graphComputer.configuration();
+        final Configuration graphComputerConfigurationClone = new BaseConfiguration();
+        ConfigurationUtils.copy(graphComputerConfiguration, graphComputerConfigurationClone);
+        assertEquals(ConfigurationConverter.getMap(graphComputerConfiguration), ConfigurationConverter.getMap(graphComputerConfigurationClone));
+        // creating a graph computer shouldn't alter the graph configuration
+        assertEquals(ConfigurationConverter.getMap(graphConfiguration), ConfigurationConverter.getMap(graph.configuration()));
+        // creating a graph computer should add the graph computer's class name
+        assertTrue(graphComputerConfiguration.containsKey(GraphComputer.GRAPH_COMPUTER));
+        assertEquals(graphComputer.getClass().getCanonicalName(), graphComputerConfiguration.getString(GraphComputer.GRAPH_COMPUTER));
+        // specifying worker count should alter the graph computer configuration
+        int workers = graphComputerConfiguration.containsKey(GraphComputer.WORKERS) ? graphComputerConfiguration.getInt(GraphComputer.WORKERS) : 1;
+        graphComputer.workers(workers + 1);
+        assertTrue(graphComputerConfiguration.containsKey(GraphComputer.WORKERS));
+        assertEquals(graphComputerConfiguration.getInt(GraphComputer.WORKERS), workers + 1);
+        assertEquals(ConfigurationConverter.getMap(graphComputerConfiguration), ConfigurationConverter.getMap(graphComputer.configuration()));
+        graphComputerConfigurationClone.clear();
+        ConfigurationUtils.copy(graphComputer.configuration(), graphComputerConfigurationClone);
+        assertEquals(ConfigurationConverter.getMap(graphComputerConfiguration), ConfigurationConverter.getMap(graphComputerConfigurationClone));
+        // execute graph computer
+        graphComputer.program(new VertexProgramG()).submit(graph);
+        // executing a graph computer should not alter the graph configuration
+        assertEquals(ConfigurationConverter.getMap(graphConfiguration), ConfigurationConverter.getMap(graph.configuration()));
+        // executing a graph computer should not alter the graph computer configuration
+        // TODO: assertEquals(ConfigurationConverter.getMap(graphComputerConfiguration), ConfigurationConverter.getMap(graphComputerConfigurationClone));
+    }
+
 
     /////////////////////////////////////////////
     @Test

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/6639e4cd/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/AbstractHadoopGraphComputer.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/AbstractHadoopGraphComputer.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/AbstractHadoopGraphComputer.java
index b95fb7e..a81efb6 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/AbstractHadoopGraphComputer.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/AbstractHadoopGraphComputer.java
@@ -81,6 +81,7 @@ public abstract class AbstractHadoopGraphComputer implements GraphComputer {
 
     protected AbstractHadoopGraphComputer(final org.apache.commons.configuration.Configuration configuration) {
         this.configuration = new HadoopConfiguration(configuration);
+        this.configuration.setProperty(GRAPH_COMPUTER, this.getClass().getCanonicalName());
         this.logger = LoggerFactory.getLogger(this.getClass());
         GraphComputerHelper.configure(this, this.configuration);
     }
@@ -140,7 +141,7 @@ public abstract class AbstractHadoopGraphComputer implements GraphComputer {
 
     @Override
     public GraphComputer configure(final String key, final Object value) {
-        this.configuration.setProperty(key,value);
+        this.configuration.setProperty(key, value);
         return this;
     }
 

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/6639e4cd/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/partitioner/SparkGraphPartitionerComputerProcessIntegrateTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/partitioner/SparkGraphPartitionerComputerProcessIntegrateTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/partitioner/SparkGraphPartitionerComputerProcessIntegrateTest.java
new file mode 100644
index 0000000..452bcd0
--- /dev/null
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/partitioner/SparkGraphPartitionerComputerProcessIntegrateTest.java
@@ -0,0 +1,33 @@
+/*
+ *  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.spark.structure.io.partitioner;
+
+import org.apache.tinkerpop.gremlin.GraphProviderClass;
+import org.apache.tinkerpop.gremlin.process.ProcessComputerSuite;
+import org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerGraph;
+import org.junit.runner.RunWith;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+@RunWith(ProcessComputerSuite.class)
+@GraphProviderClass(provider = TinkerGraphPartitionerProvider.class, graph = TinkerGraph.class)
+public class SparkGraphPartitionerComputerProcessIntegrateTest {
+}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/6639e4cd/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/partitioner/SparkGraphPartitionerComputerProcessTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/partitioner/SparkGraphPartitionerComputerProcessTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/partitioner/SparkGraphPartitionerComputerProcessTest.java
deleted file mode 100644
index 7c5b3e3..0000000
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/partitioner/SparkGraphPartitionerComputerProcessTest.java
+++ /dev/null
@@ -1,33 +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.
- */
-
-package org.apache.tinkerpop.gremlin.spark.structure.io.partitioner;
-
-import org.apache.tinkerpop.gremlin.GraphProviderClass;
-import org.apache.tinkerpop.gremlin.process.ProcessComputerSuite;
-import org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerGraph;
-import org.junit.runner.RunWith;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-@RunWith(ProcessComputerSuite.class)
-@GraphProviderClass(provider = TinkerGraphPartitionerProvider.class, graph = TinkerGraph.class)
-public class SparkGraphPartitionerComputerProcessTest {
-}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/6639e4cd/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/process/computer/TinkerGraphComputer.java
----------------------------------------------------------------------
diff --git a/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/process/computer/TinkerGraphComputer.java b/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/process/computer/TinkerGraphComputer.java
index f2d2527..e4ca684 100644
--- a/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/process/computer/TinkerGraphComputer.java
+++ b/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/process/computer/TinkerGraphComputer.java
@@ -126,7 +126,7 @@ public final class TinkerGraphComputer implements GraphComputer {
     @Override
     public GraphComputer program(final VertexProgram vertexProgram) {
         this.vertexProgram = vertexProgram;
-        this.vertexProgram.storeState(this.configuration);
+        // this.vertexProgram.storeState(this.configuration);
         return this;
     }