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/06/07 02:59:21 UTC

incubator-tinkerpop git commit: split up the GryoRegistrator and GryoSerializer based tests into two different full runs of ProcessComputerSuite. The former is an Integration test.

Repository: incubator-tinkerpop
Updated Branches:
  refs/heads/TINKERPOP-1321 08040f808 -> 293fe29c6


split up the GryoRegistrator and GryoSerializer based tests into two different full runs of ProcessComputerSuite. The former is an Integration test.


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

Branch: refs/heads/TINKERPOP-1321
Commit: 293fe29c6735952a7a6b0ed4814eea72adfdde66
Parents: 08040f8
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Mon Jun 6 20:59:21 2016 -0600
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Mon Jun 6 20:59:21 2016 -0600

----------------------------------------------------------------------
 .../gremlin/spark/AbstractSparkTest.java        |  4 ++
 ...tratorGraphComputerProcessIntegrateTest.java | 33 ++++++++++++
 ...SparkHadoopGraphGryoRegistratorProvider.java | 56 ++++++++++++++++++++
 .../computer/SparkHadoopGraphProvider.java      | 19 ++-----
 4 files changed, 97 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/293fe29c/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/AbstractSparkTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/AbstractSparkTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/AbstractSparkTest.java
index 6d2231f..4d584ff 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/AbstractSparkTest.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/AbstractSparkTest.java
@@ -26,6 +26,7 @@ import org.apache.spark.SparkContext;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.tinkerpop.gremlin.hadoop.Constants;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.HadoopPoolShimService;
 import org.apache.tinkerpop.gremlin.spark.structure.Spark;
 import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.GryoSerializer;
 import org.apache.tinkerpop.gremlin.structure.Graph;
@@ -34,6 +35,8 @@ import org.junit.Before;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader.SHIM_CLASS_SYSTEM_PROPERTY;
+
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
@@ -43,6 +46,7 @@ public abstract class AbstractSparkTest {
     @After
     @Before
     public void setupTest() {
+        System.clearProperty(SHIM_CLASS_SYSTEM_PROPERTY);
         SparkConf sparkConfiguration = new SparkConf();
         sparkConfiguration.setAppName(this.getClass().getCanonicalName() + "-setupTest");
         sparkConfiguration.set("spark.master", "local[4]");

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/293fe29c/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGryoRegistratorGraphComputerProcessIntegrateTest.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGryoRegistratorGraphComputerProcessIntegrateTest.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGryoRegistratorGraphComputerProcessIntegrateTest.java
new file mode 100644
index 0000000..29f627d
--- /dev/null
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkGryoRegistratorGraphComputerProcessIntegrateTest.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.process.computer;
+
+import org.apache.tinkerpop.gremlin.GraphProviderClass;
+import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
+import org.apache.tinkerpop.gremlin.process.ProcessComputerSuite;
+import org.junit.runner.RunWith;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+@RunWith(ProcessComputerSuite.class)
+@GraphProviderClass(provider = SparkHadoopGraphGryoRegistratorProvider.class, graph = HadoopGraph.class)
+public class SparkGryoRegistratorGraphComputerProcessIntegrateTest {
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/293fe29c/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphGryoRegistratorProvider.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphGryoRegistratorProvider.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphGryoRegistratorProvider.java
new file mode 100644
index 0000000..df36a13
--- /dev/null
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphGryoRegistratorProvider.java
@@ -0,0 +1,56 @@
+/*
+ *  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.process.computer;
+
+import org.apache.spark.serializer.KryoSerializer;
+import org.apache.tinkerpop.gremlin.LoadGraphWith;
+import org.apache.tinkerpop.gremlin.hadoop.Constants;
+import org.apache.tinkerpop.gremlin.spark.structure.Spark;
+import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.GryoRegistrator;
+import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.kryoshim.unshaded.UnshadedKryoShimService;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader;
+
+import java.util.Map;
+
+import static org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader.SHIM_CLASS_SYSTEM_PROPERTY;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public final class SparkHadoopGraphGryoRegistratorProvider extends SparkHadoopGraphProvider {
+
+    private static boolean firstTest = true;
+
+    public Map<String, Object> getBaseConfiguration(final String graphName, final Class<?> test, final String testMethodName, final LoadGraphWith.GraphData loadGraphWith) {
+        final Map<String, Object> config = super.getBaseConfiguration(graphName, test, testMethodName, loadGraphWith);
+        config.put(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, false);  // ensure the context doesn't stay open for the GryoSerializer tests
+        config.put("spark.serializer", KryoSerializer.class.getCanonicalName());
+        config.put("spark.kryo.registrator", GryoRegistrator.class.getCanonicalName());
+        //
+        if (firstTest) {
+            firstTest = false;
+            Spark.close();
+            System.setProperty(SHIM_CLASS_SYSTEM_PROPERTY, UnshadedKryoShimService.class.getCanonicalName());
+            KryoShimServiceLoader.load(true);
+        }
+        //
+        return config;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/293fe29c/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphProvider.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphProvider.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphProvider.java
index b1f610d..1fc29cf 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphProvider.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkHadoopGraphProvider.java
@@ -55,13 +55,12 @@ import static org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimSe
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
 @GraphProvider.Descriptor(computer = SparkGraphComputer.class)
-public final class SparkHadoopGraphProvider extends HadoopGraphProvider {
+public class SparkHadoopGraphProvider extends HadoopGraphProvider {
 
     @Override
     public Map<String, Object> getBaseConfiguration(final String graphName, final Class<?> test, final String testMethodName, final LoadGraphWith.GraphData loadGraphWith) {
         final Map<String, Object> config = super.getBaseConfiguration(graphName, test, testMethodName, loadGraphWith);
-        config.put(Constants.GREMLIN_SPARK_PERSIST_CONTEXT,
-                !test.equals(ProgramTest.Traversals.class) && !test.equals(GroovyProgramTest.class));  // this makes the test suite go really fast
+        config.put(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, true);  // this makes the test suite go really fast
 
         // toy graph inputRDD does not have corresponding outputRDD so where jobs chain, it fails (failing makes sense)
         if (null != loadGraphWith &&
@@ -89,20 +88,10 @@ public final class SparkHadoopGraphProvider extends HadoopGraphProvider {
             SugarTestHelper.clearRegistry(this);
         }
 
+        System.clearProperty(SHIM_CLASS_SYSTEM_PROPERTY);
         config.put(Constants.GREMLIN_HADOOP_DEFAULT_GRAPH_COMPUTER, SparkGraphComputer.class.getCanonicalName());
-
-
         config.put("spark.master", "local[4]");
-        if (RANDOM.nextBoolean()) {
-            System.setProperty(SHIM_CLASS_SYSTEM_PROPERTY, HadoopPoolShimService.class.getCanonicalName());
-            KryoShimServiceLoader.load(true);
-            config.put("spark.serializer", GryoSerializer.class.getCanonicalName());
-        } else {
-            System.setProperty(SHIM_CLASS_SYSTEM_PROPERTY, UnshadedKryoShimService.class.getCanonicalName());
-            KryoShimServiceLoader.load(true);
-            config.put("spark.serializer", KryoSerializer.class.getCanonicalName());
-            config.put("spark.kryo.registrator", GryoRegistrator.class.getCanonicalName());
-        }
+        config.put("spark.serializer", GryoSerializer.class.getCanonicalName());
         config.put("spark.kryo.registrationRequired", true);
         return config;
     }