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 13:57:02 UTC

[01/34] incubator-tinkerpop git commit: Introduce Kryo shim to support serializer reuse

Repository: incubator-tinkerpop
Updated Branches:
  refs/heads/TINKERPOP-1278 fb29b25f0 -> 5ab566cff


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphSerializer.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphSerializer.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphSerializer.java
new file mode 100644
index 0000000..d5ba90d
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphSerializer.java
@@ -0,0 +1,140 @@
+/*
+ * 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.structure.util.star;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.tinkerpop.gremlin.process.computer.GraphFilter;
+import org.apache.tinkerpop.gremlin.structure.Direction;
+import org.apache.tinkerpop.gremlin.structure.Edge;
+import org.apache.tinkerpop.gremlin.structure.T;
+import org.apache.tinkerpop.gremlin.structure.VertexProperty;
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.InputShim;
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShim;
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.OutputShim;
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.SerializerShim;
+
+public class StarGraphSerializer implements SerializerShim<StarGraph> {
+
+    private final Direction edgeDirectionToSerialize;
+    private GraphFilter graphFilter;
+
+    private final static byte VERSION_1 = Byte.MIN_VALUE;
+
+    public StarGraphSerializer(final Direction edgeDirectionToSerialize, final GraphFilter graphFilter) {
+        this.edgeDirectionToSerialize = edgeDirectionToSerialize;
+        this.graphFilter = graphFilter;
+    }
+
+    @Override
+    public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final StarGraph starGraph) {
+        output.writeByte(VERSION_1);
+        kryo.writeObjectOrNull(output, starGraph.edgeProperties, HashMap.class);
+        kryo.writeObjectOrNull(output, starGraph.metaProperties, HashMap.class);
+        kryo.writeClassAndObject(output, starGraph.starVertex.id);
+        kryo.writeObject(output, starGraph.starVertex.label);
+        writeEdges(kryo, output, starGraph, Direction.IN);
+        writeEdges(kryo, output, starGraph, Direction.OUT);
+        kryo.writeObject(output, null != starGraph.starVertex.vertexProperties);
+        if (null != starGraph.starVertex.vertexProperties) {
+            kryo.writeObject(output, starGraph.starVertex.vertexProperties.size());
+            for (final Map.Entry<String, List<VertexProperty>> vertexProperties : starGraph.starVertex.vertexProperties.entrySet()) {
+                kryo.writeObject(output, vertexProperties.getKey());
+                kryo.writeObject(output, vertexProperties.getValue().size());
+                for (final VertexProperty vertexProperty : vertexProperties.getValue()) {
+                    kryo.writeClassAndObject(output, vertexProperty.id());
+                    kryo.writeClassAndObject(output, vertexProperty.value());
+                }
+            }
+        }
+    }
+
+    /**
+     * If the returned {@link StarGraph} is null, that means that the {@link GraphFilter} filtered the vertex.
+     */
+    @Override
+    public <I extends InputShim> StarGraph read(KryoShim<I, ?> kryo, I input, Class<StarGraph> clazz) {
+        final StarGraph starGraph = StarGraph.open();
+        input.readByte();  // version field ignored for now - for future use with backward compatibility
+        starGraph.edgeProperties = kryo.readObjectOrNull(input, HashMap.class);
+        starGraph.metaProperties = kryo.readObjectOrNull(input, HashMap.class);
+        starGraph.addVertex(T.id, kryo.readClassAndObject(input), T.label, kryo.readObject(input, String.class));
+        readEdges(kryo, input, starGraph, Direction.IN);
+        readEdges(kryo, input, starGraph, Direction.OUT);
+        if (kryo.readObject(input, Boolean.class)) {
+            final int numberOfUniqueKeys = kryo.readObject(input, Integer.class);
+            for (int i = 0; i < numberOfUniqueKeys; i++) {
+                final String vertexPropertyKey = kryo.readObject(input, String.class);
+                final int numberOfVertexPropertiesWithKey = kryo.readObject(input, Integer.class);
+                for (int j = 0; j < numberOfVertexPropertiesWithKey; j++) {
+                    final Object id = kryo.readClassAndObject(input);
+                    final Object value = kryo.readClassAndObject(input);
+                    starGraph.starVertex.property(VertexProperty.Cardinality.list, vertexPropertyKey, value, T.id, id);
+                }
+            }
+        }
+        return this.graphFilter.hasFilter() ? starGraph.applyGraphFilter(this.graphFilter).orElse(null) : starGraph;
+    }
+
+    private <O extends OutputShim> void writeEdges(final KryoShim<?, O> kryo, final O output, final StarGraph starGraph, final Direction direction) {
+        // only write edges if there are some AND if the user requested them to be serialized AND if they match
+        // the direction being serialized by the format
+        final Map<String, List<Edge>> starEdges = direction.equals(Direction.OUT) ? starGraph.starVertex.outEdges : starGraph.starVertex.inEdges;
+        final boolean writeEdges = null != starEdges && edgeDirectionToSerialize != null
+                && (edgeDirectionToSerialize == direction || edgeDirectionToSerialize == Direction.BOTH);
+        kryo.writeObject(output, writeEdges);
+        if (writeEdges) {
+            kryo.writeObject(output, starEdges.size());
+            for (final Map.Entry<String, List<Edge>> edges : starEdges.entrySet()) {
+                kryo.writeObject(output, edges.getKey());
+                kryo.writeObject(output, edges.getValue().size());
+                for (final Edge edge : edges.getValue()) {
+                    kryo.writeClassAndObject(output, edge.id());
+                    kryo.writeClassAndObject(output, direction.equals(Direction.OUT) ? edge.inVertex().id() : edge.outVertex().id());
+                }
+            }
+        }
+    }
+
+    private <I extends InputShim> void readEdges(final KryoShim<I, ?> kryo, final I input, final StarGraph starGraph, final Direction direction) {
+        if (kryo.readObject(input, Boolean.class)) {
+            final int numberOfUniqueLabels = kryo.readObject(input, Integer.class);
+            for (int i = 0; i < numberOfUniqueLabels; i++) {
+                final String edgeLabel = kryo.readObject(input, String.class);
+                final int numberOfEdgesWithLabel = kryo.readObject(input, Integer.class);
+                for (int j = 0; j < numberOfEdgesWithLabel; j++) {
+                    final Object edgeId = kryo.readClassAndObject(input);
+                    final Object adjacentVertexId = kryo.readClassAndObject(input);
+                    if (this.graphFilter.checkEdgeLegality(direction, edgeLabel).positive()) {
+                        if (direction.equals(Direction.OUT))
+                            starGraph.starVertex.addOutEdge(edgeLabel, starGraph.addVertex(T.id, adjacentVertexId), T.id, edgeId);
+                        else
+                            starGraph.starVertex.addInEdge(edgeLabel, starGraph.addVertex(T.id, adjacentVertexId), T.id, edgeId);
+                    } else if (null != starGraph.edgeProperties) {
+                        starGraph.edgeProperties.remove(edgeId);
+                    }
+                }
+            }
+        }
+    }
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
new file mode 100644
index 0000000..2053280
--- /dev/null
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
@@ -0,0 +1,69 @@
+/*
+ * 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.hadoop.structure.io;
+
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShimService;
+import org.apache.tinkerpop.shaded.kryo.Kryo;
+import org.apache.tinkerpop.shaded.kryo.io.Input;
+import org.apache.tinkerpop.shaded.kryo.io.Output;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+public class HadoopPoolShimService implements KryoShimService {
+
+    public Object readClassAndObject(InputStream source) {
+
+        Kryo k = null;
+
+        try {
+            k = HadoopPools.getGryoPool().takeKryo();
+
+            return k.readClassAndObject(new Input(source));
+        } finally {
+            if (null != k) {
+                HadoopPools.getGryoPool().offerKryo(k);
+            }
+        }
+    }
+
+    public void writeClassAndObject(Object o, OutputStream sink) {
+
+        Kryo k = null;
+
+        try {
+            k = HadoopPools.getGryoPool().takeKryo();
+
+            Output output = new Output(sink);
+
+            k.writeClassAndObject(output, o);
+
+            output.flush();
+        } finally {
+            if (null != k) {
+                HadoopPools.getGryoPool().offerKryo(k);
+            }
+        }
+    }
+
+    @Override
+    public int getPriority() {
+        return 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPools.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPools.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPools.java
index f09f703..5074ad5 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPools.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPools.java
@@ -43,6 +43,7 @@ public final class HadoopPools {
             GRYO_POOL = GryoPool.build().
                     poolSize(configuration.getInt(GryoPool.CONFIG_IO_GRYO_POOL_SIZE, 256)).
                     ioRegistries(configuration.getList(GryoPool.CONFIG_IO_REGISTRY, Collections.emptyList())).
+                    initializeMapper(m -> m.registrationRequired(false)).
                     create();
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
index 9a07f75..e7a38a5 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
@@ -21,6 +21,8 @@ package org.apache.tinkerpop.gremlin.hadoop.structure.io;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.tinkerpop.gremlin.process.computer.MapReduce;
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShimServiceLoader;
+import org.apache.tinkerpop.shaded.kryo.io.Output;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -63,29 +65,14 @@ public final class ObjectWritable<T> implements WritableComparable<ObjectWritabl
 
     @Override
     public void readFields(final DataInput input) throws IOException {
-        this.t = HadoopPools.getGryoPool().doWithReader(gryoReader -> {
-            try {
-                // class argument is Object because gryo doesn't really care that we don't know the specific type.
-                // the type is embedded in the stream so it can just read it from there and return it as needed.
-                // presumably that will cast nicely to T
-                return (T) gryoReader.readObject(new ByteArrayInputStream(WritableUtils.readCompressedByteArray(input)), Object.class);
-            } catch (final IOException e) {
-                throw new IllegalStateException(e.getMessage(), e);
-            }
-        });
+        ByteArrayInputStream bais = new ByteArrayInputStream(WritableUtils.readCompressedByteArray(input));
+        this.t = KryoShimServiceLoader.readClassAndObject(bais);
     }
 
     @Override
     public void write(final DataOutput output) throws IOException {
-        HadoopPools.getGryoPool().doWithWriter(gryoWriter -> {
-            try {
-                final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
-                gryoWriter.writeObject(outputStream, this.t);
-                WritableUtils.writeCompressedByteArray(output, outputStream.toByteArray());
-            } catch (final IOException e) {
-                throw new IllegalStateException(e.getMessage(), e);
-            }
-        });
+        byte serialized[] = KryoShimServiceLoader.writeClassAndObjectToBytes(this.t);
+        WritableUtils.writeCompressedByteArray(output, serialized);
     }
 
     private void writeObject(final ObjectOutputStream outputStream) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritable.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritable.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritable.java
index ac360e9..7ac8e8c 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritable.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritable.java
@@ -21,6 +21,7 @@ package org.apache.tinkerpop.gremlin.hadoop.structure.io;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.tinkerpop.gremlin.structure.Vertex;
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShimServiceLoader;
 import org.apache.tinkerpop.gremlin.structure.util.ElementHelper;
 import org.apache.tinkerpop.gremlin.structure.util.star.StarGraph;
 
@@ -60,42 +61,15 @@ public final class VertexWritable implements Writable, Serializable {
 
     @Override
     public void readFields(final DataInput input) throws IOException {
-        try {
-            this.vertex = null;
-            this.vertex = HadoopPools.getGryoPool().doWithReader(gryoReader -> {
-                try {
-                    final ByteArrayInputStream inputStream = new ByteArrayInputStream(WritableUtils.readCompressedByteArray(input));
-                    return gryoReader.readObject(inputStream, StarGraph.class).getStarVertex(); // read the star graph
-                } catch (final IOException e) {
-                    throw new IllegalStateException(e.getMessage(), e);
-                }
-            });
-        } catch (final IllegalStateException e) {
-            if (e.getCause() instanceof IOException)
-                throw (IOException) e.getCause();
-            else
-                throw e;
-        }
+        this.vertex = null;
+        ByteArrayInputStream bais = new ByteArrayInputStream(WritableUtils.readCompressedByteArray(input));
+        this.vertex = ((StarGraph)KryoShimServiceLoader.readClassAndObject(bais)).getStarVertex(); // read the star graph;
     }
 
     @Override
     public void write(final DataOutput output) throws IOException {
-        try {
-            HadoopPools.getGryoPool().doWithWriter(gryoWriter -> {
-                try {
-                    final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
-                    gryoWriter.writeObject(outputStream, this.vertex.graph()); // write the star graph
-                    WritableUtils.writeCompressedByteArray(output, outputStream.toByteArray());
-                } catch (final IOException e) {
-                    throw new IllegalStateException(e.getMessage(), e);
-                }
-            });
-        } catch (final IllegalStateException e) {
-            if (e.getCause() instanceof IOException)
-                throw (IOException) e.getCause();
-            else
-                throw e;
-        }
+        byte serialized[] = KryoShimServiceLoader.writeClassAndObjectToBytes(this.vertex.graph());
+        WritableUtils.writeCompressedByteArray(output, serialized);
     }
 
     private void writeObject(final ObjectOutputStream outputStream) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/hadoop-gremlin/src/main/resources/META-INF/services/org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShimService
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/resources/META-INF/services/org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShimService b/hadoop-gremlin/src/main/resources/META-INF/services/org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShimService
new file mode 100644
index 0000000..0b27e72
--- /dev/null
+++ b/hadoop-gremlin/src/main/resources/META-INF/services/org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShimService
@@ -0,0 +1 @@
+org.apache.tinkerpop.gremlin.hadoop.structure.io.HadoopPoolShimService # HadoopPools provides/caches instances of TinkerPop's shaded Kryo


[34/34] incubator-tinkerpop git commit: Merge branch 'master' into TINKERPOP-1278

Posted by ok...@apache.org.
Merge branch 'master' into TINKERPOP-1278


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

Branch: refs/heads/TINKERPOP-1278
Commit: 5ab566cffcd34906ef23b5674036bc840cc316a6
Parents: fb29b25 3aa0e37
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Tue Jun 7 07:56:48 2016 -0600
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Tue Jun 7 07:56:48 2016 -0600

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |   2 +
 docs/src/dev/provider/index.asciidoc            |  10 +-
 docs/src/recipes/between-vertices.asciidoc      |  14 +-
 .../src/reference/gremlin-applications.asciidoc |   5 +-
 .../reference/implementations-hadoop.asciidoc   |   4 +-
 .../upgrade/release-3.2.x-incubating.asciidoc   |   6 +-
 .../process/computer/GiraphWorkerContext.java   |   3 +-
 .../gremlin/structure/io/gryo/GryoMapper.java   | 441 ++++++++++++-------
 .../gremlin/structure/io/gryo/GryoPool.java     |   1 +
 .../structure/io/gryo/GryoSerializers.java      |  40 +-
 .../structure/io/gryo/JavaTimeSerializers.java  | 125 ++----
 .../structure/io/gryo/PairSerializer.java       |  11 +-
 .../structure/io/gryo/TypeRegistration.java     |  82 ++++
 .../structure/io/gryo/URISerializer.java        |  23 +-
 .../structure/io/gryo/UUIDSerializer.java       |  24 +-
 .../structure/io/gryo/kryoshim/InputShim.java   |  38 ++
 .../structure/io/gryo/kryoshim/KryoShim.java    |  41 ++
 .../io/gryo/kryoshim/KryoShimService.java       |  99 +++++
 .../io/gryo/kryoshim/KryoShimServiceLoader.java | 215 +++++++++
 .../structure/io/gryo/kryoshim/OutputShim.java  |  40 ++
 .../io/gryo/kryoshim/SerializerShim.java        |  36 ++
 .../io/gryo/kryoshim/package-info.java          |  55 +++
 .../kryoshim/shaded/ShadedInputAdapter.java     |  66 +++
 .../gryo/kryoshim/shaded/ShadedKryoAdapter.java |  67 +++
 .../kryoshim/shaded/ShadedOutputAdapter.java    |  71 +++
 .../shaded/ShadedSerializerAdapter.java         |  54 +++
 .../io/gryo/kryoshim/shaded/package-info.java   |  25 ++
 .../util/star/StarGraphGryoSerializer.java      | 130 +-----
 .../util/star/StarGraphSerializer.java          | 150 +++++++
 .../gremlin/groovy/engine/GremlinExecutor.java  |  12 +-
 .../gremlin/groovy/engine/ScriptEngines.java    |  28 +-
 .../jsr223/GremlinGroovyScriptEngine.java       |   8 +-
 .../ConfigurationCustomizerProvider.java        |  83 ++++
 .../groovy/jsr223/BaseScriptForTesting.java     |  30 ++
 .../GremlinGroovyScriptEngineConfigTest.java    |  40 ++
 .../ConfigurationCustomizerProviderTest.java    |  78 ++++
 .../tinkerpop/gremlin/server/GremlinServer.java |   2 +-
 .../gremlin/server/BaseScriptForTesting.java    |  30 ++
 .../server/GremlinServerIntegrateTest.java      |  50 ++-
 .../hadoop/process/computer/HadoopCombine.java  |   3 +-
 .../hadoop/process/computer/HadoopMap.java      |   3 +-
 .../hadoop/process/computer/HadoopReduce.java   |   3 +-
 .../structure/io/HadoopPoolShimService.java     |  75 ++++
 .../hadoop/structure/io/HadoopPools.java        |   1 +
 .../structure/io/HadoopPoolsConfigurable.java   |   4 +-
 .../hadoop/structure/io/ObjectWritable.java     |  25 +-
 .../hadoop/structure/io/VertexWritable.java     |  39 +-
 .../structure/io/gryo/GryoRecordReader.java     |   3 +-
 .../structure/io/gryo/GryoRecordWriter.java     |   4 +-
 .../structure/io/script/ScriptInputFormat.java  |   5 +
 .../structure/io/script/ScriptOutputFormat.java |   6 +
 ...n.structure.io.gryo.kryoshim.KryoShimService |   1 +
 .../gremlin/hadoop/HadoopGraphProvider.java     |   3 +
 pom.xml                                         |   3 +
 .../spark/process/computer/SparkExecutor.java   |  14 +-
 .../structure/io/gryo/GryoRegistrator.java      | 238 ++++++++++
 .../spark/structure/io/gryo/GryoSerializer.java |   2 +-
 .../io/gryo/IoRegistryAwareKryoSerializer.java  | 116 +++++
 .../io/gryo/ObjectWritableSerializer.java       |  12 +-
 .../io/gryo/VertexWritableSerializer.java       |  12 +-
 .../kryoshim/unshaded/UnshadedInputAdapter.java |  78 ++++
 .../kryoshim/unshaded/UnshadedKryoAdapter.java  |  74 ++++
 .../unshaded/UnshadedKryoShimService.java       | 154 +++++++
 .../unshaded/UnshadedOutputAdapter.java         |  83 ++++
 .../unshaded/UnshadedSerializerAdapter.java     |  57 +++
 ...n.structure.io.gryo.kryoshim.KryoShimService |   1 +
 ...tratorGraphComputerProcessIntegrateTest.java |  33 ++
 ...SparkHadoopGraphGryoRegistratorProvider.java |  52 +++
 .../computer/SparkHadoopGraphProvider.java      |   8 +-
 .../spark/structure/io/ToyGraphInputRDD.java    |   3 +-
 70 files changed, 2843 insertions(+), 511 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/5ab566cf/pom.xml
----------------------------------------------------------------------


[26/34] incubator-tinkerpop git commit: worked with @dalaro to fix a bug in HadoopPoolShimService. Reverted my last work on VertexProgramHelper.

Posted by ok...@apache.org.
worked with @dalaro to fix a bug in HadoopPoolShimService. Reverted my last work on VertexProgramHelper.


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

Branch: refs/heads/TINKERPOP-1278
Commit: 90e3159969363e6a3383ffc64d58c27f76384a55
Parents: 797364c
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Mon Jun 6 16:55:42 2016 -0600
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Mon Jun 6 16:55:42 2016 -0600

----------------------------------------------------------------------
 .../computer/util/VertexProgramHelper.java      | 33 +++++---------------
 .../io/gryo/kryoshim/KryoShimServiceLoader.java | 17 +++++-----
 .../structure/io/HadoopPoolShimService.java     |  2 +-
 .../gremlin/hadoop/HadoopGraphProvider.java     |  2 +-
 .../spark/process/computer/SparkExecutor.java   | 11 +++----
 5 files changed, 23 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/90e31599/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/util/VertexProgramHelper.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/util/VertexProgramHelper.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/util/VertexProgramHelper.java
index 2b3a0b2..bc67866 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/util/VertexProgramHelper.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/util/VertexProgramHelper.java
@@ -25,13 +25,8 @@ import org.apache.tinkerpop.gremlin.process.traversal.Step;
 import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
 import org.apache.tinkerpop.gremlin.process.traversal.step.map.EdgeVertexStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.map.VertexStep;
-import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoPool;
-import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader;
 import org.apache.tinkerpop.gremlin.util.Serializer;
-import org.apache.tinkerpop.shaded.kryo.io.Input;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashSet;
@@ -42,8 +37,6 @@ import java.util.Set;
  */
 public final class VertexProgramHelper {
 
-    private static final GryoPool GRYO_POOL = GryoPool.build().create();
-
     private VertexProgramHelper() {
     }
 
@@ -74,33 +67,21 @@ public final class VertexProgramHelper {
             final String byteString = Arrays.toString(Serializer.serializeObject(object));
             configuration.setProperty(key, byteString.substring(1, byteString.length() - 1));
         } catch (final IOException e) {
-            try {
-                final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
-                GRYO_POOL.doWithWriter(kryo -> kryo.writeObject(outputStream, object));
-                String byteString = Arrays.toString(outputStream.toByteArray());
-                configuration.setProperty(key, byteString.substring(1, byteString.length() - 1));
-            } catch (final Exception e1) {
-                throw new IllegalArgumentException(e1.getMessage(), e1);
-            }
+            throw new IllegalArgumentException(e.getMessage(), e);
         }
     }
 
     public static <T> T deserialize(final Configuration configuration, final String key) {
-        final String[] stringBytes = configuration.getString(key).split(",");
-        byte[] bytes = new byte[stringBytes.length];
-        for (int i = 0; i < stringBytes.length; i++) {
-            bytes[i] = Byte.valueOf(stringBytes[i].trim());
-        }
         try {
+            final String[] stringBytes = configuration.getString(key).split(",");
+            byte[] bytes = new byte[stringBytes.length];
+            for (int i = 0; i < stringBytes.length; i++) {
+                bytes[i] = Byte.valueOf(stringBytes[i].trim());
+            }
             return (T) Serializer.deserializeObject(bytes);
         } catch (final IOException | ClassNotFoundException e) {
-            try {
-                return (T) GRYO_POOL.readWithKryo(kryo -> kryo.readClassAndObject(new Input(new ByteArrayInputStream(bytes))));
-            } catch (final Exception e1) {
-                throw new IllegalArgumentException(e1.getMessage(), e1);
-            }
+            throw new IllegalArgumentException(e.getMessage(), e);
         }
-
     }
 
     public static <S, E> Traversal.Admin<S, E> reverse(final Traversal.Admin<S, E> traversal) {

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/90e31599/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
index 9184dd0..fd57a3c 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
@@ -51,6 +51,7 @@ public class KryoShimServiceLoader {
 
     public static void applyConfiguration(Configuration conf) {
         KryoShimServiceLoader.conf = conf;
+        load(true);
     }
 
     /**
@@ -195,20 +196,20 @@ public class KryoShimServiceLoader {
 
                 if (0 == result) {
                     log.warn("Found two {} implementations with the same canonical classname: {}.  " +
-                             "This may indicate a problem with the classpath/classloader such as " +
-                             "duplicate or conflicting copies of the file " +
-                             "META-INF/services/org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimService.",
-                             a.getClass().getCanonicalName());
+                                    "This may indicate a problem with the classpath/classloader such as " +
+                                    "duplicate or conflicting copies of the file " +
+                                    "META-INF/services/org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimService.",
+                            a.getClass().getCanonicalName());
                 } else {
                     String winner = 0 < result ? a.getClass().getCanonicalName() : b.getClass().getCanonicalName();
                     log.warn("{} implementations {} and {} are tied with priority value {}.  " +
-                             "Preferring {} to the other because it has a lexicographically greater classname.  " +
-                             "Consider setting the system property \"{}\" instead of relying on priority tie-breaking.",
-                             KryoShimService.class.getSimpleName(), a, b, ap, winner, SHIM_CLASS_SYSTEM_PROPERTY);
+                                    "Preferring {} to the other because it has a lexicographically greater classname.  " +
+                                    "Consider setting the system property \"{}\" instead of relying on priority tie-breaking.",
+                            KryoShimService.class.getSimpleName(), a, b, ap, winner, SHIM_CLASS_SYSTEM_PROPERTY);
                 }
 
                 return result;
             }
         }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/90e31599/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
index 5753d90..df72b71 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
@@ -71,6 +71,6 @@ public class HadoopPoolShimService implements KryoShimService {
 
     @Override
     public void applyConfiguration(Configuration conf) {
-        KryoShimServiceLoader.applyConfiguration(conf);
+        HadoopPools.initialize(conf);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/90e31599/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java
index 57157db..e36c08d 100644
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java
@@ -112,7 +112,7 @@ public class HadoopGraphProvider extends AbstractGraphProvider {
 
     @Override
     public Map<String, Object> getBaseConfiguration(final String graphName, final Class<?> test, final String testMethodName, final LoadGraphWith.GraphData loadGraphWith) {
-        System.setProperty(SHIM_CLASS_SYSTEM_PROPERTY, HadoopPoolShimService.class.getCanonicalName());
+        System.clearProperty(SHIM_CLASS_SYSTEM_PROPERTY);
         this.graphSONInput = RANDOM.nextBoolean();
         return new HashMap<String, Object>() {{
             put(Graph.GRAPH, HadoopGraph.class.getName());

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/90e31599/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkExecutor.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkExecutor.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkExecutor.java
index 9e5ac53..4db8086 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkExecutor.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkExecutor.java
@@ -24,7 +24,6 @@ import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.function.Function2;
 import org.apache.spark.api.java.function.PairFlatMapFunction;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.HadoopPools;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
 import org.apache.tinkerpop.gremlin.process.computer.GraphFilter;
 import org.apache.tinkerpop.gremlin.process.computer.MapReduce;
@@ -171,7 +170,7 @@ public final class SparkExecutor {
             assert graphRDD.partitioner().get().equals(newViewIncomingRDD.partitioner().get());
         newViewIncomingRDD
                 .foreachPartition(partitionIterator -> {
-                    HadoopPools.initialize(apacheConfiguration);
+                    KryoShimServiceLoader.applyConfiguration(apacheConfiguration);
                 }); // need to complete a task so its BSP and the memory for this iteration is updated
         return newViewIncomingRDD;
     }
@@ -206,7 +205,7 @@ public final class SparkExecutor {
             final JavaPairRDD<Object, VertexWritable> graphRDD, final MapReduce<K, V, ?, ?, ?> mapReduce,
             final Configuration apacheConfiguration) {
         JavaPairRDD<K, V> mapRDD = graphRDD.mapPartitionsToPair(partitionIterator -> {
-            HadoopPools.initialize(apacheConfiguration);
+            KryoShimServiceLoader.applyConfiguration(apacheConfiguration);
             return () -> new MapIterator<>(MapReduce.<MapReduce<K, V, ?, ?, ?>>createMapReduce(HadoopGraph.open(apacheConfiguration), apacheConfiguration), partitionIterator);
         });
         if (mapReduce.getMapKeySort().isPresent())
@@ -217,7 +216,7 @@ public final class SparkExecutor {
     public static <K, V, OK, OV> JavaPairRDD<OK, OV> executeCombine(final JavaPairRDD<K, V> mapRDD,
                                                                     final Configuration apacheConfiguration) {
         return mapRDD.mapPartitionsToPair(partitionIterator -> {
-            HadoopPools.initialize(apacheConfiguration);
+            KryoShimServiceLoader.applyConfiguration(apacheConfiguration);
             return () -> new CombineIterator<>(MapReduce.<MapReduce<K, V, OK, OV, ?>>createMapReduce(HadoopGraph.open(apacheConfiguration), apacheConfiguration), partitionIterator);
         });
     }
@@ -226,11 +225,11 @@ public final class SparkExecutor {
             final JavaPairRDD<K, V> mapOrCombineRDD, final MapReduce<K, V, OK, OV, ?> mapReduce,
             final Configuration apacheConfiguration) {
         JavaPairRDD<OK, OV> reduceRDD = mapOrCombineRDD.groupByKey().mapPartitionsToPair(partitionIterator -> {
-            HadoopPools.initialize(apacheConfiguration);
+            KryoShimServiceLoader.applyConfiguration(apacheConfiguration);
             return () -> new ReduceIterator<>(MapReduce.<MapReduce<K, V, OK, OV, ?>>createMapReduce(HadoopGraph.open(apacheConfiguration), apacheConfiguration), partitionIterator);
         });
         if (mapReduce.getReduceKeySort().isPresent())
             reduceRDD = reduceRDD.sortByKey(mapReduce.getReduceKeySort().get(), true, 1);
         return reduceRDD;
     }
-}
+}
\ No newline at end of file


[13/34] incubator-tinkerpop git commit: Added GraphTraversalSource to core javadoc. CTR

Posted by ok...@apache.org.
Added GraphTraversalSource to core javadoc. CTR


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

Branch: refs/heads/TINKERPOP-1278
Commit: 6e17d668a8ead38d426b61beb224a42303280526
Parents: 2f4c24c
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Sat Jun 4 07:32:13 2016 -0400
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Sat Jun 4 07:32:13 2016 -0400

----------------------------------------------------------------------
 pom.xml | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/6e17d668/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 077f08b..fe9d781 100644
--- a/pom.xml
+++ b/pom.xml
@@ -938,6 +938,9 @@ limitations under the License.
                                         <include>
                                             org/apache/tinkerpop/gremlin/process/traversal/dsl/graph/GraphTraversal.java
                                         </include>
+                                        <include>
+                                            org/apache/tinkerpop/gremlin/process/traversal/dsl/graph/GraphTraversalSource.java
+                                        </include>
                                         <!-- driver -->
                                         <include>org/apache/tinkerpop/gremlin/driver/*.java
                                         </include>


[33/34] incubator-tinkerpop git commit: renamed the global System property from tinkerpop.kryo.shim to gremlin.io.kryoShimService to be consistent with our property naming convention -- cc @dalaro. CTR.

Posted by ok...@apache.org.
renamed the global System property from tinkerpop.kryo.shim to gremlin.io.kryoShimService to be consistent with our property naming convention -- cc @dalaro. CTR.


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

Branch: refs/heads/TINKERPOP-1278
Commit: 3aa0e374c1a69ace2001743745fdfb47364d10e1
Parents: 2c7b1f9
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Tue Jun 7 07:17:50 2016 -0600
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Tue Jun 7 07:17:50 2016 -0600

----------------------------------------------------------------------
 .../structure/io/gryo/kryoshim/KryoShimService.java   |  2 +-
 .../io/gryo/kryoshim/KryoShimServiceLoader.java       | 14 +++++++-------
 .../tinkerpop/gremlin/hadoop/HadoopGraphProvider.java |  5 ++---
 .../tinkerpop/gremlin/spark/AbstractSparkTest.java    |  3 ---
 .../SparkHadoopGraphGryoRegistratorProvider.java      |  6 +++---
 .../process/computer/SparkHadoopGraphProvider.java    |  7 -------
 6 files changed, 13 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3aa0e374/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java
index ee0c8b3..b8880a4 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java
@@ -64,7 +64,7 @@ public interface KryoShimService {
 
     /**
      * Returns this service's relative priority number.  Unless explicitly overridden through a
-     * system property ({@link KryoShimServiceLoader#SHIM_CLASS_SYSTEM_PROPERTY}),
+     * system property ({@link KryoShimServiceLoader#KRYO_SHIM_SERVICE}),
      * the service implementation with the numerically highest priority will be used
      * and all others ignored.  In other words, the highest priority wins (in the absence of a
      * system property override).

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3aa0e374/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
index d16b1a5..4227edc 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
@@ -47,7 +47,7 @@ public class KryoShimServiceLoader {
      * package-and-classname to force it into service.  Setting this property causes the
      * priority-selection mechanism ({@link KryoShimService#getPriority()}) to be ignored.
      */
-    public static final String SHIM_CLASS_SYSTEM_PROPERTY = "tinkerpop.kryo.shim";
+    public static final String KRYO_SHIM_SERVICE = "gremlin.io.kryoShimService";
 
     public static void applyConfiguration(final Configuration conf) {
         KryoShimServiceLoader.conf = conf;
@@ -86,14 +86,14 @@ public class KryoShimServiceLoader {
             }
         }
 
-        String shimClass = System.getProperty(SHIM_CLASS_SYSTEM_PROPERTY);
+        String shimClass = System.getProperty(KRYO_SHIM_SERVICE);
 
         if (null != shimClass) {
             for (KryoShimService kss : services) {
                 if (kss.getClass().getCanonicalName().equals(shimClass)) {
                     log.info("Set {} provider to {} ({}) from system property {}={}",
                             KryoShimService.class.getSimpleName(), kss, kss.getClass(),
-                            SHIM_CLASS_SYSTEM_PROPERTY, shimClass);
+                            KRYO_SHIM_SERVICE, shimClass);
                     result = kss;
                 }
             }
@@ -160,19 +160,19 @@ public class KryoShimServiceLoader {
      * where the {@code input} parameter is {@code source}.  Returns the deserialized object.
      *
      * @param source an input stream containing data for a serialized object class and instance
-     * @param <T> the type to which the deserialized object is cast as it is returned
+     * @param <T>    the type to which the deserialized object is cast as it is returned
      * @return the deserialized object
      */
     public static <T> T readClassAndObject(final InputStream source) {
         final KryoShimService shimService = load();
 
-        return (T)shimService.readClassAndObject(source);
+        return (T) shimService.readClassAndObject(source);
     }
 
     /**
      * Selects the service with greatest {@link KryoShimService#getPriority()}
      * (not absolute value).
-     *
+     * <p>
      * Breaks ties with lexicographical comparison of classnames where the
      * name that sorts last is considered to have highest priority.  Ideally
      * nothing should rely on that tiebreaking behavior, but it beats random
@@ -205,7 +205,7 @@ public class KryoShimServiceLoader {
                     log.warn("{} implementations {} and {} are tied with priority value {}.  " +
                                     "Preferring {} to the other because it has a lexicographically greater classname.  " +
                                     "Consider setting the system property \"{}\" instead of relying on priority tie-breaking.",
-                            KryoShimService.class.getSimpleName(), a, b, ap, winner, SHIM_CLASS_SYSTEM_PROPERTY);
+                            KryoShimService.class.getSimpleName(), a, b, ap, winner, KRYO_SHIM_SERVICE);
                 }
 
                 return result;

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3aa0e374/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java
index e36c08d..9c6a352 100644
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java
@@ -28,7 +28,6 @@ import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopProperty;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopVertex;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopVertexProperty;
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.HadoopPoolShimService;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.graphson.GraphSONInputFormat;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.gryo.GryoInputFormat;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.gryo.GryoOutputFormat;
@@ -46,7 +45,7 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 
-import static org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader.SHIM_CLASS_SYSTEM_PROPERTY;
+import static org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader.KRYO_SHIM_SERVICE;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
@@ -112,7 +111,7 @@ public class HadoopGraphProvider extends AbstractGraphProvider {
 
     @Override
     public Map<String, Object> getBaseConfiguration(final String graphName, final Class<?> test, final String testMethodName, final LoadGraphWith.GraphData loadGraphWith) {
-        System.clearProperty(SHIM_CLASS_SYSTEM_PROPERTY);
+        System.clearProperty(KRYO_SHIM_SERVICE);
         this.graphSONInput = RANDOM.nextBoolean();
         return new HashMap<String, Object>() {{
             put(Graph.GRAPH, HadoopGraph.class.getName());

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3aa0e374/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 473976b..6d2231f 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,7 +26,6 @@ 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;
@@ -35,8 +34,6 @@ 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)
  */

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3aa0e374/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
index c1832cc..fcebbd0 100644
--- 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
@@ -29,7 +29,7 @@ import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLo
 
 import java.util.Map;
 
-import static org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader.SHIM_CLASS_SYSTEM_PROPERTY;
+import static org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader.KRYO_SHIM_SERVICE;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
@@ -44,9 +44,9 @@ public final class SparkHadoopGraphGryoRegistratorProvider extends SparkHadoopGr
         config.put(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, false);
         config.put("spark.serializer", KryoSerializer.class.getCanonicalName());
         config.put("spark.kryo.registrator", GryoRegistrator.class.getCanonicalName());
-        System.setProperty(SHIM_CLASS_SYSTEM_PROPERTY, UnshadedKryoShimService.class.getCanonicalName());
+        System.setProperty(KRYO_SHIM_SERVICE, UnshadedKryoShimService.class.getCanonicalName());
         KryoShimServiceLoader.load(true);
-        System.clearProperty(SHIM_CLASS_SYSTEM_PROPERTY);
+        System.clearProperty(KRYO_SHIM_SERVICE);
         return config;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/3aa0e374/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 06119fc..d4201b5 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
@@ -18,7 +18,6 @@
  */
 package org.apache.tinkerpop.gremlin.spark.process.computer;
 
-import org.apache.spark.serializer.KryoSerializer;
 import org.apache.tinkerpop.gremlin.GraphProvider;
 import org.apache.tinkerpop.gremlin.LoadGraphWith;
 import org.apache.tinkerpop.gremlin.groovy.util.SugarTestHelper;
@@ -26,7 +25,6 @@ import org.apache.tinkerpop.gremlin.hadoop.Constants;
 import org.apache.tinkerpop.gremlin.hadoop.HadoopGraphProvider;
 import org.apache.tinkerpop.gremlin.hadoop.groovy.plugin.HadoopGremlinPluginCheck;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.FileSystemStorageCheck;
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.HadoopPoolShimService;
 import org.apache.tinkerpop.gremlin.process.computer.Computer;
 import org.apache.tinkerpop.gremlin.process.computer.GraphComputer;
 import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversalSource;
@@ -41,16 +39,11 @@ import org.apache.tinkerpop.gremlin.spark.structure.Spark;
 import org.apache.tinkerpop.gremlin.spark.structure.io.PersistedOutputRDD;
 import org.apache.tinkerpop.gremlin.spark.structure.io.SparkContextStorageCheck;
 import org.apache.tinkerpop.gremlin.spark.structure.io.ToyGraphInputRDD;
-import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.GryoRegistrator;
 import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.GryoSerializer;
-import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.kryoshim.unshaded.UnshadedKryoShimService;
 import org.apache.tinkerpop.gremlin.structure.Graph;
-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)
  */


[24/34] incubator-tinkerpop git commit: Able to now test both shim and non-shim models in Spark. Also go configuration with ProgramTest working.

Posted by ok...@apache.org.
Able to now test both shim and non-shim models in Spark. Also go configuration with ProgramTest working.


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

Branch: refs/heads/TINKERPOP-1278
Commit: e7003635e27c625b3f30492111f20f4fe4e24eb5
Parents: 0cd31bf
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Mon Jun 6 14:52:53 2016 -0600
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Mon Jun 6 14:52:53 2016 -0600

----------------------------------------------------------------------
 .../structure/io/gryo/GryoSerializers.java      |  8 +-
 .../structure/io/gryo/GryoRegistrator.java      | 90 ++++++++++++++++----
 .../computer/SparkHadoopGraphProvider.java      | 11 ++-
 3 files changed, 88 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/e7003635/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoSerializers.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoSerializers.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoSerializers.java
index 16fbe85..2042a4a 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoSerializers.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoSerializers.java
@@ -47,7 +47,7 @@ public final class GryoSerializers {
     /**
      * Serializes any {@link Edge} implementation encountered to a {@link DetachedEdge}.
      */
-    final static class EdgeSerializer implements SerializerShim<Edge> {
+    public final static class EdgeSerializer implements SerializerShim<Edge> {
         @Override
         public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Edge edge) {
             kryo.writeClassAndObject(output, DetachedFactory.detach(edge, true));
@@ -63,7 +63,7 @@ public final class GryoSerializers {
     /**
      * Serializes any {@link Vertex} implementation encountered to an {@link DetachedVertex}.
      */
-    final static class VertexSerializer implements SerializerShim<Vertex> {
+    public final static class VertexSerializer implements SerializerShim<Vertex> {
         @Override
         public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Vertex vertex) {
             kryo.writeClassAndObject(output, DetachedFactory.detach(vertex, true));
@@ -78,7 +78,7 @@ public final class GryoSerializers {
     /**
      * Serializes any {@link Property} implementation encountered to an {@link DetachedProperty}.
      */
-    final static class PropertySerializer implements SerializerShim<Property> {
+    public final static class PropertySerializer implements SerializerShim<Property> {
         @Override
         public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Property property) {
             kryo.writeClassAndObject(output, property instanceof VertexProperty ? DetachedFactory.detach((VertexProperty) property, true) : DetachedFactory.detach(property));
@@ -93,7 +93,7 @@ public final class GryoSerializers {
     /**
      * Serializes any {@link VertexProperty} implementation encountered to an {@link DetachedVertexProperty}.
      */
-    final static class VertexPropertySerializer implements SerializerShim<VertexProperty> {
+    public final static class VertexPropertySerializer implements SerializerShim<VertexProperty> {
         @Override
         public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, VertexProperty vertexProperty) {
             kryo.writeClassAndObject(output, DetachedFactory.detach(vertexProperty, true));

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/e7003635/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java
index 1ae8c5c..9563408 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java
@@ -22,19 +22,44 @@ import com.esotericsoftware.kryo.Kryo;
 import com.esotericsoftware.kryo.Serializer;
 import com.esotericsoftware.kryo.serializers.JavaSerializer;
 import org.apache.spark.serializer.KryoRegistrator;
+import org.apache.spark.util.SerializableConfiguration;
+import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopConfiguration;
+import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopEdge;
+import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopProperty;
+import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopVertex;
+import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopVertexProperty;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
+import org.apache.tinkerpop.gremlin.process.computer.util.ComputerGraph;
+import org.apache.tinkerpop.gremlin.process.traversal.Path;
+import org.apache.tinkerpop.gremlin.process.traversal.step.util.ImmutablePath;
+import org.apache.tinkerpop.gremlin.process.traversal.step.util.MutablePath;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.B_LP_O_P_S_SE_SL_Traverser;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.B_LP_O_S_SE_SL_Traverser;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.B_O_S_SE_SL_Traverser;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.B_O_Traverser;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.LP_O_OB_P_S_SE_SL_Traverser;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.LP_O_OB_S_SE_SL_Traverser;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.O_OB_S_SE_SL_Traverser;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.O_Traverser;
 import org.apache.tinkerpop.gremlin.spark.process.computer.payload.MessagePayload;
 import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewIncomingPayload;
 import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewOutgoingPayload;
 import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewPayload;
 import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.kryoshim.unshaded.UnshadedSerializerAdapter;
+import org.apache.tinkerpop.gremlin.structure.Edge;
+import org.apache.tinkerpop.gremlin.structure.Property;
+import org.apache.tinkerpop.gremlin.structure.Vertex;
+import org.apache.tinkerpop.gremlin.structure.VertexProperty;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoMapper;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoSerializers;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.TypeRegistration;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
 import org.apache.tinkerpop.gremlin.structure.util.star.StarGraph;
+import org.apache.tinkerpop.gremlin.structure.util.star.StarGraphSerializer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import scala.collection.mutable.WrappedArray;
 
 import java.util.*;
 
@@ -55,11 +80,11 @@ public class GryoRegistrator implements KryoRegistrator {
      * Register TinkerPop's classes with the supplied {@link Kryo} instance
      * while honoring optional overrides and optional class blacklist ("blackset"?).
      *
-     * @param kryo the Kryo serializer instance with which to register types
+     * @param kryo                the Kryo serializer instance with which to register types
      * @param serializerOverrides serializer mappings that override this class's defaults
-     * @param blacklist classes which should not be registered at all, even if there is an override entry
-     *                  or if they would be registered by this class by default (does not affect Kryo's
-     *                  built-in registrations, e.g. String.class).
+     * @param blacklist           classes which should not be registered at all, even if there is an override entry
+     *                            or if they would be registered by this class by default (does not affect Kryo's
+     *                            built-in registrations, e.g. String.class).
      */
     public void registerClasses(Kryo kryo, Map<Class<?>, Serializer<?>> serializerOverrides, Set<Class<?>> blacklist) {
         // Apply TinkerPop type registrations copied from GyroSerializer's constructor
@@ -111,11 +136,11 @@ public class GryoRegistrator implements KryoRegistrator {
                 } else {
                     // There's supposed to be a check in GryoMapper that prevents this from happening
                     log.error("GryoMapper's default serialization registration for {} is a {}. " +
-                              "This is probably a bug in TinkerPop (this is not a valid default registration). " +
-                              "I am configuring Spark to use Kryo's default serializer for this class, " +
-                              "but this may cause serialization failures at runtime.",
-                              tr.getTargetClass(),
-                              org.apache.tinkerpop.shaded.kryo.Serializer.class.getCanonicalName());
+                                    "This is probably a bug in TinkerPop (this is not a valid default registration). " +
+                                    "I am configuring Spark to use Kryo's default serializer for this class, " +
+                                    "but this may cause serialization failures at runtime.",
+                            tr.getTargetClass(),
+                            org.apache.tinkerpop.shaded.kryo.Serializer.class.getCanonicalName());
                     kryo.register(tr.getTargetClass());
                 }
             } else if (null != serializerShim) {
@@ -127,12 +152,12 @@ public class GryoRegistrator implements KryoRegistrator {
             } else if (null != functionOfShadedKryo) {
                 // As with shaded serializers, there's supposed to be a check in GryoMapper that prevents this from happening
                 log.error("GryoMapper's default serialization registration for {} is a Function<{},{}>.  " +
-                          "This is probably a bug in TinkerPop (this is not a valid default registration). " +
-                          "I am configuring Spark to use Kryo's default serializer instead of this function, " +
-                          "but this may cause serialization failures at runtime.",
-                          tr.getTargetClass(),
-                          org.apache.tinkerpop.shaded.kryo.Kryo.class.getCanonicalName(),
-                          org.apache.tinkerpop.shaded.kryo.Serializer.class.getCanonicalName());
+                                "This is probably a bug in TinkerPop (this is not a valid default registration). " +
+                                "I am configuring Spark to use Kryo's default serializer instead of this function, " +
+                                "but this may cause serialization failures at runtime.",
+                        tr.getTargetClass(),
+                        org.apache.tinkerpop.shaded.kryo.Kryo.class.getCanonicalName(),
+                        org.apache.tinkerpop.shaded.kryo.Serializer.class.getCanonicalName());
                 kryo.register(tr.getTargetClass());
             } else {
                 // Register all other classes with the default behavior (FieldSerializer)
@@ -164,13 +189,46 @@ public class GryoRegistrator implements KryoRegistrator {
         // duplication, but it would be a bit cumbersome to do so without disturbing
         // the ordering of the existing entries in that constructor, since not all
         // of the entries are for TinkerPop (and the ordering is significant).
+        if (Boolean.valueOf(System.getProperty("is.testing", "false"))) {
+            try {
+                m.put(Class.forName("scala.reflect.ClassTag$$anon$1"), new JavaSerializer());
+                m.put(Class.forName("scala.reflect.ManifestFactory$$anon$1"), new JavaSerializer());
+            } catch (final ClassNotFoundException e) {
+                throw new IllegalStateException(e.getMessage(), e);
+            }
+        }
+        m.put(WrappedArray.ofRef.class, null);
         m.put(MessagePayload.class, null);
         m.put(ViewIncomingPayload.class, null);
         m.put(ViewOutgoingPayload.class, null);
         m.put(ViewPayload.class, null);
         m.put(VertexWritable.class, new UnshadedSerializerAdapter<>(new VertexWritableSerializer()));
         m.put(ObjectWritable.class, new UnshadedSerializerAdapter<>(new ObjectWritableSerializer<>()));
-
+        //
+        m.put(HadoopConfiguration.class, null);
+        //
+        m.put(HadoopVertex.class, new UnshadedSerializerAdapter<>(new GryoSerializers.VertexSerializer()));
+        m.put(HadoopVertexProperty.class, new UnshadedSerializerAdapter<>(new GryoSerializers.VertexPropertySerializer()));
+        m.put(HadoopProperty.class, new UnshadedSerializerAdapter<>(new GryoSerializers.PropertySerializer()));
+        m.put(HadoopEdge.class, new UnshadedSerializerAdapter<>(new GryoSerializers.EdgeSerializer()));
+        //
+        m.put(ComputerGraph.ComputerVertex.class, new UnshadedSerializerAdapter<>(new GryoSerializers.VertexSerializer()));
+        m.put(ComputerGraph.ComputerVertexProperty.class, new UnshadedSerializerAdapter<>(new GryoSerializers.VertexPropertySerializer()));
+        m.put(ComputerGraph.ComputerProperty.class, new UnshadedSerializerAdapter<>(new GryoSerializers.PropertySerializer()));
+        m.put(ComputerGraph.ComputerEdge.class, new UnshadedSerializerAdapter<>(new GryoSerializers.EdgeSerializer()));
+        //
+        m.put(StarGraph.StarEdge.class, new UnshadedSerializerAdapter<>(new GryoSerializers.EdgeSerializer()));
+        m.put(StarGraph.StarVertex.class, new UnshadedSerializerAdapter<>(new GryoSerializers.VertexSerializer()));
+        m.put(StarGraph.StarProperty.class, new UnshadedSerializerAdapter<>(new GryoSerializers.PropertySerializer()));
+        m.put(StarGraph.StarVertexProperty.class, new UnshadedSerializerAdapter<>(new GryoSerializers.VertexPropertySerializer()));
+        //
+        m.put(MutablePath.class, new UnshadedSerializerAdapter<>(new GryoSerializers.PathSerializer()));
+        m.put(ImmutablePath.class, new UnshadedSerializerAdapter<>(new GryoSerializers.PathSerializer()));
+        try {
+            m.put(Class.forName(ImmutablePath.class.getCanonicalName() + "$TailPath"), new UnshadedSerializerAdapter<>(new GryoSerializers.PathSerializer()));
+        } catch (final ClassNotFoundException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
         return m;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/e7003635/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 85552ce..7737d1e 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
@@ -26,6 +26,7 @@ import org.apache.tinkerpop.gremlin.hadoop.Constants;
 import org.apache.tinkerpop.gremlin.hadoop.HadoopGraphProvider;
 import org.apache.tinkerpop.gremlin.hadoop.groovy.plugin.HadoopGremlinPluginCheck;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.FileSystemStorageCheck;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.HadoopPoolShimService;
 import org.apache.tinkerpop.gremlin.process.computer.Computer;
 import org.apache.tinkerpop.gremlin.process.computer.GraphComputer;
 import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversalSource;
@@ -40,10 +41,14 @@ import org.apache.tinkerpop.gremlin.spark.structure.io.SparkContextStorageCheck;
 import org.apache.tinkerpop.gremlin.spark.structure.io.ToyGraphInputRDD;
 import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.GryoRegistrator;
 import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.GryoSerializer;
+import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.kryoshim.unshaded.UnshadedKryoShimService;
 import org.apache.tinkerpop.gremlin.structure.Graph;
+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)
  */
@@ -83,9 +88,13 @@ public final class SparkHadoopGraphProvider extends HadoopGraphProvider {
 
 
         config.put("spark.master", "local[4]");
-        if (false) {
+        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());
         }


[08/34] incubator-tinkerpop git commit: TINKERPOP-1301 Provide Javadoc for ScriptInput/OutputFormat's ported to tp31 branch

Posted by ok...@apache.org.
TINKERPOP-1301 Provide Javadoc for ScriptInput/OutputFormat's ported to tp31 branch


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

Branch: refs/heads/TINKERPOP-1278
Commit: d5d2c48ef67e31671cb177d30c6a15c84b585d24
Parents: 1275437
Author: Lewis John McGibbney <le...@gmail.com>
Authored: Fri Jun 3 09:00:08 2016 -0700
Committer: Lewis John McGibbney <le...@gmail.com>
Committed: Fri Jun 3 09:00:08 2016 -0700

----------------------------------------------------------------------
 .../gremlin/hadoop/structure/io/script/ScriptInputFormat.java  | 5 +++++
 .../gremlin/hadoop/structure/io/script/ScriptOutputFormat.java | 6 ++++++
 2 files changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/d5d2c48e/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/script/ScriptInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/script/ScriptInputFormat.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/script/ScriptInputFormat.java
index f6d6c4c..b7c6876 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/script/ScriptInputFormat.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/script/ScriptInputFormat.java
@@ -32,6 +32,11 @@ import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
 import java.io.IOException;
 
 /**
+ * ScriptInputFormat and {@link org.apache.tinkerpop.gremlin.hadoop.structure.io.script.ScriptOutputFormat}
+ * take an arbitrary script and use that script to either read or write Vertex objects,
+ * respectively. This can be considered the most general InputFormat/OutputFormat
+ * possible in that Hadoop-Gremlin uses the user provided script for all reading/writing.
+ * @see <a href="http://tinkerpop.apache.org/docs/current/reference/#script-io-format">Script I/O Format Reference Documentation</a>
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  * @author Daniel Kuppitz (http://gremlin.guru)
  */

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/d5d2c48e/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/script/ScriptOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/script/ScriptOutputFormat.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/script/ScriptOutputFormat.java
index b57e43e..85dc106 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/script/ScriptOutputFormat.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/script/ScriptOutputFormat.java
@@ -29,6 +29,12 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 
 /**
+ * {@link org.apache.tinkerpop.gremlin.hadoop.structure.io.script.ScriptInputFormat}
+ * and ScriptOutputFormat take an arbitrary script and use that script to either
+ * read or write Vertex objects, respectively. This can be considered the most
+ * general InputFormat/OutputFormat possible in that Hadoop-Gremlin uses the user
+ * provided script for all reading/writing.
+ * @see <a href="http://tinkerpop.apache.org/docs/current/reference/#script-io-format">Script I/O Format Reference Documentation</a>
  * @author Daniel Kuppitz (http://gremlin.guru)
  */
 public final class ScriptOutputFormat extends CommonFileOutputFormat implements HadoopPoolsConfigurable {


[18/34] incubator-tinkerpop git commit: Merge remote-tracking branch 'origin/TINKERPOP-1322'

Posted by ok...@apache.org.
Merge remote-tracking branch 'origin/TINKERPOP-1322'


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

Branch: refs/heads/TINKERPOP-1278
Commit: 092dcb991a2cdbfb310c5597e0942d9500402c31
Parents: dac7f11 e32347c
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Mon Jun 6 06:25:13 2016 -0400
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Mon Jun 6 06:25:13 2016 -0400

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |  1 +
 .../src/reference/gremlin-applications.asciidoc |  5 +-
 .../gremlin/groovy/engine/ScriptEngines.java    | 28 ++++++-
 .../jsr223/GremlinGroovyScriptEngine.java       |  8 +-
 .../ConfigurationCustomizerProvider.java        | 83 ++++++++++++++++++++
 .../groovy/jsr223/BaseScriptForTesting.java     | 30 +++++++
 .../GremlinGroovyScriptEngineConfigTest.java    | 40 ++++++++++
 .../ConfigurationCustomizerProviderTest.java    | 78 ++++++++++++++++++
 .../gremlin/server/BaseScriptForTesting.java    | 30 +++++++
 .../server/GremlinServerIntegrateTest.java      | 50 +++++++++---
 10 files changed, 338 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/092dcb99/CHANGELOG.asciidoc
----------------------------------------------------------------------


[29/34] 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.

Posted by ok...@apache.org.
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-1278
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;
     }


[10/34] incubator-tinkerpop git commit: Merge branch 'tp31'

Posted by ok...@apache.org.
Merge branch 'tp31'


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

Branch: refs/heads/TINKERPOP-1278
Commit: f24e35851e6b5122b32650d559b295086a5cf399
Parents: cb4fae1 0445e6d
Author: Daniel Kuppitz <da...@hotmail.com>
Authored: Sat Jun 4 01:07:27 2016 +0200
Committer: Daniel Kuppitz <da...@hotmail.com>
Committed: Sat Jun 4 01:07:27 2016 +0200

----------------------------------------------------------------------
 .../gremlin/hadoop/structure/io/script/ScriptInputFormat.java  | 5 +++++
 .../gremlin/hadoop/structure/io/script/ScriptOutputFormat.java | 6 ++++++
 2 files changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f24e3585/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/script/ScriptInputFormat.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f24e3585/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/script/ScriptOutputFormat.java
----------------------------------------------------------------------


[21/34] incubator-tinkerpop git commit: Merge branch 'serialization-flexibility' of https://github.com/dalaro/incubator-tinkerpop into TINKERPOP-1321-dan

Posted by ok...@apache.org.
Merge branch 'serialization-flexibility' of https://github.com/dalaro/incubator-tinkerpop into TINKERPOP-1321-dan


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

Branch: refs/heads/TINKERPOP-1278
Commit: 0cd31bf8e86a26ac92f232fdcb6a29eda9d6dfd9
Parents: 55eb1e6 88ac304
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Mon Jun 6 13:36:54 2016 -0600
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Mon Jun 6 13:36:54 2016 -0600

----------------------------------------------------------------------
 .../process/computer/GiraphWorkerContext.java   |   3 +-
 .../gremlin/structure/io/gryo/GryoMapper.java   | 440 ++++++++++++-------
 .../gremlin/structure/io/gryo/GryoPool.java     |   1 +
 .../structure/io/gryo/GryoSerializers.java      |  40 +-
 .../structure/io/gryo/JavaTimeSerializers.java  | 125 ++----
 .../structure/io/gryo/PairSerializer.java       |  11 +-
 .../structure/io/gryo/TypeRegistration.java     |  82 ++++
 .../structure/io/gryo/URISerializer.java        |  23 +-
 .../structure/io/gryo/UUIDSerializer.java       |  24 +-
 .../structure/io/gryo/kryoshim/InputShim.java   |  38 ++
 .../structure/io/gryo/kryoshim/KryoShim.java    |  41 ++
 .../io/gryo/kryoshim/KryoShimService.java       |  99 +++++
 .../io/gryo/kryoshim/KryoShimServiceLoader.java | 214 +++++++++
 .../structure/io/gryo/kryoshim/OutputShim.java  |  40 ++
 .../io/gryo/kryoshim/SerializerShim.java        |  36 ++
 .../io/gryo/kryoshim/package-info.java          |  55 +++
 .../kryoshim/shaded/ShadedInputAdapter.java     |  66 +++
 .../gryo/kryoshim/shaded/ShadedKryoAdapter.java |  67 +++
 .../kryoshim/shaded/ShadedOutputAdapter.java    |  72 +++
 .../shaded/ShadedSerializerAdapter.java         |  54 +++
 .../io/gryo/kryoshim/shaded/package-info.java   |  25 ++
 .../util/star/StarGraphGryoSerializer.java      | 130 +-----
 .../util/star/StarGraphSerializer.java          | 150 +++++++
 .../hadoop/process/computer/HadoopCombine.java  |   3 +-
 .../hadoop/process/computer/HadoopMap.java      |   3 +-
 .../hadoop/process/computer/HadoopReduce.java   |   3 +-
 .../structure/io/HadoopPoolShimService.java     |  76 ++++
 .../hadoop/structure/io/HadoopPools.java        |   1 +
 .../structure/io/HadoopPoolsConfigurable.java   |   4 +-
 .../hadoop/structure/io/ObjectWritable.java     |  25 +-
 .../hadoop/structure/io/VertexWritable.java     |  39 +-
 .../structure/io/gryo/GryoRecordReader.java     |   3 +-
 .../structure/io/gryo/GryoRecordWriter.java     |   4 +-
 ...n.structure.io.gryo.kryoshim.KryoShimService |   1 +
 .../spark/process/computer/SparkExecutor.java   |   3 +-
 .../structure/io/gryo/GryoRegistrator.java      | 194 ++++++++
 .../spark/structure/io/gryo/GryoSerializer.java |   2 +-
 .../io/gryo/IoRegistryAwareKryoSerializer.java  | 116 +++++
 .../io/gryo/ObjectWritableSerializer.java       |  12 +-
 .../io/gryo/VertexWritableSerializer.java       |  12 +-
 .../kryoshim/unshaded/UnshadedInputAdapter.java |  78 ++++
 .../kryoshim/unshaded/UnshadedKryoAdapter.java  |  74 ++++
 .../unshaded/UnshadedKryoShimService.java       | 154 +++++++
 .../unshaded/UnshadedOutputAdapter.java         |  83 ++++
 .../unshaded/UnshadedSerializerAdapter.java     |  57 +++
 ...n.structure.io.gryo.kryoshim.KryoShimService |   1 +
 .../computer/SparkHadoopGraphProvider.java      |  11 +-
 .../spark/structure/io/ToyGraphInputRDD.java    |   3 +-
 48 files changed, 2328 insertions(+), 470 deletions(-)
----------------------------------------------------------------------



[19/34] incubator-tinkerpop git commit: This closes #314

Posted by ok...@apache.org.
This closes #314


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

Branch: refs/heads/TINKERPOP-1278
Commit: 55eb1e6428fecdf6ee80e9a27a603e867d015507
Parents: 092dcb9
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Mon Jun 6 06:32:41 2016 -0400
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Mon Jun 6 06:32:41 2016 -0400

----------------------------------------------------------------------

----------------------------------------------------------------------



[15/34] incubator-tinkerpop git commit: Kryo shim configuration tweaks

Posted by ok...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ToyGraphInputRDD.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ToyGraphInputRDD.java b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ToyGraphInputRDD.java
index ea3636f..4cd8cea 100644
--- a/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ToyGraphInputRDD.java
+++ b/spark-gremlin/src/test/java/org/apache/tinkerpop/gremlin/spark/structure/io/ToyGraphInputRDD.java
@@ -30,6 +30,7 @@ import org.apache.tinkerpop.gremlin.structure.io.GraphReader;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoIo;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoReader;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoResourceAccess;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader;
 import org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerFactory;
 import org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerGraph;
 import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
@@ -46,7 +47,7 @@ public final class ToyGraphInputRDD implements InputRDD {
 
     @Override
     public JavaPairRDD<Object, VertexWritable> readGraphRDD(final Configuration configuration, final JavaSparkContext sparkContext) {
-        HadoopPools.initialize(TinkerGraph.open().configuration());
+        KryoShimServiceLoader.applyConfiguration(TinkerGraph.open().configuration());
         final List<VertexWritable> vertices;
         if (configuration.getString(Constants.GREMLIN_HADOOP_INPUT_LOCATION).contains("modern"))
             vertices = IteratorUtils.list(IteratorUtils.map(TinkerFactory.createModern().vertices(), VertexWritable::new));


[32/34] incubator-tinkerpop git commit: removed GraphFilter from GryoMapper. It was added as another way of solving a problem, but that solution wasn't used so removing it. Also, realized a cleaner way of doing the KryoShim tests in Spark -- minor config

Posted by ok...@apache.org.
removed GraphFilter from GryoMapper. It was added as another way of solving a problem, but that solution wasn't used so removing it. Also, realized a cleaner way of doing the KryoShim tests in Spark -- minor configuration changes.


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

Branch: refs/heads/TINKERPOP-1278
Commit: 2c7b1f92ba0a591ef3f25f75447b37a0edb60fd9
Parents: 4bb9f36
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Tue Jun 7 07:14:40 2016 -0600
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Tue Jun 7 07:14:40 2016 -0600

----------------------------------------------------------------------
 .../gremlin/structure/io/gryo/GryoMapper.java     |  4 +---
 .../gremlin/spark/AbstractSparkTest.java          |  1 -
 .../SparkHadoopGraphGryoRegistratorProvider.java  | 18 +++++++-----------
 .../computer/SparkHadoopGraphProvider.java        |  1 -
 4 files changed, 8 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/2c7b1f92/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
index 577c1ce..5b4a3a8 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
@@ -330,8 +330,6 @@ public final class GryoMapper implements Mapper<Kryo> {
             add(GryoTypeReg.of(AtomicLong.class, 79));
             add(GryoTypeReg.of(Pair.class, 88, new PairSerializer()));
             add(GryoTypeReg.of(TraversalExplanation.class, 106, new JavaSerializer()));
-            add(GryoTypeReg.of(GraphFilter.class, 120, new JavaSerializer())); // ***LAST ID***
-            //add(GryoTypeReg.of(PureTraversal.class, 121, new JavaSerializer()));
 
             add(GryoTypeReg.of(Duration.class, 93, new JavaTimeSerializers.DurationSerializer()));
             add(GryoTypeReg.of(Instant.class, 94, new JavaTimeSerializers.InstantSerializer()));
@@ -357,7 +355,7 @@ public final class GryoMapper implements Mapper<Kryo> {
             add(GryoTypeReg.of(GroupStepV3d0.GroupBiOperatorV3d0.class, 113));
             add(GryoTypeReg.of(RangeGlobalStep.RangeBiOperator.class, 114));
             add(GryoTypeReg.of(OrderGlobalStep.OrderBiOperator.class, 118, new JavaSerializer())); // because they contain traversals
-            add(GryoTypeReg.of(ProfileStep.ProfileBiOperator.class, 119));
+            add(GryoTypeReg.of(ProfileStep.ProfileBiOperator.class, 119)); // ***LAST ID***
         }};
 
         private final List<IoRegistry> registries = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/2c7b1f92/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 4d584ff..473976b 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
@@ -46,7 +46,6 @@ 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/2c7b1f92/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
index df36a13..c1832cc 100644
--- 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
@@ -36,21 +36,17 @@ import static org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimSe
  */
 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) {
+        Spark.close();
         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
+        // ensure the context doesn't stay open for the GryoSerializer tests to follow
+        // this is primarily to ensure that the KryoShimService loaded specifically in these tests don't leak to the other tests
+        config.put(Constants.GREMLIN_SPARK_PERSIST_CONTEXT, false);
         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);
-        }
-        //
+        System.setProperty(SHIM_CLASS_SYSTEM_PROPERTY, UnshadedKryoShimService.class.getCanonicalName());
+        KryoShimServiceLoader.load(true);
+        System.clearProperty(SHIM_CLASS_SYSTEM_PROPERTY);
         return config;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/2c7b1f92/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 1fc29cf..06119fc 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
@@ -88,7 +88,6 @@ public 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]");
         config.put("spark.serializer", GryoSerializer.class.getCanonicalName());


[11/34] incubator-tinkerpop git commit: The timeout function to the GremlinExecutor was not executing in the same thread as the script

Posted by ok...@apache.org.
The timeout function to the GremlinExecutor was not executing in the same thread as the script

This would mean that if someone were to override the timeout and they were expecting to rollback a transaction or perform some other action that needed to occur in the same thread as the script they would not get the expected behavior. CTR


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

Branch: refs/heads/TINKERPOP-1278
Commit: 2f4c24c71ebf4a5e455ee96ad78ca6b5e7d1306a
Parents: 0445e6d
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Fri Jun 3 20:43:02 2016 -0400
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Fri Jun 3 20:43:02 2016 -0400

----------------------------------------------------------------------
 CHANGELOG.asciidoc                                      |  1 +
 .../gremlin/groovy/engine/GremlinExecutor.java          | 12 +++++-------
 2 files changed, 6 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/2f4c24c7/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index ebe3efd..601022a 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -26,6 +26,7 @@ image::https://raw.githubusercontent.com/apache/incubator-tinkerpop/master/docs/
 TinkerPop 3.1.3 (NOT OFFICIALLY RELEASED YET)
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
+* Fixed a bug where timeout functions provided to the `GremlinExecutor` were not executing in the same thread as the script evaluation.
 * Optimized a few special cases in `RangeByIsCountStrategy`.
 * Named the thread pool used by Gremlin Server sessions: "gremlin-server-session-$n".
 * Fixed a bug in `BulkSet.equals()` which made itself apparent when using `store()` and `aggregate()` with labeled `cap()`.

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/2f4c24c7/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/engine/GremlinExecutor.java
----------------------------------------------------------------------
diff --git a/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/engine/GremlinExecutor.java b/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/engine/GremlinExecutor.java
index 5ef8667..2476114 100644
--- a/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/engine/GremlinExecutor.java
+++ b/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/engine/GremlinExecutor.java
@@ -249,7 +249,7 @@ public class GremlinExecutor implements AutoCloseable {
         bindings.putAll(boundVars);
 
         // override the timeout if the lifecycle has a value assigned
-        final long seto = lifeCycle.scriptEvaluationTimeoutOverride.orElse(scriptEvaluationTimeout);
+        final long seto = lifeCycle.getScriptEvaluationTimeoutOverride().orElse(scriptEvaluationTimeout);
 
         final CompletableFuture<Object> evaluationFuture = new CompletableFuture<>();
         final FutureTask<Void> f = new FutureTask<>(() -> {
@@ -283,10 +283,11 @@ public class GremlinExecutor implements AutoCloseable {
 
                 // thread interruptions will typically come as the result of a timeout, so in those cases,
                 // check for that situation and convert to TimeoutException
-                if (root instanceof InterruptedException)
+                if (root instanceof InterruptedException) {
+                    lifeCycle.getAfterTimeout().orElse(afterTimeout).accept(bindings);
                     evaluationFuture.completeExceptionally(new TimeoutException(
                             String.format("Script evaluation exceeded the configured 'scriptEvaluationTimeout' threshold of %s ms for request [%s]: %s", seto, script, root.getMessage())));
-                else {
+                } else {
                     lifeCycle.getAfterFailure().orElse(afterFailure).accept(bindings, root);
                     evaluationFuture.completeExceptionally(root);
                 }
@@ -301,10 +302,7 @@ public class GremlinExecutor implements AutoCloseable {
             // Schedule a timeout in the thread pool for future execution
             final ScheduledFuture<?> sf = scheduledExecutorService.schedule(() -> {
                 logger.warn("Timing out script - {} - in thread [{}]", script, Thread.currentThread().getName());
-                if (!f.isDone()) {
-                    lifeCycle.getAfterTimeout().orElse(afterTimeout).accept(bindings);
-                    f.cancel(true);
-                }
+                if (!f.isDone()) f.cancel(true);
             }, seto, TimeUnit.MILLISECONDS);
 
             // Cancel the scheduled timeout if the eval future is complete or the script evaluation failed


[03/34] incubator-tinkerpop git commit: Added ConfigurationCustomizerCompiler

Posted by ok...@apache.org.
Added ConfigurationCustomizerCompiler

This CustomizerCompiler implementation makes it possible to set low-level configurations in the GremlinGroovyScriptEngine on the Groovy CompilerConfiguration which ultimately controls script compilation settings used by the classloader.


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

Branch: refs/heads/TINKERPOP-1278
Commit: e32347c13e6c4917ebc691aa70e8531cb6b36e1a
Parents: ddddc5f
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Thu Jun 2 16:39:51 2016 -0400
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Jun 2 16:39:51 2016 -0400

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |  1 +
 .../src/reference/gremlin-applications.asciidoc |  5 +-
 .../gremlin/groovy/engine/ScriptEngines.java    | 28 ++++++-
 .../jsr223/GremlinGroovyScriptEngine.java       |  8 +-
 .../ConfigurationCustomizerProvider.java        | 83 ++++++++++++++++++++
 .../groovy/jsr223/BaseScriptForTesting.java     | 30 +++++++
 .../GremlinGroovyScriptEngineConfigTest.java    | 40 ++++++++++
 .../ConfigurationCustomizerProviderTest.java    | 78 ++++++++++++++++++
 .../gremlin/server/BaseScriptForTesting.java    | 30 +++++++
 .../server/GremlinServerIntegrateTest.java      | 50 +++++++++---
 10 files changed, 338 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/e32347c1/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index 9590a90..6b53bb2 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -34,6 +34,7 @@ TinkerPop 3.2.1 (NOT OFFICIALLY RELEASED YET)
 * Added `EmptyMemory` for ease of use when no memory exists.
 * Updated `VertexComputing.generateProgram()` API to include `Memory`. (*breaking*)
 * `ImmutablePath.TailPath` is now serializable like `ImmutablePath`.
+* Added `ConfigurationCompilerProvider` which allows fine-grained control of some of the internal `GremlinGroovyScriptEngine` settings at the Groovy compilation level.
 * Intoduced the `application/vnd.gremlin-v1.0+gryo-lite` serialization type to Gremlin Server which users "reference" elements rather than "detached".
 * `GryoMapper` allows overrides of existing serializers on calls to `addCustom` on the builder.
 * Added a traversal style guide to the recipes cookbook.

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/e32347c1/docs/src/reference/gremlin-applications.asciidoc
----------------------------------------------------------------------
diff --git a/docs/src/reference/gremlin-applications.asciidoc b/docs/src/reference/gremlin-applications.asciidoc
index a8070ba..8c987fe 100644
--- a/docs/src/reference/gremlin-applications.asciidoc
+++ b/docs/src/reference/gremlin-applications.asciidoc
@@ -979,8 +979,8 @@ run Gremlin Server with Ganglia monitoring, download the `org.acplt:oncrpc` jar
 link:http://repo1.maven.org/maven2/org/acplt/oncrpc/1.0.7/[here] and copy it to the Gremlin Server `/lib` directory
 before starting the server.
 
-Security
-^^^^^^^^
+Security and Execution
+^^^^^^^^^^^^^^^^^^^^^^
 
 image:gremlin-server-secure.png[width=175,float=right] Gremlin Server provides for several features that aid in the
 security of the graphs that it exposes.  It has built in SSL support and a pluggable authentication framework using
@@ -1151,6 +1151,7 @@ There are a number of pre-packaged `CustomizerProvider` implementations:
 |=========================================================
 |Customizer |Description
 |`CompileStaticCustomizerProvider` |Applies `CompileStatic` annotations to incoming scripts thus removing dynamic dispatch. More information about static compilation can be found in the link:http://docs.groovy-lang.org/latest/html/documentation/#_static_compilation[Groovy Documentation].  It is possible to configure this `CustomizerProvider` by specifying a comma separated list of link:http://docs.groovy-lang.org/latest/html/documentation/#Typecheckingextensions-Workingwithextensions[type checking extensions] that can have the effect of securing calls to various methods.
+|`ConfigurationCustomizerProvider` |Allows configuration of the the Groovy `CompilerConfiguration` object by taking a `Map` of key/value pairs where the "key" is a property to set on the `CompilerConfiguration`.
 |`ThreadInterruptCustomizerProvider` |Injects checks for thread interruption, thus allowing the thread to potentially respect calls to `Thread.interrupt()`
 |`TimedInterruptCustomizerProvider` |Injects checks into loops to interrupt them if they exceed the configured timeout in milliseconds.
 |`TypeCheckedCustomizerProvider` |Similar to the above mentioned, `CompileStaticCustomizerProvider`, the `TypeCheckedCustomizerProvider` injects `TypeChecked` annotations to incoming scripts.  More information on the nature of this annotation can be found in the link:http://docs.groovy-lang.org/latest/html/documentation/#_the_code_typechecked_code_annotation[Groovy Documentation].  It too takes a comma separated list of link:http://docs.groovy-lang.org/latest/html/documentation/#Typecheckingextensions-Workingwithextensions[type checking extensions].

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/e32347c1/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/engine/ScriptEngines.java
----------------------------------------------------------------------
diff --git a/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/engine/ScriptEngines.java b/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/engine/ScriptEngines.java
index 1113eb5..3d54bea 100644
--- a/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/engine/ScriptEngines.java
+++ b/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/engine/ScriptEngines.java
@@ -23,6 +23,7 @@ import org.apache.tinkerpop.gremlin.groovy.DefaultImportCustomizerProvider;
 import org.apache.tinkerpop.gremlin.groovy.jsr223.DependencyManager;
 import org.apache.tinkerpop.gremlin.groovy.jsr223.GremlinGroovyScriptEngine;
 import org.apache.tinkerpop.gremlin.groovy.jsr223.GremlinGroovyScriptEngineFactory;
+import org.apache.tinkerpop.gremlin.groovy.jsr223.customizer.ConfigurationCustomizerProvider;
 import org.apache.tinkerpop.gremlin.groovy.plugin.GremlinPlugin;
 import org.apache.tinkerpop.gremlin.groovy.plugin.IllegalEnvironmentException;
 import org.slf4j.Logger;
@@ -382,8 +383,17 @@ public class ScriptEngines implements AutoCloseable {
 
                         final Class<?>[] argClasses = new Class<?>[args.length];
                         Stream.of(args).map(a -> a.getClass()).collect(Collectors.toList()).toArray(argClasses);
-                        final Constructor constructor = providerClass.getConstructor(argClasses);
-                        providers.add((CompilerCustomizerProvider) constructor.newInstance(args));
+
+                        final Optional<Constructor> constructor = Stream.of(providerClass.getConstructors())
+                                .filter(c -> c.getParameterCount() == argClasses.length &&
+                                             allMatch(c.getParameterTypes(), argClasses))
+                                .findFirst();
+
+                        if (constructor.isPresent()) providers.add((CompilerCustomizerProvider)
+                                constructor.get().newInstance(args));
+                        else
+                            throw new IllegalStateException(String.format("Could not configure %s with the supplied options %s",
+                                    ConfigurationCustomizerProvider.class.getName(), Arrays.asList(args)));
                     } else {
                         providers.add((CompilerCustomizerProvider) providerClass.newInstance());
                     }
@@ -400,6 +410,20 @@ public class ScriptEngines implements AutoCloseable {
     }
 
     /**
+     * Determine if the constructor argument types match the arg types that are going to be passed in to that
+     * constructor.
+     */
+    private static boolean allMatch(final Class<?>[] constructorArgTypes, final Class<?>[] argTypes) {
+        for (int ix = 0; ix < constructorArgTypes.length; ix++) {
+            if (!constructorArgTypes[ix].isAssignableFrom(argTypes[ix])) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    /**
      * Takes the bindings from a request for eval and merges them with the {@code ENGINE_SCOPE} bindings.
      */
     private static Bindings mergeBindings(final Bindings bindings, final ScriptEngine engine) {

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/e32347c1/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 23240cb..0069103 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.ConfigurationCustomizerProvider;
 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;
@@ -610,7 +611,12 @@ public class GremlinGroovyScriptEngine extends GroovyScriptEngineImpl implements
         final CompilerConfiguration conf = new CompilerConfiguration();
         conf.addCompilationCustomizers(this.importCustomizerProvider.create());
 
-        customizerProviders.forEach(p -> conf.addCompilationCustomizers(p.create()));
+        // ConfigurationCustomizerProvider is treated separately
+        customizerProviders.stream().filter(cp -> !(cp instanceof ConfigurationCustomizerProvider))
+                .forEach(p -> conf.addCompilationCustomizers(p.create()));
+
+        customizerProviders.stream().filter(cp -> cp instanceof ConfigurationCustomizerProvider).findFirst()
+                .ifPresent(cp -> ((ConfigurationCustomizerProvider) cp).applyCustomization(conf));
 
         this.loader = new GremlinGroovyClassLoader(getParentLoader(), conf);
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/e32347c1/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/jsr223/customizer/ConfigurationCustomizerProvider.java
----------------------------------------------------------------------
diff --git a/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/jsr223/customizer/ConfigurationCustomizerProvider.java b/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/jsr223/customizer/ConfigurationCustomizerProvider.java
new file mode 100644
index 0000000..ba9855c
--- /dev/null
+++ b/gremlin-groovy/src/main/java/org/apache/tinkerpop/gremlin/groovy/jsr223/customizer/ConfigurationCustomizerProvider.java
@@ -0,0 +1,83 @@
+/*
+ * 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.structure.util.ElementHelper;
+import org.codehaus.groovy.control.CompilerConfiguration;
+import org.codehaus.groovy.control.customizers.CompilationCustomizer;
+
+import java.lang.reflect.Method;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Allows configurations to be directly supplied to a groovy {@code CompilerConfiguration} when a
+ * {@link org.apache.tinkerpop.gremlin.groovy.jsr223.GremlinGroovyScriptEngine} is initialized, providing fine-grained
+ * control over its internals.
+ *
+ * @author Stephen Mallette (http://stephen.genoprime.com)
+ */
+public class ConfigurationCustomizerProvider implements CompilerCustomizerProvider {
+
+    private final Map<String,Object> properties;
+
+    /**
+     * Creates a new instance using configuration values specified
+     */
+    public ConfigurationCustomizerProvider(final Object... keyValues) {
+        if (null == keyValues || keyValues.length == 0)
+            throw new IllegalArgumentException("ConfigurationCustomizerProvider must have key/values specified");
+
+        if (keyValues.length % 2 != 0)
+            throw new IllegalArgumentException("The keyValues must have an even number of values");
+
+        properties = ElementHelper.asMap(keyValues);
+    }
+
+    /**
+     * Creates a new instance using configuration values specified
+     */
+    public ConfigurationCustomizerProvider(final Map<String,Object> keyValues) {
+        properties = keyValues;
+    }
+
+    public CompilerConfiguration applyCustomization(final CompilerConfiguration compilerConfiguration) {
+        final Class<CompilerConfiguration> clazz = CompilerConfiguration.class;
+        final List<Method> methods = Arrays.asList(clazz.getMethods());
+        for (Map.Entry<String,Object> entry : properties.entrySet()) {
+            final Method method = methods.stream().filter(m -> m.getName().equals("set" + entry.getKey())).findFirst()
+                   .orElseThrow(() -> new IllegalStateException("Invalid setting [" + entry.getKey() + "] for CompilerConfiguration"));
+
+            try {
+                method.invoke(compilerConfiguration, entry.getValue());
+            } catch (Exception ex) {
+                throw new IllegalStateException(ex);
+            }
+        }
+
+        return compilerConfiguration;
+    }
+
+    @Override
+    public CompilationCustomizer create() {
+        throw new UnsupportedOperationException("This is a marker implementation that does not create a CompilationCustomizer instance");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/e32347c1/gremlin-groovy/src/test/java/org/apache/tinkerpop/gremlin/groovy/jsr223/BaseScriptForTesting.java
----------------------------------------------------------------------
diff --git a/gremlin-groovy/src/test/java/org/apache/tinkerpop/gremlin/groovy/jsr223/BaseScriptForTesting.java b/gremlin-groovy/src/test/java/org/apache/tinkerpop/gremlin/groovy/jsr223/BaseScriptForTesting.java
new file mode 100644
index 0000000..98c8e8c
--- /dev/null
+++ b/gremlin-groovy/src/test/java/org/apache/tinkerpop/gremlin/groovy/jsr223/BaseScriptForTesting.java
@@ -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;
+
+import groovy.lang.Script;
+
+/**
+ * @author Stephen Mallette (http://stephen.genoprime.com)
+ */
+public abstract class BaseScriptForTesting extends Script {
+    public String hello(final String name) {
+        return "hello, " + name;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/e32347c1/gremlin-groovy/src/test/java/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinGroovyScriptEngineConfigTest.java
----------------------------------------------------------------------
diff --git a/gremlin-groovy/src/test/java/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinGroovyScriptEngineConfigTest.java b/gremlin-groovy/src/test/java/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinGroovyScriptEngineConfigTest.java
new file mode 100644
index 0000000..d354ffa
--- /dev/null
+++ b/gremlin-groovy/src/test/java/org/apache/tinkerpop/gremlin/groovy/jsr223/GremlinGroovyScriptEngineConfigTest.java
@@ -0,0 +1,40 @@
+/*
+ * 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.apache.tinkerpop.gremlin.groovy.DefaultImportCustomizerProvider;
+import org.apache.tinkerpop.gremlin.groovy.jsr223.customizer.ConfigurationCustomizerProvider;
+import org.junit.Test;
+
+import javax.script.ScriptEngine;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * @author Stephen Mallette (http://stephen.genoprime.com)
+ */
+public class GremlinGroovyScriptEngineConfigTest {
+    @Test
+    public void shouldAddBaseScriptClass() throws Exception {
+        final ScriptEngine engine = new GremlinGroovyScriptEngine(
+                new ConfigurationCustomizerProvider("ScriptBaseClass", BaseScriptForTesting.class.getName()), new DefaultImportCustomizerProvider());
+
+        assertEquals("hello, stephen", engine.eval("hello('stephen')"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/e32347c1/gremlin-groovy/src/test/java/org/apache/tinkerpop/gremlin/groovy/jsr223/customizer/ConfigurationCustomizerProviderTest.java
----------------------------------------------------------------------
diff --git a/gremlin-groovy/src/test/java/org/apache/tinkerpop/gremlin/groovy/jsr223/customizer/ConfigurationCustomizerProviderTest.java b/gremlin-groovy/src/test/java/org/apache/tinkerpop/gremlin/groovy/jsr223/customizer/ConfigurationCustomizerProviderTest.java
new file mode 100644
index 0000000..54b55b1
--- /dev/null
+++ b/gremlin-groovy/src/test/java/org/apache/tinkerpop/gremlin/groovy/jsr223/customizer/ConfigurationCustomizerProviderTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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.commons.lang.exception.ExceptionUtils;
+import org.codehaus.groovy.control.CompilerConfiguration;
+import org.junit.Test;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.IsInstanceOf.instanceOf;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+/**
+ * @author Stephen Mallette (http://stephen.genoprime.com)
+ */
+public class ConfigurationCustomizerProviderTest {
+    @Test(expected = IllegalArgumentException.class)
+    public void shouldThrowExceptionForNoSettings() {
+        new ConfigurationCustomizerProvider();
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void shouldThrowExceptionForInvalidSettings() {
+        new ConfigurationCustomizerProvider("only-one-arg");
+    }
+
+    @Test
+    public void shouldThrowExceptionForNotFoundSetting() {
+        final CompilerConfiguration configuration = new CompilerConfiguration();
+        try {
+            final ConfigurationCustomizerProvider provider = new ConfigurationCustomizerProvider(
+                    "Tolerance", 3,
+                    "NotRealSettingThatWouldEverOccur2", new java.util.Date());
+
+            provider.applyCustomization(configuration);
+        } catch (Exception ex) {
+            assertThat(ex, instanceOf(IllegalStateException.class));
+            assertEquals("Invalid setting [NotRealSettingThatWouldEverOccur2] for CompilerConfiguration", ex.getMessage());
+        }
+    }
+
+    @Test
+    public void shouldApplyConfigurationChanges() {
+        final CompilerConfiguration configuration = new CompilerConfiguration();
+
+        assertEquals(10, configuration.getTolerance());
+        assertNull(configuration.getScriptBaseClass());
+        assertEquals(false, configuration.getDebug());
+
+        final ConfigurationCustomizerProvider provider = new ConfigurationCustomizerProvider(
+                "Tolerance", 3,
+                "ScriptBaseClass", "Something",
+                "Debug", true);
+
+        provider.applyCustomization(configuration);
+
+        assertEquals(3, configuration.getTolerance());
+        assertEquals("Something", configuration.getScriptBaseClass());
+        assertEquals(true, configuration.getDebug());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/e32347c1/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/BaseScriptForTesting.java
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/BaseScriptForTesting.java b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/BaseScriptForTesting.java
new file mode 100644
index 0000000..ae4c713
--- /dev/null
+++ b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/BaseScriptForTesting.java
@@ -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.server;
+
+import groovy.lang.Script;
+
+/**
+ * @author Stephen Mallette (http://stephen.genoprime.com)
+ */
+public abstract class BaseScriptForTesting extends Script {
+    public String hello(final String name) {
+        return "hello, " + name;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/e32347c1/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 16cbdee..a2b415d 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
@@ -41,6 +41,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.ConfigurationCustomizerProvider;
 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;
@@ -55,7 +56,6 @@ import org.junit.Before;
 import org.junit.Test;
 
 import java.nio.channels.ClosedChannelException;
-import java.security.cert.CertificateException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -72,12 +72,15 @@ import java.util.stream.IntStream;
 
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.core.IsInstanceOf.instanceOf;
 import static org.hamcrest.core.IsNot.not;
 import static org.hamcrest.core.StringEndsWith.endsWith;
 import static org.hamcrest.core.StringStartsWith.startsWith;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
 import static org.junit.Assume.assumeThat;
+import static org.junit.Assert.assertEquals;
 
 /**
  * Integration tests for server-side settings and processing.
@@ -161,6 +164,9 @@ public class GremlinServerIntegrateTest extends AbstractGremlinServerIntegration
             case "shouldReceiveFailureTimeOutOnScriptEvalOfOutOfControlLoop":
                 settings.scriptEngines.get("gremlin-groovy").config = getScriptEngineConfForTimedInterrupt();
                 break;
+            case "shouldUseBaseScript":
+                settings.scriptEngines.get("gremlin-groovy").config = getScriptEngineConfForBaseScript();
+                break;
         }
 
         return settings;
@@ -206,6 +212,30 @@ public class GremlinServerIntegrateTest extends AbstractGremlinServerIntegration
         return scriptEngineConf;
     }
 
+    private static Map<String, Object> getScriptEngineConfForBaseScript() {
+        final Map<String,Object> scriptEngineConf = new HashMap<>();
+        final Map<String,Object> compilerCustomizerProviderConf = new HashMap<>();
+        final List<Object> keyValues = new ArrayList<>();
+
+        final Map<String,Object> properties = new HashMap<>();
+        properties.put("ScriptBaseClass", BaseScriptForTesting.class.getName());
+        keyValues.add(properties);
+
+        compilerCustomizerProviderConf.put(ConfigurationCustomizerProvider.class.getName(), keyValues);
+        scriptEngineConf.put("compilerCustomizerProviders", compilerCustomizerProviderConf);
+        return scriptEngineConf;
+    }
+
+    @Test
+    public void shouldUseBaseScript() throws Exception {
+        final Cluster cluster = Cluster.open();
+        final Client client = cluster.connect(name.getMethodName());
+
+        assertEquals("hello, stephen", client.submit("hello('stephen')").all().get().get(0).getString());
+
+        cluster.close();
+    }
+
     @Test
     public void shouldUseInterpreterMode() throws Exception {
         final Cluster cluster = Cluster.open();
@@ -368,12 +398,12 @@ public class GremlinServerIntegrateTest extends AbstractGremlinServerIntegration
                 });
             });
 
-            assertTrue(latch.await(30000, TimeUnit.MILLISECONDS));
+            assertThat(latch.await(30000, TimeUnit.MILLISECONDS), is(true));
             assertEquals(0, latch.getCount());
-            assertFalse(faulty.get());
-            assertTrue(expected.get());
+            assertThat(faulty.get(), is(false));
+            assertThat(expected.get(), is(true));
 
-            assertTrue(recordingAppender.getMessages().stream().anyMatch(m -> m.contains("Pausing response writing as writeBufferHighWaterMark exceeded on")));
+            assertThat(recordingAppender.getMessages().stream().anyMatch(m -> m.contains("Pausing response writing as writeBufferHighWaterMark exceeded on")), is(true));
         } catch (Exception ex) {
             fail("Shouldn't have tossed an exception");
         } finally {
@@ -410,7 +440,7 @@ public class GremlinServerIntegrateTest extends AbstractGremlinServerIntegration
 
             if (!latch.await(3000, TimeUnit.MILLISECONDS))
                 fail("Request should have returned error, but instead timed out");
-            assertTrue(pass.get());
+            assertThat(pass.get(), is(true));
         }
     }
 
@@ -433,7 +463,7 @@ public class GremlinServerIntegrateTest extends AbstractGremlinServerIntegration
 
             if (!latch.await(3000, TimeUnit.MILLISECONDS))
                 fail("Request should have returned error, but instead timed out");
-            assertTrue(pass.get());
+            assertThat(pass.get(), is(true));
         }
     }
 
@@ -456,7 +486,7 @@ public class GremlinServerIntegrateTest extends AbstractGremlinServerIntegration
 
             if (!latch.await(3000, TimeUnit.MILLISECONDS))
                 fail("Request should have returned error, but instead timed out");
-            assertTrue(pass.get());
+            assertThat(pass.get(), is(true));
         }
     }
 
@@ -691,7 +721,7 @@ public class GremlinServerIntegrateTest extends AbstractGremlinServerIntegration
             client.submit("1+1").all().join();
             fail();
         } catch (RuntimeException re) {
-            assertTrue(re.getCause().getCause() instanceof ClosedChannelException);
+            assertThat(re.getCause().getCause() instanceof ClosedChannelException, is(true));
 
             //
             // should recover when the server comes back


[27/34] incubator-tinkerpop git commit: remove uneeded import.

Posted by ok...@apache.org.
remove uneeded import.


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

Branch: refs/heads/TINKERPOP-1278
Commit: 359f04bbc314d69d5b03f0f241c69f629dbac13a
Parents: 90e3159
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Mon Jun 6 16:57:16 2016 -0600
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Mon Jun 6 16:57:16 2016 -0600

----------------------------------------------------------------------
 .../gremlin/hadoop/structure/io/HadoopPoolShimService.java          | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/359f04bb/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
index df72b71..12b5d07 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
@@ -20,7 +20,6 @@ package org.apache.tinkerpop.gremlin.hadoop.structure.io;
 
 import org.apache.commons.configuration.Configuration;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimService;
-import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader;
 import org.apache.tinkerpop.shaded.kryo.Kryo;
 import org.apache.tinkerpop.shaded.kryo.io.Input;
 import org.apache.tinkerpop.shaded.kryo.io.Output;


[30/34] incubator-tinkerpop git commit: Use "final" where required to be consistent with rest of codebase.

Posted by ok...@apache.org.
Use "final" where required to be consistent with rest of codebase.


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

Branch: refs/heads/TINKERPOP-1278
Commit: 45abdcc35e9a80036ca669cc1d65ad16d09d2461
Parents: 293fe29
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Tue Jun 7 08:06:30 2016 -0400
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Tue Jun 7 08:06:30 2016 -0400

----------------------------------------------------------------------
 .../gremlin/structure/io/gryo/GryoMapper.java   | 32 ++++++------
 .../structure/io/gryo/GryoSerializers.java      | 20 ++++----
 .../structure/io/gryo/JavaTimeSerializers.java  | 52 ++++++++++----------
 .../structure/io/gryo/PairSerializer.java       |  4 +-
 .../structure/io/gryo/TypeRegistration.java     |  2 +-
 .../structure/io/gryo/kryoshim/InputShim.java   | 12 ++---
 .../structure/io/gryo/kryoshim/KryoShim.java    | 12 ++---
 .../io/gryo/kryoshim/KryoShimService.java       |  8 +--
 .../io/gryo/kryoshim/KryoShimServiceLoader.java | 30 +++++------
 .../structure/io/gryo/kryoshim/OutputShim.java  | 14 +++---
 .../io/gryo/kryoshim/SerializerShim.java        |  6 +--
 .../kryoshim/shaded/ShadedInputAdapter.java     |  4 +-
 .../gryo/kryoshim/shaded/ShadedKryoAdapter.java | 14 +++---
 .../kryoshim/shaded/ShadedOutputAdapter.java    | 17 +++----
 .../shaded/ShadedSerializerAdapter.java         | 14 +++---
 .../util/star/StarGraphSerializer.java          |  2 +-
 .../structure/io/HadoopPoolShimService.java     |  8 +--
 .../hadoop/structure/io/ObjectWritable.java     |  4 +-
 .../hadoop/structure/io/VertexWritable.java     |  4 +-
 .../structure/io/gryo/GryoRegistrator.java      | 16 +++---
 .../io/gryo/IoRegistryAwareKryoSerializer.java  |  8 +--
 .../io/gryo/ObjectWritableSerializer.java       |  4 +-
 .../io/gryo/VertexWritableSerializer.java       |  4 +-
 .../kryoshim/unshaded/UnshadedInputAdapter.java |  4 +-
 .../kryoshim/unshaded/UnshadedKryoAdapter.java  | 14 +++---
 .../unshaded/UnshadedKryoShimService.java       | 22 ++++-----
 .../unshaded/UnshadedOutputAdapter.java         | 14 +++---
 .../unshaded/UnshadedSerializerAdapter.java     |  6 +--
 28 files changed, 175 insertions(+), 176 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
index 7bf9b7d..577c1ce 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
@@ -48,7 +48,6 @@ import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.TraverserSe
 import org.apache.tinkerpop.gremlin.process.traversal.util.DefaultTraversalMetrics;
 import org.apache.tinkerpop.gremlin.process.traversal.util.ImmutableMetrics;
 import org.apache.tinkerpop.gremlin.process.traversal.util.MutableMetrics;
-import org.apache.tinkerpop.gremlin.process.traversal.util.PureTraversal;
 import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalExplanation;
 import org.apache.tinkerpop.gremlin.structure.Direction;
 import org.apache.tinkerpop.gremlin.structure.Edge;
@@ -380,7 +379,7 @@ public final class GryoMapper implements Mapper<Kryo> {
                         null == tr.getSerializerShim() /* a shim serializer is acceptable */ &&
                         !(tr.getShadedSerializer() instanceof JavaSerializer) /* shaded JavaSerializer is acceptable */) {
                     // everything else is invalid
-                    String msg = String.format("The default GryoMapper type registration %s is invalid.  " +
+                    final String msg = String.format("The default GryoMapper type registration %s is invalid.  " +
                                     "It must supply either an implementation of %s or %s, but supplies neither.  " +
                                     "This is probably a bug in GryoMapper's default serialization class registrations.", tr,
                             SerializerShim.class.getCanonicalName(), JavaSerializer.class.getCanonicalName());
@@ -505,11 +504,12 @@ public final class GryoMapper implements Mapper<Kryo> {
             return new GryoMapper(this);
         }
 
-        private <T> void addOrOverrideRegistration(Class<?> clazz, Function<Integer, TypeRegistration<T>> newRegistrationBuilder) {
-            Iterator<TypeRegistration<?>> iter = typeRegistrations.iterator();
+        private <T> void addOrOverrideRegistration(final Class<?> clazz,
+                                                   final Function<Integer, TypeRegistration<T>> newRegistrationBuilder) {
+            final Iterator<TypeRegistration<?>> iter = typeRegistrations.iterator();
             Integer registrationId = null;
             while (iter.hasNext()) {
-                TypeRegistration<?> existingRegistration = iter.next();
+                final TypeRegistration<?> existingRegistration = iter.next();
                 if (existingRegistration.getTargetClass().equals(clazz)) {
                     // when overridding a registration, use the old id
                     registrationId = existingRegistration.getId();
@@ -534,11 +534,11 @@ public final class GryoMapper implements Mapper<Kryo> {
         private final Function<Kryo, Serializer> functionOfShadedKryo;
         private final int id;
 
-        private GryoTypeReg(Class<T> clazz,
-                            Serializer<T> shadedSerializer,
-                            SerializerShim<T> serializerShim,
-                            Function<Kryo, Serializer> functionOfShadedKryo,
-                            int id) {
+        private GryoTypeReg(final Class<T> clazz,
+                            final Serializer<T> shadedSerializer,
+                            final SerializerShim<T> serializerShim,
+                            final Function<Kryo, Serializer> functionOfShadedKryo,
+                            final int id) {
             this.clazz = clazz;
             this.shadedSerializer = shadedSerializer;
             this.serializerShim = serializerShim;
@@ -554,7 +554,7 @@ public final class GryoMapper implements Mapper<Kryo> {
                 serializerCount++;
 
             if (1 < serializerCount) {
-                String msg = String.format(
+                final String msg = String.format(
                         "GryoTypeReg accepts at most one kind of serializer, but multiple " +
                                 "serializers were supplied for class %s (id %s).  " +
                                 "Shaded serializer: %s.  Shim serializer: %s.  Shaded serializer function: %s.",
@@ -564,19 +564,19 @@ public final class GryoMapper implements Mapper<Kryo> {
             }
         }
 
-        private static <T> GryoTypeReg<T> of(Class<T> clazz, int id) {
+        private static <T> GryoTypeReg<T> of(final Class<T> clazz, final int id) {
             return new GryoTypeReg<>(clazz, null, null, null, id);
         }
 
-        private static <T> GryoTypeReg<T> of(Class<T> clazz, int id, Serializer<T> shadedSerializer) {
+        private static <T> GryoTypeReg<T> of(final Class<T> clazz, final int id, final Serializer<T> shadedSerializer) {
             return new GryoTypeReg<>(clazz, shadedSerializer, null, null, id);
         }
 
-        private static <T> GryoTypeReg<T> of(Class<T> clazz, int id, SerializerShim<T> serializerShim) {
+        private static <T> GryoTypeReg<T> of(final Class<T> clazz, final int id, final SerializerShim<T> serializerShim) {
             return new GryoTypeReg<>(clazz, null, serializerShim, null, id);
         }
 
-        private static <T> GryoTypeReg<T> of(Class clazz, int id, Function<Kryo, Serializer> fct) {
+        private static <T> GryoTypeReg<T> of(final Class clazz, final int id, final Function<Kryo, Serializer> fct) {
             return new GryoTypeReg<>(clazz, null, null, fct, id);
         }
 
@@ -606,7 +606,7 @@ public final class GryoMapper implements Mapper<Kryo> {
         }
 
         @Override
-        public Kryo registerWith(Kryo kryo) {
+        public Kryo registerWith(final Kryo kryo) {
             if (null != functionOfShadedKryo)
                 kryo.register(clazz, functionOfShadedKryo.apply(kryo), id);
             else if (null != shadedSerializer)

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoSerializers.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoSerializers.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoSerializers.java
index 2042a4a..da4fd7a 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoSerializers.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoSerializers.java
@@ -49,12 +49,12 @@ public final class GryoSerializers {
      */
     public final static class EdgeSerializer implements SerializerShim<Edge> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Edge edge) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final Edge edge) {
             kryo.writeClassAndObject(output, DetachedFactory.detach(edge, true));
         }
 
         @Override
-        public <I extends InputShim> Edge read(KryoShim<I, ?> kryo, I input, Class<Edge> edgeClass) {
+        public <I extends InputShim> Edge read(final KryoShim<I, ?> kryo, final I input, final Class<Edge> edgeClass) {
             final Object o = kryo.readClassAndObject(input);
             return (Edge) o;
         }
@@ -65,12 +65,12 @@ public final class GryoSerializers {
      */
     public final static class VertexSerializer implements SerializerShim<Vertex> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Vertex vertex) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final Vertex vertex) {
             kryo.writeClassAndObject(output, DetachedFactory.detach(vertex, true));
         }
 
         @Override
-        public <I extends InputShim> Vertex read(KryoShim<I, ?> kryo, I input, Class<Vertex> vertexClass) {
+        public <I extends InputShim> Vertex read(final KryoShim<I, ?> kryo, final I input, final Class<Vertex> vertexClass) {
             return (Vertex) kryo.readClassAndObject(input);
         }
     }
@@ -80,12 +80,12 @@ public final class GryoSerializers {
      */
     public final static class PropertySerializer implements SerializerShim<Property> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Property property) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final Property property) {
             kryo.writeClassAndObject(output, property instanceof VertexProperty ? DetachedFactory.detach((VertexProperty) property, true) : DetachedFactory.detach(property));
         }
 
         @Override
-        public <I extends InputShim> Property read(KryoShim<I, ?> kryo, I input, Class<Property> propertyClass) {
+        public <I extends InputShim> Property read(final KryoShim<I, ?> kryo, final I input, final Class<Property> propertyClass) {
             return (Property) kryo.readClassAndObject(input);
         }
     }
@@ -95,12 +95,12 @@ public final class GryoSerializers {
      */
     public final static class VertexPropertySerializer implements SerializerShim<VertexProperty> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, VertexProperty vertexProperty) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final VertexProperty vertexProperty) {
             kryo.writeClassAndObject(output, DetachedFactory.detach(vertexProperty, true));
         }
 
         @Override
-        public <I extends InputShim> VertexProperty read(KryoShim<I, ?> kryo, I input, Class<VertexProperty> vertexPropertyClass) {
+        public <I extends InputShim> VertexProperty read(final KryoShim<I, ?> kryo, final I input, final Class<VertexProperty> vertexPropertyClass) {
             return (VertexProperty) kryo.readClassAndObject(input);
         }
     }
@@ -110,12 +110,12 @@ public final class GryoSerializers {
      */
     public final static class PathSerializer implements SerializerShim<Path> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Path path) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final Path path) {
             kryo.writeClassAndObject(output, DetachedFactory.detach(path, false));
         }
 
         @Override
-        public <I extends InputShim> Path read(KryoShim<I, ?> kryo, I input, Class<Path> pathClass) {
+        public <I extends InputShim> Path read(final KryoShim<I, ?> kryo, final I input, final Class<Path> pathClass) {
             return (Path) kryo.readClassAndObject(input);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/JavaTimeSerializers.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/JavaTimeSerializers.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/JavaTimeSerializers.java
index 24df032..85bb3c0 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/JavaTimeSerializers.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/JavaTimeSerializers.java
@@ -50,12 +50,12 @@ final class JavaTimeSerializers {
      */
     final static class DurationSerializer implements SerializerShim<Duration> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Duration duration) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final Duration duration) {
             output.writeLong(duration.toNanos());
         }
 
         @Override
-        public <I extends InputShim> Duration read(KryoShim<I, ?> kryo, I input, Class<Duration> durationClass) {
+        public <I extends InputShim> Duration read(final KryoShim<I, ?> kryo, final I input, final Class<Duration> durationClass) {
             return Duration.ofNanos(input.readLong());
         }
     }
@@ -65,13 +65,13 @@ final class JavaTimeSerializers {
      */
     final static class InstantSerializer implements SerializerShim<Instant> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Instant instant) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final Instant instant) {
             output.writeLong(instant.getEpochSecond());
             output.writeInt(instant.getNano());
         }
 
         @Override
-        public <I extends InputShim> Instant read(KryoShim<I, ?> kryo, I input, Class<Instant> aClass) {
+        public <I extends InputShim> Instant read(final KryoShim<I, ?> kryo, final I input, final Class<Instant> aClass) {
             return Instant.ofEpochSecond(input.readLong(), input.readInt());
         }
     }
@@ -81,12 +81,12 @@ final class JavaTimeSerializers {
      */
     final static class LocalDateSerializer implements SerializerShim<LocalDate> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, LocalDate localDate) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final LocalDate localDate) {
             output.writeLong(localDate.toEpochDay());
         }
 
         @Override
-        public <I extends InputShim> LocalDate read(KryoShim<I, ?> kryo, I input, Class<LocalDate> clazz) {
+        public <I extends InputShim> LocalDate read(final KryoShim<I, ?> kryo, final I input, final Class<LocalDate> clazz) {
             return LocalDate.ofEpochDay(input.readLong());
         }
     }
@@ -96,7 +96,7 @@ final class JavaTimeSerializers {
      */
     final static class LocalDateTimeSerializer implements SerializerShim<LocalDateTime> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, LocalDateTime localDateTime) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final LocalDateTime localDateTime) {
             output.writeInt(localDateTime.getYear());
             output.writeInt(localDateTime.getMonthValue());
             output.writeInt(localDateTime.getDayOfMonth());
@@ -107,7 +107,7 @@ final class JavaTimeSerializers {
         }
 
         @Override
-        public <I extends InputShim> LocalDateTime read(KryoShim<I, ?> kryo, I input, Class<LocalDateTime> clazz) {
+        public <I extends InputShim> LocalDateTime read(final KryoShim<I, ?> kryo, final I input, final Class<LocalDateTime> clazz) {
             return LocalDateTime.of(input.readInt(), input.readInt(), input.readInt(), input.readInt(), input.readInt(), input.readInt(), input.readInt());
         }
     }
@@ -117,12 +117,12 @@ final class JavaTimeSerializers {
      */
     final static class LocalTimeSerializer implements SerializerShim<LocalTime> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, LocalTime localTime) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final LocalTime localTime) {
             output.writeLong(localTime.toNanoOfDay());
         }
 
         @Override
-        public <I extends InputShim> LocalTime read(KryoShim<I, ?> kryo, I input, Class<LocalTime> clazz) {
+        public <I extends InputShim> LocalTime read(final KryoShim<I, ?> kryo, final I input, final Class<LocalTime> clazz) {
             return LocalTime.ofNanoOfDay(input.readLong());
         }
     }
@@ -132,13 +132,13 @@ final class JavaTimeSerializers {
      */
     final static class MonthDaySerializer implements SerializerShim<MonthDay> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, MonthDay monthDay) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final MonthDay monthDay) {
             output.writeInt(monthDay.getMonthValue());
             output.writeInt(monthDay.getDayOfMonth());
         }
 
         @Override
-        public <I extends InputShim> MonthDay read(KryoShim<I, ?> kryo, I input, Class<MonthDay> clazz) {
+        public <I extends InputShim> MonthDay read(final KryoShim<I, ?> kryo, final I input, final Class<MonthDay> clazz) {
             return MonthDay.of(input.readInt(), input.readInt());
         }
     }
@@ -148,13 +148,13 @@ final class JavaTimeSerializers {
      */
     final static class OffsetDateTimeSerializer implements SerializerShim<OffsetDateTime> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, OffsetDateTime offsetDateTime) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final OffsetDateTime offsetDateTime) {
             kryo.writeObject(output, offsetDateTime.toLocalDateTime());
             kryo.writeObject(output, offsetDateTime.getOffset());
         }
 
         @Override
-        public <I extends InputShim> OffsetDateTime read(KryoShim<I, ?> kryo, I input, Class<OffsetDateTime> clazz) {
+        public <I extends InputShim> OffsetDateTime read(final KryoShim<I, ?> kryo, final I input, final Class<OffsetDateTime> clazz) {
             return OffsetDateTime.of(kryo.readObject(input, LocalDateTime.class), kryo.readObject(input, ZoneOffset.class));
         }
     }
@@ -164,13 +164,13 @@ final class JavaTimeSerializers {
      */
     final static class OffsetTimeSerializer implements SerializerShim<OffsetTime> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, OffsetTime offsetTime) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final OffsetTime offsetTime) {
             kryo.writeObject(output, offsetTime.toLocalTime());
             kryo.writeObject(output, offsetTime.getOffset());
         }
 
         @Override
-        public <I extends InputShim> OffsetTime read(KryoShim<I, ?> kryo, I input, Class<OffsetTime> clazz) {
+        public <I extends InputShim> OffsetTime read(final KryoShim<I, ?> kryo, final I input, final Class<OffsetTime> clazz) {
             return OffsetTime.of(kryo.readObject(input, LocalTime.class), kryo.readObject(input, ZoneOffset.class));
         }
     }
@@ -180,14 +180,14 @@ final class JavaTimeSerializers {
      */
     final static class PeriodSerializer implements SerializerShim<Period> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Period period) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final Period period) {
             output.writeInt(period.getYears());
             output.writeInt(period.getMonths());
             output.writeInt(period.getDays());
         }
 
         @Override
-        public <I extends InputShim> Period read(KryoShim<I, ?> kryo, I input, Class<Period> clazz) {
+        public <I extends InputShim> Period read(final KryoShim<I, ?> kryo, final I input, final Class<Period> clazz) {
             return Period.of(input.readInt(), input.readInt(), input.readInt());
         }
     }
@@ -197,12 +197,12 @@ final class JavaTimeSerializers {
      */
     final static class YearSerializer implements SerializerShim<Year> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Year year) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final Year year) {
             output.writeInt(year.getValue());
         }
 
         @Override
-        public <I extends InputShim> Year read(KryoShim<I, ?> kryo, I input, Class<Year> clazz) {
+        public <I extends InputShim> Year read(final KryoShim<I, ?> kryo, final I input, final Class<Year> clazz) {
             return Year.of(input.readInt());
         }
     }
@@ -212,13 +212,13 @@ final class JavaTimeSerializers {
      */
     final static class YearMonthSerializer implements SerializerShim<YearMonth> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, YearMonth monthDay) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final YearMonth monthDay) {
             output.writeInt(monthDay.getYear());
             output.writeInt(monthDay.getMonthValue());
         }
 
         @Override
-        public <I extends InputShim> YearMonth read(KryoShim<I, ?> kryo, I input, Class<YearMonth> clazz) {
+        public <I extends InputShim> YearMonth read(final KryoShim<I, ?> kryo, final I input, final Class<YearMonth> clazz) {
             return YearMonth.of(input.readInt(), input.readInt());
         }
     }
@@ -228,7 +228,7 @@ final class JavaTimeSerializers {
      */
     final static class ZonedDateTimeSerializer implements SerializerShim<ZonedDateTime> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, ZonedDateTime zonedDateTime) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final ZonedDateTime zonedDateTime) {
             output.writeInt(zonedDateTime.getYear());
             output.writeInt(zonedDateTime.getMonthValue());
             output.writeInt(zonedDateTime.getDayOfMonth());
@@ -240,7 +240,7 @@ final class JavaTimeSerializers {
         }
 
         @Override
-        public <I extends InputShim> ZonedDateTime read(KryoShim<I, ?> kryo, I input, Class<ZonedDateTime> clazz) {
+        public <I extends InputShim> ZonedDateTime read(final KryoShim<I, ?> kryo, final I input, final Class<ZonedDateTime> clazz) {
             return ZonedDateTime.of(input.readInt(), input.readInt(), input.readInt(),
                     input.readInt(), input.readInt(), input.readInt(), input.readInt(),
                     ZoneId.of(input.readString()));
@@ -252,12 +252,12 @@ final class JavaTimeSerializers {
      */
     final static class ZoneOffsetSerializer implements SerializerShim<ZoneOffset> {
         @Override
-        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, ZoneOffset zoneOffset) {
+        public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final ZoneOffset zoneOffset) {
             output.writeString(zoneOffset.getId());
         }
 
         @Override
-        public <I extends InputShim> ZoneOffset read(KryoShim<I, ?> kryo, I input, Class<ZoneOffset> clazz) {
+        public <I extends InputShim> ZoneOffset read(final KryoShim<I, ?> kryo, final I input, final Class<ZoneOffset> clazz) {
             return ZoneOffset.of(input.readString());
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/PairSerializer.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/PairSerializer.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/PairSerializer.java
index 0464b22..1465b24 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/PairSerializer.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/PairSerializer.java
@@ -33,13 +33,13 @@ import org.javatuples.Pair;
  */
 final class PairSerializer implements SerializerShim<Pair> {
     @Override
-    public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Pair pair) {
+    public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final Pair pair) {
         kryo.writeClassAndObject(output, pair.getValue0());
         kryo.writeClassAndObject(output, pair.getValue1());
     }
 
     @Override
-    public <I extends InputShim> Pair read(KryoShim<I, ?> kryo, I input, Class<Pair> pairClass) {
+    public <I extends InputShim> Pair read(final KryoShim<I, ?> kryo, final I input, final Class<Pair> pairClass) {
         return Pair.with(kryo.readClassAndObject(input), kryo.readClassAndObject(input));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java
index 1f41c0d..a01d967 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java
@@ -66,7 +66,7 @@ public interface TypeRegistration<T> {
      * @param kryo Kryo instance into which this type is registered
      * @return the sole parameter
      */
-    Kryo registerWith(Kryo kryo);
+    Kryo registerWith(final Kryo kryo);
 
     /**
      * Returns true if at least one of {@link #getShadedSerializer()}, {@link #getSerializerShim()}, or

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/InputShim.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/InputShim.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/InputShim.java
index 23bec16..424b1bf 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/InputShim.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/InputShim.java
@@ -24,15 +24,15 @@ package org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim;
  */
 public interface InputShim {
 
-    byte readByte();
+    public byte readByte();
 
-    byte[] readBytes(int size);
+    public byte[] readBytes(final int size);
 
-    String readString();
+    public String readString();
 
-    long readLong();
+    public long readLong();
 
-    int readInt();
+    public int readInt();
 
-    double readDouble();
+    public double readDouble();
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShim.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShim.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShim.java
index a0f3f87..a4eb2cb 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShim.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShim.java
@@ -27,15 +27,15 @@ package org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim;
  */
 public interface KryoShim<I extends InputShim, O extends OutputShim> {
 
-    <T> T readObject(I input, Class<T> type);
+    public <T> T readObject(final I input, final Class<T> type);
 
-    Object readClassAndObject(I input);
+    public Object readClassAndObject(final I input);
 
-    void writeObject(O output, Object object);
+    public void writeObject(final O output, final Object object);
 
-    void writeClassAndObject(O output, Object object);
+    public void writeClassAndObject(final O output, final Object object);
 
-    <T> T readObjectOrNull(I input, Class<T> type);
+    public <T> T readObjectOrNull(final I input, final Class<T> type);
 
-    void writeObjectOrNull(O output, Object object, Class type);
+    public void writeObjectOrNull(final O output, final Object object, final Class type);
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java
index 7783856..ee0c8b3 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java
@@ -52,7 +52,7 @@ public interface KryoShimService {
      * @param source the stream from which to read an object's serialized form
      * @return the first deserialized object available from {@code source}
      */
-    Object readClassAndObject(InputStream source);
+    public Object readClassAndObject(final InputStream source);
 
     /**
      * Serializes an object to an output stream.  This may flush the output stream.
@@ -60,7 +60,7 @@ public interface KryoShimService {
      * @param o the object to serialize
      * @param sink the stream into which the serialized object is written
      */
-    void writeClassAndObject(Object o, OutputStream sink);
+    public void writeClassAndObject(final Object o, final OutputStream sink);
 
     /**
      * Returns this service's relative priority number.  Unless explicitly overridden through a
@@ -81,7 +81,7 @@ public interface KryoShimService {
      *
      * @return this implementation's priority value
      */
-    int getPriority();
+    public int getPriority();
 
     /**
      * Attempt to incorporate the supplied configuration in future read/write calls.
@@ -95,5 +95,5 @@ public interface KryoShimService {
      *
      * @param conf the configuration to apply to this service's internal serializer
      */
-    void applyConfiguration(Configuration conf);
+    public void applyConfiguration(final Configuration conf);
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
index fd57a3c..d16b1a5 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
@@ -49,7 +49,7 @@ public class KryoShimServiceLoader {
      */
     public static final String SHIM_CLASS_SYSTEM_PROPERTY = "tinkerpop.kryo.shim";
 
-    public static void applyConfiguration(Configuration conf) {
+    public static void applyConfiguration(final Configuration conf) {
         KryoShimServiceLoader.conf = conf;
         load(true);
     }
@@ -64,15 +64,15 @@ public class KryoShimServiceLoader {
      *                    before selecting a new service to return
      * @return the shim service
      */
-    public static KryoShimService load(boolean forceReload) {
+    public static KryoShimService load(final boolean forceReload) {
 
         if (null != cachedShimService && !forceReload) {
             return cachedShimService;
         }
 
-        ArrayList<KryoShimService> services = new ArrayList<>();
+        final ArrayList<KryoShimService> services = new ArrayList<>();
 
-        ServiceLoader<KryoShimService> sl = ServiceLoader.load(KryoShimService.class);
+        final ServiceLoader<KryoShimService> sl = ServiceLoader.load(KryoShimService.class);
 
         KryoShimService result = null;
 
@@ -117,7 +117,7 @@ public class KryoShimServiceLoader {
         log.info("Set {} provider to {} ({}) because its priority value ({}) is the highest available",
                 KryoShimService.class.getSimpleName(), result, result.getClass(), result.getPriority());
 
-        Configuration userConf = conf;
+        final Configuration userConf = conf;
 
         if (null != userConf) {
             log.info("Configuring {} provider {} with user-provided configuration",
@@ -145,10 +145,10 @@ public class KryoShimServiceLoader {
      * @param o an object for which the instance and class are serialized
      * @return the serialized form
      */
-    public static byte[] writeClassAndObjectToBytes(Object o) {
-        KryoShimService shimService = load();
+    public static byte[] writeClassAndObjectToBytes(final Object o) {
+        final KryoShimService shimService = load();
 
-        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
 
         shimService.writeClassAndObject(o, baos);
 
@@ -163,8 +163,8 @@ public class KryoShimServiceLoader {
      * @param <T> the type to which the deserialized object is cast as it is returned
      * @return the deserialized object
      */
-    public static <T> T readClassAndObject(InputStream source) {
-        KryoShimService shimService = load();
+    public static <T> T readClassAndObject(final InputStream source) {
+        final KryoShimService shimService = load();
 
         return (T)shimService.readClassAndObject(source);
     }
@@ -183,16 +183,16 @@ public class KryoShimServiceLoader {
         INSTANCE;
 
         @Override
-        public int compare(KryoShimService a, KryoShimService b) {
-            int ap = a.getPriority();
-            int bp = b.getPriority();
+        public int compare(final KryoShimService a, final KryoShimService b) {
+            final int ap = a.getPriority();
+            final int bp = b.getPriority();
 
             if (ap < bp) {
                 return -1;
             } else if (bp < ap) {
                 return 1;
             } else {
-                int result = a.getClass().getCanonicalName().compareTo(b.getClass().getCanonicalName());
+                final int result = a.getClass().getCanonicalName().compareTo(b.getClass().getCanonicalName());
 
                 if (0 == result) {
                     log.warn("Found two {} implementations with the same canonical classname: {}.  " +
@@ -201,7 +201,7 @@ public class KryoShimServiceLoader {
                                     "META-INF/services/org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimService.",
                             a.getClass().getCanonicalName());
                 } else {
-                    String winner = 0 < result ? a.getClass().getCanonicalName() : b.getClass().getCanonicalName();
+                    final String winner = 0 < result ? a.getClass().getCanonicalName() : b.getClass().getCanonicalName();
                     log.warn("{} implementations {} and {} are tied with priority value {}.  " +
                                     "Preferring {} to the other because it has a lexicographically greater classname.  " +
                                     "Consider setting the system property \"{}\" instead of relying on priority tie-breaking.",

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/OutputShim.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/OutputShim.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/OutputShim.java
index e4ca3d5..e3647f3 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/OutputShim.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/OutputShim.java
@@ -24,17 +24,17 @@ package org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim;
  */
 public interface OutputShim {
 
-    void writeByte(byte b);
+    public void writeByte(final byte b);
 
-    void writeBytes(byte[] array, int offset, int count);
+    public void writeBytes(final byte[] array, final int offset, final int count);
 
-    void writeString(String s);
+    public void writeString(final String s);
 
-    void writeLong(long l);
+    public void writeLong(final long l);
 
-    void writeInt(int i);
+    public void writeInt(final int i);
 
-    void writeDouble(double d);
+    public void writeDouble(final double d);
 
-    void flush();
+    public void flush();
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/SerializerShim.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/SerializerShim.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/SerializerShim.java
index e5f9005..c36cfee 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/SerializerShim.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/SerializerShim.java
@@ -26,11 +26,11 @@ package org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim;
  */
 public interface SerializerShim<T> {
 
-    <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, T object);
+    public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final T object);
 
-    <I extends InputShim> T read(KryoShim<I, ?> kryo, I input, Class<T> clazz);
+    public <I extends InputShim> T read(final KryoShim<I, ?> kryo, final I input, final Class<T> clazz);
 
-    default boolean isImmutable() {
+    public default boolean isImmutable() {
         return false;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedInputAdapter.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedInputAdapter.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedInputAdapter.java
index d11c1c4..29524b9 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedInputAdapter.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedInputAdapter.java
@@ -25,7 +25,7 @@ public class ShadedInputAdapter implements InputShim {
 
     private final Input shadedInput;
 
-    public ShadedInputAdapter(Input shadedInput) {
+    public ShadedInputAdapter(final Input shadedInput) {
         this.shadedInput = shadedInput;
     }
 
@@ -40,7 +40,7 @@ public class ShadedInputAdapter implements InputShim {
     }
 
     @Override
-    public byte[] readBytes(int size) {
+    public byte[] readBytes(final int size) {
         return shadedInput.readBytes(size);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedKryoAdapter.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedKryoAdapter.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedKryoAdapter.java
index 4283298..5ce0f6d 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedKryoAdapter.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedKryoAdapter.java
@@ -25,42 +25,42 @@ public class ShadedKryoAdapter implements KryoShim<ShadedInputAdapter, ShadedOut
 
     private final Kryo shadedKryo;
 
-    public ShadedKryoAdapter(Kryo shadedKryo) {
+    public ShadedKryoAdapter(final Kryo shadedKryo) {
         this.shadedKryo = shadedKryo;
     }
 
     @Override
-    public <T> T readObject(ShadedInputAdapter input, Class<T> type)
+    public <T> T readObject(final ShadedInputAdapter input, final Class<T> type)
     {
         return shadedKryo.readObject(input.getShadedInput(), type);
     }
 
     @Override
-    public Object readClassAndObject(ShadedInputAdapter input)
+    public Object readClassAndObject(final ShadedInputAdapter input)
     {
         return shadedKryo.readClassAndObject(input.getShadedInput());
     }
 
     @Override
-    public void writeObject(ShadedOutputAdapter output, Object object)
+    public void writeObject(final ShadedOutputAdapter output, final Object object)
     {
         shadedKryo.writeObject(output.getShadedOutput(), object);
     }
 
     @Override
-    public void writeClassAndObject(ShadedOutputAdapter output, Object object)
+    public void writeClassAndObject(final ShadedOutputAdapter output, final Object object)
     {
         shadedKryo.writeClassAndObject(output.getShadedOutput(), object);
     }
 
     @Override
-    public <T> T readObjectOrNull(ShadedInputAdapter input, Class<T> type)
+    public <T> T readObjectOrNull(final ShadedInputAdapter input, final Class<T> type)
     {
         return shadedKryo.readObjectOrNull(input.getShadedInput(), type);
     }
 
     @Override
-    public void writeObjectOrNull(ShadedOutputAdapter output, Object object, Class type)
+    public void writeObjectOrNull(final ShadedOutputAdapter output, final Object object, final Class type)
     {
         shadedKryo.writeObjectOrNull(output.getShadedOutput(), object, type);
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedOutputAdapter.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedOutputAdapter.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedOutputAdapter.java
index 7547466..b05f528 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedOutputAdapter.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedOutputAdapter.java
@@ -25,38 +25,38 @@ public class ShadedOutputAdapter implements OutputShim {
 
     private final Output shadedOutput;
 
-    public ShadedOutputAdapter(Output shadedOutput) {
+    public ShadedOutputAdapter(final Output shadedOutput) {
         this.shadedOutput = shadedOutput;
     }
 
     @Override
-    public void writeByte(byte b)
+    public void writeByte(final byte b)
     {
         shadedOutput.writeByte(b);
     }
 
     @Override
-    public void writeBytes(byte[] array, int offset, int count) {
+    public void writeBytes(final byte[] array, final int offset, final int count) {
         shadedOutput.writeBytes(array, offset, count);
     }
 
     @Override
-    public void writeString(String s) {
+    public void writeString(final String s) {
         shadedOutput.writeString(s);
     }
 
     @Override
-    public void writeLong(long l) {
+    public void writeLong(final long l) {
         shadedOutput.writeLong(l);
     }
 
     @Override
-    public void writeInt(int i) {
+    public void writeInt(final int i) {
         shadedOutput.writeInt(i);
     }
 
     @Override
-    public void writeDouble(double d) {
+    public void writeDouble(final double d) {
         shadedOutput.writeDouble(d);
     }
 
@@ -65,8 +65,7 @@ public class ShadedOutputAdapter implements OutputShim {
         shadedOutput.flush();
     }
 
-    Output getShadedOutput()
-    {
+    Output getShadedOutput() {
         return shadedOutput;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedSerializerAdapter.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedSerializerAdapter.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedSerializerAdapter.java
index 4ce27b1..28a44bd 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedSerializerAdapter.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedSerializerAdapter.java
@@ -28,27 +28,27 @@ public class ShadedSerializerAdapter<T> extends Serializer<T> {
 
     SerializerShim<T> serializer;
 
-    public ShadedSerializerAdapter(SerializerShim<T> serializer) {
+    public ShadedSerializerAdapter(final SerializerShim<T> serializer) {
         this.serializer = serializer;
         setImmutable(this.serializer.isImmutable());
     }
 
     @Override
-    public void write(Kryo kryo, Output output, T t) {
+    public void write(final Kryo kryo, final Output output, final T t) {
         /* These adapters could be cached pretty efficiently in instance fields if it were guaranteed that this
          * class was never subject to concurrent use.  That's true of Kryo instances, but it is not clear that
          * it is true of Serializer instances.
          */
-        ShadedKryoAdapter shadedKryoAdapter = new ShadedKryoAdapter(kryo);
-        ShadedOutputAdapter shadedOutputAdapter = new ShadedOutputAdapter(output);
+        final ShadedKryoAdapter shadedKryoAdapter = new ShadedKryoAdapter(kryo);
+        final ShadedOutputAdapter shadedOutputAdapter = new ShadedOutputAdapter(output);
         serializer.write(shadedKryoAdapter, shadedOutputAdapter, t);
     }
 
     @Override
-    public T read(Kryo kryo, Input input, Class<T> aClass) {
+    public T read(final Kryo kryo, final Input input, final Class<T> aClass) {
         // Same caching opportunity as in write(...)
-        ShadedKryoAdapter shadedKryoAdapter = new ShadedKryoAdapter(kryo);
-        ShadedInputAdapter shadedInputAdapter = new ShadedInputAdapter(input);
+        final ShadedKryoAdapter shadedKryoAdapter = new ShadedKryoAdapter(kryo);
+        final ShadedInputAdapter shadedInputAdapter = new ShadedInputAdapter(input);
         return serializer.read(shadedKryoAdapter, shadedInputAdapter, aClass);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphSerializer.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphSerializer.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphSerializer.java
index 431e1eb..f4ffa7d 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphSerializer.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphSerializer.java
@@ -81,7 +81,7 @@ public class StarGraphSerializer implements SerializerShim<StarGraph> {
      * If the returned {@link StarGraph} is null, that means that the {@link GraphFilter} filtered the vertex.
      */
     @Override
-    public <I extends InputShim> StarGraph read(KryoShim<I, ?> kryo, I input, Class<StarGraph> clazz) {
+    public <I extends InputShim> StarGraph read(final KryoShim<I, ?> kryo, final I input, final Class<StarGraph> clazz) {
         final StarGraph starGraph = StarGraph.open();
         input.readByte();  // version field ignored for now - for future use with backward compatibility
         starGraph.edgeProperties = kryo.readObjectOrNull(input, HashMap.class);

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
index 12b5d07..3fad4fd 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
@@ -29,7 +29,7 @@ import java.io.OutputStream;
 
 public class HadoopPoolShimService implements KryoShimService {
 
-    public Object readClassAndObject(InputStream source) {
+    public Object readClassAndObject(final InputStream source) {
 
         Kryo k = null;
 
@@ -44,14 +44,14 @@ public class HadoopPoolShimService implements KryoShimService {
         }
     }
 
-    public void writeClassAndObject(Object o, OutputStream sink) {
+    public void writeClassAndObject(final Object o, final OutputStream sink) {
 
         Kryo k = null;
 
         try {
             k = HadoopPools.getGryoPool().takeKryo();
 
-            Output output = new Output(sink);
+            final Output output = new Output(sink);
 
             k.writeClassAndObject(output, o);
 
@@ -69,7 +69,7 @@ public class HadoopPoolShimService implements KryoShimService {
     }
 
     @Override
-    public void applyConfiguration(Configuration conf) {
+    public void applyConfiguration(final Configuration conf) {
         HadoopPools.initialize(conf);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
index 88f7ee1..0379ee6 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
@@ -63,13 +63,13 @@ public final class ObjectWritable<T> implements WritableComparable<ObjectWritabl
 
     @Override
     public void readFields(final DataInput input) throws IOException {
-        ByteArrayInputStream bais = new ByteArrayInputStream(WritableUtils.readCompressedByteArray(input));
+        final ByteArrayInputStream bais = new ByteArrayInputStream(WritableUtils.readCompressedByteArray(input));
         this.t = KryoShimServiceLoader.readClassAndObject(bais);
     }
 
     @Override
     public void write(final DataOutput output) throws IOException {
-        byte serialized[] = KryoShimServiceLoader.writeClassAndObjectToBytes(this.t);
+        final byte serialized[] = KryoShimServiceLoader.writeClassAndObjectToBytes(this.t);
         WritableUtils.writeCompressedByteArray(output, serialized);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritable.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritable.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritable.java
index 2252ded..be09941 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritable.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritable.java
@@ -61,13 +61,13 @@ public final class VertexWritable implements Writable, Serializable {
     @Override
     public void readFields(final DataInput input) throws IOException {
         this.vertex = null;
-        ByteArrayInputStream bais = new ByteArrayInputStream(WritableUtils.readCompressedByteArray(input));
+        final ByteArrayInputStream bais = new ByteArrayInputStream(WritableUtils.readCompressedByteArray(input));
         this.vertex = ((StarGraph)KryoShimServiceLoader.readClassAndObject(bais)).getStarVertex(); // read the star graph;
     }
 
     @Override
     public void write(final DataOutput output) throws IOException {
-        byte serialized[] = KryoShimServiceLoader.writeClassAndObjectToBytes(this.vertex.graph());
+        final byte serialized[] = KryoShimServiceLoader.writeClassAndObjectToBytes(this.vertex.graph());
         WritableUtils.writeCompressedByteArray(output, serialized);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java
index 68112d7..412fd3b 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java
@@ -74,11 +74,11 @@ public class GryoRegistrator implements KryoRegistrator {
      *                            or if they would be registered by this class by default (does not affect Kryo's
      *                            built-in registrations, e.g. String.class).
      */
-    public void registerClasses(Kryo kryo, Map<Class<?>, Serializer<?>> serializerOverrides, Set<Class<?>> blacklist) {
+    public void registerClasses(final Kryo kryo, final Map<Class<?>, Serializer<?>> serializerOverrides, final Set<Class<?>> blacklist) {
         // Apply TinkerPop type registrations copied from GyroSerializer's constructor
         for (Map.Entry<Class<?>, Serializer<?>> ent : getExtraRegistrations().entrySet()) {
-            Class<?> targetClass = ent.getKey();
-            Serializer<?> ser = ent.getValue();
+            final Class<?> targetClass = ent.getKey();
+            final Serializer<?> ser = ent.getValue();
 
             // Is this class blacklisted?  Skip it. (takes precedence over serializerOverrides)
             if (blacklist.contains(targetClass)) {
@@ -97,7 +97,7 @@ public class GryoRegistrator implements KryoRegistrator {
             }
         }
 
-        Set<Class<?>> shimmedClassesFromGryoMapper = new HashSet<>();
+        final Set<Class<?>> shimmedClassesFromGryoMapper = new HashSet<>();
 
         // Apply GryoMapper's default registrations
         for (TypeRegistration<?> tr : GryoMapper.build().create().getTypeRegistrations()) {
@@ -171,7 +171,7 @@ public class GryoRegistrator implements KryoRegistrator {
          * iteration in key-insertion-order).
          */
 
-        LinkedHashMap<Class<?>, Serializer<?>> m = new LinkedHashMap<>();
+        final LinkedHashMap<Class<?>, Serializer<?>> m = new LinkedHashMap<>();
         // The following entries were copied from GryoSerializer's constructor
         // This could be turned into a static collection on GryoSerializer to avoid
         // duplication, but it would be a bit cumbersome to do so without disturbing
@@ -218,10 +218,10 @@ public class GryoRegistrator implements KryoRegistrator {
         return m;
     }
 
-    private boolean checkForAndApplySerializerOverride(Map<Class<?>, Serializer<?>> serializerOverrides,
-                                                       Kryo kryo, Class<?> targetClass) {
+    private boolean checkForAndApplySerializerOverride(final Map<Class<?>, Serializer<?>> serializerOverrides,
+                                                       final Kryo kryo, Class<?> targetClass) {
         if (serializerOverrides.containsKey(targetClass)) {
-            Serializer<?> ser = serializerOverrides.get(targetClass);
+            final Serializer<?> ser = serializerOverrides.get(targetClass);
             if (null == ser) {
                 // null means use Kryo's default serializer
                 log.debug("Registering {} with default serializer per overrides", targetClass);

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/IoRegistryAwareKryoSerializer.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/IoRegistryAwareKryoSerializer.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/IoRegistryAwareKryoSerializer.java
index 8b21e21..bf71fae 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/IoRegistryAwareKryoSerializer.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/IoRegistryAwareKryoSerializer.java
@@ -46,7 +46,7 @@ public class IoRegistryAwareKryoSerializer extends KryoSerializer {
 
     private static final Logger log = LoggerFactory.getLogger(IoRegistryAwareKryoSerializer.class);
 
-    public IoRegistryAwareKryoSerializer(SparkConf conf) {
+    public IoRegistryAwareKryoSerializer(final SparkConf conf) {
         super(conf);
         // store conf so that we can access its registry (if one is present) in newKryo()
         this.conf = conf;
@@ -54,19 +54,19 @@ public class IoRegistryAwareKryoSerializer extends KryoSerializer {
 
     @Override
     public Kryo newKryo() {
-        Kryo kryo = super.newKryo();
+        final Kryo kryo = super.newKryo();
 
         return applyIoRegistryIfPresent(kryo);
     }
 
-    private Kryo applyIoRegistryIfPresent(Kryo kryo) {
+    private Kryo applyIoRegistryIfPresent(final Kryo kryo) {
         if (!conf.contains(GryoPool.CONFIG_IO_REGISTRY)) {
             log.info("SparkConf {} does not contain setting {}, skipping {} handling",
                     GryoPool.CONFIG_IO_REGISTRY, conf, IoRegistry.class.getCanonicalName());
             return kryo;
         }
 
-        String registryClassnames = conf.get(GryoPool.CONFIG_IO_REGISTRY);
+        final String registryClassnames = conf.get(GryoPool.CONFIG_IO_REGISTRY);
 
         for (String registryClassname : registryClassnames.split(",")) {
             final IoRegistry registry;

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/ObjectWritableSerializer.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/ObjectWritableSerializer.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/ObjectWritableSerializer.java
index 4ceb045..01be50d 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/ObjectWritableSerializer.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/ObjectWritableSerializer.java
@@ -35,13 +35,13 @@ import org.apache.tinkerpop.shaded.kryo.io.Output;
 public final class ObjectWritableSerializer<T> implements SerializerShim<ObjectWritable<T>> {
 
     @Override
-    public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, ObjectWritable<T> starGraph) {
+    public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final ObjectWritable<T> starGraph) {
         kryo.writeClassAndObject(output, starGraph.get());
         output.flush();
     }
 
     @Override
-    public <I extends InputShim> ObjectWritable<T> read(KryoShim<I, ?> kryo, I input, Class<ObjectWritable<T>> clazz) {
+    public <I extends InputShim> ObjectWritable<T> read(final KryoShim<I, ?> kryo, final I input, final Class<ObjectWritable<T>> clazz) {
         return new ObjectWritable(kryo.readClassAndObject(input));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/VertexWritableSerializer.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/VertexWritableSerializer.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/VertexWritableSerializer.java
index f3c1b15..c89fb05 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/VertexWritableSerializer.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/VertexWritableSerializer.java
@@ -36,13 +36,13 @@ import org.apache.tinkerpop.shaded.kryo.io.Output;
 public final class VertexWritableSerializer implements SerializerShim<VertexWritable> {
 
     @Override
-    public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, VertexWritable vertexWritable) {
+    public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final VertexWritable vertexWritable) {
         kryo.writeObject(output, vertexWritable.get().graph());
         output.flush();
     }
 
     @Override
-    public <I extends InputShim> VertexWritable read(KryoShim<I, ?> kryo, I input, Class<VertexWritable> clazz) {
+    public <I extends InputShim> VertexWritable read(final KryoShim<I, ?> kryo, final I input, final Class<VertexWritable> clazz) {
         return new VertexWritable(kryo.readObject(input, StarGraph.class).getStarVertex());
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedInputAdapter.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedInputAdapter.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedInputAdapter.java
index c533af7..0f13bb2 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedInputAdapter.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedInputAdapter.java
@@ -32,7 +32,7 @@ public class UnshadedInputAdapter implements InputShim
 
     private final Input unshadedInput;
 
-    public UnshadedInputAdapter(Input unshadedInput)
+    public UnshadedInputAdapter(final Input unshadedInput)
     {
         this.unshadedInput = unshadedInput;
     }
@@ -49,7 +49,7 @@ public class UnshadedInputAdapter implements InputShim
     }
 
     @Override
-    public byte[] readBytes(int size) {
+    public byte[] readBytes(final int size) {
         return unshadedInput.readBytes(size);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoAdapter.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoAdapter.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoAdapter.java
index b14abe0..94918a3 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoAdapter.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoAdapter.java
@@ -31,43 +31,43 @@ public class UnshadedKryoAdapter implements KryoShim<UnshadedInputAdapter, Unsha
 {
     private final Kryo unshadedKryo;
 
-    public UnshadedKryoAdapter(Kryo unshadedKryo)
+    public UnshadedKryoAdapter(final Kryo unshadedKryo)
     {
         this.unshadedKryo = unshadedKryo;
     }
 
     @Override
-    public <T> T readObject(UnshadedInputAdapter input, Class<T> type)
+    public <T> T readObject(final UnshadedInputAdapter input, final Class<T> type)
     {
         return unshadedKryo.readObject(input.getUnshadedInput(), type);
     }
 
     @Override
-    public Object readClassAndObject(UnshadedInputAdapter input)
+    public Object readClassAndObject(final UnshadedInputAdapter input)
     {
         return unshadedKryo.readClassAndObject(input.getUnshadedInput());
     }
 
     @Override
-    public void writeObject(UnshadedOutputAdapter output, Object object)
+    public void writeObject(final UnshadedOutputAdapter output, final Object object)
     {
         unshadedKryo.writeObject(output.getUnshadedOutput(), object);
     }
 
     @Override
-    public void writeClassAndObject(UnshadedOutputAdapter output, Object object)
+    public void writeClassAndObject(final UnshadedOutputAdapter output, final Object object)
     {
         unshadedKryo.writeClassAndObject(output.getUnshadedOutput(), object);
     }
 
     @Override
-    public <T> T readObjectOrNull(UnshadedInputAdapter input, Class<T> type)
+    public <T> T readObjectOrNull(final UnshadedInputAdapter input, final Class<T> type)
     {
         return unshadedKryo.readObjectOrNull(input.getUnshadedInput(), type);
     }
 
     @Override
-    public void writeObjectOrNull(UnshadedOutputAdapter output, Object object, Class type)
+    public void writeObjectOrNull(final UnshadedOutputAdapter output, final Object object, final Class type)
     {
         unshadedKryo.writeObjectOrNull(output.getUnshadedOutput(), object, type);
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java
index 95ebb0d..41e0001 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java
@@ -51,9 +51,9 @@ public class UnshadedKryoShimService implements KryoShimService {
     public UnshadedKryoShimService() { }
 
     @Override
-    public Object readClassAndObject(InputStream source) {
+    public Object readClassAndObject(final InputStream source) {
 
-        LinkedBlockingQueue<Kryo> kryos = initialize();
+        final LinkedBlockingQueue<Kryo> kryos = initialize();
 
         Kryo k = null;
         try {
@@ -72,15 +72,15 @@ public class UnshadedKryoShimService implements KryoShimService {
     }
 
     @Override
-    public void writeClassAndObject(Object o, OutputStream sink) {
+    public void writeClassAndObject(final Object o, OutputStream sink) {
 
-        LinkedBlockingQueue<Kryo> kryos = initialize();
+        final LinkedBlockingQueue<Kryo> kryos = initialize();
 
         Kryo k = null;
         try {
             k = kryos.take();
 
-            Output kryoOutput = new Output(sink);
+            final Output kryoOutput = new Output(sink);
 
             k.writeClassAndObject(kryoOutput, o);
 
@@ -102,7 +102,7 @@ public class UnshadedKryoShimService implements KryoShimService {
     }
 
     @Override
-    public void applyConfiguration(Configuration conf) {
+    public void applyConfiguration(final Configuration conf) {
         initialize(conf);
     }
 
@@ -110,22 +110,22 @@ public class UnshadedKryoShimService implements KryoShimService {
         return initialize(new BaseConfiguration());
     }
 
-    private LinkedBlockingQueue<Kryo> initialize(Configuration conf) {
+    private LinkedBlockingQueue<Kryo> initialize(final Configuration conf) {
         // DCL is safe in this case due to volatility
         if (!initialized) {
             synchronized (UnshadedKryoShimService.class) {
                 if (!initialized) {
-                    SparkConf sparkConf = new SparkConf();
+                    final SparkConf sparkConf = new SparkConf();
 
                     // Copy the user's IoRegistry from the param conf to the SparkConf we just created
-                    String regStr = conf.getString(GryoPool.CONFIG_IO_REGISTRY);
+                    final String regStr = conf.getString(GryoPool.CONFIG_IO_REGISTRY);
                     if (null != regStr) { // SparkConf rejects null values with NPE, so this has to be checked before set(...)
                         sparkConf.set(GryoPool.CONFIG_IO_REGISTRY, regStr);
                     }
                     // Setting spark.serializer here almost certainly isn't necessary, but it doesn't hurt
                     sparkConf.set("spark.serializer", IoRegistryAwareKryoSerializer.class.getCanonicalName());
 
-                    String registrator = conf.getString("spark.kryo.registrator");
+                    final String registrator = conf.getString("spark.kryo.registrator");
                     if (null != registrator) {
                         sparkConf.set("spark.kryo.registrator", registrator);
                         log.info("Copied spark.kryo.registrator: {}", registrator);
@@ -134,7 +134,7 @@ public class UnshadedKryoShimService implements KryoShimService {
                     }
 
                     // Reuse Gryo poolsize for Kryo poolsize (no need to copy this to SparkConf)
-                    int poolSize = conf.getInt(GryoPool.CONFIG_IO_GRYO_POOL_SIZE,
+                    final int poolSize = conf.getInt(GryoPool.CONFIG_IO_GRYO_POOL_SIZE,
                             GryoPool.CONFIG_IO_GRYO_POOL_SIZE_DEFAULT);
                     // Instantiate the spark.serializer
                     final IoRegistryAwareKryoSerializer ioReg = new IoRegistryAwareKryoSerializer(sparkConf);

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedOutputAdapter.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedOutputAdapter.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedOutputAdapter.java
index 9cc59d4..8dd8eb3 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedOutputAdapter.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedOutputAdapter.java
@@ -31,7 +31,7 @@ public class UnshadedOutputAdapter implements OutputShim
 {
     private final Output unshadedOutput;
 
-    public UnshadedOutputAdapter(Output unshadedOutput)
+    public UnshadedOutputAdapter(final Output unshadedOutput)
     {
         this.unshadedOutput = unshadedOutput;
     }
@@ -42,35 +42,35 @@ public class UnshadedOutputAdapter implements OutputShim
     }
 
     @Override
-    public void writeByte(byte b)
+    public void writeByte(final byte b)
     {
         unshadedOutput.writeByte(b);
     }
 
     @Override
-    public void writeBytes(byte[] array, int offset, int count) {
+    public void writeBytes(final byte[] array, final int offset, final int count) {
         unshadedOutput.writeBytes(array, offset, count);
     }
 
     @Override
-    public void writeString(String s)
+    public void writeString(final String s)
     {
         unshadedOutput.writeString(s);
     }
 
     @Override
-    public void writeLong(long l)
+    public void writeLong(final long l)
     {
         unshadedOutput.writeLong(l);
     }
 
     @Override
-    public void writeInt(int i) {
+    public void writeInt(final int i) {
         unshadedOutput.writeInt(i);
     }
 
     @Override
-    public void writeDouble(double d)
+    public void writeDouble(final double d)
     {
         unshadedOutput.writeDouble(d);
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/45abdcc3/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedSerializerAdapter.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedSerializerAdapter.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedSerializerAdapter.java
index efc9a4f..a5f8b05 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedSerializerAdapter.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedSerializerAdapter.java
@@ -35,20 +35,20 @@ public class UnshadedSerializerAdapter<T> extends Serializer<T>
 
     SerializerShim<T> serializer;
 
-    public UnshadedSerializerAdapter(SerializerShim<T> serializer) {
+    public UnshadedSerializerAdapter(final SerializerShim<T> serializer) {
         this.serializer = serializer;
         setImmutable(this.serializer.isImmutable());
     }
 
     @Override
-    public void write(Kryo kryo, Output output, T t) {
+    public void write(final Kryo kryo, final Output output, final T t) {
         UnshadedKryoAdapter shadedKryoAdapter = new UnshadedKryoAdapter(kryo);
         UnshadedOutputAdapter shadedOutputAdapter = new UnshadedOutputAdapter(output);
         serializer.write(shadedKryoAdapter, shadedOutputAdapter, t);
     }
 
     @Override
-    public T read(Kryo kryo, Input input, Class<T> aClass)
+    public T read(final Kryo kryo, final Input input, final Class<T> aClass)
     {
         UnshadedKryoAdapter shadedKryoAdapter = new UnshadedKryoAdapter(kryo);
         UnshadedInputAdapter shadedInputAdapter = new UnshadedInputAdapter(input);


[14/34] incubator-tinkerpop git commit: Merge remote-tracking branch 'origin/tp31'

Posted by ok...@apache.org.
Merge remote-tracking branch 'origin/tp31'


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

Branch: refs/heads/TINKERPOP-1278
Commit: dac7f114985ce4faf4a4c7c0fb21e88ccfa4c0a3
Parents: d02562c 6e17d66
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Sat Jun 4 07:32:48 2016 -0400
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Sat Jun 4 07:32:48 2016 -0400

----------------------------------------------------------------------
 pom.xml | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/dac7f114/pom.xml
----------------------------------------------------------------------


[17/34] incubator-tinkerpop git commit: Fix MonthDay serializer

Posted by ok...@apache.org.
Fix MonthDay serializer

I carelessly broke the read leg of this serializer when porting it to
the shim.  Ireplaced actual deserialization logic with autogenerated
"return null" when inserting a method template with the shim's
slightly different method signature.  Trivial fix though.


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

Branch: refs/heads/TINKERPOP-1278
Commit: ebd2f8576acdb472fa8696b58db128eac191f0ef
Parents: 9321a3e
Author: Dan LaRocque <da...@hopcount.org>
Authored: Mon Jun 6 04:35:34 2016 -0400
Committer: Dan LaRocque <da...@hopcount.org>
Committed: Mon Jun 6 04:35:34 2016 -0400

----------------------------------------------------------------------
 .../tinkerpop/gremlin/structure/io/gryo/JavaTimeSerializers.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ebd2f857/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/JavaTimeSerializers.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/JavaTimeSerializers.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/JavaTimeSerializers.java
index 8b14345..24df032 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/JavaTimeSerializers.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/JavaTimeSerializers.java
@@ -139,7 +139,7 @@ final class JavaTimeSerializers {
 
         @Override
         public <I extends InputShim> MonthDay read(KryoShim<I, ?> kryo, I input, Class<MonthDay> clazz) {
-            return null;
+            return MonthDay.of(input.readInt(), input.readInt());
         }
     }
 


[20/34] incubator-tinkerpop git commit: TINKERPOP-1321 Shim change and test tweaks

Posted by ok...@apache.org.
TINKERPOP-1321 Shim change and test tweaks


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

Branch: refs/heads/TINKERPOP-1278
Commit: 88ac30472cac2c7ca7c604de5de85e6709791611
Parents: ebd2f85
Author: Dan LaRocque <da...@hopcount.org>
Authored: Mon Jun 6 15:32:37 2016 -0400
Committer: Dan LaRocque <da...@hopcount.org>
Committed: Mon Jun 6 15:34:43 2016 -0400

----------------------------------------------------------------------
 .../structure/io/gryo/GryoRegistrator.java      | 194 +++++++++++++++++++
 .../io/gryo/TinkerPopKryoRegistrator.java       | 194 -------------------
 .../unshaded/UnshadedKryoShimService.java       |   2 +-
 .../computer/SparkHadoopGraphProvider.java      |  11 +-
 4 files changed, 204 insertions(+), 197 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/88ac3047/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java
new file mode 100644
index 0000000..1ae8c5c
--- /dev/null
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java
@@ -0,0 +1,194 @@
+/*
+ * 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.gryo;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.serializers.JavaSerializer;
+import org.apache.spark.serializer.KryoRegistrator;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
+import org.apache.tinkerpop.gremlin.spark.process.computer.payload.MessagePayload;
+import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewIncomingPayload;
+import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewOutgoingPayload;
+import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewPayload;
+import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.kryoshim.unshaded.UnshadedSerializerAdapter;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoMapper;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.TypeRegistration;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
+import org.apache.tinkerpop.gremlin.structure.util.star.StarGraph;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+
+/**
+ * A spark.kryo.registrator implementation that installs TinkerPop types.
+ * This is intended for use with spark.serializer=KryoSerializer, not GryoSerializer.
+ */
+public class GryoRegistrator implements KryoRegistrator {
+
+    private static final Logger log = LoggerFactory.getLogger(GryoRegistrator.class);
+
+    @Override
+    public void registerClasses(Kryo kryo) {
+        registerClasses(kryo, Collections.emptyMap(), Collections.emptySet());
+    }
+
+    /**
+     * Register TinkerPop's classes with the supplied {@link Kryo} instance
+     * while honoring optional overrides and optional class blacklist ("blackset"?).
+     *
+     * @param kryo the Kryo serializer instance with which to register types
+     * @param serializerOverrides serializer mappings that override this class's defaults
+     * @param blacklist classes which should not be registered at all, even if there is an override entry
+     *                  or if they would be registered by this class by default (does not affect Kryo's
+     *                  built-in registrations, e.g. String.class).
+     */
+    public void registerClasses(Kryo kryo, Map<Class<?>, Serializer<?>> serializerOverrides, Set<Class<?>> blacklist) {
+        // Apply TinkerPop type registrations copied from GyroSerializer's constructor
+        for (Map.Entry<Class<?>, Serializer<?>> ent : getExtraRegistrations().entrySet()) {
+            Class<?> targetClass = ent.getKey();
+            Serializer<?> ser = ent.getValue();
+
+            // Is this class blacklisted?  Skip it. (takes precedence over serializerOverrides)
+            if (blacklist.contains(targetClass)) {
+                log.debug("Not registering serializer for {} (blacklisted)", targetClass);
+                continue;
+            }
+
+            if (checkForAndApplySerializerOverride(serializerOverrides, kryo, targetClass)) {
+                // do nothing but skip the remaining else(-if) clauses
+            } else if (null == ser) {
+                log.debug("Registering {} with default serializer", targetClass);
+                kryo.register(targetClass);
+            } else {
+                log.debug("Registering {} with serializer {}", targetClass, ser);
+                kryo.register(targetClass, ser);
+            }
+        }
+
+        Set<Class<?>> shimmedClassesFromGryoMapper = new HashSet<>();
+
+        // Apply GryoMapper's default registrations
+        for (TypeRegistration<?> tr : GryoMapper.build().create().getTypeRegistrations()) {
+            // Is this class blacklisted?  Skip it. (takes precedence over serializerOverrides)
+            if (blacklist.contains(tr.getTargetClass())) {
+                log.debug("Not registering serializer for {} (blacklisted)", tr.getTargetClass());
+                continue;
+            }
+
+            final org.apache.tinkerpop.shaded.kryo.Serializer<?> shadedSerializer = tr.getShadedSerializer();
+            final SerializerShim<?> serializerShim = tr.getSerializerShim();
+            final java.util.function.Function<
+                    org.apache.tinkerpop.shaded.kryo.Kryo,
+                    org.apache.tinkerpop.shaded.kryo.Serializer> functionOfShadedKryo = tr.getFunctionOfShadedKryo();
+
+            // Apply overrides with the highest case-precedence
+            if (checkForAndApplySerializerOverride(serializerOverrides, kryo, tr.getTargetClass())) {
+                // do nothing but skip the remaining else(-if) clauses
+            } else if (null != shadedSerializer) {
+                if (shadedSerializer.getClass().equals(org.apache.tinkerpop.shaded.kryo.serializers.JavaSerializer.class)) {
+                    // Convert GryoMapper's shaded JavaSerializer mappings to their unshaded equivalents
+                    log.debug("Registering {} with JavaSerializer", tr.getTargetClass());
+                    kryo.register(tr.getTargetClass(), new JavaSerializer());
+                } else {
+                    // There's supposed to be a check in GryoMapper that prevents this from happening
+                    log.error("GryoMapper's default serialization registration for {} is a {}. " +
+                              "This is probably a bug in TinkerPop (this is not a valid default registration). " +
+                              "I am configuring Spark to use Kryo's default serializer for this class, " +
+                              "but this may cause serialization failures at runtime.",
+                              tr.getTargetClass(),
+                              org.apache.tinkerpop.shaded.kryo.Serializer.class.getCanonicalName());
+                    kryo.register(tr.getTargetClass());
+                }
+            } else if (null != serializerShim) {
+                // Wrap shim serializers in an adapter for Spark's unshaded Kryo
+                log.debug("Registering {} to serializer shim {} (serializer shim {})",
+                        tr.getTargetClass(), serializerShim, serializerShim.getClass());
+                kryo.register(tr.getTargetClass(), new UnshadedSerializerAdapter<>(serializerShim));
+                shimmedClassesFromGryoMapper.add(tr.getTargetClass());
+            } else if (null != functionOfShadedKryo) {
+                // As with shaded serializers, there's supposed to be a check in GryoMapper that prevents this from happening
+                log.error("GryoMapper's default serialization registration for {} is a Function<{},{}>.  " +
+                          "This is probably a bug in TinkerPop (this is not a valid default registration). " +
+                          "I am configuring Spark to use Kryo's default serializer instead of this function, " +
+                          "but this may cause serialization failures at runtime.",
+                          tr.getTargetClass(),
+                          org.apache.tinkerpop.shaded.kryo.Kryo.class.getCanonicalName(),
+                          org.apache.tinkerpop.shaded.kryo.Serializer.class.getCanonicalName());
+                kryo.register(tr.getTargetClass());
+            } else {
+                // Register all other classes with the default behavior (FieldSerializer)
+                log.debug("Registering {} with default serializer", tr.getTargetClass());
+                kryo.register(tr.getTargetClass());
+            }
+        }
+
+        // StarGraph's shim serializer is especially important on Spark for efficiency reasons,
+        // so log a warning if we failed to register it somehow
+        if (!shimmedClassesFromGryoMapper.contains(StarGraph.class)) {
+            log.warn("No SerializerShim found for StarGraph");
+        }
+    }
+
+    private LinkedHashMap<Class<?>, Serializer<?>> getExtraRegistrations() {
+
+        /* The map returned by this method MUST have a fixed iteration order!
+         *
+         * The order itself is irrelevant, so long as it is completely stable at runtime.
+         *
+         * LinkedHashMap satisfies this requirement (its contract specifies
+         * iteration in key-insertion-order).
+         */
+
+        LinkedHashMap<Class<?>, Serializer<?>> m = new LinkedHashMap<>();
+        // The following entries were copied from GryoSerializer's constructor
+        // This could be turned into a static collection on GryoSerializer to avoid
+        // duplication, but it would be a bit cumbersome to do so without disturbing
+        // the ordering of the existing entries in that constructor, since not all
+        // of the entries are for TinkerPop (and the ordering is significant).
+        m.put(MessagePayload.class, null);
+        m.put(ViewIncomingPayload.class, null);
+        m.put(ViewOutgoingPayload.class, null);
+        m.put(ViewPayload.class, null);
+        m.put(VertexWritable.class, new UnshadedSerializerAdapter<>(new VertexWritableSerializer()));
+        m.put(ObjectWritable.class, new UnshadedSerializerAdapter<>(new ObjectWritableSerializer<>()));
+
+        return m;
+    }
+
+    private boolean checkForAndApplySerializerOverride(Map<Class<?>, Serializer<?>> serializerOverrides,
+                                                       Kryo kryo, Class<?> targetClass) {
+        if (serializerOverrides.containsKey(targetClass)) {
+            Serializer<?> ser = serializerOverrides.get(targetClass);
+            if (null == ser) {
+                // null means use Kryo's default serializer
+                log.debug("Registering {} with default serializer per overrides", targetClass);
+                kryo.register(targetClass);
+            } else {
+                // nonnull means use that serializer
+                log.debug("Registering {} with serializer {} per overrides", targetClass, ser);
+                kryo.register(targetClass, ser);
+            }
+            return true;
+        }
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/88ac3047/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/TinkerPopKryoRegistrator.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/TinkerPopKryoRegistrator.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/TinkerPopKryoRegistrator.java
deleted file mode 100644
index bdb80fd..0000000
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/TinkerPopKryoRegistrator.java
+++ /dev/null
@@ -1,194 +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.gryo;
-
-import com.esotericsoftware.kryo.Kryo;
-import com.esotericsoftware.kryo.Serializer;
-import com.esotericsoftware.kryo.serializers.JavaSerializer;
-import org.apache.spark.serializer.KryoRegistrator;
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable;
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
-import org.apache.tinkerpop.gremlin.spark.process.computer.payload.MessagePayload;
-import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewIncomingPayload;
-import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewOutgoingPayload;
-import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewPayload;
-import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.kryoshim.unshaded.UnshadedSerializerAdapter;
-import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoMapper;
-import org.apache.tinkerpop.gremlin.structure.io.gryo.TypeRegistration;
-import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
-import org.apache.tinkerpop.gremlin.structure.util.star.StarGraph;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.*;
-
-/**
- * A spark.kryo.registrator implementation that installs TinkerPop types.
- * This is intended for use with spark.serializer=KryoSerializer, not GryoSerializer.
- */
-public class TinkerPopKryoRegistrator implements KryoRegistrator {
-
-    private static final Logger log = LoggerFactory.getLogger(TinkerPopKryoRegistrator.class);
-
-    @Override
-    public void registerClasses(Kryo kryo) {
-        registerClasses(kryo, Collections.emptyMap(), Collections.emptySet());
-    }
-
-    /**
-     * Register TinkerPop's classes with the supplied {@link Kryo} instance
-     * while honoring optional overrides and optional class blacklist ("blackset"?).
-     *
-     * @param kryo the Kryo serializer instance with which to register types
-     * @param serializerOverrides serializer mappings that override this class's defaults
-     * @param blacklist classes which should not be registered at all, even if there is an override entry
-     *                  or if they would be registered by this class by default (does not affect Kryo's
-     *                  built-in registrations, e.g. String.class).
-     */
-    public void registerClasses(Kryo kryo, Map<Class<?>, Serializer<?>> serializerOverrides, Set<Class<?>> blacklist) {
-        // Apply TinkerPop type registrations copied from GyroSerializer's constructor
-        for (Map.Entry<Class<?>, Serializer<?>> ent : getExtraRegistrations().entrySet()) {
-            Class<?> targetClass = ent.getKey();
-            Serializer<?> ser = ent.getValue();
-
-            // Is this class blacklisted?  Skip it. (takes precedence over serializerOverrides)
-            if (blacklist.contains(targetClass)) {
-                log.debug("Not registering serializer for {} (blacklisted)", targetClass);
-                continue;
-            }
-
-            if (checkForAndApplySerializerOverride(serializerOverrides, kryo, targetClass)) {
-                // do nothing but skip the remaining else(-if) clauses
-            } else if (null == ser) {
-                log.debug("Registering {} with default serializer", targetClass);
-                kryo.register(targetClass);
-            } else {
-                log.debug("Registering {} with serializer {}", targetClass, ser);
-                kryo.register(targetClass, ser);
-            }
-        }
-
-        Set<Class<?>> shimmedClassesFromGryoMapper = new HashSet<>();
-
-        // Apply GryoMapper's default registrations
-        for (TypeRegistration<?> tr : GryoMapper.build().create().getTypeRegistrations()) {
-            // Is this class blacklisted?  Skip it. (takes precedence over serializerOverrides)
-            if (blacklist.contains(tr.getTargetClass())) {
-                log.debug("Not registering serializer for {} (blacklisted)", tr.getTargetClass());
-                continue;
-            }
-
-            final org.apache.tinkerpop.shaded.kryo.Serializer<?> shadedSerializer = tr.getShadedSerializer();
-            final SerializerShim<?> serializerShim = tr.getSerializerShim();
-            final java.util.function.Function<
-                    org.apache.tinkerpop.shaded.kryo.Kryo,
-                    org.apache.tinkerpop.shaded.kryo.Serializer> functionOfShadedKryo = tr.getFunctionOfShadedKryo();
-
-            // Apply overrides with the highest case-precedence
-            if (checkForAndApplySerializerOverride(serializerOverrides, kryo, tr.getTargetClass())) {
-                // do nothing but skip the remaining else(-if) clauses
-            } else if (null != shadedSerializer) {
-                if (shadedSerializer.getClass().equals(org.apache.tinkerpop.shaded.kryo.serializers.JavaSerializer.class)) {
-                    // Convert GryoMapper's shaded JavaSerializer mappings to their unshaded equivalents
-                    log.debug("Registering {} with JavaSerializer", tr.getTargetClass());
-                    kryo.register(tr.getTargetClass(), new JavaSerializer());
-                } else {
-                    // There's supposed to be a check in GryoMapper that prevents this from happening
-                    log.error("GryoMapper's default serialization registration for {} is a {}. " +
-                              "This is probably a bug in TinkerPop (this is not a valid default registration). " +
-                              "I am configuring Spark to use Kryo's default serializer for this class, " +
-                              "but this may cause serialization failures at runtime.",
-                              tr.getTargetClass(),
-                              org.apache.tinkerpop.shaded.kryo.Serializer.class.getCanonicalName());
-                    kryo.register(tr.getTargetClass());
-                }
-            } else if (null != serializerShim) {
-                // Wrap shim serializers in an adapter for Spark's unshaded Kryo
-                log.debug("Registering {} to serializer shim {} (serializer shim {})",
-                        tr.getTargetClass(), serializerShim, serializerShim.getClass());
-                kryo.register(tr.getTargetClass(), new UnshadedSerializerAdapter<>(serializerShim));
-                shimmedClassesFromGryoMapper.add(tr.getTargetClass());
-            } else if (null != functionOfShadedKryo) {
-                // As with shaded serializers, there's supposed to be a check in GryoMapper that prevents this from happening
-                log.error("GryoMapper's default serialization registration for {} is a Function<{},{}>.  " +
-                          "This is probably a bug in TinkerPop (this is not a valid default registration). " +
-                          "I am configuring Spark to use Kryo's default serializer instead of this function, " +
-                          "but this may cause serialization failures at runtime.",
-                          tr.getTargetClass(),
-                          org.apache.tinkerpop.shaded.kryo.Kryo.class.getCanonicalName(),
-                          org.apache.tinkerpop.shaded.kryo.Serializer.class.getCanonicalName());
-                kryo.register(tr.getTargetClass());
-            } else {
-                // Register all other classes with the default behavior (FieldSerializer)
-                log.debug("Registering {} with default serializer", tr.getTargetClass());
-                kryo.register(tr.getTargetClass());
-            }
-        }
-
-        // StarGraph's shim serializer is especially important on Spark for efficiency reasons,
-        // so log a warning if we failed to register it somehow
-        if (!shimmedClassesFromGryoMapper.contains(StarGraph.class)) {
-            log.warn("No SerializerShim found for StarGraph");
-        }
-    }
-
-    private LinkedHashMap<Class<?>, Serializer<?>> getExtraRegistrations() {
-
-        /* The map returned by this method MUST have a fixed iteration order!
-         *
-         * The order itself is irrelevant, so long as it is completely stable at runtime.
-         *
-         * LinkedHashMap satisfies this requirement (its contract specifies
-         * iteration in key-insertion-order).
-         */
-
-        LinkedHashMap<Class<?>, Serializer<?>> m = new LinkedHashMap<>();
-        // The following entries were copied from GryoSerializer's constructor
-        // This could be turned into a static collection on GryoSerializer to avoid
-        // duplication, but it would be a bit cumbersome to do so without disturbing
-        // the ordering of the existing entries in that constructor, since not all
-        // of the entries are for TinkerPop (and the ordering is significant).
-        m.put(MessagePayload.class, null);
-        m.put(ViewIncomingPayload.class, null);
-        m.put(ViewOutgoingPayload.class, null);
-        m.put(ViewPayload.class, null);
-        m.put(VertexWritable.class, new UnshadedSerializerAdapter<>(new VertexWritableSerializer()));
-        m.put(ObjectWritable.class, new UnshadedSerializerAdapter<>(new ObjectWritableSerializer<>()));
-
-        return m;
-    }
-
-    private boolean checkForAndApplySerializerOverride(Map<Class<?>, Serializer<?>> serializerOverrides,
-                                                       Kryo kryo, Class<?> targetClass) {
-        if (serializerOverrides.containsKey(targetClass)) {
-            Serializer<?> ser = serializerOverrides.get(targetClass);
-            if (null == ser) {
-                // null means use Kryo's default serializer
-                log.debug("Registering {} with default serializer per overrides", targetClass);
-                kryo.register(targetClass);
-            } else {
-                // nonnull means use that serializer
-                log.debug("Registering {} with serializer {} per overrides", targetClass, ser);
-                kryo.register(targetClass, ser);
-            }
-            return true;
-        }
-        return false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/88ac3047/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java
index a524a97..95ebb0d 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java
@@ -98,7 +98,7 @@ public class UnshadedKryoShimService implements KryoShimService {
 
     @Override
     public int getPriority() {
-        return 50;
+        return -50;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/88ac3047/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 cf76027..85552ce 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
@@ -18,6 +18,7 @@
  */
 package org.apache.tinkerpop.gremlin.spark.process.computer;
 
+import org.apache.spark.serializer.KryoSerializer;
 import org.apache.tinkerpop.gremlin.GraphProvider;
 import org.apache.tinkerpop.gremlin.LoadGraphWith;
 import org.apache.tinkerpop.gremlin.groovy.util.SugarTestHelper;
@@ -37,6 +38,7 @@ import org.apache.tinkerpop.gremlin.spark.structure.Spark;
 import org.apache.tinkerpop.gremlin.spark.structure.io.PersistedOutputRDD;
 import org.apache.tinkerpop.gremlin.spark.structure.io.SparkContextStorageCheck;
 import org.apache.tinkerpop.gremlin.spark.structure.io.ToyGraphInputRDD;
+import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.GryoRegistrator;
 import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.GryoSerializer;
 import org.apache.tinkerpop.gremlin.structure.Graph;
 
@@ -79,9 +81,14 @@ public final class SparkHadoopGraphProvider extends HadoopGraphProvider {
 
         config.put(Constants.GREMLIN_HADOOP_DEFAULT_GRAPH_COMPUTER, SparkGraphComputer.class.getCanonicalName());
 
-        /// spark configuration
+
         config.put("spark.master", "local[4]");
-        config.put("spark.serializer", GryoSerializer.class.getCanonicalName());
+        if (false) {
+            config.put("spark.serializer", GryoSerializer.class.getCanonicalName());
+        } else {
+            config.put("spark.serializer", KryoSerializer.class.getCanonicalName());
+            config.put("spark.kryo.registrator", GryoRegistrator.class.getCanonicalName());
+        }
         config.put("spark.kryo.registrationRequired", true);
         return config;
     }


[05/34] incubator-tinkerpop git commit: Kryo shim refactoring and documentation

Posted by ok...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphSerializer.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphSerializer.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphSerializer.java
index d5ba90d..431e1eb 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphSerializer.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphSerializer.java
@@ -27,11 +27,21 @@ import org.apache.tinkerpop.gremlin.structure.Direction;
 import org.apache.tinkerpop.gremlin.structure.Edge;
 import org.apache.tinkerpop.gremlin.structure.T;
 import org.apache.tinkerpop.gremlin.structure.VertexProperty;
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.InputShim;
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShim;
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.OutputShim;
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.SerializerShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.InputShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.OutputShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
 
+/**
+ * Kryo serializer for {@link StarGraph}.  Implements an internal versioning capability for backward compatibility.
+ * The single byte at the front of the serialization stream denotes the version.  That version can be used to choose
+ * the correct deserialization mechanism.  The limitation is that this versioning won't help with backward
+ * compatibility for custom serializers from providers.  Providers should be encouraged to write their serializers
+ * with backward compatibility in mind.
+ *
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ * @author Stephen Mallette (http://stephen.genoprime.com)
+ */
 public class StarGraphSerializer implements SerializerShim<StarGraph> {
 
     private final Direction edgeDirectionToSerialize;

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
index 2053280..c19b914 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
@@ -18,7 +18,7 @@
  */
 package org.apache.tinkerpop.gremlin.hadoop.structure.io;
 
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShimService;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimService;
 import org.apache.tinkerpop.shaded.kryo.Kryo;
 import org.apache.tinkerpop.shaded.kryo.io.Input;
 import org.apache.tinkerpop.shaded.kryo.io.Output;

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
index e7a38a5..88f7ee1 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/ObjectWritable.java
@@ -21,11 +21,9 @@ package org.apache.tinkerpop.gremlin.hadoop.structure.io;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.tinkerpop.gremlin.process.computer.MapReduce;
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShimServiceLoader;
-import org.apache.tinkerpop.shaded.kryo.io.Output;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader;
 
 import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritable.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritable.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritable.java
index 7ac8e8c..2252ded 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritable.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/VertexWritable.java
@@ -21,12 +21,11 @@ package org.apache.tinkerpop.gremlin.hadoop.structure.io;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.tinkerpop.gremlin.structure.Vertex;
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShimServiceLoader;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader;
 import org.apache.tinkerpop.gremlin.structure.util.ElementHelper;
 import org.apache.tinkerpop.gremlin.structure.util.star.StarGraph;
 
 import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/hadoop-gremlin/src/main/resources/META-INF/services/org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimService
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/resources/META-INF/services/org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimService b/hadoop-gremlin/src/main/resources/META-INF/services/org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimService
new file mode 100644
index 0000000..0b27e72
--- /dev/null
+++ b/hadoop-gremlin/src/main/resources/META-INF/services/org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimService
@@ -0,0 +1 @@
+org.apache.tinkerpop.gremlin.hadoop.structure.io.HadoopPoolShimService # HadoopPools provides/caches instances of TinkerPop's shaded Kryo

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/hadoop-gremlin/src/main/resources/META-INF/services/org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShimService
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/resources/META-INF/services/org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShimService b/hadoop-gremlin/src/main/resources/META-INF/services/org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShimService
deleted file mode 100644
index 0b27e72..0000000
--- a/hadoop-gremlin/src/main/resources/META-INF/services/org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShimService
+++ /dev/null
@@ -1 +0,0 @@
-org.apache.tinkerpop.gremlin.hadoop.structure.io.HadoopPoolShimService # HadoopPools provides/caches instances of TinkerPop's shaded Kryo

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/TinkerPopKryoRegistrator.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/TinkerPopKryoRegistrator.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/TinkerPopKryoRegistrator.java
new file mode 100644
index 0000000..4c99e70
--- /dev/null
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/TinkerPopKryoRegistrator.java
@@ -0,0 +1,121 @@
+/*
+ * 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;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.serializers.JavaSerializer;
+import com.google.common.base.Preconditions;
+import org.apache.spark.serializer.KryoRegistrator;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
+import org.apache.tinkerpop.gremlin.process.traversal.step.map.GroupStep;
+import org.apache.tinkerpop.gremlin.process.traversal.step.map.OrderGlobalStep;
+import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalExplanation;
+import org.apache.tinkerpop.gremlin.spark.process.computer.payload.MessagePayload;
+import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewIncomingPayload;
+import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewOutgoingPayload;
+import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewPayload;
+import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.ObjectWritableSerializer;
+import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.VertexWritableSerializer;
+import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.kryoshim.unshaded.UnshadedSerializerAdapter;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoMapper;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.TypeRegistration;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
+import org.apache.tinkerpop.gremlin.structure.util.star.StarGraph;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A spark.kryo.registrator implementation that installs TinkerPop types.
+ * This is intended for use with spark.serializer=KryoSerializer, not GryoSerializer.
+ */
+public class TinkerPopKryoRegistrator implements KryoRegistrator {
+
+    private static final Logger log = LoggerFactory.getLogger(TinkerPopKryoRegistrator.class);
+
+    @Override
+    public void registerClasses(Kryo kryo) {
+        // TinkerPop type registrations copied from GyroSerializer's constructor
+        kryo.register(MessagePayload.class);
+        kryo.register(ViewIncomingPayload.class);
+        kryo.register(ViewOutgoingPayload.class);
+        kryo.register(ViewPayload.class);
+        kryo.register(VertexWritable.class, new UnshadedSerializerAdapter<>(new VertexWritableSerializer()));
+        kryo.register(ObjectWritable.class, new UnshadedSerializerAdapter<>(new ObjectWritableSerializer<>()));
+
+        Set<Class<?>> shimmedClasses = new HashSet<>();
+
+        Set<Class<?>> javaSerializationClasses = new HashSet<>();
+
+        // Copy GryoMapper's default registrations
+        for (TypeRegistration<?> tr : GryoMapper.build().create().getTypeRegistrations()) {
+            // Special case for JavaSerializer, which is generally implemented in terms of TinkerPop's
+            // problematic static GryoMapper/GryoSerializer pool (these are handled below the loop)
+            org.apache.tinkerpop.shaded.kryo.Serializer<?> shadedSerializer = tr.getShadedSerializer();
+            SerializerShim<?> serializerShim = tr.getSerializerShim();
+            if (null != shadedSerializer &&
+                    shadedSerializer.getClass().equals(org.apache.tinkerpop.shaded.kryo.serializers.JavaSerializer.class)) {
+                javaSerializationClasses.add(tr.getTargetClass());
+            } else if (null != serializerShim) {
+                log.debug("Registering class {} to serializer shim {} (serializer shim class {})",
+                        tr.getTargetClass(), serializerShim, serializerShim.getClass());
+                kryo.register(tr.getTargetClass(), new UnshadedSerializerAdapter<>(serializerShim));
+                shimmedClasses.add(tr.getTargetClass());
+            } else {
+                // Register with the default behavior (FieldSerializer)
+                log.debug("Registering class {} with default serializer", tr.getTargetClass());
+                kryo.register(tr.getTargetClass());
+            }
+        }
+
+        Map<Class<?>, Serializer<?>> javaSerializerReplacements = new HashMap<>();
+        javaSerializerReplacements.put(GroupStep.GroupBiOperator.class, new JavaSerializer());
+        javaSerializerReplacements.put(OrderGlobalStep.OrderBiOperator.class, null);
+        javaSerializerReplacements.put(TraversalExplanation.class, null);
+
+        for (Map.Entry<Class<?>, Serializer<?>> e : javaSerializerReplacements.entrySet()) {
+            Class<?> c = e.getKey();
+            Serializer<?> s = e.getValue();
+
+            if (javaSerializationClasses.remove(c)) {
+                if (null != s) {
+                    log.debug("Registering class {} with serializer {}", c, s);
+                    kryo.register(c, s);
+                } else {
+                    log.debug("Registering class {} with default serializer", c);
+                    kryo.register(c);
+                }
+            } else {
+                log.debug("Registering class {} with JavaSerializer", c);
+                kryo.register(c, new JavaSerializer());
+            }
+        }
+
+        // We really care about StarGraph's shim serializer, so make sure we registered it
+        if (!shimmedClasses.contains(StarGraph.class)) {
+            log.warn("No SerializerShim found for StarGraph");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/ObjectWritableSerializer.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/ObjectWritableSerializer.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/ObjectWritableSerializer.java
index 21cbc60..4ceb045 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/ObjectWritableSerializer.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/ObjectWritableSerializer.java
@@ -20,6 +20,10 @@
 package org.apache.tinkerpop.gremlin.spark.structure.io.gryo;
 
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.InputShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.OutputShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
 import org.apache.tinkerpop.shaded.kryo.Kryo;
 import org.apache.tinkerpop.shaded.kryo.Serializer;
 import org.apache.tinkerpop.shaded.kryo.io.Input;
@@ -28,16 +32,16 @@ import org.apache.tinkerpop.shaded.kryo.io.Output;
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
-public final class ObjectWritableSerializer<T> extends Serializer<ObjectWritable<T>> {
+public final class ObjectWritableSerializer<T> implements SerializerShim<ObjectWritable<T>> {
 
     @Override
-    public void write(final Kryo kryo, final Output output, final ObjectWritable<T> objectWritable) {
-        kryo.writeClassAndObject(output, objectWritable.get());
+    public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, ObjectWritable<T> starGraph) {
+        kryo.writeClassAndObject(output, starGraph.get());
         output.flush();
     }
 
     @Override
-    public ObjectWritable<T> read(final Kryo kryo, final Input input, final Class<ObjectWritable<T>> clazz) {
+    public <I extends InputShim> ObjectWritable<T> read(KryoShim<I, ?> kryo, I input, Class<ObjectWritable<T>> clazz) {
         return new ObjectWritable(kryo.readClassAndObject(input));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/VertexWritableSerializer.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/VertexWritableSerializer.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/VertexWritableSerializer.java
index 97891f3..f3c1b15 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/VertexWritableSerializer.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/VertexWritableSerializer.java
@@ -20,6 +20,10 @@
 package org.apache.tinkerpop.gremlin.spark.structure.io.gryo;
 
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.InputShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.OutputShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
 import org.apache.tinkerpop.gremlin.structure.util.star.StarGraph;
 import org.apache.tinkerpop.shaded.kryo.Kryo;
 import org.apache.tinkerpop.shaded.kryo.Serializer;
@@ -29,14 +33,16 @@ import org.apache.tinkerpop.shaded.kryo.io.Output;
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
-public final class VertexWritableSerializer extends Serializer<VertexWritable> {
+public final class VertexWritableSerializer implements SerializerShim<VertexWritable> {
+
     @Override
-    public void write(final Kryo kryo, final Output output, final VertexWritable vertexWritable) {
+    public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, VertexWritable vertexWritable) {
         kryo.writeObject(output, vertexWritable.get().graph());
+        output.flush();
     }
 
     @Override
-    public VertexWritable read(final Kryo kryo, final Input input, final Class<VertexWritable> aClass) {
+    public <I extends InputShim> VertexWritable read(KryoShim<I, ?> kryo, I input, Class<VertexWritable> clazz) {
         return new VertexWritable(kryo.readObject(input, StarGraph.class).getStarVertex());
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedInputAdapter.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedInputAdapter.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedInputAdapter.java
new file mode 100644
index 0000000..c533af7
--- /dev/null
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedInputAdapter.java
@@ -0,0 +1,78 @@
+/*
+ * 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.
+ */
+
+/**
+ * Copyright DataStax, Inc.
+ *
+ * Please see the included license file for details.
+ */
+package org.apache.tinkerpop.gremlin.spark.structure.io.gryo.kryoshim.unshaded;
+
+import com.esotericsoftware.kryo.io.Input;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.InputShim;
+
+public class UnshadedInputAdapter implements InputShim
+{
+
+    private final Input unshadedInput;
+
+    public UnshadedInputAdapter(Input unshadedInput)
+    {
+        this.unshadedInput = unshadedInput;
+    }
+
+    Input getUnshadedInput()
+    {
+        return unshadedInput;
+    }
+
+    @Override
+    public byte readByte()
+    {
+        return unshadedInput.readByte();
+    }
+
+    @Override
+    public byte[] readBytes(int size) {
+        return unshadedInput.readBytes(size);
+    }
+
+    @Override
+    public String readString()
+    {
+        return unshadedInput.readString();
+    }
+
+    @Override
+    public long readLong()
+    {
+        return unshadedInput.readLong();
+    }
+
+    @Override
+    public int readInt() {
+        return unshadedInput.readInt();
+    }
+
+    @Override
+    public double readDouble()
+    {
+        return unshadedInput.readDouble();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoAdapter.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoAdapter.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoAdapter.java
new file mode 100644
index 0000000..b14abe0
--- /dev/null
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoAdapter.java
@@ -0,0 +1,74 @@
+/*
+ * 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.
+ */
+
+/**
+ * Copyright DataStax, Inc.
+ *
+ * Please see the included license file for details.
+ */
+package org.apache.tinkerpop.gremlin.spark.structure.io.gryo.kryoshim.unshaded;
+
+import com.esotericsoftware.kryo.Kryo;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShim;
+
+public class UnshadedKryoAdapter implements KryoShim<UnshadedInputAdapter, UnshadedOutputAdapter>
+{
+    private final Kryo unshadedKryo;
+
+    public UnshadedKryoAdapter(Kryo unshadedKryo)
+    {
+        this.unshadedKryo = unshadedKryo;
+    }
+
+    @Override
+    public <T> T readObject(UnshadedInputAdapter input, Class<T> type)
+    {
+        return unshadedKryo.readObject(input.getUnshadedInput(), type);
+    }
+
+    @Override
+    public Object readClassAndObject(UnshadedInputAdapter input)
+    {
+        return unshadedKryo.readClassAndObject(input.getUnshadedInput());
+    }
+
+    @Override
+    public void writeObject(UnshadedOutputAdapter output, Object object)
+    {
+        unshadedKryo.writeObject(output.getUnshadedOutput(), object);
+    }
+
+    @Override
+    public void writeClassAndObject(UnshadedOutputAdapter output, Object object)
+    {
+        unshadedKryo.writeClassAndObject(output.getUnshadedOutput(), object);
+    }
+
+    @Override
+    public <T> T readObjectOrNull(UnshadedInputAdapter input, Class<T> type)
+    {
+        return unshadedKryo.readObjectOrNull(input.getUnshadedInput(), type);
+    }
+
+    @Override
+    public void writeObjectOrNull(UnshadedOutputAdapter output, Object object, Class type)
+    {
+        unshadedKryo.writeObjectOrNull(output.getUnshadedOutput(), object, type);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java
new file mode 100644
index 0000000..d0411e8
--- /dev/null
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java
@@ -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.
+ */
+
+/**
+ * Copyright DataStax, Inc.
+ * <p>
+ * Please see the included license file for details.
+ */
+package org.apache.tinkerpop.gremlin.spark.structure.io.gryo.kryoshim.unshaded;
+
+import com.twitter.chill.KryoInstantiator;
+import com.twitter.chill.KryoPool;
+import com.twitter.chill.SerDeState;
+import org.apache.spark.SparkConf;
+import org.apache.spark.serializer.KryoSerializer;
+import org.apache.tinkerpop.gremlin.spark.structure.io.TinkerPopKryoRegistrator;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+public class UnshadedKryoShimService implements KryoShimService {
+
+    public static final String SPARK_KRYO_POOL_SIZE_SYSTEM_PROPERTY = "tinkerpop.kryo.poolsize";
+
+    private static final Logger log = LoggerFactory.getLogger(UnshadedKryoShimService.class);
+    private static final int SPARK_KRYO_POOL_SIZE_DEFAULT = 8;
+
+    private final KryoSerializer sparkKryoSerializer;
+    private final KryoPool kryoPool;
+
+    public UnshadedKryoShimService() {
+        this(TinkerPopKryoRegistrator.class.getCanonicalName(), getDefaultKryoPoolSize());
+    }
+
+    public UnshadedKryoShimService(String sparkKryoRegistratorClassname, int kryoPoolSize) {
+        SparkConf sparkConf = new SparkConf();
+        sparkConf.set("spark.serializer", KryoSerializer.class.getCanonicalName());
+        sparkConf.set("spark.kryo.registrator", sparkKryoRegistratorClassname);
+        sparkKryoSerializer = new KryoSerializer(sparkConf);
+        kryoPool = KryoPool.withByteArrayOutputStream(kryoPoolSize, new KryoInstantiator());
+    }
+
+    @Override
+    public Object readClassAndObject(InputStream source) {
+        SerDeState sds = null;
+        try {
+            sds = kryoPool.borrow();
+
+            sds.setInput(source);
+
+            return sds.readClassAndObject();
+        } finally {
+            kryoPool.release(sds);
+        }
+    }
+
+    @Override
+    public void writeClassAndObject(Object o, OutputStream sink) {
+        SerDeState sds = null;
+        try {
+            sds = kryoPool.borrow();
+
+            sds.writeClassAndObject(o); // this writes to an internal buffer
+
+            sds.writeOutputTo(sink); // this copies the internal buffer to sink
+
+            sink.flush();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        } finally {
+            kryoPool.release(sds);
+        }
+    }
+
+    @Override
+    public int getPriority() {
+        return 1024;
+    }
+
+    private static int getDefaultKryoPoolSize() {
+        String raw = System.getProperty(SPARK_KRYO_POOL_SIZE_SYSTEM_PROPERTY);
+
+        int size = SPARK_KRYO_POOL_SIZE_DEFAULT;
+        try {
+            size = Integer.valueOf(raw);
+            log.info("Setting kryo pool size to {} according to system property {}", size,
+                    SPARK_KRYO_POOL_SIZE_SYSTEM_PROPERTY);
+        } catch (NumberFormatException e) {
+            log.error("System property {}={} could not be parsed as an integer, using default value {}",
+                    SPARK_KRYO_POOL_SIZE_SYSTEM_PROPERTY, raw, size, e);
+        }
+
+        return size;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedOutputAdapter.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedOutputAdapter.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedOutputAdapter.java
new file mode 100644
index 0000000..9cc59d4
--- /dev/null
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedOutputAdapter.java
@@ -0,0 +1,83 @@
+/*
+ * 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.
+ */
+
+/**
+ * Copyright DataStax, Inc.
+ *
+ * Please see the included license file for details.
+ */
+package org.apache.tinkerpop.gremlin.spark.structure.io.gryo.kryoshim.unshaded;
+
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.OutputShim;
+
+public class UnshadedOutputAdapter implements OutputShim
+{
+    private final Output unshadedOutput;
+
+    public UnshadedOutputAdapter(Output unshadedOutput)
+    {
+        this.unshadedOutput = unshadedOutput;
+    }
+
+    Output getUnshadedOutput()
+    {
+        return unshadedOutput;
+    }
+
+    @Override
+    public void writeByte(byte b)
+    {
+        unshadedOutput.writeByte(b);
+    }
+
+    @Override
+    public void writeBytes(byte[] array, int offset, int count) {
+        unshadedOutput.writeBytes(array, offset, count);
+    }
+
+    @Override
+    public void writeString(String s)
+    {
+        unshadedOutput.writeString(s);
+    }
+
+    @Override
+    public void writeLong(long l)
+    {
+        unshadedOutput.writeLong(l);
+    }
+
+    @Override
+    public void writeInt(int i) {
+        unshadedOutput.writeInt(i);
+    }
+
+    @Override
+    public void writeDouble(double d)
+    {
+        unshadedOutput.writeDouble(d);
+    }
+
+    @Override
+    public void flush()
+    {
+        unshadedOutput.flush();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedSerializerAdapter.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedSerializerAdapter.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedSerializerAdapter.java
new file mode 100644
index 0000000..efc9a4f
--- /dev/null
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedSerializerAdapter.java
@@ -0,0 +1,57 @@
+/*
+ * 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.
+ */
+
+/**
+ * Copyright DataStax, Inc.
+ *
+ * Please see the included license file for details.
+ */
+package org.apache.tinkerpop.gremlin.spark.structure.io.gryo.kryoshim.unshaded;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
+
+public class UnshadedSerializerAdapter<T> extends Serializer<T>
+{
+
+    SerializerShim<T> serializer;
+
+    public UnshadedSerializerAdapter(SerializerShim<T> serializer) {
+        this.serializer = serializer;
+        setImmutable(this.serializer.isImmutable());
+    }
+
+    @Override
+    public void write(Kryo kryo, Output output, T t) {
+        UnshadedKryoAdapter shadedKryoAdapter = new UnshadedKryoAdapter(kryo);
+        UnshadedOutputAdapter shadedOutputAdapter = new UnshadedOutputAdapter(output);
+        serializer.write(shadedKryoAdapter, shadedOutputAdapter, t);
+    }
+
+    @Override
+    public T read(Kryo kryo, Input input, Class<T> aClass)
+    {
+        UnshadedKryoAdapter shadedKryoAdapter = new UnshadedKryoAdapter(kryo);
+        UnshadedInputAdapter shadedInputAdapter = new UnshadedInputAdapter(input);
+        return serializer.read(shadedKryoAdapter, shadedInputAdapter, aClass);
+    }
+}


[28/34] incubator-tinkerpop git commit: made SparkHadoopGraphProvider more robust.

Posted by ok...@apache.org.
made SparkHadoopGraphProvider more robust.


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

Branch: refs/heads/TINKERPOP-1278
Commit: 08040f8083aecb5b18eafbdf367f93b72a5356f0
Parents: 359f04b
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Mon Jun 6 18:47:45 2016 -0600
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Mon Jun 6 18:47:45 2016 -0600

----------------------------------------------------------------------
 .../spark/process/computer/SparkHadoopGraphProvider.java      | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/08040f80/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 7737d1e..b1f610d 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
@@ -33,8 +33,10 @@ import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversalSo
 import org.apache.tinkerpop.gremlin.process.traversal.engine.ComputerTraversalEngine;
 import org.apache.tinkerpop.gremlin.process.traversal.step.map.GroovyPageRankTest;
 import org.apache.tinkerpop.gremlin.process.traversal.step.map.GroovyPeerPressureTest;
+import org.apache.tinkerpop.gremlin.process.traversal.step.map.GroovyProgramTest;
 import org.apache.tinkerpop.gremlin.process.traversal.step.map.PageRankTest;
 import org.apache.tinkerpop.gremlin.process.traversal.step.map.PeerPressureTest;
+import org.apache.tinkerpop.gremlin.process.traversal.step.map.ProgramTest;
 import org.apache.tinkerpop.gremlin.spark.structure.Spark;
 import org.apache.tinkerpop.gremlin.spark.structure.io.PersistedOutputRDD;
 import org.apache.tinkerpop.gremlin.spark.structure.io.SparkContextStorageCheck;
@@ -58,10 +60,13 @@ public final 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, true);  // this makes the test suite go really fast
+        config.put(Constants.GREMLIN_SPARK_PERSIST_CONTEXT,
+                !test.equals(ProgramTest.Traversals.class) && !test.equals(GroovyProgramTest.class));  // 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 &&
+                !test.equals(ProgramTest.Traversals.class) &&
+                !test.equals(GroovyProgramTest.Traversals.class) &&
                 !test.equals(PageRankTest.Traversals.class) &&
                 !test.equals(GroovyPageRankTest.Traversals.class) &&
                 !test.equals(PeerPressureTest.Traversals.class) &&


[23/34] incubator-tinkerpop git commit: Merge remote-tracking branch 'origin/tp31'

Posted by ok...@apache.org.
Merge remote-tracking branch 'origin/tp31'


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

Branch: refs/heads/TINKERPOP-1278
Commit: 406956db52125dee2fa4f3622fe967f09b1e2413
Parents: 55eb1e6 f7e7df0
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Mon Jun 6 16:18:21 2016 -0400
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Mon Jun 6 16:18:21 2016 -0400

----------------------------------------------------------------------
 .../java/org/apache/tinkerpop/gremlin/server/GremlinServer.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[12/34] incubator-tinkerpop git commit: Merge remote-tracking branch 'origin/tp31'

Posted by ok...@apache.org.
Merge remote-tracking branch 'origin/tp31'


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

Branch: refs/heads/TINKERPOP-1278
Commit: d02562cb7ed795b2c64a59fa02db7bd3700b9c8f
Parents: f24e358 2f4c24c
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Fri Jun 3 20:45:42 2016 -0400
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Fri Jun 3 20:45:42 2016 -0400

----------------------------------------------------------------------
 CHANGELOG.asciidoc                                      |  1 +
 .../gremlin/groovy/engine/GremlinExecutor.java          | 12 +++++-------
 2 files changed, 6 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/d02562cb/CHANGELOG.asciidoc
----------------------------------------------------------------------


[07/34] incubator-tinkerpop git commit: Added another example to "between vertices" recipes CTR

Posted by ok...@apache.org.
Added another example to "between vertices" recipes CTR


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

Branch: refs/heads/TINKERPOP-1278
Commit: cb4fae19785c71f7369cd14f9258483fbda7a3b1
Parents: 2a58313
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Fri Jun 3 07:49:08 2016 -0400
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Fri Jun 3 07:49:08 2016 -0400

----------------------------------------------------------------------
 docs/src/recipes/between-vertices.asciidoc | 14 +++++++++++++-
 1 file changed, 13 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/cb4fae19/docs/src/recipes/between-vertices.asciidoc
----------------------------------------------------------------------
diff --git a/docs/src/recipes/between-vertices.asciidoc b/docs/src/recipes/between-vertices.asciidoc
index 03a4b3c..021e85b 100644
--- a/docs/src/recipes/between-vertices.asciidoc
+++ b/docs/src/recipes/between-vertices.asciidoc
@@ -45,7 +45,19 @@ vertices with identifiers "2" and "3".
 two vertices.
 
 The basic pattern of using `where()` step to find the "other" known vertex can be applied in far more complex
-scenarios. Consider the following schema:
+scenarios. For one such example, consider the following traversal that finds all the paths between a group of defined
+vertices:
+
+[gremlin-groovy,modern]
+----
+ids = [2,4,6].toArray()
+g.V(ids).as("a").
+  repeat(bothE().otherV().simplePath()).times(5).emit(hasId(within(ids))).as("b").
+  filter(select(last,"a","b").by(id).where("a", lt("b"))).
+  path().by().by(label)
+----
+
+For another example, consider the following schema:
 
 image:recipe-job-schema.png[width=750]
 


[06/34] incubator-tinkerpop git commit: Kryo shim refactoring and documentation

Posted by ok...@apache.org.
Kryo shim refactoring and documentation


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

Branch: refs/heads/TINKERPOP-1278
Commit: 218d7909b9c35a4488102f53c444c92be737972c
Parents: ef52869
Author: Dan LaRocque <da...@hopcount.org>
Authored: Fri Jun 3 07:41:50 2016 -0400
Committer: Dan LaRocque <da...@hopcount.org>
Committed: Fri Jun 3 07:41:50 2016 -0400

----------------------------------------------------------------------
 .../gremlin/structure/io/gryo/GryoMapper.java   |  29 +--
 .../structure/io/gryo/TypeRegistration.java     |   2 +-
 .../structure/io/gryo/URISerializer.java        |   8 +-
 .../structure/io/gryo/UUIDSerializer.java       |  12 +-
 .../structure/io/gryo/kryoshim/InputShim.java   |  38 ++++
 .../structure/io/gryo/kryoshim/KryoShim.java    |  41 ++++
 .../io/gryo/kryoshim/KryoShimService.java       |  83 ++++++++
 .../io/gryo/kryoshim/KryoShimServiceLoader.java | 199 +++++++++++++++++++
 .../structure/io/gryo/kryoshim/OutputShim.java  |  40 ++++
 .../io/gryo/kryoshim/SerializerShim.java        |  36 ++++
 .../io/gryo/kryoshim/package-info.java          |  55 +++++
 .../kryoshim/shaded/ShadedInputAdapter.java     |  66 ++++++
 .../gryo/kryoshim/shaded/ShadedKryoAdapter.java |  67 +++++++
 .../kryoshim/shaded/ShadedOutputAdapter.java    |  72 +++++++
 .../shaded/ShadedSerializerAdapter.java         |  54 +++++
 .../io/gryo/kryoshim/shaded/package-info.java   |  25 +++
 .../structure/io/kryoshim/InputShim.java        |  37 ----
 .../gremlin/structure/io/kryoshim/KryoShim.java |  40 ----
 .../structure/io/kryoshim/KryoShimService.java  |  83 --------
 .../io/kryoshim/KryoShimServiceLoader.java      | 137 -------------
 .../structure/io/kryoshim/OutputShim.java       |  41 ----
 .../structure/io/kryoshim/SerializerShim.java   |  35 ----
 .../structure/io/kryoshim/package-info.java     |  54 -----
 .../io/kryoshim/shaded/ShadedInputAdapter.java  |  66 ------
 .../io/kryoshim/shaded/ShadedKryoAdapter.java   |  67 -------
 .../io/kryoshim/shaded/ShadedOutputAdapter.java |  72 -------
 .../shaded/ShadedSerializerAdapter.java         |  54 -----
 .../io/kryoshim/shaded/package-info.java        |  25 ---
 .../util/star/StarGraphGryoSerializer.java      |   9 +-
 .../util/star/StarGraphSerializer.java          |  18 +-
 .../structure/io/HadoopPoolShimService.java     |   2 +-
 .../hadoop/structure/io/ObjectWritable.java     |   4 +-
 .../hadoop/structure/io/VertexWritable.java     |   3 +-
 ...n.structure.io.gryo.kryoshim.KryoShimService |   1 +
 ...remlin.structure.io.kryoshim.KryoShimService |   1 -
 .../structure/io/TinkerPopKryoRegistrator.java  | 121 +++++++++++
 .../io/gryo/ObjectWritableSerializer.java       |  12 +-
 .../io/gryo/VertexWritableSerializer.java       |  12 +-
 .../kryoshim/unshaded/UnshadedInputAdapter.java |  78 ++++++++
 .../kryoshim/unshaded/UnshadedKryoAdapter.java  |  74 +++++++
 .../unshaded/UnshadedKryoShimService.java       | 115 +++++++++++
 .../unshaded/UnshadedOutputAdapter.java         |  83 ++++++++
 .../unshaded/UnshadedSerializerAdapter.java     |  57 ++++++
 43 files changed, 1368 insertions(+), 760 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
index 5bc71da..851b03c 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
@@ -57,8 +57,8 @@ import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.structure.VertexProperty;
 import org.apache.tinkerpop.gremlin.structure.io.IoRegistry;
 import org.apache.tinkerpop.gremlin.structure.io.Mapper;
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.SerializerShim;
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.shaded.ShadedSerializerAdapter;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.shaded.ShadedSerializerAdapter;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedEdge;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedPath;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedProperty;
@@ -70,8 +70,6 @@ import org.apache.tinkerpop.gremlin.structure.util.reference.ReferenceProperty;
 import org.apache.tinkerpop.gremlin.structure.util.reference.ReferenceVertex;
 import org.apache.tinkerpop.gremlin.structure.util.reference.ReferenceVertexProperty;
 import org.apache.tinkerpop.gremlin.structure.util.star.StarGraph;
-import org.apache.tinkerpop.gremlin.structure.util.star.StarGraphGryoSerializer;
-import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 import org.apache.tinkerpop.gremlin.structure.util.star.StarGraphSerializer;
 import org.apache.tinkerpop.shaded.kryo.ClassResolver;
 import org.apache.tinkerpop.shaded.kryo.Kryo;
@@ -115,7 +113,6 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
-import java.util.Optional;
 import java.util.Set;
 import java.util.TimeZone;
 import java.util.TreeMap;
@@ -406,7 +403,7 @@ public final class GryoMapper implements Mapper<Kryo> {
         public Builder addCustom(final Class... custom) {
             if (custom != null && custom.length > 0) {
                 for (Class c : custom) {
-                    addOrOverrideRegistration(GryoTypeReg.of(c, currentSerializationId.getAndIncrement()));
+                    addOrOverrideRegistration(c, id -> GryoTypeReg.of(c, id));
                 }
             }
             return this;
@@ -417,7 +414,7 @@ public final class GryoMapper implements Mapper<Kryo> {
          * a class that is already registered will override that registration.
          */
         public Builder addCustom(final Class clazz, final Serializer serializer) {
-            addOrOverrideRegistration(GryoTypeReg.of(clazz, currentSerializationId.getAndIncrement(), serializer));
+            addOrOverrideRegistration(clazz, id -> GryoTypeReg.of(clazz, id, serializer));
             return this;
         }
 
@@ -425,7 +422,7 @@ public final class GryoMapper implements Mapper<Kryo> {
          * Register custom class to serialize with a custom serialization shim.
          */
         public Builder addCustom(final Class clazz, final SerializerShim serializer) {
-            addOrOverrideRegistration(GryoTypeReg.of(clazz, currentSerializationId.getAndIncrement(), serializer));
+            addOrOverrideRegistration(clazz, id -> GryoTypeReg.of(clazz, id, serializer));
             return this;
         }
 
@@ -434,7 +431,7 @@ public final class GryoMapper implements Mapper<Kryo> {
          * that calling this method for a class that is already registered will override that registration.
          */
         public Builder addCustom(final Class clazz, final Function<Kryo, Serializer> functionOfKryo) {
-            addOrOverrideRegistration(GryoTypeReg.of(clazz, currentSerializationId.getAndIncrement(), functionOfKryo));
+            addOrOverrideRegistration(clazz, id -> GryoTypeReg.of(clazz, id, functionOfKryo));
             return this;
         }
 
@@ -490,16 +487,24 @@ public final class GryoMapper implements Mapper<Kryo> {
             return new GryoMapper(this);
         }
 
-        private <T> void addOrOverrideRegistration(TypeRegistration<T> newRegistration) {
+        private <T> void addOrOverrideRegistration(Class<?> clazz, Function<Integer, TypeRegistration<T>> newRegistrationBuilder) {
             Iterator<TypeRegistration<?>> iter = typeRegistrations.iterator();
+            Integer registrationId = null;
             while (iter.hasNext()) {
                 TypeRegistration<?> existingRegistration = iter.next();
-                if (existingRegistration.getTargetClass().equals(newRegistration.getTargetClass())) {
+                if (existingRegistration.getTargetClass().equals(clazz)) {
+                    // when overridding a registration, use the old id
+                    registrationId = existingRegistration.getId();
+                    // remove the old registration (we install its override below)
                     iter.remove();
                     break;
                 }
             }
-            typeRegistrations.add(newRegistration);
+            if (null == registrationId) {
+                // when not overridding a registration, get an id from the counter
+                registrationId = currentSerializationId.getAndIncrement();
+            }
+            typeRegistrations.add(newRegistrationBuilder.apply(registrationId));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java
index 5ca3f31..ef105ce 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java
@@ -18,7 +18,7 @@
  */
 package org.apache.tinkerpop.gremlin.structure.io.gryo;
 
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.SerializerShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
 import org.apache.tinkerpop.shaded.kryo.Kryo;
 import org.apache.tinkerpop.shaded.kryo.Serializer;
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/URISerializer.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/URISerializer.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/URISerializer.java
index de08061..a65bcea 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/URISerializer.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/URISerializer.java
@@ -18,10 +18,10 @@
  */
 package org.apache.tinkerpop.gremlin.structure.io.gryo;
 
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.InputShim;
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShim;
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.OutputShim;
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.SerializerShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.InputShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.OutputShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
 
 import java.net.URI;
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/UUIDSerializer.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/UUIDSerializer.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/UUIDSerializer.java
index b86ddc2..d1d59d3 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/UUIDSerializer.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/UUIDSerializer.java
@@ -18,14 +18,10 @@
  */
 package org.apache.tinkerpop.gremlin.structure.io.gryo;
 
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.InputShim;
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShim;
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.OutputShim;
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.SerializerShim;
-import org.apache.tinkerpop.shaded.kryo.Kryo;
-import org.apache.tinkerpop.shaded.kryo.Serializer;
-import org.apache.tinkerpop.shaded.kryo.io.Input;
-import org.apache.tinkerpop.shaded.kryo.io.Output;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.InputShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.OutputShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
 
 import java.util.UUID;
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/InputShim.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/InputShim.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/InputShim.java
new file mode 100644
index 0000000..23bec16
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/InputShim.java
@@ -0,0 +1,38 @@
+/*
+ * 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.structure.io.gryo.kryoshim;
+
+/**
+ * A minimal {@link org.apache.tinkerpop.shaded.kryo.io.Input}-like abstraction.
+ * See that class for method documentation.
+ */
+public interface InputShim {
+
+    byte readByte();
+
+    byte[] readBytes(int size);
+
+    String readString();
+
+    long readLong();
+
+    int readInt();
+
+    double readDouble();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShim.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShim.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShim.java
new file mode 100644
index 0000000..a0f3f87
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShim.java
@@ -0,0 +1,41 @@
+/*
+ * 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.structure.io.gryo.kryoshim;
+
+/**
+ * A minimal {@link org.apache.tinkerpop.shaded.kryo.Kryo}-like abstraction.
+ * See that class for method documentation.
+ *
+ * @param <I> this interface's complementary InputShim
+ * @param <O> this interface's complementary OutputShim
+ */
+public interface KryoShim<I extends InputShim, O extends OutputShim> {
+
+    <T> T readObject(I input, Class<T> type);
+
+    Object readClassAndObject(I input);
+
+    void writeObject(O output, Object object);
+
+    void writeClassAndObject(O output, Object object);
+
+    <T> T readObjectOrNull(I input, Class<T> type);
+
+    void writeObjectOrNull(O output, Object object, Class type);
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java
new file mode 100644
index 0000000..959605c
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java
@@ -0,0 +1,83 @@
+/*
+ * 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.structure.io.gryo.kryoshim;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This interface exists to decouple HadoopPools from TinkerPop's shaded Kryo.
+ * <p>
+ * VertexWritable and ObjectWritable formerly implemented Serializable by
+ * resorting to statically-pooled shaded Kryo instances maintained by the HadoopPools class.
+ * This is awkward because those shaded Kryo instances require class registration by default.
+ * <p>
+ * Consider what happens with custom property datatypes reachable from the reference graph rooted at an ObjectWritable
+ * or VertexWritable instance.  It is not enough for these property classes to merely implement
+ * Serializable, though one think that from skimming ObjectWritable/VertexWritable.  Those classes
+ * must also register with TinkerPop's internal, shaded Kryo instances as maintained by HadoopPools,
+ * or else configure those instances to accept unregistered classes.
+ * Otherwise, TinkerPop's shaded Kryo will refuse to serialize those properties (even though
+ * they implement Serializable, and even though the user might think they are only using
+ * Java's standard Serialization mechanism!).
+ * <p>
+ * By hiding the mechanics of serialization behind this interface instead of hardcoding it in
+ * HadoopPools, the user can decide how to implement serialization for ObjectWritable/VertexWritable
+ * (and whatever other classes in TinkerPop decide to implement Serializable but then delegate
+ * all of the implementation details, like ObjectWritable/VertexWritable do now).
+ */
+public interface KryoShimService {
+
+    /**
+     * Deserializes an object from an input stream.
+     *
+     * @param source the stream from which to read an object's serialized form
+     * @return the first deserialized object available from {@code source}
+     */
+    Object readClassAndObject(InputStream source);
+
+    /**
+     * Serializes an object to an output stream.  This may flush the output stream.
+     *
+     * @param o the object to serialize
+     * @param sink the stream into which the serialized object is written
+     */
+    void writeClassAndObject(Object o, OutputStream sink);
+
+    /**
+     * Returns this service's relative priority number.  Unless explicitly overridden through a
+     * system property ({@link KryoShimServiceLoader#SHIM_CLASS_SYSTEM_PROPERTY}),
+     * the service implementation with the numerically highest priority will be used
+     * and all others ignored.  In other words, the highest priority wins (in the absence of a
+     * system property override).
+     * <p>
+     * TinkerPop's current default implementation uses priority value zero.
+     * <p>
+     * Third-party implementations of this interface should (but are not technically required)
+     * to use a priority value with absolute value greater than 100.
+     * <p>
+     * The implementation currently breaks priority ties by lexicographical comparison of
+     * fully-qualified package-and-classname, but this tie-breaking behavior should be
+     * considered undefined and subject to future change.  Ties are ignored if the service
+     * is explicitly set through the system property mentioned above.
+     *
+     * @return this implementation's priority value
+     */
+    int getPriority();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
new file mode 100644
index 0000000..9ccf2de
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
@@ -0,0 +1,199 @@
+/*
+ * 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.structure.io.gryo.kryoshim;
+
+import org.apache.tinkerpop.shaded.kryo.io.Input;
+import org.apache.tinkerpop.shaded.kryo.io.Output;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.ServiceLoader;
+
+/**
+ * Loads the highest-priority or user-selected {@link KryoShimService}.
+ */
+public class KryoShimServiceLoader {
+
+    private static volatile KryoShimService CACHED_SHIM_SERVICE;
+
+    private static final Logger log = LoggerFactory.getLogger(KryoShimServiceLoader.class);
+
+    /**
+     * Set this system property to the fully-qualified name of a {@link KryoShimService}
+     * package-and-classname to force it into service.  Setting this property causes the
+     * priority-selection mechanism ({@link KryoShimService#getPriority()}) to be ignored.
+     */
+    public static final String SHIM_CLASS_SYSTEM_PROPERTY = "tinkerpop.kryo.shim";
+
+    /**
+     * Return a reference to the shim service.  This method may return a cached shim service
+     * unless {@code forceReload} is true.  Calls to this method need not be externally
+     * synchonized.
+     *
+     * @param forceReload if false, this method may use its internal service cache; if true,
+     *                    this method must ignore cache, and it must invoke {@link ServiceLoader#reload()}
+     *                    before selecting a new service to return
+     * @return the shim service
+     */
+    public static KryoShimService load(boolean forceReload) {
+
+        if (null != CACHED_SHIM_SERVICE && !forceReload) {
+            return CACHED_SHIM_SERVICE;
+        }
+
+        ArrayList<KryoShimService> services = new ArrayList<>();
+
+        ServiceLoader<KryoShimService> sl = ServiceLoader.load(KryoShimService.class);
+
+        KryoShimService result = null;
+
+        synchronized (KryoShimServiceLoader.class) {
+            if (forceReload) {
+                sl.reload();
+            }
+
+            for (KryoShimService kss : sl) {
+                services.add(kss);
+            }
+        }
+
+        String shimClass = System.getProperty(SHIM_CLASS_SYSTEM_PROPERTY);
+
+        if (null != shimClass) {
+            for (KryoShimService kss : services) {
+                if (kss.getClass().getCanonicalName().equals(shimClass)) {
+                    log.info("Set {} provider to {} ({}) from system property {}={}",
+                            KryoShimService.class.getSimpleName(), kss, kss.getClass(),
+                            SHIM_CLASS_SYSTEM_PROPERTY, shimClass);
+                    result = kss;
+                }
+            }
+        } else {
+            Collections.sort(services, KryoShimServiceComparator.INSTANCE);
+
+            for (KryoShimService kss : services) {
+                log.debug("Found Kryo shim service class {} (priority {})", kss.getClass(), kss.getPriority());
+            }
+
+            if (0 != services.size()) {
+                result = services.get(services.size() - 1);
+            }
+        }
+
+
+        if (null == result) {
+            throw new IllegalStateException("Unable to load KryoShimService");
+        }
+
+        log.info("Set {} provider to {} ({}) because its priority value ({}) is the highest available",
+                KryoShimService.class.getSimpleName(), result, result.getClass(), result.getPriority());
+
+        return CACHED_SHIM_SERVICE = result;
+    }
+
+    /**
+     * Equivalent to {@link #load(boolean)} with the parameter {@code true}.
+     *
+     * @return the (possibly cached) shim service
+     */
+    public static KryoShimService load() {
+        return load(false);
+    }
+
+    /**
+     * A loose abstraction of {@link org.apache.tinkerpop.shaded.kryo.Kryo#writeClassAndObject(Output, Object)},
+     * where the {@code output} parameter is an internally-created {@link ByteArrayOutputStream}.  Returns
+     * the byte array underlying that stream.
+     *
+     * @param o an object for which the instance and class are serialized
+     * @return the serialized form
+     */
+    public static byte[] writeClassAndObjectToBytes(Object o) {
+        KryoShimService shimService = load();
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+        shimService.writeClassAndObject(o, baos);
+
+        return baos.toByteArray();
+    }
+
+    /**
+     * A loose abstraction of {@link org.apache.tinkerpop.shaded.kryo.Kryo#readClassAndObject(Input)},
+     * where the {@code input} parameter is {@code source}.  Returns the deserialized object.
+     *
+     * @param source an input stream containing data for a serialized object class and instance
+     * @param <T> the type to which the deserialized object is cast as it is returned
+     * @return the deserialized object
+     */
+    public static <T> T readClassAndObject(InputStream source) {
+        KryoShimService shimService = load();
+
+        return (T)shimService.readClassAndObject(source);
+    }
+
+    /**
+     * Selects the service with greatest {@link KryoShimService#getPriority()}
+     * (not absolute value).
+     *
+     * Breaks ties with lexicographical comparison of classnames where the
+     * name that sorts last is considered to have highest priority.  Ideally
+     * nothing should rely on that tiebreaking behavior, but it beats random
+     * selection in case a user ever gets into that situation by accident and
+     * tries to figure out what's going on.
+     */
+    private enum KryoShimServiceComparator implements Comparator<KryoShimService> {
+        INSTANCE;
+
+        @Override
+        public int compare(KryoShimService a, KryoShimService b) {
+            int ap = a.getPriority();
+            int bp = b.getPriority();
+
+            if (ap < bp) {
+                return -1;
+            } else if (bp < ap) {
+                return 1;
+            } else {
+                int result = a.getClass().getCanonicalName().compareTo(b.getClass().getCanonicalName());
+
+                if (0 == result) {
+                    log.warn("Found two {} implementations with the same canonical classname: {}.  " +
+                             "This may indicate a problem with the classpath/classloader such as " +
+                             "duplicate or conflicting copies of the file " +
+                             "META-INF/services/org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimService.",
+                             a.getClass().getCanonicalName());
+                } else {
+                    String winner = 0 < result ? a.getClass().getCanonicalName() : b.getClass().getCanonicalName();
+                    log.warn("{} implementations {} and {} are tied with priority value {}.  " +
+                             "Preferring {} to the other because it has a lexicographically greater classname.  " +
+                             "Consider setting the system property \"{}\" instead of relying on priority tie-breaking.",
+                             KryoShimService.class.getSimpleName(), a, b, ap, winner, SHIM_CLASS_SYSTEM_PROPERTY);
+                }
+
+                return result;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/OutputShim.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/OutputShim.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/OutputShim.java
new file mode 100644
index 0000000..e4ca3d5
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/OutputShim.java
@@ -0,0 +1,40 @@
+/*
+ * 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.structure.io.gryo.kryoshim;
+
+/**
+ * A minimal {@link org.apache.tinkerpop.shaded.kryo.io.Output}-like abstraction.
+ * See that class for method documentation.
+ */
+public interface OutputShim {
+
+    void writeByte(byte b);
+
+    void writeBytes(byte[] array, int offset, int count);
+
+    void writeString(String s);
+
+    void writeLong(long l);
+
+    void writeInt(int i);
+
+    void writeDouble(double d);
+
+    void flush();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/SerializerShim.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/SerializerShim.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/SerializerShim.java
new file mode 100644
index 0000000..191cdd8
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/SerializerShim.java
@@ -0,0 +1,36 @@
+/*
+ * 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.structure.io.gryo.kryoshim;
+
+/**
+ * A minimal {@link org.apache.tinkerpop.shaded.kryo.Serializer}-like abstraction.
+ * See that class for method documentation.
+ *
+ * @param <T> the class this serializer reads/writes from/to bytes.
+ */
+public interface SerializerShim<T> {
+
+    <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, T starGraph);
+
+    <I extends InputShim> T read(KryoShim<I, ?> kryo, I input, Class<T> clazz);
+
+    default boolean isImmutable() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/package-info.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/package-info.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/package-info.java
new file mode 100644
index 0000000..70675da
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/package-info.java
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ */
+/**
+ * Abstracts a minimal subset of Kryo types and methods.
+ * <p>
+ * Kryo is often shaded.  For instance, TinkerPop's Gryo
+ * serializer relies on a shaded Kryo package.
+ * TinkerPop serializers written against a particular shaded
+ * Kryo package (or an unshaded Kryo package) are compatible
+ * only with that package.  In contrast, TinkerPop serializers written
+ * against this abstraction can be used with any shaded or
+ * unshaded Kryo package, so long as the signatures and behavior
+ * of the methods in this package remain stable.
+ * <p>
+ * To show how this is useful, consider
+ * {@link org.apache.tinkerpop.gremlin.structure.util.star.StarGraphSerializer}.
+ * This class has logic unique to TinkerPop that performs
+ * efficient and forward-compatible serialization of
+ * {@link org.apache.tinkerpop.gremlin.structure.util.star.StarGraph}
+ * instances.  It takes advantage of package-level visibility
+ * and the fact that it shares a package with its target,
+ * so it would be challenging to cleanly and naturally replicate
+ * (i.e. without package spoofing or runtime visibility overrides).
+ * By implementing
+ * {@link org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim}
+ * instead of, say, Gryo's shaded
+ * {@link org.apache.tinkerpop.shaded.kryo.Serializer},
+ * such a serializer can be used with anybody's Kryo package,
+ * regardless of whether
+ * that package is shaded or not.  This lets third-parties reuse
+ * TinkerPop's efficient, internals-aware StarGraph serializer on
+ * their own serialization platform (and without altering
+ * TinkerPop's bytecode, let alone its source).
+ * <p>
+ * The number of types and methods in this
+ * package is deliberately small to reduce the likelihood of a
+ * new Kryo release introducing an incompatible change.
+ */
+package org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedInputAdapter.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedInputAdapter.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedInputAdapter.java
new file mode 100644
index 0000000..d11c1c4
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedInputAdapter.java
@@ -0,0 +1,66 @@
+/*
+ * 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.structure.io.gryo.kryoshim.shaded;
+
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.InputShim;
+import org.apache.tinkerpop.shaded.kryo.io.Input;
+
+public class ShadedInputAdapter implements InputShim {
+
+    private final Input shadedInput;
+
+    public ShadedInputAdapter(Input shadedInput) {
+        this.shadedInput = shadedInput;
+    }
+
+    Input getShadedInput() {
+        return shadedInput;
+    }
+
+    @Override
+    public byte readByte()
+    {
+        return shadedInput.readByte();
+    }
+
+    @Override
+    public byte[] readBytes(int size) {
+        return shadedInput.readBytes(size);
+    }
+
+    @Override
+    public String readString() {
+        return shadedInput.readString();
+    }
+
+    @Override
+    public long readLong() {
+        return shadedInput.readLong();
+    }
+
+    @Override
+    public int readInt() {
+        return shadedInput.readInt();
+    }
+
+    @Override
+    public double readDouble() {
+        return shadedInput.readDouble();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedKryoAdapter.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedKryoAdapter.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedKryoAdapter.java
new file mode 100644
index 0000000..4283298
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedKryoAdapter.java
@@ -0,0 +1,67 @@
+/*
+ * 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.structure.io.gryo.kryoshim.shaded;
+
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShim;
+import org.apache.tinkerpop.shaded.kryo.Kryo;
+
+public class ShadedKryoAdapter implements KryoShim<ShadedInputAdapter, ShadedOutputAdapter> {
+
+    private final Kryo shadedKryo;
+
+    public ShadedKryoAdapter(Kryo shadedKryo) {
+        this.shadedKryo = shadedKryo;
+    }
+
+    @Override
+    public <T> T readObject(ShadedInputAdapter input, Class<T> type)
+    {
+        return shadedKryo.readObject(input.getShadedInput(), type);
+    }
+
+    @Override
+    public Object readClassAndObject(ShadedInputAdapter input)
+    {
+        return shadedKryo.readClassAndObject(input.getShadedInput());
+    }
+
+    @Override
+    public void writeObject(ShadedOutputAdapter output, Object object)
+    {
+        shadedKryo.writeObject(output.getShadedOutput(), object);
+    }
+
+    @Override
+    public void writeClassAndObject(ShadedOutputAdapter output, Object object)
+    {
+        shadedKryo.writeClassAndObject(output.getShadedOutput(), object);
+    }
+
+    @Override
+    public <T> T readObjectOrNull(ShadedInputAdapter input, Class<T> type)
+    {
+        return shadedKryo.readObjectOrNull(input.getShadedInput(), type);
+    }
+
+    @Override
+    public void writeObjectOrNull(ShadedOutputAdapter output, Object object, Class type)
+    {
+        shadedKryo.writeObjectOrNull(output.getShadedOutput(), object, type);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedOutputAdapter.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedOutputAdapter.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedOutputAdapter.java
new file mode 100644
index 0000000..7547466
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedOutputAdapter.java
@@ -0,0 +1,72 @@
+/*
+ * 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.structure.io.gryo.kryoshim.shaded;
+
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.OutputShim;
+import org.apache.tinkerpop.shaded.kryo.io.Output;
+
+public class ShadedOutputAdapter implements OutputShim {
+
+    private final Output shadedOutput;
+
+    public ShadedOutputAdapter(Output shadedOutput) {
+        this.shadedOutput = shadedOutput;
+    }
+
+    @Override
+    public void writeByte(byte b)
+    {
+        shadedOutput.writeByte(b);
+    }
+
+    @Override
+    public void writeBytes(byte[] array, int offset, int count) {
+        shadedOutput.writeBytes(array, offset, count);
+    }
+
+    @Override
+    public void writeString(String s) {
+        shadedOutput.writeString(s);
+    }
+
+    @Override
+    public void writeLong(long l) {
+        shadedOutput.writeLong(l);
+    }
+
+    @Override
+    public void writeInt(int i) {
+        shadedOutput.writeInt(i);
+    }
+
+    @Override
+    public void writeDouble(double d) {
+        shadedOutput.writeDouble(d);
+    }
+
+    @Override
+    public void flush() {
+        shadedOutput.flush();
+    }
+
+    Output getShadedOutput()
+    {
+        return shadedOutput;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedSerializerAdapter.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedSerializerAdapter.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedSerializerAdapter.java
new file mode 100644
index 0000000..4ce27b1
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/ShadedSerializerAdapter.java
@@ -0,0 +1,54 @@
+/*
+ * 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.structure.io.gryo.kryoshim.shaded;
+
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
+import org.apache.tinkerpop.shaded.kryo.Kryo;
+import org.apache.tinkerpop.shaded.kryo.Serializer;
+import org.apache.tinkerpop.shaded.kryo.io.Input;
+import org.apache.tinkerpop.shaded.kryo.io.Output;
+
+public class ShadedSerializerAdapter<T> extends Serializer<T> {
+
+    SerializerShim<T> serializer;
+
+    public ShadedSerializerAdapter(SerializerShim<T> serializer) {
+        this.serializer = serializer;
+        setImmutable(this.serializer.isImmutable());
+    }
+
+    @Override
+    public void write(Kryo kryo, Output output, T t) {
+        /* These adapters could be cached pretty efficiently in instance fields if it were guaranteed that this
+         * class was never subject to concurrent use.  That's true of Kryo instances, but it is not clear that
+         * it is true of Serializer instances.
+         */
+        ShadedKryoAdapter shadedKryoAdapter = new ShadedKryoAdapter(kryo);
+        ShadedOutputAdapter shadedOutputAdapter = new ShadedOutputAdapter(output);
+        serializer.write(shadedKryoAdapter, shadedOutputAdapter, t);
+    }
+
+    @Override
+    public T read(Kryo kryo, Input input, Class<T> aClass) {
+        // Same caching opportunity as in write(...)
+        ShadedKryoAdapter shadedKryoAdapter = new ShadedKryoAdapter(kryo);
+        ShadedInputAdapter shadedInputAdapter = new ShadedInputAdapter(input);
+        return serializer.read(shadedKryoAdapter, shadedInputAdapter, aClass);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/package-info.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/package-info.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/package-info.java
new file mode 100644
index 0000000..d91a3f1
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/shaded/package-info.java
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+/**
+ * Implementations of the interfaces in
+ * {@link org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim}
+ * using TinkerPop's shaded copy of Kryo.
+ */
+package org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.shaded;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/InputShim.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/InputShim.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/InputShim.java
deleted file mode 100644
index 6d00884..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/InputShim.java
+++ /dev/null
@@ -1,37 +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.structure.io.kryoshim;
-
-/**
- * A minimal {@link org.apache.tinkerpop.shaded.kryo.io.Input}-like abstraction.
- */
-public interface InputShim {
-
-    byte readByte();
-
-    byte[] readBytes(int size);
-
-    String readString();
-
-    long readLong();
-
-    int readInt();
-
-    double readDouble();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShim.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShim.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShim.java
deleted file mode 100644
index e2a95e6..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShim.java
+++ /dev/null
@@ -1,40 +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.structure.io.kryoshim;
-
-/**
- * A minimal {@link org.apache.tinkerpop.shaded.kryo.Kryo}-like abstraction.
- *
- * @param <I> this interface's complementary InputShim
- * @param <O> this interface's complementary OutputShim
- */
-public interface KryoShim<I extends InputShim, O extends OutputShim> {
-
-    <T> T readObject(I input, Class<T> type);
-
-    Object readClassAndObject(I input);
-
-    void writeObject(O output, Object object);
-
-    void writeClassAndObject(O output, Object object);
-
-    <T> T readObjectOrNull(I input, Class<T> type);
-
-    void writeObjectOrNull(O output, Object object, Class type);
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShimService.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShimService.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShimService.java
deleted file mode 100644
index 024d40c..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShimService.java
+++ /dev/null
@@ -1,83 +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.structure.io.kryoshim;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-
-/**
- * This interface exists to decouple HadoopPools from TinkerPop's shaded Kryo.
- * <p>
- * VertexWritable and ObjectWritable formerly implemented Serializable by
- * resorting to statically-pooled shaded Kryo instances maintained by the HadoopPools class.
- * This is awkward because those shaded Kryo instances require class registration by default.
- * <p>
- * Consider what happens with custom property datatypes reachable from the reference graph rooted at an ObjectWritable
- * or VertexWritable instance.  It is not enough for these property classes to merely implement
- * Serializable, though one think that from skimming ObjectWritable/VertexWritable.  Those classes
- * must also register with TinkerPop's internal, shaded Kryo instances as maintained by HadoopPools,
- * or else configure those instances to accept unregistered classes.
- * Otherwise, TinkerPop's shaded Kryo will refuse to serialize those properties (even though
- * they implement Serializable, and even though the user might think they are only using
- * Java's standard Serialization mechanism!).
- * <p>
- * By hiding the mechanics of serialization behind this interface instead of hardcoding it in
- * HadoopPools, the user can decide how to implement serialization for ObjectWritable/VertexWritable
- * (and whatever other classes in TinkerPop decide to implement Serializable but then delegate
- * all of the implementation details, like ObjectWritable/VertexWritable do now).
- */
-public interface KryoShimService {
-
-    /**
-     * Deserializes an object from an input stream.
-     *
-     * @param source the stream from which to read an object's serialized form
-     * @return the first deserialized object available from {@code source}
-     */
-    Object readClassAndObject(InputStream source);
-
-    /**
-     * Serializes an object to an output stream.  This may flush the output stream.
-     *
-     * @param o the object to serialize
-     * @param sink the stream into which the serialized object is written
-     */
-    void writeClassAndObject(Object o, OutputStream sink);
-
-    /**
-     * Returns this service's relative priority number.  Unless explicitly overridden through a
-     * system property ({@link KryoShimServiceLoader#SHIM_CLASS_SYSTEM_PROPERTY}),
-     * the service implementation with the numerically highest priority will be used
-     * and all others ignored.  In other words, the highest priority wins (in the absence of a
-     * system property override).
-     * <p>
-     * TinkerPop's current default implementation uses priority value zero.
-     * <p>
-     * Third-party implementations of this interface should (but are not technically required)
-     * to use a priority value with absolute value greater than 100.
-     * <p>
-     * The implementation currently breaks priority ties by lexicographical comparison of
-     * fully-qualified package-and-classname, but this tie-breaking behavior should be
-     * considered undefined and subject to future change.  Ties are ignored if the service
-     * is explicitly set through the system property mentioned above.
-     *
-     * @return this implementation's priority value
-     */
-    int getPriority();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShimServiceLoader.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShimServiceLoader.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShimServiceLoader.java
deleted file mode 100644
index 1d5413d..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShimServiceLoader.java
+++ /dev/null
@@ -1,137 +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.structure.io.kryoshim;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.ByteArrayOutputStream;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.ServiceLoader;
-
-public class KryoShimServiceLoader {
-
-    private static volatile KryoShimService CACHED_SHIM_SERVICE;
-
-    private static final Logger log = LoggerFactory.getLogger(KryoShimServiceLoader.class);
-
-    /**
-     * Set this system property to the fully-qualified name of a {@link KryoShimService}
-     * package-and-classname to force it into service.  Setting this property causes the
-     * priority-selection mechanism ({@link KryoShimService#getPriority()}) to be ignored.
-     */
-    public static final String SHIM_CLASS_SYSTEM_PROPERTY = "tinkerpop.kryo.shim";
-
-    public static KryoShimService load(boolean forceReload) {
-
-        if (null != CACHED_SHIM_SERVICE && !forceReload) {
-            return CACHED_SHIM_SERVICE;
-        }
-
-        ArrayList<KryoShimService> services = new ArrayList<>();
-
-        ServiceLoader<KryoShimService> sl = ServiceLoader.load(KryoShimService.class);
-
-        KryoShimService result = null;
-
-        synchronized (KryoShimServiceLoader.class) {
-            if (forceReload) {
-                sl.reload();
-            }
-
-            for (KryoShimService kss : sl) {
-                services.add(kss);
-            }
-        }
-
-        String shimClass = System.getProperty(SHIM_CLASS_SYSTEM_PROPERTY);
-
-        if (null != shimClass) {
-            for (KryoShimService kss : services) {
-                if (kss.getClass().getCanonicalName().equals(shimClass)) {
-                    log.info("Set {} provider to {} ({}) from system property {}={}",
-                            KryoShimService.class.getSimpleName(), kss, kss.getClass(),
-                            SHIM_CLASS_SYSTEM_PROPERTY, shimClass);
-                    result = kss;
-                }
-            }
-        } else {
-            Collections.sort(services, KryoShimServiceComparator.INSTANCE);
-
-            for (KryoShimService kss : services) {
-                log.debug("Found Kryo shim service class {} (priority {})", kss.getClass(), kss.getPriority());
-            }
-
-            if (0 != services.size()) {
-                result = services.get(services.size() - 1);
-            }
-        }
-
-
-        if (null == result) {
-            throw new IllegalStateException("Unable to load KryoShimService");
-        }
-
-        log.info("Set {} provider to {} ({}) because its priority value ({}) is the highest available",
-                KryoShimService.class.getSimpleName(), result, result.getClass(), result.getPriority());
-
-        return CACHED_SHIM_SERVICE = result;
-    }
-
-    public static KryoShimService load() {
-        return load(false);
-    }
-
-    public static byte[] writeClassAndObjectToBytes(Object o) {
-        KryoShimService shimService = load();
-
-        ByteArrayOutputStream baos = new ByteArrayOutputStream();
-
-        shimService.writeClassAndObject(o, baos);
-
-        return baos.toByteArray();
-    }
-
-    public static <T> T readClassAndObject(InputStream source) {
-        KryoShimService shimService = load();
-
-        return (T)shimService.readClassAndObject(source);
-    }
-
-    private enum KryoShimServiceComparator implements Comparator<KryoShimService> {
-        INSTANCE;
-
-        @Override
-        public int compare(KryoShimService a, KryoShimService b) {
-            int ap = a.getPriority();
-            int bp = b.getPriority();
-
-            if (ap < bp) {
-                return -1;
-            } else if (bp < ap) {
-                return 1;
-            } else {
-                return a.getClass().getCanonicalName().compareTo(b.getClass().getCanonicalName());
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/OutputShim.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/OutputShim.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/OutputShim.java
deleted file mode 100644
index 4468434..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/OutputShim.java
+++ /dev/null
@@ -1,41 +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.structure.io.kryoshim;
-
-import java.io.IOException;
-
-/**
- * A minimal {@link org.apache.tinkerpop.shaded.kryo.io.Output}-like abstraction.
- */
-public interface OutputShim {
-
-    void writeByte(byte b);
-
-    void writeBytes(byte[] array, int offset, int count);
-
-    void writeString(String s);
-
-    void writeLong(long l);
-
-    void writeInt(int i);
-
-    void writeDouble(double d);
-
-    void flush();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/SerializerShim.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/SerializerShim.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/SerializerShim.java
deleted file mode 100644
index 686350d..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/SerializerShim.java
+++ /dev/null
@@ -1,35 +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.structure.io.kryoshim;
-
-/**
- * A minimal {@link org.apache.tinkerpop.shaded.kryo.Serializer}-like abstraction.
- *
- * @param <T> the class this serializer reads/writes from/to bytes.
- */
-public interface SerializerShim<T> {
-
-    <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, T starGraph);
-
-    <I extends InputShim> T read(KryoShim<I, ?> kryo, I input, Class<T> clazz);
-
-    default boolean isImmutable() {
-        return false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/package-info.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/package-info.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/package-info.java
deleted file mode 100644
index 436f117..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/package-info.java
+++ /dev/null
@@ -1,54 +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.
- */
-/**
- * Abstracts a minimal subset of Kryo types and methods.
- * <p>
- * Kryo is often shaded.  For instance, TinkerPop's Gryo
- * serializer relies on a shaded Kryo package.
- * TinkerPop serializers written against a particular shaded
- * Kryo package (or the unshaded Kryo package) are compatible
- * only with that package.  In contrast, TinkerPop serializers written
- * against this abstraction can be used with any shaded or
- * unshaded Kryo package, so long as the signatures and behavior
- * of the methods in this package remain stable.
- * <p>
- * To show how this is useful, consider
- * {@link org.apache.tinkerpop.gremlin.structure.util.star.StarGraphSerializer}.
- * This class has logic unique to TinkerPop that performs
- * efficient and forward-compatible serialization of
- * {@link org.apache.tinkerpop.gremlin.structure.util.star.StarGraph}
- * instances.  It takes advantage of package-level visibility
- * and the fact that it shares a package with its target,
- * so it would be challenging to cleanly and naturally replicate
- * (i.e. without package spoofing or runtime visibility overrides).
- * By implementing
- * {@link org.apache.tinkerpop.gremlin.structure.io.kryoshim.SerializerShim}
- * instead of, say, Gryo's shaded
- * {@link org.apache.tinkerpop.shaded.kryo.Serializer},
- * it can be used with anybody's Kryo package, regardless of whether
- * that package is shaded or not.  This lets third-parties reuse
- * TinkerPop's efficient, internals-aware StarGraph serializer on
- * their own serialization platform (and without altering
- * TinkerPop's bytecode, let alone its source).
- * <p>
- * The number of types and methods in this
- * package is deliberately small to reduce the likelihood of a
- * new Kryo release introducing an incompatible change.
- */
-package org.apache.tinkerpop.gremlin.structure.io.kryoshim;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedInputAdapter.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedInputAdapter.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedInputAdapter.java
deleted file mode 100644
index 2872326..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedInputAdapter.java
+++ /dev/null
@@ -1,66 +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.structure.io.kryoshim.shaded;
-
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.InputShim;
-import org.apache.tinkerpop.shaded.kryo.io.Input;
-
-public class ShadedInputAdapter implements InputShim {
-
-    private final Input shadedInput;
-
-    public ShadedInputAdapter(Input shadedInput) {
-        this.shadedInput = shadedInput;
-    }
-
-    Input getShadedInput() {
-        return shadedInput;
-    }
-
-    @Override
-    public byte readByte()
-    {
-        return shadedInput.readByte();
-    }
-
-    @Override
-    public byte[] readBytes(int size) {
-        return shadedInput.readBytes(size);
-    }
-
-    @Override
-    public String readString() {
-        return shadedInput.readString();
-    }
-
-    @Override
-    public long readLong() {
-        return shadedInput.readLong();
-    }
-
-    @Override
-    public int readInt() {
-        return shadedInput.readInt();
-    }
-
-    @Override
-    public double readDouble() {
-        return shadedInput.readDouble();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedKryoAdapter.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedKryoAdapter.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedKryoAdapter.java
deleted file mode 100644
index 0e85f6f..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedKryoAdapter.java
+++ /dev/null
@@ -1,67 +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.structure.io.kryoshim.shaded;
-
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShim;
-import org.apache.tinkerpop.shaded.kryo.Kryo;
-
-public class ShadedKryoAdapter implements KryoShim<ShadedInputAdapter, ShadedOutputAdapter> {
-
-    private final Kryo shadedKryo;
-
-    public ShadedKryoAdapter(Kryo shadedKryo) {
-        this.shadedKryo = shadedKryo;
-    }
-
-    @Override
-    public <T> T readObject(ShadedInputAdapter input, Class<T> type)
-    {
-        return shadedKryo.readObject(input.getShadedInput(), type);
-    }
-
-    @Override
-    public Object readClassAndObject(ShadedInputAdapter input)
-    {
-        return shadedKryo.readClassAndObject(input.getShadedInput());
-    }
-
-    @Override
-    public void writeObject(ShadedOutputAdapter output, Object object)
-    {
-        shadedKryo.writeObject(output.getShadedOutput(), object);
-    }
-
-    @Override
-    public void writeClassAndObject(ShadedOutputAdapter output, Object object)
-    {
-        shadedKryo.writeClassAndObject(output.getShadedOutput(), object);
-    }
-
-    @Override
-    public <T> T readObjectOrNull(ShadedInputAdapter input, Class<T> type)
-    {
-        return shadedKryo.readObjectOrNull(input.getShadedInput(), type);
-    }
-
-    @Override
-    public void writeObjectOrNull(ShadedOutputAdapter output, Object object, Class type)
-    {
-        shadedKryo.writeObjectOrNull(output.getShadedOutput(), object, type);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedOutputAdapter.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedOutputAdapter.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedOutputAdapter.java
deleted file mode 100644
index 5ca0ecb..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedOutputAdapter.java
+++ /dev/null
@@ -1,72 +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.structure.io.kryoshim.shaded;
-
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.OutputShim;
-import org.apache.tinkerpop.shaded.kryo.io.Output;
-
-public class ShadedOutputAdapter implements OutputShim {
-
-    private final Output shadedOutput;
-
-    public ShadedOutputAdapter(Output shadedOutput) {
-        this.shadedOutput = shadedOutput;
-    }
-
-    @Override
-    public void writeByte(byte b)
-    {
-        shadedOutput.writeByte(b);
-    }
-
-    @Override
-    public void writeBytes(byte[] array, int offset, int count) {
-        shadedOutput.writeBytes(array, offset, count);
-    }
-
-    @Override
-    public void writeString(String s) {
-        shadedOutput.writeString(s);
-    }
-
-    @Override
-    public void writeLong(long l) {
-        shadedOutput.writeLong(l);
-    }
-
-    @Override
-    public void writeInt(int i) {
-        shadedOutput.writeInt(i);
-    }
-
-    @Override
-    public void writeDouble(double d) {
-        shadedOutput.writeDouble(d);
-    }
-
-    @Override
-    public void flush() {
-        shadedOutput.flush();
-    }
-
-    Output getShadedOutput()
-    {
-        return shadedOutput;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedSerializerAdapter.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedSerializerAdapter.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedSerializerAdapter.java
deleted file mode 100644
index ebfd0fb..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedSerializerAdapter.java
+++ /dev/null
@@ -1,54 +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.structure.io.kryoshim.shaded;
-
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.SerializerShim;
-import org.apache.tinkerpop.shaded.kryo.Kryo;
-import org.apache.tinkerpop.shaded.kryo.Serializer;
-import org.apache.tinkerpop.shaded.kryo.io.Input;
-import org.apache.tinkerpop.shaded.kryo.io.Output;
-
-public class ShadedSerializerAdapter<T> extends Serializer<T> {
-
-    SerializerShim<T> serializer;
-
-    public ShadedSerializerAdapter(SerializerShim<T> serializer) {
-        this.serializer = serializer;
-        setImmutable(this.serializer.isImmutable());
-    }
-
-    @Override
-    public void write(Kryo kryo, Output output, T t) {
-        /* These adapters could be cached pretty efficiently in instance fields if it were guaranteed that this
-         * class was never subject to concurrent use.  That's true of Kryo instances, but it is not clear that
-         * it is true of Serializer instances.
-         */
-        ShadedKryoAdapter shadedKryoAdapter = new ShadedKryoAdapter(kryo);
-        ShadedOutputAdapter shadedOutputAdapter = new ShadedOutputAdapter(output);
-        serializer.write(shadedKryoAdapter, shadedOutputAdapter, t);
-    }
-
-    @Override
-    public T read(Kryo kryo, Input input, Class<T> aClass) {
-        // Same caching opportunity as in write(...)
-        ShadedKryoAdapter shadedKryoAdapter = new ShadedKryoAdapter(kryo);
-        ShadedInputAdapter shadedInputAdapter = new ShadedInputAdapter(input);
-        return serializer.read(shadedKryoAdapter, shadedInputAdapter, aClass);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/package-info.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/package-info.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/package-info.java
deleted file mode 100644
index ea70003..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/package-info.java
+++ /dev/null
@@ -1,25 +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.
- */
-
-/**
- * Implementations of the interfaces in
- * {@link org.apache.tinkerpop.gremlin.structure.io.kryoshim}
- * using TinkerPop's shaded copy of Kryo.
- */
-package org.apache.tinkerpop.gremlin.structure.io.kryoshim.shaded;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/218d7909/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphGryoSerializer.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphGryoSerializer.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphGryoSerializer.java
index 2acd770..b2379ce 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphGryoSerializer.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphGryoSerializer.java
@@ -23,14 +23,11 @@ import java.util.Map;
 
 import org.apache.tinkerpop.gremlin.process.computer.GraphFilter;
 import org.apache.tinkerpop.gremlin.structure.Direction;
-import org.apache.tinkerpop.gremlin.structure.io.kryoshim.shaded.ShadedSerializerAdapter;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.shaded.ShadedSerializerAdapter;
 
 /**
- * Kryo serializer for {@link StarGraph}.  Implements an internal versioning capability for backward compatibility.
- * The single byte at the front of the serialization stream denotes the version.  That version can be used to choose
- * the correct deserialization mechanism.  The limitation is that this versioning won't help with backward
- * compatibility for custom serializers from providers.  Providers should be encouraged to write their serializers
- * with backward compatibility in mind.
+ * A wrapper for {@link StarGraphSerializer} that makes it compatible with TinkerPop's
+ * shaded Kryo.
  *
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  * @author Stephen Mallette (http://stephen.genoprime.com)


[04/34] incubator-tinkerpop git commit: Fixed bad header formatting - wrong levels CTR

Posted by ok...@apache.org.
Fixed bad header formatting - wrong levels CTR


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

Branch: refs/heads/TINKERPOP-1278
Commit: 2a58313238eb65f8eb6490f703a7206e6b95b21a
Parents: 9c56fa3
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Fri Jun 3 07:34:13 2016 -0400
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Fri Jun 3 07:34:13 2016 -0400

----------------------------------------------------------------------
 docs/src/dev/provider/index.asciidoc               | 10 +++++-----
 docs/src/reference/implementations-hadoop.asciidoc |  4 ++--
 docs/src/upgrade/release-3.2.x-incubating.asciidoc |  6 +++---
 3 files changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/2a583132/docs/src/dev/provider/index.asciidoc
----------------------------------------------------------------------
diff --git a/docs/src/dev/provider/index.asciidoc b/docs/src/dev/provider/index.asciidoc
index 675b1c3..94b5e52 100644
--- a/docs/src/dev/provider/index.asciidoc
+++ b/docs/src/dev/provider/index.asciidoc
@@ -714,13 +714,13 @@ Gremlin Server will send:
 |=========================================================
 
 OpProcessors Arguments
-^^^^^^^^^^^^^^^^^^^^^^
+~~~~~~~~~~~~~~~~~~~~~~
 
 The following sections define a non-exhaustive list of available operations and arguments for embedded `OpProcessors`
 (i.e. ones packaged with Gremlin Server).
 
 Common
-++++++
+^^^^^^
 
 All `OpProcessor` instances support these arguments.
 
@@ -731,7 +731,7 @@ All `OpProcessor` instances support these arguments.
 |=========================================================
 
 Standard OpProcessor
-++++++++++++++++++++
+^^^^^^^^^^^^^^^^^^^^
 
 The "standard" `OpProcessor` handles requests for the primary function of Gremlin Server - executing Gremlin.
 Requests made to this `OpProcessor` are "sessionless" in the sense that a request must encapsulate the entirety
@@ -773,7 +773,7 @@ to send an alias pair with key of "g" and value of "g2" and thus allow the scrip
 |=========================================================
 
 Session OpProcessor
-+++++++++++++++++++
+^^^^^^^^^^^^^^^^^^^
 
 The "session" `OpProcessor` handles requests for the primary function of Gremlin Server - executing Gremlin. It is
 like the "standard" `OpProcessor`, but instead maintains state between sessions and allows the option to leave all
@@ -828,7 +828,7 @@ to send an alias pair with key of "g" and value of "g2" and thus allow the scrip
 |=========================================================
 
 Authentication
-^^^^^^^^^^^^^^
+~~~~~~~~~~~~~~
 
 Gremlin Server supports link:https://en.wikipedia.org/wiki/Simple_Authentication_and_Security_Layer[SASL-based]
 authentication.  A SASL implementation provides a series of challenges and responses that a driver must comply with

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/2a583132/docs/src/reference/implementations-hadoop.asciidoc
----------------------------------------------------------------------
diff --git a/docs/src/reference/implementations-hadoop.asciidoc b/docs/src/reference/implementations-hadoop.asciidoc
index 9e388ea..140b4d1 100644
--- a/docs/src/reference/implementations-hadoop.asciidoc
+++ b/docs/src/reference/implementations-hadoop.asciidoc
@@ -927,7 +927,7 @@ determine how to read and write data to and from Hadoop. For instance, `gremlin.
 `gremlin.hadoop.graphWriter`.
 
 GraphFilterAware Interface
-++++++++++++++++++++++++++
+^^^^^^^^^^^^^^^^^^^^^^^^^^
 
 <<graph-filter,Graph filters>> by OLAP processors to only pull a subgraph of the full graph from the graph data source. For instance, the
 example below constructs a `GraphFilter` that will only pull the the "knows"-graph amongst people into the `GraphComputer`
@@ -949,7 +949,7 @@ if (configuration.containsKey(Constants.GREMLIN_HADOOP_GRAPH_FILTER))
 ----
 
 PersistResultGraphAware Interface
-+++++++++++++++++++++++++++++++++
+^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
 
 A graph system provider's `OutputFormat` should implement the `PersistResultGraphAware` interface which
 determines which persistence options are available to the user. For the standard file-based `OutputFormats` provided

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/2a583132/docs/src/upgrade/release-3.2.x-incubating.asciidoc
----------------------------------------------------------------------
diff --git a/docs/src/upgrade/release-3.2.x-incubating.asciidoc b/docs/src/upgrade/release-3.2.x-incubating.asciidoc
index 9e387de..ac94960 100644
--- a/docs/src/upgrade/release-3.2.x-incubating.asciidoc
+++ b/docs/src/upgrade/release-3.2.x-incubating.asciidoc
@@ -120,7 +120,7 @@ Graph System Providers
 ^^^^^^^^^^^^^^^^^^^^^^
 
 VertexComputing API Change
-+++++++++++++++++++++++
+++++++++++++++++++++++++++
 
 The `VertexComputing` API is used by steps that wrap a `VertexProgram`. There is a method called
 `VertexComputing.generateProgram()` that has changed which now takes a second argument of `Memory`. To  upgrade, simply
@@ -218,7 +218,7 @@ Upgrading for Users
 ~~~~~~~~~~~~~~~~~~~
 
 Hadoop FileSystem Variable
-++++++++++++++++++++++++++
+^^^^^^^^^^^^^^^^^^^^^^^^^^
 
 The `HadoopGremlinPlugin` defines two variables: `hdfs` and `fs`. The first is a reference to the HDFS `FileSystemStorage`
 and the latter is a reference to the the local `FileSystemStorage`. Prior to 3.2.x, `fs` was called `local`. However,
@@ -226,7 +226,7 @@ there was a variable name conflict with `Scope.local`. As such `local` is now `f
 but was not realized until this release. Finally, this only effects Gremlin Console users.
 
 Hadoop Configurations
-+++++++++++++++++++++
+^^^^^^^^^^^^^^^^^^^^^
 
 Note that `gremlin.hadoop.graphInputFormat`, `gremlin.hadoop.graphOutputFormat`, `gremlin.spark.graphInputRDD`, and
 `gremlin.spark.graphOuputRDD` have all been deprecated. Using them still works, but moving forward, users only need to


[31/34] incubator-tinkerpop git commit: Merge branch 'TINKERPOP-1321'

Posted by ok...@apache.org.
Merge branch 'TINKERPOP-1321'


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

Branch: refs/heads/TINKERPOP-1278
Commit: 4bb9f36ae11304fc469cd2fb4184bd31bd3a4d27
Parents: 406956d 45abdcc
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Tue Jun 7 06:46:29 2016 -0600
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Tue Jun 7 06:46:29 2016 -0600

----------------------------------------------------------------------
 .../process/computer/GiraphWorkerContext.java   |   3 +-
 .../gremlin/structure/io/gryo/GryoMapper.java   | 443 ++++++++++++-------
 .../gremlin/structure/io/gryo/GryoPool.java     |   1 +
 .../structure/io/gryo/GryoSerializers.java      |  40 +-
 .../structure/io/gryo/JavaTimeSerializers.java  | 125 ++----
 .../structure/io/gryo/PairSerializer.java       |  11 +-
 .../structure/io/gryo/TypeRegistration.java     |  82 ++++
 .../structure/io/gryo/URISerializer.java        |  23 +-
 .../structure/io/gryo/UUIDSerializer.java       |  24 +-
 .../structure/io/gryo/kryoshim/InputShim.java   |  38 ++
 .../structure/io/gryo/kryoshim/KryoShim.java    |  41 ++
 .../io/gryo/kryoshim/KryoShimService.java       |  99 +++++
 .../io/gryo/kryoshim/KryoShimServiceLoader.java | 215 +++++++++
 .../structure/io/gryo/kryoshim/OutputShim.java  |  40 ++
 .../io/gryo/kryoshim/SerializerShim.java        |  36 ++
 .../io/gryo/kryoshim/package-info.java          |  55 +++
 .../kryoshim/shaded/ShadedInputAdapter.java     |  66 +++
 .../gryo/kryoshim/shaded/ShadedKryoAdapter.java |  67 +++
 .../kryoshim/shaded/ShadedOutputAdapter.java    |  71 +++
 .../shaded/ShadedSerializerAdapter.java         |  54 +++
 .../io/gryo/kryoshim/shaded/package-info.java   |  25 ++
 .../util/star/StarGraphGryoSerializer.java      | 130 +-----
 .../util/star/StarGraphSerializer.java          | 150 +++++++
 .../hadoop/process/computer/HadoopCombine.java  |   3 +-
 .../hadoop/process/computer/HadoopMap.java      |   3 +-
 .../hadoop/process/computer/HadoopReduce.java   |   3 +-
 .../structure/io/HadoopPoolShimService.java     |  75 ++++
 .../hadoop/structure/io/HadoopPools.java        |   1 +
 .../structure/io/HadoopPoolsConfigurable.java   |   4 +-
 .../hadoop/structure/io/ObjectWritable.java     |  25 +-
 .../hadoop/structure/io/VertexWritable.java     |  39 +-
 .../structure/io/gryo/GryoRecordReader.java     |   3 +-
 .../structure/io/gryo/GryoRecordWriter.java     |   4 +-
 ...n.structure.io.gryo.kryoshim.KryoShimService |   1 +
 .../gremlin/hadoop/HadoopGraphProvider.java     |   4 +
 .../spark/process/computer/SparkExecutor.java   |  14 +-
 .../structure/io/gryo/GryoRegistrator.java      | 238 ++++++++++
 .../spark/structure/io/gryo/GryoSerializer.java |   2 +-
 .../io/gryo/IoRegistryAwareKryoSerializer.java  | 116 +++++
 .../io/gryo/ObjectWritableSerializer.java       |  12 +-
 .../io/gryo/VertexWritableSerializer.java       |  12 +-
 .../kryoshim/unshaded/UnshadedInputAdapter.java |  78 ++++
 .../kryoshim/unshaded/UnshadedKryoAdapter.java  |  74 ++++
 .../unshaded/UnshadedKryoShimService.java       | 154 +++++++
 .../unshaded/UnshadedOutputAdapter.java         |  83 ++++
 .../unshaded/UnshadedSerializerAdapter.java     |  57 +++
 ...n.structure.io.gryo.kryoshim.KryoShimService |   1 +
 .../gremlin/spark/AbstractSparkTest.java        |   4 +
 ...tratorGraphComputerProcessIntegrateTest.java |  33 ++
 ...SparkHadoopGraphGryoRegistratorProvider.java |  56 +++
 .../computer/SparkHadoopGraphProvider.java      |  16 +-
 .../spark/structure/io/ToyGraphInputRDD.java    |   3 +-
 52 files changed, 2480 insertions(+), 477 deletions(-)
----------------------------------------------------------------------



[16/34] incubator-tinkerpop git commit: Kryo shim configuration tweaks

Posted by ok...@apache.org.
Kryo shim configuration tweaks


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

Branch: refs/heads/TINKERPOP-1278
Commit: 9321a3e14eab4ed05f8ef5f4c77de481a4011b81
Parents: 218d790
Author: Dan LaRocque <da...@hopcount.org>
Authored: Mon Jun 6 02:24:12 2016 -0400
Committer: Dan LaRocque <da...@hopcount.org>
Committed: Mon Jun 6 03:10:03 2016 -0400

----------------------------------------------------------------------
 .../process/computer/GiraphWorkerContext.java   |   3 +-
 .../gremlin/structure/io/gryo/GryoMapper.java   |  30 ++-
 .../gremlin/structure/io/gryo/GryoPool.java     |   1 +
 .../structure/io/gryo/GryoSerializers.java      |  40 ++--
 .../structure/io/gryo/JavaTimeSerializers.java  | 127 +++++-------
 .../structure/io/gryo/PairSerializer.java       |  11 +-
 .../structure/io/gryo/TypeRegistration.java     |  12 ++
 .../io/gryo/kryoshim/KryoShimService.java       |  16 ++
 .../io/gryo/kryoshim/KryoShimServiceLoader.java |  23 ++-
 .../io/gryo/kryoshim/SerializerShim.java        |   2 +-
 .../hadoop/process/computer/HadoopCombine.java  |   3 +-
 .../hadoop/process/computer/HadoopMap.java      |   3 +-
 .../hadoop/process/computer/HadoopReduce.java   |   3 +-
 .../structure/io/HadoopPoolShimService.java     |   7 +
 .../structure/io/HadoopPoolsConfigurable.java   |   4 +-
 .../structure/io/gryo/GryoRecordReader.java     |   3 +-
 .../structure/io/gryo/GryoRecordWriter.java     |   4 +-
 .../spark/process/computer/SparkExecutor.java   |   3 +-
 .../structure/io/TinkerPopKryoRegistrator.java  | 121 ------------
 .../spark/structure/io/gryo/GryoSerializer.java |   2 +-
 .../io/gryo/IoRegistryAwareKryoSerializer.java  | 116 +++++++++++
 .../io/gryo/TinkerPopKryoRegistrator.java       | 194 +++++++++++++++++++
 .../unshaded/UnshadedKryoShimService.java       | 131 ++++++++-----
 ...n.structure.io.gryo.kryoshim.KryoShimService |   1 +
 .../spark/structure/io/ToyGraphInputRDD.java    |   3 +-
 25 files changed, 572 insertions(+), 291 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphWorkerContext.java
----------------------------------------------------------------------
diff --git a/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphWorkerContext.java b/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphWorkerContext.java
index 86b733c..0122ab4 100644
--- a/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphWorkerContext.java
+++ b/giraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/giraph/process/computer/GiraphWorkerContext.java
@@ -28,6 +28,7 @@ import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
 import org.apache.tinkerpop.gremlin.process.computer.VertexProgram;
 import org.apache.tinkerpop.gremlin.process.computer.util.ImmutableMemory;
 import org.apache.tinkerpop.gremlin.process.computer.util.VertexProgramPool;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader;
 
 import java.util.Iterator;
 
@@ -45,7 +46,7 @@ public final class GiraphWorkerContext extends WorkerContext {
 
     public void preApplication() throws InstantiationException, IllegalAccessException {
         final Configuration apacheConfiguration = ConfUtil.makeApacheConfiguration(this.getContext().getConfiguration());
-        HadoopPools.initialize(apacheConfiguration);
+        KryoShimServiceLoader.applyConfiguration(apacheConfiguration);
         final VertexProgram vertexProgram = VertexProgram.createVertexProgram(HadoopGraph.open(apacheConfiguration), apacheConfiguration);
         this.vertexProgramPool = new VertexProgramPool(vertexProgram, this.getContext().getConfiguration().getInt(GiraphConstants.NUM_COMPUTE_THREADS.getKey(), 1));
         this.memory = new GiraphMemory(this, vertexProgram);

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
index 851b03c..41ca44d 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
@@ -18,6 +18,7 @@
  */
 package org.apache.tinkerpop.gremlin.structure.io.gryo;
 
+import org.apache.commons.lang.builder.ToStringBuilder;
 import org.apache.tinkerpop.gremlin.process.computer.GraphFilter;
 import org.apache.tinkerpop.gremlin.process.computer.MapReduce;
 import org.apache.tinkerpop.gremlin.process.computer.util.MapMemory;
@@ -369,6 +370,20 @@ public final class GryoMapper implements Mapper<Kryo> {
         private Supplier<ClassResolver> classResolver = GryoClassResolver::new;
 
         private Builder() {
+            // Validate the default registrations
+            // For justification of these default registration rules, see TinkerPopKryoRegistrator
+            for (TypeRegistration<?> tr : typeRegistrations) {
+                if (tr.hasSerializer() /* no serializer is acceptable */ &&
+                    null == tr.getSerializerShim() /* a shim serializer is acceptable */ &&
+                    !(tr.getShadedSerializer() instanceof JavaSerializer) /* shaded JavaSerializer is acceptable */) {
+                    // everything else is invalid
+                    String msg = String.format("The default GryoMapper type registration %s is invalid.  " +
+                            "It must supply either an implementation of %s or %s, but supplies neither.  " +
+                            "This is probably a bug in GryoMapper's default serialization class registrations.", tr,
+                            SerializerShim.class.getCanonicalName(), JavaSerializer.class.getCanonicalName());
+                    throw new IllegalStateException(msg);
+                }
+            }
         }
 
         /**
@@ -538,8 +553,8 @@ public final class GryoMapper implements Mapper<Kryo> {
             if (1 < serializerCount) {
                 String msg = String.format(
                         "GryoTypeReg accepts at most one kind of serializer, but multiple " +
-                                "serializers were supplied for class %s (id %s).  " +
-                                "Shaded serializer: %s.  Shim serializer: %s.  Shaded serializer function: %s.",
+                        "serializers were supplied for class %s (id %s).  " +
+                        "Shaded serializer: %s.  Shim serializer: %s.  Shaded serializer function: %s.",
                         this.clazz.getCanonicalName(), id,
                         this.shadedSerializer, this.serializerShim, this.functionOfShadedKryo);
                 throw new IllegalArgumentException(msg);
@@ -603,5 +618,16 @@ public final class GryoMapper implements Mapper<Kryo> {
 
             return kryo;
         }
+
+        @Override
+        public String toString() {
+            return new ToStringBuilder(this)
+                    .append("targetClass", clazz)
+                    .append("id", id)
+                    .append("shadedSerializer", shadedSerializer)
+                    .append("serializerShim", serializerShim)
+                    .append("functionOfShadedKryo", functionOfShadedKryo)
+                    .toString();
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoPool.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoPool.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoPool.java
index e7bf636..59f8a5d 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoPool.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoPool.java
@@ -40,6 +40,7 @@ import java.util.function.Function;
 public final class GryoPool {
     public static final String CONFIG_IO_REGISTRY = "gremlin.io.registry";
     public static final String CONFIG_IO_GRYO_POOL_SIZE = "gremlin.io.gryo.poolSize";
+    public static final int CONFIG_IO_GRYO_POOL_SIZE_DEFAULT = 256;
 
     public enum Type {READER, WRITER, READER_WRITER}
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoSerializers.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoSerializers.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoSerializers.java
index ae99ac6..16fbe85 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoSerializers.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoSerializers.java
@@ -23,16 +23,16 @@ import org.apache.tinkerpop.gremlin.structure.Edge;
 import org.apache.tinkerpop.gremlin.structure.Property;
 import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.structure.VertexProperty;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.InputShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.OutputShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedEdge;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedFactory;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedPath;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedProperty;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedVertex;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedVertexProperty;
-import org.apache.tinkerpop.shaded.kryo.Kryo;
-import org.apache.tinkerpop.shaded.kryo.Serializer;
-import org.apache.tinkerpop.shaded.kryo.io.Input;
-import org.apache.tinkerpop.shaded.kryo.io.Output;
 
 /**
  * Class used to serialize graph-based objects such as vertices, edges, properties, and paths. These objects are
@@ -42,19 +42,19 @@ import org.apache.tinkerpop.shaded.kryo.io.Output;
  * @author Stephen Mallette (http://stephen.genoprime.com)
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
-final class GryoSerializers {
+public final class GryoSerializers {
 
     /**
      * Serializes any {@link Edge} implementation encountered to a {@link DetachedEdge}.
      */
-    final static class EdgeSerializer extends Serializer<Edge> {
+    final static class EdgeSerializer implements SerializerShim<Edge> {
         @Override
-        public void write(final Kryo kryo, final Output output, final Edge edge) {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Edge edge) {
             kryo.writeClassAndObject(output, DetachedFactory.detach(edge, true));
         }
 
         @Override
-        public Edge read(final Kryo kryo, final Input input, final Class<Edge> edgeClass) {
+        public <I extends InputShim> Edge read(KryoShim<I, ?> kryo, I input, Class<Edge> edgeClass) {
             final Object o = kryo.readClassAndObject(input);
             return (Edge) o;
         }
@@ -63,14 +63,14 @@ final class GryoSerializers {
     /**
      * Serializes any {@link Vertex} implementation encountered to an {@link DetachedVertex}.
      */
-    final static class VertexSerializer extends Serializer<Vertex> {
+    final static class VertexSerializer implements SerializerShim<Vertex> {
         @Override
-        public void write(final Kryo kryo, final Output output, final Vertex vertex) {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Vertex vertex) {
             kryo.writeClassAndObject(output, DetachedFactory.detach(vertex, true));
         }
 
         @Override
-        public Vertex read(final Kryo kryo, final Input input, final Class<Vertex> vertexClass) {
+        public <I extends InputShim> Vertex read(KryoShim<I, ?> kryo, I input, Class<Vertex> vertexClass) {
             return (Vertex) kryo.readClassAndObject(input);
         }
     }
@@ -78,14 +78,14 @@ final class GryoSerializers {
     /**
      * Serializes any {@link Property} implementation encountered to an {@link DetachedProperty}.
      */
-    final static class PropertySerializer extends Serializer<Property> {
+    final static class PropertySerializer implements SerializerShim<Property> {
         @Override
-        public void write(final Kryo kryo, final Output output, final Property property) {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Property property) {
             kryo.writeClassAndObject(output, property instanceof VertexProperty ? DetachedFactory.detach((VertexProperty) property, true) : DetachedFactory.detach(property));
         }
 
         @Override
-        public Property read(final Kryo kryo, final Input input, final Class<Property> propertyClass) {
+        public <I extends InputShim> Property read(KryoShim<I, ?> kryo, I input, Class<Property> propertyClass) {
             return (Property) kryo.readClassAndObject(input);
         }
     }
@@ -93,14 +93,14 @@ final class GryoSerializers {
     /**
      * Serializes any {@link VertexProperty} implementation encountered to an {@link DetachedVertexProperty}.
      */
-    final static class VertexPropertySerializer extends Serializer<VertexProperty> {
+    final static class VertexPropertySerializer implements SerializerShim<VertexProperty> {
         @Override
-        public void write(final Kryo kryo, final Output output, final VertexProperty vertexProperty) {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, VertexProperty vertexProperty) {
             kryo.writeClassAndObject(output, DetachedFactory.detach(vertexProperty, true));
         }
 
         @Override
-        public VertexProperty read(final Kryo kryo, final Input input, final Class<VertexProperty> vertexPropertyClass) {
+        public <I extends InputShim> VertexProperty read(KryoShim<I, ?> kryo, I input, Class<VertexProperty> vertexPropertyClass) {
             return (VertexProperty) kryo.readClassAndObject(input);
         }
     }
@@ -108,14 +108,14 @@ final class GryoSerializers {
     /**
      * Serializes any {@link Path} implementation encountered to an {@link DetachedPath}.
      */
-    final static class PathSerializer extends Serializer<Path> {
+    public final static class PathSerializer implements SerializerShim<Path> {
         @Override
-        public void write(final Kryo kryo, final Output output, final Path path) {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Path path) {
             kryo.writeClassAndObject(output, DetachedFactory.detach(path, false));
         }
 
         @Override
-        public Path read(final Kryo kryo, final Input input, final Class<Path> pathClass) {
+        public <I extends InputShim> Path read(KryoShim<I, ?> kryo, I input, Class<Path> pathClass) {
             return (Path) kryo.readClassAndObject(input);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/JavaTimeSerializers.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/JavaTimeSerializers.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/JavaTimeSerializers.java
index 1d4e236..8b14345 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/JavaTimeSerializers.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/JavaTimeSerializers.java
@@ -18,10 +18,10 @@
  */
 package org.apache.tinkerpop.gremlin.structure.io.gryo;
 
-import org.apache.tinkerpop.shaded.kryo.Kryo;
-import org.apache.tinkerpop.shaded.kryo.Serializer;
-import org.apache.tinkerpop.shaded.kryo.io.Input;
-import org.apache.tinkerpop.shaded.kryo.io.Output;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.InputShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.OutputShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
 
 import java.time.Duration;
 import java.time.Instant;
@@ -48,17 +48,14 @@ final class JavaTimeSerializers {
     /**
      * Serializer for the {@link Duration} class.
      */
-    final static class DurationSerializer extends Serializer<Duration>
-    {
+    final static class DurationSerializer implements SerializerShim<Duration> {
         @Override
-        public void write(final Kryo kryo, final Output output, final Duration duration)
-        {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Duration duration) {
             output.writeLong(duration.toNanos());
         }
 
         @Override
-        public Duration read(final Kryo kryo, final Input input, final Class<Duration> durationClass)
-        {
+        public <I extends InputShim> Duration read(KryoShim<I, ?> kryo, I input, Class<Duration> durationClass) {
             return Duration.ofNanos(input.readLong());
         }
     }
@@ -66,18 +63,15 @@ final class JavaTimeSerializers {
     /**
      * Serializer for the {@link Instant} class.
      */
-    final static class InstantSerializer extends Serializer<Instant>
-    {
+    final static class InstantSerializer implements SerializerShim<Instant> {
         @Override
-        public void write(Kryo kryo, Output output, Instant instant)
-        {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Instant instant) {
             output.writeLong(instant.getEpochSecond());
             output.writeInt(instant.getNano());
         }
 
         @Override
-        public Instant read(Kryo kryo, Input input, Class<Instant> aClass)
-        {
+        public <I extends InputShim> Instant read(KryoShim<I, ?> kryo, I input, Class<Instant> aClass) {
             return Instant.ofEpochSecond(input.readLong(), input.readInt());
         }
     }
@@ -85,17 +79,14 @@ final class JavaTimeSerializers {
     /**
      * Serializer for the {@link LocalDate} class.
      */
-    final static class LocalDateSerializer extends Serializer<LocalDate>
-    {
+    final static class LocalDateSerializer implements SerializerShim<LocalDate> {
         @Override
-        public void write(final Kryo kryo, final Output output, final LocalDate localDate)
-        {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, LocalDate localDate) {
             output.writeLong(localDate.toEpochDay());
         }
 
         @Override
-        public LocalDate read(final Kryo kryo, final Input input, final Class<LocalDate> clazz)
-        {
+        public <I extends InputShim> LocalDate read(KryoShim<I, ?> kryo, I input, Class<LocalDate> clazz) {
             return LocalDate.ofEpochDay(input.readLong());
         }
     }
@@ -103,11 +94,9 @@ final class JavaTimeSerializers {
     /**
      * Serializer for the {@link LocalDateTime} class.
      */
-    final static class LocalDateTimeSerializer extends Serializer<LocalDateTime>
-    {
+    final static class LocalDateTimeSerializer implements SerializerShim<LocalDateTime> {
         @Override
-        public void write(final Kryo kryo, final Output output, final LocalDateTime localDateTime)
-        {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, LocalDateTime localDateTime) {
             output.writeInt(localDateTime.getYear());
             output.writeInt(localDateTime.getMonthValue());
             output.writeInt(localDateTime.getDayOfMonth());
@@ -118,8 +107,7 @@ final class JavaTimeSerializers {
         }
 
         @Override
-        public LocalDateTime read(final Kryo kryo, final Input input, final Class<LocalDateTime> clazz)
-        {
+        public <I extends InputShim> LocalDateTime read(KryoShim<I, ?> kryo, I input, Class<LocalDateTime> clazz) {
             return LocalDateTime.of(input.readInt(), input.readInt(), input.readInt(), input.readInt(), input.readInt(), input.readInt(), input.readInt());
         }
     }
@@ -127,17 +115,14 @@ final class JavaTimeSerializers {
     /**
      * Serializer for the {@link LocalTime} class.
      */
-    final static class LocalTimeSerializer extends Serializer<LocalTime>
-    {
+    final static class LocalTimeSerializer implements SerializerShim<LocalTime> {
         @Override
-        public void write(final Kryo kryo, final Output output, final LocalTime localTime)
-        {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, LocalTime localTime) {
             output.writeLong(localTime.toNanoOfDay());
         }
 
         @Override
-        public LocalTime read(final Kryo kryo, final Input input, final Class<LocalTime> clazz)
-        {
+        public <I extends InputShim> LocalTime read(KryoShim<I, ?> kryo, I input, Class<LocalTime> clazz) {
             return LocalTime.ofNanoOfDay(input.readLong());
         }
     }
@@ -145,37 +130,31 @@ final class JavaTimeSerializers {
     /**
      * Serializer for the {@link MonthDay} class.
      */
-    final static class MonthDaySerializer extends Serializer<MonthDay>
-    {
+    final static class MonthDaySerializer implements SerializerShim<MonthDay> {
         @Override
-        public void write(final Kryo kryo, final Output output, final MonthDay monthDay)
-        {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, MonthDay monthDay) {
             output.writeInt(monthDay.getMonthValue());
             output.writeInt(monthDay.getDayOfMonth());
         }
 
         @Override
-        public MonthDay read(final Kryo kryo, final Input input, final Class<MonthDay> clazz)
-        {
-            return MonthDay.of(input.readInt(), input.readInt());
+        public <I extends InputShim> MonthDay read(KryoShim<I, ?> kryo, I input, Class<MonthDay> clazz) {
+            return null;
         }
     }
 
     /**
      * Serializer for the {@link OffsetDateTime} class.
      */
-    final static class OffsetDateTimeSerializer extends Serializer<OffsetDateTime>
-    {
+    final static class OffsetDateTimeSerializer implements SerializerShim<OffsetDateTime> {
         @Override
-        public void write(final Kryo kryo, final Output output, final OffsetDateTime offsetDateTime)
-        {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, OffsetDateTime offsetDateTime) {
             kryo.writeObject(output, offsetDateTime.toLocalDateTime());
             kryo.writeObject(output, offsetDateTime.getOffset());
         }
 
         @Override
-        public OffsetDateTime read(final Kryo kryo, final Input input, final Class<OffsetDateTime> clazz)
-        {
+        public <I extends InputShim> OffsetDateTime read(KryoShim<I, ?> kryo, I input, Class<OffsetDateTime> clazz) {
             return OffsetDateTime.of(kryo.readObject(input, LocalDateTime.class), kryo.readObject(input, ZoneOffset.class));
         }
     }
@@ -183,18 +162,15 @@ final class JavaTimeSerializers {
     /**
      * Serializer for the {@link OffsetTime} class.
      */
-    final static class OffsetTimeSerializer extends Serializer<OffsetTime>
-    {
+    final static class OffsetTimeSerializer implements SerializerShim<OffsetTime> {
         @Override
-        public void write(final Kryo kryo, final Output output, final OffsetTime offsetTime)
-        {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, OffsetTime offsetTime) {
             kryo.writeObject(output, offsetTime.toLocalTime());
             kryo.writeObject(output, offsetTime.getOffset());
         }
 
         @Override
-        public OffsetTime read(final Kryo kryo, final Input input, final Class<OffsetTime> clazz)
-        {
+        public <I extends InputShim> OffsetTime read(KryoShim<I, ?> kryo, I input, Class<OffsetTime> clazz) {
             return OffsetTime.of(kryo.readObject(input, LocalTime.class), kryo.readObject(input, ZoneOffset.class));
         }
     }
@@ -202,19 +178,16 @@ final class JavaTimeSerializers {
     /**
      * Serializer for the {@link Period} class.
      */
-    final static class PeriodSerializer extends Serializer<Period>
-    {
+    final static class PeriodSerializer implements SerializerShim<Period> {
         @Override
-        public void write(final Kryo kryo, final Output output, final Period period)
-        {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Period period) {
             output.writeInt(period.getYears());
             output.writeInt(period.getMonths());
             output.writeInt(period.getDays());
         }
 
         @Override
-        public Period read(final Kryo kryo, final Input input, final Class<Period> clazz)
-        {
+        public <I extends InputShim> Period read(KryoShim<I, ?> kryo, I input, Class<Period> clazz) {
             return Period.of(input.readInt(), input.readInt(), input.readInt());
         }
     }
@@ -222,17 +195,14 @@ final class JavaTimeSerializers {
     /**
      * Serializer for the {@link Year} class.
      */
-    final static class YearSerializer extends Serializer<Year>
-    {
+    final static class YearSerializer implements SerializerShim<Year> {
         @Override
-        public void write(final Kryo kryo, final Output output, final Year year)
-        {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Year year) {
             output.writeInt(year.getValue());
         }
 
         @Override
-        public Year read(final Kryo kryo, final Input input, final Class<Year> clazz)
-        {
+        public <I extends InputShim> Year read(KryoShim<I, ?> kryo, I input, Class<Year> clazz) {
             return Year.of(input.readInt());
         }
     }
@@ -240,18 +210,15 @@ final class JavaTimeSerializers {
     /**
      * Serializer for the {@link YearMonth} class.
      */
-    final static class YearMonthSerializer extends Serializer<YearMonth>
-    {
+    final static class YearMonthSerializer implements SerializerShim<YearMonth> {
         @Override
-        public void write(final Kryo kryo, final Output output, final YearMonth monthDay)
-        {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, YearMonth monthDay) {
             output.writeInt(monthDay.getYear());
             output.writeInt(monthDay.getMonthValue());
         }
 
         @Override
-        public YearMonth read(final Kryo kryo, final Input input, final Class<YearMonth> clazz)
-        {
+        public <I extends InputShim> YearMonth read(KryoShim<I, ?> kryo, I input, Class<YearMonth> clazz) {
             return YearMonth.of(input.readInt(), input.readInt());
         }
     }
@@ -259,11 +226,9 @@ final class JavaTimeSerializers {
     /**
      * Serializer for the {@link ZonedDateTime} class.
      */
-    final static class ZonedDateTimeSerializer extends Serializer<ZonedDateTime>
-    {
+    final static class ZonedDateTimeSerializer implements SerializerShim<ZonedDateTime> {
         @Override
-        public void write(final Kryo kryo, final Output output, final ZonedDateTime zonedDateTime)
-        {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, ZonedDateTime zonedDateTime) {
             output.writeInt(zonedDateTime.getYear());
             output.writeInt(zonedDateTime.getMonthValue());
             output.writeInt(zonedDateTime.getDayOfMonth());
@@ -275,8 +240,7 @@ final class JavaTimeSerializers {
         }
 
         @Override
-        public ZonedDateTime read(final Kryo kryo, final Input input, final Class<ZonedDateTime> clazz)
-        {
+        public <I extends InputShim> ZonedDateTime read(KryoShim<I, ?> kryo, I input, Class<ZonedDateTime> clazz) {
             return ZonedDateTime.of(input.readInt(), input.readInt(), input.readInt(),
                     input.readInt(), input.readInt(), input.readInt(), input.readInt(),
                     ZoneId.of(input.readString()));
@@ -286,17 +250,14 @@ final class JavaTimeSerializers {
     /**
      * Serializer for the {@link ZoneOffset} class.
      */
-    final static class ZoneOffsetSerializer extends Serializer<ZoneOffset>
-    {
+    final static class ZoneOffsetSerializer implements SerializerShim<ZoneOffset> {
         @Override
-        public void write(final Kryo kryo, final Output output, final ZoneOffset zoneOffset)
-        {
+        public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, ZoneOffset zoneOffset) {
             output.writeString(zoneOffset.getId());
         }
 
         @Override
-        public ZoneOffset read(final Kryo kryo, final Input input, final Class<ZoneOffset> clazz)
-        {
+        public <I extends InputShim> ZoneOffset read(KryoShim<I, ?> kryo, I input, Class<ZoneOffset> clazz) {
             return ZoneOffset.of(input.readString());
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/PairSerializer.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/PairSerializer.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/PairSerializer.java
index e5e92e7..0464b22 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/PairSerializer.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/PairSerializer.java
@@ -18,6 +18,10 @@
  */
 package org.apache.tinkerpop.gremlin.structure.io.gryo;
 
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.InputShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.OutputShim;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
 import org.apache.tinkerpop.shaded.kryo.Kryo;
 import org.apache.tinkerpop.shaded.kryo.Serializer;
 import org.apache.tinkerpop.shaded.kryo.io.Input;
@@ -27,16 +31,15 @@ import org.javatuples.Pair;
 /**
  * @author Daniel Kuppitz (http://gremlin.guru)
  */
-final class PairSerializer extends Serializer<Pair> {
-
+final class PairSerializer implements SerializerShim<Pair> {
     @Override
-    public void write(final Kryo kryo, final Output output, final Pair pair) {
+    public <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, Pair pair) {
         kryo.writeClassAndObject(output, pair.getValue0());
         kryo.writeClassAndObject(output, pair.getValue1());
     }
 
     @Override
-    public Pair read(final Kryo kryo, final Input input, final Class<Pair> pairClass) {
+    public <I extends InputShim> Pair read(KryoShim<I, ?> kryo, I input, Class<Pair> pairClass) {
         return Pair.with(kryo.readClassAndObject(input), kryo.readClassAndObject(input));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java
index ef105ce..1f41c0d 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java
@@ -67,4 +67,16 @@ public interface TypeRegistration<T> {
      * @return the sole parameter
      */
     Kryo registerWith(Kryo kryo);
+
+    /**
+     * Returns true if at least one of {@link #getShadedSerializer()}, {@link #getSerializerShim()}, or
+     * {@link #getFunctionOfShadedKryo()} is non null.  Returns false if all are null.
+     *
+     * @return whether a serializer is defined for this type registration
+     */
+    default boolean hasSerializer() {
+        return null != getFunctionOfShadedKryo() ||
+                null != getSerializerShim() ||
+                null != getShadedSerializer();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java
index 959605c..7783856 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimService.java
@@ -18,6 +18,8 @@
  */
 package org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim;
 
+import org.apache.commons.configuration.Configuration;
+
 import java.io.InputStream;
 import java.io.OutputStream;
 
@@ -80,4 +82,18 @@ public interface KryoShimService {
      * @return this implementation's priority value
      */
     int getPriority();
+
+    /**
+     * Attempt to incorporate the supplied configuration in future read/write calls.
+     * <p>
+     * This method is a wart that exists essentially just to support the old
+     * {@link HadoopPools#initialize(Configuration)} use-case.
+     * <p>
+     * This method is not guaranteed to have any effect on an instance of this interface
+     * after {@link #writeClassAndObject(Object, OutputStream)} or {@link #readClassAndObject(InputStream)}
+     * has been invoked on that particular instance.
+     *
+     * @param conf the configuration to apply to this service's internal serializer
+     */
+    void applyConfiguration(Configuration conf);
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
index 9ccf2de..9184dd0 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/KryoShimServiceLoader.java
@@ -18,6 +18,7 @@
  */
 package org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim;
 
+import org.apache.commons.configuration.Configuration;
 import org.apache.tinkerpop.shaded.kryo.io.Input;
 import org.apache.tinkerpop.shaded.kryo.io.Output;
 import org.slf4j.Logger;
@@ -35,7 +36,9 @@ import java.util.ServiceLoader;
  */
 public class KryoShimServiceLoader {
 
-    private static volatile KryoShimService CACHED_SHIM_SERVICE;
+    private static volatile KryoShimService cachedShimService;
+
+    private static volatile Configuration conf;
 
     private static final Logger log = LoggerFactory.getLogger(KryoShimServiceLoader.class);
 
@@ -46,6 +49,10 @@ public class KryoShimServiceLoader {
      */
     public static final String SHIM_CLASS_SYSTEM_PROPERTY = "tinkerpop.kryo.shim";
 
+    public static void applyConfiguration(Configuration conf) {
+        KryoShimServiceLoader.conf = conf;
+    }
+
     /**
      * Return a reference to the shim service.  This method may return a cached shim service
      * unless {@code forceReload} is true.  Calls to this method need not be externally
@@ -58,8 +65,8 @@ public class KryoShimServiceLoader {
      */
     public static KryoShimService load(boolean forceReload) {
 
-        if (null != CACHED_SHIM_SERVICE && !forceReload) {
-            return CACHED_SHIM_SERVICE;
+        if (null != cachedShimService && !forceReload) {
+            return cachedShimService;
         }
 
         ArrayList<KryoShimService> services = new ArrayList<>();
@@ -109,7 +116,15 @@ public class KryoShimServiceLoader {
         log.info("Set {} provider to {} ({}) because its priority value ({}) is the highest available",
                 KryoShimService.class.getSimpleName(), result, result.getClass(), result.getPriority());
 
-        return CACHED_SHIM_SERVICE = result;
+        Configuration userConf = conf;
+
+        if (null != userConf) {
+            log.info("Configuring {} provider {} with user-provided configuration",
+                    KryoShimService.class.getSimpleName(), result);
+            result.applyConfiguration(userConf);
+        }
+
+        return cachedShimService = result;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/SerializerShim.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/SerializerShim.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/SerializerShim.java
index 191cdd8..e5f9005 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/SerializerShim.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/kryoshim/SerializerShim.java
@@ -26,7 +26,7 @@ package org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim;
  */
 public interface SerializerShim<T> {
 
-    <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, T starGraph);
+    <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, T object);
 
     <I extends InputShim> T read(KryoShim<I, ?> kryo, I input, Class<T> clazz);
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/HadoopCombine.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/HadoopCombine.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/HadoopCombine.java
index de1e2f9..06778e6 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/HadoopCombine.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/HadoopCombine.java
@@ -25,6 +25,7 @@ import org.apache.tinkerpop.gremlin.hadoop.structure.io.HadoopPools;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable;
 import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
 import org.apache.tinkerpop.gremlin.process.computer.MapReduce;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader;
 import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -47,7 +48,7 @@ public final class HadoopCombine extends Reducer<ObjectWritable, ObjectWritable,
     @Override
     public void setup(final Reducer<ObjectWritable, ObjectWritable, ObjectWritable, ObjectWritable>.Context context) {
         final Configuration apacheConfiguration = ConfUtil.makeApacheConfiguration(context.getConfiguration());
-        HadoopPools.initialize(apacheConfiguration);
+        KryoShimServiceLoader.applyConfiguration(apacheConfiguration);
         this.mapReduce = MapReduce.createMapReduce(HadoopGraph.open(apacheConfiguration), apacheConfiguration);
         this.mapReduce.workerStart(MapReduce.Stage.COMBINE);
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/HadoopMap.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/HadoopMap.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/HadoopMap.java
index 9e6fac3..5fc7026 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/HadoopMap.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/HadoopMap.java
@@ -28,6 +28,7 @@ import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
 import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
 import org.apache.tinkerpop.gremlin.process.computer.MapReduce;
 import org.apache.tinkerpop.gremlin.process.computer.util.ComputerGraph;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -50,7 +51,7 @@ public final class HadoopMap extends Mapper<NullWritable, VertexWritable, Object
     @Override
     public void setup(final Mapper<NullWritable, VertexWritable, ObjectWritable, ObjectWritable>.Context context) {
         final Configuration apacheConfiguration = ConfUtil.makeApacheConfiguration(context.getConfiguration());
-        HadoopPools.initialize(apacheConfiguration);
+        KryoShimServiceLoader.applyConfiguration(apacheConfiguration);
         this.mapReduce = MapReduce.createMapReduce(HadoopGraph.open(apacheConfiguration), apacheConfiguration);
         this.mapReduce.workerStart(MapReduce.Stage.MAP);
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/HadoopReduce.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/HadoopReduce.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/HadoopReduce.java
index 06dfba1..6ca7b8f 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/HadoopReduce.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/process/computer/HadoopReduce.java
@@ -25,6 +25,7 @@ import org.apache.tinkerpop.gremlin.hadoop.structure.io.HadoopPools;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable;
 import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
 import org.apache.tinkerpop.gremlin.process.computer.MapReduce;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader;
 import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -47,7 +48,7 @@ public final class HadoopReduce extends Reducer<ObjectWritable, ObjectWritable,
     @Override
     public void setup(final Reducer<ObjectWritable, ObjectWritable, ObjectWritable, ObjectWritable>.Context context) {
         final Configuration apacheConfiguration = ConfUtil.makeApacheConfiguration(context.getConfiguration());
-        HadoopPools.initialize(apacheConfiguration);
+        KryoShimServiceLoader.applyConfiguration(apacheConfiguration);
         this.mapReduce = MapReduce.createMapReduce(HadoopGraph.open(apacheConfiguration), apacheConfiguration);
         this.mapReduce.workerStart(MapReduce.Stage.REDUCE);
     }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
index c19b914..5753d90 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolShimService.java
@@ -18,7 +18,9 @@
  */
 package org.apache.tinkerpop.gremlin.hadoop.structure.io;
 
+import org.apache.commons.configuration.Configuration;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimService;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader;
 import org.apache.tinkerpop.shaded.kryo.Kryo;
 import org.apache.tinkerpop.shaded.kryo.io.Input;
 import org.apache.tinkerpop.shaded.kryo.io.Output;
@@ -66,4 +68,9 @@ public class HadoopPoolShimService implements KryoShimService {
     public int getPriority() {
         return 0;
     }
+
+    @Override
+    public void applyConfiguration(Configuration conf) {
+        KryoShimServiceLoader.applyConfiguration(conf);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolsConfigurable.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolsConfigurable.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolsConfigurable.java
index f3a1bac..0e5f135 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolsConfigurable.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/HadoopPoolsConfigurable.java
@@ -20,6 +20,8 @@ package org.apache.tinkerpop.gremlin.hadoop.structure.io;
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader;
 
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
@@ -28,7 +30,7 @@ public interface HadoopPoolsConfigurable extends Configurable {
 
     @Override
     public default void setConf(final Configuration configuration) {
-        HadoopPools.initialize(configuration);
+        KryoShimServiceLoader.applyConfiguration(ConfUtil.makeApacheConfiguration(configuration));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/gryo/GryoRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/gryo/GryoRecordReader.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/gryo/GryoRecordReader.java
index d7ed46b..a1daddf 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/gryo/GryoRecordReader.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/gryo/GryoRecordReader.java
@@ -37,6 +37,7 @@ import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoMapper;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoReader;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.VertexTerminator;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -71,7 +72,7 @@ public final class GryoRecordReader extends RecordReader<NullWritable, VertexWri
         final Configuration configuration = context.getConfiguration();
         if (configuration.get(Constants.GREMLIN_HADOOP_GRAPH_FILTER, null) != null)
             this.graphFilter = VertexProgramHelper.deserialize(ConfUtil.makeApacheConfiguration(configuration), Constants.GREMLIN_HADOOP_GRAPH_FILTER);
-        HadoopPools.initialize(configuration);
+        KryoShimServiceLoader.applyConfiguration(ConfUtil.makeApacheConfiguration(configuration));
         this.gryoReader = HadoopPools.getGryoPool().takeReader();
         long start = split.getStart();
         final Path file = split.getPath();

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/gryo/GryoRecordWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/gryo/GryoRecordWriter.java b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/gryo/GryoRecordWriter.java
index 67a8339..2ea3394 100644
--- a/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/gryo/GryoRecordWriter.java
+++ b/hadoop-gremlin/src/main/java/org/apache/tinkerpop/gremlin/hadoop/structure/io/gryo/GryoRecordWriter.java
@@ -25,8 +25,10 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.tinkerpop.gremlin.hadoop.Constants;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.HadoopPools;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
+import org.apache.tinkerpop.gremlin.hadoop.structure.util.ConfUtil;
 import org.apache.tinkerpop.gremlin.structure.Direction;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoWriter;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -43,7 +45,7 @@ public final class GryoRecordWriter extends RecordWriter<NullWritable, VertexWri
     public GryoRecordWriter(final DataOutputStream outputStream, final Configuration configuration) {
         this.outputStream = outputStream;
         this.hasEdges = configuration.getBoolean(Constants.GREMLIN_HADOOP_GRAPH_WRITER_HAS_EDGES, true);
-        HadoopPools.initialize(configuration);
+        KryoShimServiceLoader.applyConfiguration(ConfUtil.makeApacheConfiguration(configuration));
         this.gryoWriter = HadoopPools.getGryoPool().takeWriter();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkExecutor.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkExecutor.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkExecutor.java
index c2b85dd..9e5ac53 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkExecutor.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/process/computer/SparkExecutor.java
@@ -38,6 +38,7 @@ import org.apache.tinkerpop.gremlin.spark.process.computer.payload.Payload;
 import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewIncomingPayload;
 import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewOutgoingPayload;
 import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewPayload;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader;
 import org.apache.tinkerpop.gremlin.structure.util.Attachable;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedFactory;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedVertexProperty;
@@ -89,7 +90,7 @@ public final class SparkExecutor {
                 graphRDD.leftOuterJoin(viewIncomingRDD))                                                   // every other iteration may have views and messages
                 // for each partition of vertices emit a view and their outgoing messages
                 .mapPartitionsToPair(partitionIterator -> {
-                    HadoopPools.initialize(apacheConfiguration);
+                    KryoShimServiceLoader.applyConfiguration(apacheConfiguration);
                     final VertexProgram<M> workerVertexProgram = VertexProgram.<VertexProgram<M>>createVertexProgram(HadoopGraph.open(apacheConfiguration), apacheConfiguration); // each partition(Spark)/worker(TP3) has a local copy of the vertex program (a worker's task)
                     final String[] vertexComputeKeysArray = VertexProgramHelper.vertexComputeKeysAsArray(workerVertexProgram.getVertexComputeKeys()); // the compute keys as an array
                     final SparkMessenger<M> messenger = new SparkMessenger<>();

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/TinkerPopKryoRegistrator.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/TinkerPopKryoRegistrator.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/TinkerPopKryoRegistrator.java
deleted file mode 100644
index 4c99e70..0000000
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/TinkerPopKryoRegistrator.java
+++ /dev/null
@@ -1,121 +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;
-
-import com.esotericsoftware.kryo.Kryo;
-import com.esotericsoftware.kryo.Serializer;
-import com.esotericsoftware.kryo.serializers.JavaSerializer;
-import com.google.common.base.Preconditions;
-import org.apache.spark.serializer.KryoRegistrator;
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable;
-import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
-import org.apache.tinkerpop.gremlin.process.traversal.step.map.GroupStep;
-import org.apache.tinkerpop.gremlin.process.traversal.step.map.OrderGlobalStep;
-import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalExplanation;
-import org.apache.tinkerpop.gremlin.spark.process.computer.payload.MessagePayload;
-import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewIncomingPayload;
-import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewOutgoingPayload;
-import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewPayload;
-import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.ObjectWritableSerializer;
-import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.VertexWritableSerializer;
-import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.kryoshim.unshaded.UnshadedSerializerAdapter;
-import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoMapper;
-import org.apache.tinkerpop.gremlin.structure.io.gryo.TypeRegistration;
-import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
-import org.apache.tinkerpop.gremlin.structure.util.star.StarGraph;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * A spark.kryo.registrator implementation that installs TinkerPop types.
- * This is intended for use with spark.serializer=KryoSerializer, not GryoSerializer.
- */
-public class TinkerPopKryoRegistrator implements KryoRegistrator {
-
-    private static final Logger log = LoggerFactory.getLogger(TinkerPopKryoRegistrator.class);
-
-    @Override
-    public void registerClasses(Kryo kryo) {
-        // TinkerPop type registrations copied from GyroSerializer's constructor
-        kryo.register(MessagePayload.class);
-        kryo.register(ViewIncomingPayload.class);
-        kryo.register(ViewOutgoingPayload.class);
-        kryo.register(ViewPayload.class);
-        kryo.register(VertexWritable.class, new UnshadedSerializerAdapter<>(new VertexWritableSerializer()));
-        kryo.register(ObjectWritable.class, new UnshadedSerializerAdapter<>(new ObjectWritableSerializer<>()));
-
-        Set<Class<?>> shimmedClasses = new HashSet<>();
-
-        Set<Class<?>> javaSerializationClasses = new HashSet<>();
-
-        // Copy GryoMapper's default registrations
-        for (TypeRegistration<?> tr : GryoMapper.build().create().getTypeRegistrations()) {
-            // Special case for JavaSerializer, which is generally implemented in terms of TinkerPop's
-            // problematic static GryoMapper/GryoSerializer pool (these are handled below the loop)
-            org.apache.tinkerpop.shaded.kryo.Serializer<?> shadedSerializer = tr.getShadedSerializer();
-            SerializerShim<?> serializerShim = tr.getSerializerShim();
-            if (null != shadedSerializer &&
-                    shadedSerializer.getClass().equals(org.apache.tinkerpop.shaded.kryo.serializers.JavaSerializer.class)) {
-                javaSerializationClasses.add(tr.getTargetClass());
-            } else if (null != serializerShim) {
-                log.debug("Registering class {} to serializer shim {} (serializer shim class {})",
-                        tr.getTargetClass(), serializerShim, serializerShim.getClass());
-                kryo.register(tr.getTargetClass(), new UnshadedSerializerAdapter<>(serializerShim));
-                shimmedClasses.add(tr.getTargetClass());
-            } else {
-                // Register with the default behavior (FieldSerializer)
-                log.debug("Registering class {} with default serializer", tr.getTargetClass());
-                kryo.register(tr.getTargetClass());
-            }
-        }
-
-        Map<Class<?>, Serializer<?>> javaSerializerReplacements = new HashMap<>();
-        javaSerializerReplacements.put(GroupStep.GroupBiOperator.class, new JavaSerializer());
-        javaSerializerReplacements.put(OrderGlobalStep.OrderBiOperator.class, null);
-        javaSerializerReplacements.put(TraversalExplanation.class, null);
-
-        for (Map.Entry<Class<?>, Serializer<?>> e : javaSerializerReplacements.entrySet()) {
-            Class<?> c = e.getKey();
-            Serializer<?> s = e.getValue();
-
-            if (javaSerializationClasses.remove(c)) {
-                if (null != s) {
-                    log.debug("Registering class {} with serializer {}", c, s);
-                    kryo.register(c, s);
-                } else {
-                    log.debug("Registering class {} with default serializer", c);
-                    kryo.register(c);
-                }
-            } else {
-                log.debug("Registering class {} with JavaSerializer", c);
-                kryo.register(c, new JavaSerializer());
-            }
-        }
-
-        // We really care about StarGraph's shim serializer, so make sure we registered it
-        if (!shimmedClasses.contains(StarGraph.class)) {
-            log.warn("No SerializerShim found for StarGraph");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoSerializer.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoSerializer.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoSerializer.java
index 2c1dfa2..28a4d55 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoSerializer.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoSerializer.java
@@ -78,7 +78,7 @@ public final class GryoSerializer extends Serializer {
             }
         }
         this.gryoPool = GryoPool.build().
-                poolSize(sparkConfiguration.getInt(GryoPool.CONFIG_IO_GRYO_POOL_SIZE, 256)).
+                poolSize(sparkConfiguration.getInt(GryoPool.CONFIG_IO_GRYO_POOL_SIZE, GryoPool.CONFIG_IO_GRYO_POOL_SIZE_DEFAULT)).
                 ioRegistries(makeApacheConfiguration(sparkConfiguration).getList(GryoPool.CONFIG_IO_REGISTRY, Collections.emptyList())).
                 initializeMapper(builder -> {
                     try {

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/IoRegistryAwareKryoSerializer.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/IoRegistryAwareKryoSerializer.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/IoRegistryAwareKryoSerializer.java
new file mode 100644
index 0000000..8b21e21
--- /dev/null
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/IoRegistryAwareKryoSerializer.java
@@ -0,0 +1,116 @@
+/*
+ * 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.
+ */
+
+/**
+ * Copyright DataStax, Inc.
+ * <p>
+ * Please see the included license file for details.
+ */
+package org.apache.tinkerpop.gremlin.spark.structure.io.gryo;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import org.apache.spark.SparkConf;
+import org.apache.spark.serializer.KryoSerializer;
+import org.apache.tinkerpop.gremlin.structure.io.IoRegistry;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoIo;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoPool;
+import org.javatuples.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+/**
+ * A {@link KryoSerializer} that attempts to honor {@link GryoPool#CONFIG_IO_REGISTRY}.
+ */
+public class IoRegistryAwareKryoSerializer extends KryoSerializer {
+
+    private final SparkConf conf;
+
+    private static final Logger log = LoggerFactory.getLogger(IoRegistryAwareKryoSerializer.class);
+
+    public IoRegistryAwareKryoSerializer(SparkConf conf) {
+        super(conf);
+        // store conf so that we can access its registry (if one is present) in newKryo()
+        this.conf = conf;
+    }
+
+    @Override
+    public Kryo newKryo() {
+        Kryo kryo = super.newKryo();
+
+        return applyIoRegistryIfPresent(kryo);
+    }
+
+    private Kryo applyIoRegistryIfPresent(Kryo kryo) {
+        if (!conf.contains(GryoPool.CONFIG_IO_REGISTRY)) {
+            log.info("SparkConf {} does not contain setting {}, skipping {} handling",
+                    GryoPool.CONFIG_IO_REGISTRY, conf, IoRegistry.class.getCanonicalName());
+            return kryo;
+        }
+
+        String registryClassnames = conf.get(GryoPool.CONFIG_IO_REGISTRY);
+
+        for (String registryClassname : registryClassnames.split(",")) {
+            final IoRegistry registry;
+
+            try {
+                registry = (IoRegistry) Class.forName(registryClassname).newInstance();
+                log.info("Instantiated {}", registryClassname);
+            } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
+                log.error("Unable to reflectively instantiate the {} implementation named {}",
+                        IoRegistry.class.getCanonicalName(), registryClassname, e);
+                return kryo;
+            }
+
+            // Left is the class targeted for serialization, right is a mess of potential types, including
+            // a shaded Serializer impl, unshaded Serializer impl, or Function<shaded.Kryo,shaded.Serializer>
+            final List<Pair<Class, Object>> serializers = registry.find(GryoIo.class);
+
+            if (null == serializers) {
+                log.info("Invoking find({}.class) returned null on registry {}; ignoring this registry",
+                        GryoIo.class.getCanonicalName(), registry);
+                return kryo;
+            }
+
+            for (Pair<Class, Object> p : serializers) {
+                if (null == p.getValue1()) {
+                    // null on the right is fine
+                    log.info("Registering {} with default serializer", p.getValue0());
+                    kryo.register(p.getValue0());
+                } else if (p.getValue1() instanceof Serializer) {
+                    // unshaded serializer on the right is fine
+                    log.info("Registering {} with serializer {}", p.getValue0(), p.getValue1());
+                    kryo.register(p.getValue0(), (Serializer) p.getValue1());
+                } else {
+                    // anything else on the right is unsupported with Spark
+                    log.error("Serializer {} found in {} must implement {} " +
+                                    "(the shaded interface {} is not supported on Spark).  This class will be registered with " +
+                                    "the default behavior of Spark's KryoSerializer.",
+                            p.getValue1(), registryClassname, Serializer.class.getCanonicalName(),
+                            org.apache.tinkerpop.shaded.kryo.Serializer.class.getCanonicalName());
+                    kryo.register(p.getValue0());
+                }
+            }
+        }
+
+        return kryo;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/TinkerPopKryoRegistrator.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/TinkerPopKryoRegistrator.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/TinkerPopKryoRegistrator.java
new file mode 100644
index 0000000..bdb80fd
--- /dev/null
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/TinkerPopKryoRegistrator.java
@@ -0,0 +1,194 @@
+/*
+ * 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.gryo;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.serializers.JavaSerializer;
+import org.apache.spark.serializer.KryoRegistrator;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
+import org.apache.tinkerpop.gremlin.spark.process.computer.payload.MessagePayload;
+import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewIncomingPayload;
+import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewOutgoingPayload;
+import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewPayload;
+import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.kryoshim.unshaded.UnshadedSerializerAdapter;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoMapper;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.TypeRegistration;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
+import org.apache.tinkerpop.gremlin.structure.util.star.StarGraph;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+
+/**
+ * A spark.kryo.registrator implementation that installs TinkerPop types.
+ * This is intended for use with spark.serializer=KryoSerializer, not GryoSerializer.
+ */
+public class TinkerPopKryoRegistrator implements KryoRegistrator {
+
+    private static final Logger log = LoggerFactory.getLogger(TinkerPopKryoRegistrator.class);
+
+    @Override
+    public void registerClasses(Kryo kryo) {
+        registerClasses(kryo, Collections.emptyMap(), Collections.emptySet());
+    }
+
+    /**
+     * Register TinkerPop's classes with the supplied {@link Kryo} instance
+     * while honoring optional overrides and optional class blacklist ("blackset"?).
+     *
+     * @param kryo the Kryo serializer instance with which to register types
+     * @param serializerOverrides serializer mappings that override this class's defaults
+     * @param blacklist classes which should not be registered at all, even if there is an override entry
+     *                  or if they would be registered by this class by default (does not affect Kryo's
+     *                  built-in registrations, e.g. String.class).
+     */
+    public void registerClasses(Kryo kryo, Map<Class<?>, Serializer<?>> serializerOverrides, Set<Class<?>> blacklist) {
+        // Apply TinkerPop type registrations copied from GyroSerializer's constructor
+        for (Map.Entry<Class<?>, Serializer<?>> ent : getExtraRegistrations().entrySet()) {
+            Class<?> targetClass = ent.getKey();
+            Serializer<?> ser = ent.getValue();
+
+            // Is this class blacklisted?  Skip it. (takes precedence over serializerOverrides)
+            if (blacklist.contains(targetClass)) {
+                log.debug("Not registering serializer for {} (blacklisted)", targetClass);
+                continue;
+            }
+
+            if (checkForAndApplySerializerOverride(serializerOverrides, kryo, targetClass)) {
+                // do nothing but skip the remaining else(-if) clauses
+            } else if (null == ser) {
+                log.debug("Registering {} with default serializer", targetClass);
+                kryo.register(targetClass);
+            } else {
+                log.debug("Registering {} with serializer {}", targetClass, ser);
+                kryo.register(targetClass, ser);
+            }
+        }
+
+        Set<Class<?>> shimmedClassesFromGryoMapper = new HashSet<>();
+
+        // Apply GryoMapper's default registrations
+        for (TypeRegistration<?> tr : GryoMapper.build().create().getTypeRegistrations()) {
+            // Is this class blacklisted?  Skip it. (takes precedence over serializerOverrides)
+            if (blacklist.contains(tr.getTargetClass())) {
+                log.debug("Not registering serializer for {} (blacklisted)", tr.getTargetClass());
+                continue;
+            }
+
+            final org.apache.tinkerpop.shaded.kryo.Serializer<?> shadedSerializer = tr.getShadedSerializer();
+            final SerializerShim<?> serializerShim = tr.getSerializerShim();
+            final java.util.function.Function<
+                    org.apache.tinkerpop.shaded.kryo.Kryo,
+                    org.apache.tinkerpop.shaded.kryo.Serializer> functionOfShadedKryo = tr.getFunctionOfShadedKryo();
+
+            // Apply overrides with the highest case-precedence
+            if (checkForAndApplySerializerOverride(serializerOverrides, kryo, tr.getTargetClass())) {
+                // do nothing but skip the remaining else(-if) clauses
+            } else if (null != shadedSerializer) {
+                if (shadedSerializer.getClass().equals(org.apache.tinkerpop.shaded.kryo.serializers.JavaSerializer.class)) {
+                    // Convert GryoMapper's shaded JavaSerializer mappings to their unshaded equivalents
+                    log.debug("Registering {} with JavaSerializer", tr.getTargetClass());
+                    kryo.register(tr.getTargetClass(), new JavaSerializer());
+                } else {
+                    // There's supposed to be a check in GryoMapper that prevents this from happening
+                    log.error("GryoMapper's default serialization registration for {} is a {}. " +
+                              "This is probably a bug in TinkerPop (this is not a valid default registration). " +
+                              "I am configuring Spark to use Kryo's default serializer for this class, " +
+                              "but this may cause serialization failures at runtime.",
+                              tr.getTargetClass(),
+                              org.apache.tinkerpop.shaded.kryo.Serializer.class.getCanonicalName());
+                    kryo.register(tr.getTargetClass());
+                }
+            } else if (null != serializerShim) {
+                // Wrap shim serializers in an adapter for Spark's unshaded Kryo
+                log.debug("Registering {} to serializer shim {} (serializer shim {})",
+                        tr.getTargetClass(), serializerShim, serializerShim.getClass());
+                kryo.register(tr.getTargetClass(), new UnshadedSerializerAdapter<>(serializerShim));
+                shimmedClassesFromGryoMapper.add(tr.getTargetClass());
+            } else if (null != functionOfShadedKryo) {
+                // As with shaded serializers, there's supposed to be a check in GryoMapper that prevents this from happening
+                log.error("GryoMapper's default serialization registration for {} is a Function<{},{}>.  " +
+                          "This is probably a bug in TinkerPop (this is not a valid default registration). " +
+                          "I am configuring Spark to use Kryo's default serializer instead of this function, " +
+                          "but this may cause serialization failures at runtime.",
+                          tr.getTargetClass(),
+                          org.apache.tinkerpop.shaded.kryo.Kryo.class.getCanonicalName(),
+                          org.apache.tinkerpop.shaded.kryo.Serializer.class.getCanonicalName());
+                kryo.register(tr.getTargetClass());
+            } else {
+                // Register all other classes with the default behavior (FieldSerializer)
+                log.debug("Registering {} with default serializer", tr.getTargetClass());
+                kryo.register(tr.getTargetClass());
+            }
+        }
+
+        // StarGraph's shim serializer is especially important on Spark for efficiency reasons,
+        // so log a warning if we failed to register it somehow
+        if (!shimmedClassesFromGryoMapper.contains(StarGraph.class)) {
+            log.warn("No SerializerShim found for StarGraph");
+        }
+    }
+
+    private LinkedHashMap<Class<?>, Serializer<?>> getExtraRegistrations() {
+
+        /* The map returned by this method MUST have a fixed iteration order!
+         *
+         * The order itself is irrelevant, so long as it is completely stable at runtime.
+         *
+         * LinkedHashMap satisfies this requirement (its contract specifies
+         * iteration in key-insertion-order).
+         */
+
+        LinkedHashMap<Class<?>, Serializer<?>> m = new LinkedHashMap<>();
+        // The following entries were copied from GryoSerializer's constructor
+        // This could be turned into a static collection on GryoSerializer to avoid
+        // duplication, but it would be a bit cumbersome to do so without disturbing
+        // the ordering of the existing entries in that constructor, since not all
+        // of the entries are for TinkerPop (and the ordering is significant).
+        m.put(MessagePayload.class, null);
+        m.put(ViewIncomingPayload.class, null);
+        m.put(ViewOutgoingPayload.class, null);
+        m.put(ViewPayload.class, null);
+        m.put(VertexWritable.class, new UnshadedSerializerAdapter<>(new VertexWritableSerializer()));
+        m.put(ObjectWritable.class, new UnshadedSerializerAdapter<>(new ObjectWritableSerializer<>()));
+
+        return m;
+    }
+
+    private boolean checkForAndApplySerializerOverride(Map<Class<?>, Serializer<?>> serializerOverrides,
+                                                       Kryo kryo, Class<?> targetClass) {
+        if (serializerOverrides.containsKey(targetClass)) {
+            Serializer<?> ser = serializerOverrides.get(targetClass);
+            if (null == ser) {
+                // null means use Kryo's default serializer
+                log.debug("Registering {} with default serializer per overrides", targetClass);
+                kryo.register(targetClass);
+            } else {
+                // nonnull means use that serializer
+                log.debug("Registering {} with serializer {} per overrides", targetClass, ser);
+                kryo.register(targetClass, ser);
+            }
+            return true;
+        }
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java
index d0411e8..a524a97 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/kryoshim/unshaded/UnshadedKryoShimService.java
@@ -24,92 +24,131 @@
  */
 package org.apache.tinkerpop.gremlin.spark.structure.io.gryo.kryoshim.unshaded;
 
-import com.twitter.chill.KryoInstantiator;
-import com.twitter.chill.KryoPool;
-import com.twitter.chill.SerDeState;
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.commons.configuration.BaseConfiguration;
+import org.apache.commons.configuration.Configuration;
 import org.apache.spark.SparkConf;
-import org.apache.spark.serializer.KryoSerializer;
-import org.apache.tinkerpop.gremlin.spark.structure.io.TinkerPopKryoRegistrator;
+import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.IoRegistryAwareKryoSerializer;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoPool;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.concurrent.LinkedBlockingQueue;
 
 public class UnshadedKryoShimService implements KryoShimService {
 
-    public static final String SPARK_KRYO_POOL_SIZE_SYSTEM_PROPERTY = "tinkerpop.kryo.poolsize";
-
     private static final Logger log = LoggerFactory.getLogger(UnshadedKryoShimService.class);
-    private static final int SPARK_KRYO_POOL_SIZE_DEFAULT = 8;
 
-    private final KryoSerializer sparkKryoSerializer;
-    private final KryoPool kryoPool;
+    private static final LinkedBlockingQueue<Kryo> KRYOS = new LinkedBlockingQueue<>();
 
-    public UnshadedKryoShimService() {
-        this(TinkerPopKryoRegistrator.class.getCanonicalName(), getDefaultKryoPoolSize());
-    }
+    private static volatile boolean initialized;
 
-    public UnshadedKryoShimService(String sparkKryoRegistratorClassname, int kryoPoolSize) {
-        SparkConf sparkConf = new SparkConf();
-        sparkConf.set("spark.serializer", KryoSerializer.class.getCanonicalName());
-        sparkConf.set("spark.kryo.registrator", sparkKryoRegistratorClassname);
-        sparkKryoSerializer = new KryoSerializer(sparkConf);
-        kryoPool = KryoPool.withByteArrayOutputStream(kryoPoolSize, new KryoInstantiator());
-    }
+    public UnshadedKryoShimService() { }
 
     @Override
     public Object readClassAndObject(InputStream source) {
-        SerDeState sds = null;
-        try {
-            sds = kryoPool.borrow();
 
-            sds.setInput(source);
+        LinkedBlockingQueue<Kryo> kryos = initialize();
 
-            return sds.readClassAndObject();
+        Kryo k = null;
+        try {
+            k = kryos.take();
+
+            return k.readClassAndObject(new Input(source));
+        } catch (InterruptedException e) {
+            throw new RuntimeException(e);
         } finally {
-            kryoPool.release(sds);
+            try {
+                kryos.put(k);
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
         }
     }
 
     @Override
     public void writeClassAndObject(Object o, OutputStream sink) {
-        SerDeState sds = null;
+
+        LinkedBlockingQueue<Kryo> kryos = initialize();
+
+        Kryo k = null;
         try {
-            sds = kryoPool.borrow();
+            k = kryos.take();
 
-            sds.writeClassAndObject(o); // this writes to an internal buffer
+            Output kryoOutput = new Output(sink);
 
-            sds.writeOutputTo(sink); // this copies the internal buffer to sink
+            k.writeClassAndObject(kryoOutput, o);
 
-            sink.flush();
-        } catch (IOException e) {
+            kryoOutput.flush();
+        } catch (InterruptedException e) {
             throw new RuntimeException(e);
         } finally {
-            kryoPool.release(sds);
+            try {
+                kryos.put(k);
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
         }
     }
 
     @Override
     public int getPriority() {
-        return 1024;
+        return 50;
     }
 
-    private static int getDefaultKryoPoolSize() {
-        String raw = System.getProperty(SPARK_KRYO_POOL_SIZE_SYSTEM_PROPERTY);
+    @Override
+    public void applyConfiguration(Configuration conf) {
+        initialize(conf);
+    }
 
-        int size = SPARK_KRYO_POOL_SIZE_DEFAULT;
-        try {
-            size = Integer.valueOf(raw);
-            log.info("Setting kryo pool size to {} according to system property {}", size,
-                    SPARK_KRYO_POOL_SIZE_SYSTEM_PROPERTY);
-        } catch (NumberFormatException e) {
-            log.error("System property {}={} could not be parsed as an integer, using default value {}",
-                    SPARK_KRYO_POOL_SIZE_SYSTEM_PROPERTY, raw, size, e);
+    private LinkedBlockingQueue<Kryo> initialize() {
+        return initialize(new BaseConfiguration());
+    }
+
+    private LinkedBlockingQueue<Kryo> initialize(Configuration conf) {
+        // DCL is safe in this case due to volatility
+        if (!initialized) {
+            synchronized (UnshadedKryoShimService.class) {
+                if (!initialized) {
+                    SparkConf sparkConf = new SparkConf();
+
+                    // Copy the user's IoRegistry from the param conf to the SparkConf we just created
+                    String regStr = conf.getString(GryoPool.CONFIG_IO_REGISTRY);
+                    if (null != regStr) { // SparkConf rejects null values with NPE, so this has to be checked before set(...)
+                        sparkConf.set(GryoPool.CONFIG_IO_REGISTRY, regStr);
+                    }
+                    // Setting spark.serializer here almost certainly isn't necessary, but it doesn't hurt
+                    sparkConf.set("spark.serializer", IoRegistryAwareKryoSerializer.class.getCanonicalName());
+
+                    String registrator = conf.getString("spark.kryo.registrator");
+                    if (null != registrator) {
+                        sparkConf.set("spark.kryo.registrator", registrator);
+                        log.info("Copied spark.kryo.registrator: {}", registrator);
+                    } else {
+                        log.info("Not copying spark.kryo.registrator");
+                    }
+
+                    // Reuse Gryo poolsize for Kryo poolsize (no need to copy this to SparkConf)
+                    int poolSize = conf.getInt(GryoPool.CONFIG_IO_GRYO_POOL_SIZE,
+                            GryoPool.CONFIG_IO_GRYO_POOL_SIZE_DEFAULT);
+                    // Instantiate the spark.serializer
+                    final IoRegistryAwareKryoSerializer ioReg = new IoRegistryAwareKryoSerializer(sparkConf);
+                    // Setup a pool backed by our spark.serializer instance
+
+                    for (int i = 0; i < poolSize; i++) {
+                        KRYOS.add(ioReg.newKryo());
+                    }
+
+                    initialized = true;
+                }
+            }
         }
 
-        return size;
+        return KRYOS;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/9321a3e1/spark-gremlin/src/main/resources/META-INF/services/org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimService
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/resources/META-INF/services/org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimService b/spark-gremlin/src/main/resources/META-INF/services/org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimService
new file mode 100644
index 0000000..68712a6
--- /dev/null
+++ b/spark-gremlin/src/main/resources/META-INF/services/org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimService
@@ -0,0 +1 @@
+org.apache.tinkerpop.gremlin.spark.structure.io.gryo.kryoshim.unshaded.UnshadedKryoShimService # Supports Spark



[09/34] incubator-tinkerpop git commit: Merge remote-tracking branch 'lewismc/TINKERPOP-1301tp31' into tp31

Posted by ok...@apache.org.
Merge remote-tracking branch 'lewismc/TINKERPOP-1301tp31' into tp31


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

Branch: refs/heads/TINKERPOP-1278
Commit: 0445e6d59940e71a5ddab78181410e124dcfca7a
Parents: 029f56b d5d2c48
Author: Daniel Kuppitz <da...@hotmail.com>
Authored: Sat Jun 4 01:06:43 2016 +0200
Committer: Daniel Kuppitz <da...@hotmail.com>
Committed: Sat Jun 4 01:06:43 2016 +0200

----------------------------------------------------------------------
 .../gremlin/hadoop/structure/io/script/ScriptInputFormat.java  | 5 +++++
 .../gremlin/hadoop/structure/io/script/ScriptOutputFormat.java | 6 ++++++
 2 files changed, 11 insertions(+)
----------------------------------------------------------------------



[02/34] incubator-tinkerpop git commit: Introduce Kryo shim to support serializer reuse

Posted by ok...@apache.org.
Introduce Kryo shim to support serializer reuse


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

Branch: refs/heads/TINKERPOP-1278
Commit: ef52869788ebf5b8b825f78ef21e1d38423d9aa0
Parents: aa673db
Author: Dan LaRocque <da...@hopcount.org>
Authored: Thu Jun 2 02:09:29 2016 -0500
Committer: Dan LaRocque <da...@hopcount.org>
Committed: Thu Jun 2 03:09:29 2016 -0400

----------------------------------------------------------------------
 .../gremlin/structure/io/gryo/GryoMapper.java   | 403 ++++++++++++-------
 .../structure/io/gryo/TypeRegistration.java     |  70 ++++
 .../structure/io/gryo/URISerializer.java        |  23 +-
 .../structure/io/gryo/UUIDSerializer.java       |  20 +-
 .../structure/io/kryoshim/InputShim.java        |  37 ++
 .../gremlin/structure/io/kryoshim/KryoShim.java |  40 ++
 .../structure/io/kryoshim/KryoShimService.java  |  83 ++++
 .../io/kryoshim/KryoShimServiceLoader.java      | 137 +++++++
 .../structure/io/kryoshim/OutputShim.java       |  41 ++
 .../structure/io/kryoshim/SerializerShim.java   |  35 ++
 .../structure/io/kryoshim/package-info.java     |  54 +++
 .../io/kryoshim/shaded/ShadedInputAdapter.java  |  66 +++
 .../io/kryoshim/shaded/ShadedKryoAdapter.java   |  67 +++
 .../io/kryoshim/shaded/ShadedOutputAdapter.java |  72 ++++
 .../shaded/ShadedSerializerAdapter.java         |  54 +++
 .../io/kryoshim/shaded/package-info.java        |  25 ++
 .../util/star/StarGraphGryoSerializer.java      | 123 +-----
 .../util/star/StarGraphSerializer.java          | 140 +++++++
 .../structure/io/HadoopPoolShimService.java     |  69 ++++
 .../hadoop/structure/io/HadoopPools.java        |   1 +
 .../hadoop/structure/io/ObjectWritable.java     |  25 +-
 .../hadoop/structure/io/VertexWritable.java     |  38 +-
 ...remlin.structure.io.kryoshim.KryoShimService |   1 +
 23 files changed, 1293 insertions(+), 331 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
index 9cae845..5bc71da 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
@@ -18,6 +18,7 @@
  */
 package org.apache.tinkerpop.gremlin.structure.io.gryo;
 
+import org.apache.tinkerpop.gremlin.process.computer.GraphFilter;
 import org.apache.tinkerpop.gremlin.process.computer.MapReduce;
 import org.apache.tinkerpop.gremlin.process.computer.util.MapMemory;
 import org.apache.tinkerpop.gremlin.process.traversal.Contains;
@@ -56,6 +57,8 @@ import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.structure.VertexProperty;
 import org.apache.tinkerpop.gremlin.structure.io.IoRegistry;
 import org.apache.tinkerpop.gremlin.structure.io.Mapper;
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.SerializerShim;
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.shaded.ShadedSerializerAdapter;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedEdge;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedPath;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedProperty;
@@ -69,6 +72,7 @@ import org.apache.tinkerpop.gremlin.structure.util.reference.ReferenceVertexProp
 import org.apache.tinkerpop.gremlin.structure.util.star.StarGraph;
 import org.apache.tinkerpop.gremlin.structure.util.star.StarGraphGryoSerializer;
 import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
+import org.apache.tinkerpop.gremlin.structure.util.star.StarGraphSerializer;
 import org.apache.tinkerpop.shaded.kryo.ClassResolver;
 import org.apache.tinkerpop.shaded.kryo.Kryo;
 import org.apache.tinkerpop.shaded.kryo.KryoSerializable;
@@ -77,7 +81,6 @@ import org.apache.tinkerpop.shaded.kryo.serializers.JavaSerializer;
 import org.apache.tinkerpop.shaded.kryo.util.DefaultStreamFactory;
 import org.apache.tinkerpop.shaded.kryo.util.MapReferenceResolver;
 import org.javatuples.Pair;
-import org.javatuples.Triplet;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
@@ -156,13 +159,13 @@ import java.util.stream.Collectors;
 public final class GryoMapper implements Mapper<Kryo> {
     public static final byte[] GIO = "gio".getBytes();
     public static final byte[] HEADER = Arrays.copyOf(GIO, 16);
-    private final List<Triplet<Class, Function<Kryo, Serializer>, Integer>> serializationList;
+    private final List<TypeRegistration<?>> typeRegistrations;
     private final boolean registrationRequired;
     private final boolean referenceTracking;
     private final Supplier<ClassResolver> classResolver;
 
     private GryoMapper(final Builder builder) {
-        this.serializationList = builder.serializationList;
+        this.typeRegistrations = builder.typeRegistrations;
         validate();
 
         this.registrationRequired = builder.registrationRequired;
@@ -176,19 +179,17 @@ public final class GryoMapper implements Mapper<Kryo> {
         kryo.addDefaultSerializer(Map.Entry.class, new EntrySerializer());
         kryo.setRegistrationRequired(registrationRequired);
         kryo.setReferences(referenceTracking);
-
-        serializationList.forEach(p -> {
-            final Function<Kryo, Serializer> serializer = p.getValue1();
-            if (null == serializer)
-                kryo.register(p.getValue0(), kryo.getDefaultSerializer(p.getValue0()), p.getValue2());
-            else
-                kryo.register(p.getValue0(), serializer.apply(kryo), p.getValue2());
-        });
+        for (TypeRegistration tr : typeRegistrations)
+            tr.registerWith(kryo);
         return kryo;
     }
 
     public List<Class> getRegisteredClasses() {
-        return this.serializationList.stream().map(Triplet::getValue0).collect(Collectors.toList());
+        return this.typeRegistrations.stream().map(TypeRegistration::getTargetClass).collect(Collectors.toList());
+    }
+
+    public List<TypeRegistration<?>> getTypeRegistrations() {
+        return typeRegistrations;
     }
 
     public static Builder build() {
@@ -199,11 +200,11 @@ public final class GryoMapper implements Mapper<Kryo> {
         final Set<Integer> duplicates = new HashSet<>();
 
         final Set<Integer> ids = new HashSet<>();
-        serializationList.forEach(t -> {
-            if (!ids.contains(t.getValue2()))
-                ids.add(t.getValue2());
+        typeRegistrations.forEach(t -> {
+            if (!ids.contains(t.getId()))
+                ids.add(t.getId());
             else
-                duplicates.add(t.getValue2());
+                duplicates.add(t.getId());
         });
 
         if (duplicates.size() > 0)
@@ -244,119 +245,119 @@ public final class GryoMapper implements Mapper<Kryo> {
          * Note that the following are pre-registered boolean, Boolean, byte, Byte, char, Character, double, Double,
          * int, Integer, float, Float, long, Long, short, Short, String, void.
          */
-        private final List<Triplet<Class, Function<Kryo, Serializer>, Integer>> serializationList = new ArrayList<Triplet<Class, Function<Kryo, Serializer>, Integer>>() {{
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(byte[].class, null, 25));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(char[].class, null, 26));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(short[].class, null, 27));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(int[].class, null, 28));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(long[].class, null, 29));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(float[].class, null, 30));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(double[].class, null, 31));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(String[].class, null, 32));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Object[].class, null, 33));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(ArrayList.class, null, 10));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(BigInteger.class, null, 34));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(BigDecimal.class, null, 35));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Calendar.class, null, 39));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Class.class, null, 41));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Collection.class, null, 37));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Collections.EMPTY_LIST.getClass(), null, 51));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Collections.EMPTY_MAP.getClass(), null, 52));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Collections.EMPTY_SET.getClass(), null, 53));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Collections.singleton(null).getClass(), null, 54));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Collections.singletonList(null).getClass(), null, 24));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Collections.singletonMap(null, null).getClass(), null, 23));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Contains.class, null, 49));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Currency.class, null, 40));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Date.class, null, 38));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Direction.class, null, 12));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(DetachedEdge.class, null, 21));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(DetachedVertexProperty.class, null, 20));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(DetachedProperty.class, null, 18));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(DetachedVertex.class, null, 19));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(DetachedPath.class, null, 60));
+        private final List<TypeRegistration<?>> typeRegistrations = new ArrayList<TypeRegistration<?>>() {{
+            add(GryoTypeReg.of(byte[].class, 25));
+            add(GryoTypeReg.of(char[].class, 26));
+            add(GryoTypeReg.of(short[].class, 27));
+            add(GryoTypeReg.of(int[].class, 28));
+            add(GryoTypeReg.of(long[].class, 29));
+            add(GryoTypeReg.of(float[].class, 30));
+            add(GryoTypeReg.of(double[].class, 31));
+            add(GryoTypeReg.of(String[].class, 32));
+            add(GryoTypeReg.of(Object[].class, 33));
+            add(GryoTypeReg.of(ArrayList.class, 10));
+            add(GryoTypeReg.of(BigInteger.class, 34));
+            add(GryoTypeReg.of(BigDecimal.class, 35));
+            add(GryoTypeReg.of(Calendar.class, 39));
+            add(GryoTypeReg.of(Class.class, 41));
+            add(GryoTypeReg.of(Collection.class, 37));
+            add(GryoTypeReg.of(Collections.EMPTY_LIST.getClass(), 51));
+            add(GryoTypeReg.of(Collections.EMPTY_MAP.getClass(), 52));
+            add(GryoTypeReg.of(Collections.EMPTY_SET.getClass(), 53));
+            add(GryoTypeReg.of(Collections.singleton(null).getClass(), 54));
+            add(GryoTypeReg.of(Collections.singletonList(null).getClass(), 24));
+            add(GryoTypeReg.of(Collections.singletonMap(null, null).getClass(), 23));
+            add(GryoTypeReg.of(Contains.class, 49));
+            add(GryoTypeReg.of(Currency.class, 40));
+            add(GryoTypeReg.of(Date.class, 38));
+            add(GryoTypeReg.of(Direction.class, 12));
+            add(GryoTypeReg.of(DetachedEdge.class, 21));
+            add(GryoTypeReg.of(DetachedVertexProperty.class, 20));
+            add(GryoTypeReg.of(DetachedProperty.class, 18));
+            add(GryoTypeReg.of(DetachedVertex.class, 19));
+            add(GryoTypeReg.of(DetachedPath.class, 60));
             // skip 14
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(EnumSet.class, null, 46));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(HashMap.class, null, 11));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(HashMap.Entry.class, null, 16));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(HASH_MAP_NODE, null, 92));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(KryoSerializable.class, null, 36));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(LinkedHashMap.class, null, 47));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(LinkedHashSet.class, null, 71));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(LinkedList.class, null, 116));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(LINKED_HASH_MAP_ENTRY_CLASS, null, 15));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Locale.class, null, 22));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(StringBuffer.class, null, 43));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(StringBuilder.class, null, 44));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(T.class, null, 48));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(TimeZone.class, null, 42));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(TreeMap.class, null, 45));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(TreeSet.class, null, 50));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(UUID.class, kryo -> new UUIDSerializer(), 17));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(URI.class, kryo -> new URISerializer(), 72));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(VertexTerminator.class, null, 13));
-
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(ReferenceEdge.class, null, 81));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(ReferenceVertexProperty.class, null, 82));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(ReferenceProperty.class, null, 83));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(ReferenceVertex.class, null, 84));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(ReferencePath.class, null, 85));
-
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(StarGraph.class, kryo -> StarGraphGryoSerializer.with(Direction.BOTH), 86));
-
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Edge.class, kryo -> new GryoSerializers.EdgeSerializer(), 65));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Vertex.class, kryo -> new GryoSerializers.VertexSerializer(), 66));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Property.class, kryo -> new GryoSerializers.PropertySerializer(), 67));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(VertexProperty.class, kryo -> new GryoSerializers.VertexPropertySerializer(), 68));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Path.class, kryo -> new GryoSerializers.PathSerializer(), 59));
+            add(GryoTypeReg.of(EnumSet.class, 46));
+            add(GryoTypeReg.of(HashMap.class, 11));
+            add(GryoTypeReg.of(HashMap.Entry.class, 16));
+            add(GryoTypeReg.of(HASH_MAP_NODE, 92));
+            add(GryoTypeReg.of(KryoSerializable.class, 36));
+            add(GryoTypeReg.of(LinkedHashMap.class, 47));
+            add(GryoTypeReg.of(LinkedHashSet.class, 71));
+            add(GryoTypeReg.of(LinkedList.class, 116));
+            add(GryoTypeReg.of(LINKED_HASH_MAP_ENTRY_CLASS, 15));
+            add(GryoTypeReg.of(Locale.class, 22));
+            add(GryoTypeReg.of(StringBuffer.class, 43));
+            add(GryoTypeReg.of(StringBuilder.class, 44));
+            add(GryoTypeReg.of(T.class, 48));
+            add(GryoTypeReg.of(TimeZone.class, 42));
+            add(GryoTypeReg.of(TreeMap.class, 45));
+            add(GryoTypeReg.of(TreeSet.class, 50));
+            add(GryoTypeReg.of(UUID.class, 17, new UUIDSerializer()));
+            add(GryoTypeReg.of(URI.class, 72, new URISerializer()));
+            add(GryoTypeReg.of(VertexTerminator.class, 13));
+
+            add(GryoTypeReg.of(ReferenceEdge.class, 81));
+            add(GryoTypeReg.of(ReferenceVertexProperty.class, 82));
+            add(GryoTypeReg.of(ReferenceProperty.class, 83));
+            add(GryoTypeReg.of(ReferenceVertex.class, 84));
+            add(GryoTypeReg.of(ReferencePath.class, 85));
+
+            add(GryoTypeReg.of(StarGraph.class, 86, new StarGraphSerializer(Direction.BOTH, new GraphFilter())));
+
+            add(GryoTypeReg.of(Edge.class, 65, new GryoSerializers.EdgeSerializer()));
+            add(GryoTypeReg.of(Vertex.class, 66, new GryoSerializers.VertexSerializer()));
+            add(GryoTypeReg.of(Property.class, 67, new GryoSerializers.PropertySerializer()));
+            add(GryoTypeReg.of(VertexProperty.class, 68, new GryoSerializers.VertexPropertySerializer()));
+            add(GryoTypeReg.of(Path.class, 59, new GryoSerializers.PathSerializer()));
             // skip 55
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(B_O_Traverser.class, null, 75));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(O_Traverser.class, null, 76));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(B_LP_O_P_S_SE_SL_Traverser.class, null, 77));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(B_O_S_SE_SL_Traverser.class, null, 78));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(B_LP_O_S_SE_SL_Traverser.class, null, 87));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(O_OB_S_SE_SL_Traverser.class, null, 89));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(LP_O_OB_S_SE_SL_Traverser.class, null, 90));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(LP_O_OB_P_S_SE_SL_Traverser.class, null, 91));
-
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(TraverserSet.class, null, 58));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Tree.class, null, 61));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(HashSet.class, null, 62));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(BulkSet.class, null, 64));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(MutableMetrics.class, null, 69));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(ImmutableMetrics.class, null, 115));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(DefaultTraversalMetrics.class, null, 70));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(MapMemory.class, null, 73));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(MapReduce.NullObject.class, null, 74));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(AtomicLong.class, null, 79));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Pair.class, kryo -> new PairSerializer(), 88));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(TraversalExplanation.class, kryo -> new JavaSerializer(), 106));
-
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Duration.class, kryo -> new JavaTimeSerializers.DurationSerializer(), 93));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Instant.class, kryo -> new JavaTimeSerializers.InstantSerializer(), 94));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(LocalDate.class, kryo -> new JavaTimeSerializers.LocalDateSerializer(), 95));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(LocalDateTime.class, kryo -> new JavaTimeSerializers.LocalDateTimeSerializer(), 96));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(LocalTime.class, kryo -> new JavaTimeSerializers.LocalTimeSerializer(), 97));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(MonthDay.class, kryo -> new JavaTimeSerializers.MonthDaySerializer(), 98));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(OffsetDateTime.class, kryo -> new JavaTimeSerializers.OffsetDateTimeSerializer(), 99));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(OffsetTime.class, kryo -> new JavaTimeSerializers.OffsetTimeSerializer(), 100));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Period.class, kryo -> new JavaTimeSerializers.PeriodSerializer(), 101));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Year.class, kryo -> new JavaTimeSerializers.YearSerializer(), 102));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(YearMonth.class, kryo -> new JavaTimeSerializers.YearMonthSerializer(), 103));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(ZonedDateTime.class, kryo -> new JavaTimeSerializers.ZonedDateTimeSerializer(), 104));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(ZoneOffset.class, kryo -> new JavaTimeSerializers.ZoneOffsetSerializer(), 105));
-
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(Operator.class, null, 107));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(FoldStep.FoldBiOperator.class, null, 108));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(GroupCountStep.GroupCountBiOperator.class, null, 109));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(GroupStep.GroupBiOperator.class, kryo -> new JavaSerializer(), 117)); // because they contain traversals
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(MeanGlobalStep.MeanGlobalBiOperator.class, null, 110));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(MeanGlobalStep.MeanNumber.class, null, 111));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(TreeStep.TreeBiOperator.class, null, 112));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(GroupStepV3d0.GroupBiOperatorV3d0.class, null, 113));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(RangeGlobalStep.RangeBiOperator.class, null, 114));
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(OrderGlobalStep.OrderBiOperator.class, kryo -> new JavaSerializer(), 118)); // because they contain traversals
-            add(Triplet.<Class, Function<Kryo, Serializer>, Integer>with(ProfileStep.ProfileBiOperator.class, null, 119)); // ***LAST ID***
+            add(GryoTypeReg.of(B_O_Traverser.class, 75));
+            add(GryoTypeReg.of(O_Traverser.class, 76));
+            add(GryoTypeReg.of(B_LP_O_P_S_SE_SL_Traverser.class, 77));
+            add(GryoTypeReg.of(B_O_S_SE_SL_Traverser.class, 78));
+            add(GryoTypeReg.of(B_LP_O_S_SE_SL_Traverser.class, 87));
+            add(GryoTypeReg.of(O_OB_S_SE_SL_Traverser.class, 89));
+            add(GryoTypeReg.of(LP_O_OB_S_SE_SL_Traverser.class, 90));
+            add(GryoTypeReg.of(LP_O_OB_P_S_SE_SL_Traverser.class, 91));
+
+            add(GryoTypeReg.of(TraverserSet.class, 58));
+            add(GryoTypeReg.of(Tree.class, 61));
+            add(GryoTypeReg.of(HashSet.class, 62));
+            add(GryoTypeReg.of(BulkSet.class, 64));
+            add(GryoTypeReg.of(MutableMetrics.class, 69));
+            add(GryoTypeReg.of(ImmutableMetrics.class, 115));
+            add(GryoTypeReg.of(DefaultTraversalMetrics.class, 70));
+            add(GryoTypeReg.of(MapMemory.class, 73));
+            add(GryoTypeReg.of(MapReduce.NullObject.class, 74));
+            add(GryoTypeReg.of(AtomicLong.class, 79));
+            add(GryoTypeReg.of(Pair.class, 88, new PairSerializer()));
+            add(GryoTypeReg.of(TraversalExplanation.class, 106, new JavaSerializer()));
+
+            add(GryoTypeReg.of(Duration.class, 93, new JavaTimeSerializers.DurationSerializer()));
+            add(GryoTypeReg.of(Instant.class, 94, new JavaTimeSerializers.InstantSerializer()));
+            add(GryoTypeReg.of(LocalDate.class, 95, new JavaTimeSerializers.LocalDateSerializer()));
+            add(GryoTypeReg.of(LocalDateTime.class, 96, new JavaTimeSerializers.LocalDateTimeSerializer()));
+            add(GryoTypeReg.of(LocalTime.class, 97, new JavaTimeSerializers.LocalTimeSerializer()));
+            add(GryoTypeReg.of(MonthDay.class, 98, new JavaTimeSerializers.MonthDaySerializer()));
+            add(GryoTypeReg.of(OffsetDateTime.class, 99, new JavaTimeSerializers.OffsetDateTimeSerializer()));
+            add(GryoTypeReg.of(OffsetTime.class, 100, new JavaTimeSerializers.OffsetTimeSerializer()));
+            add(GryoTypeReg.of(Period.class, 101, new JavaTimeSerializers.PeriodSerializer()));
+            add(GryoTypeReg.of(Year.class, 102, new JavaTimeSerializers.YearSerializer()));
+            add(GryoTypeReg.of(YearMonth.class, 103, new JavaTimeSerializers.YearMonthSerializer()));
+            add(GryoTypeReg.of(ZonedDateTime.class, 104, new JavaTimeSerializers.ZonedDateTimeSerializer()));
+            add(GryoTypeReg.of(ZoneOffset.class, 105, new JavaTimeSerializers.ZoneOffsetSerializer()));
+
+            add(GryoTypeReg.of(Operator.class, 107));
+            add(GryoTypeReg.of(FoldStep.FoldBiOperator.class, 108));
+            add(GryoTypeReg.of(GroupCountStep.GroupCountBiOperator.class, 109));
+            add(GryoTypeReg.of(GroupStep.GroupBiOperator.class, 117, new JavaSerializer())); // because they contain traversals
+            add(GryoTypeReg.of(MeanGlobalStep.MeanGlobalBiOperator.class, 110));
+            add(GryoTypeReg.of(MeanGlobalStep.MeanNumber.class, 111));
+            add(GryoTypeReg.of(TreeStep.TreeBiOperator.class, 112));
+            add(GryoTypeReg.of(GroupStepV3d0.GroupBiOperatorV3d0.class, 113));
+            add(GryoTypeReg.of(RangeGlobalStep.RangeBiOperator.class, 114));
+            add(GryoTypeReg.of(OrderGlobalStep.OrderBiOperator.class, 118, new JavaSerializer())); // because they contain traversals
+            add(GryoTypeReg.of(ProfileStep.ProfileBiOperator.class, 119)); // ***LAST ID***
         }};
 
         private final List<IoRegistry> registries = new ArrayList<>();
@@ -404,8 +405,8 @@ public final class GryoMapper implements Mapper<Kryo> {
          */
         public Builder addCustom(final Class... custom) {
             if (custom != null && custom.length > 0) {
-                for (Class clazz : custom) {
-                    addCustom(clazz, (Function<Kryo, Serializer>) null);
+                for (Class c : custom) {
+                    addOrOverrideRegistration(GryoTypeReg.of(c, currentSerializationId.getAndIncrement()));
                 }
             }
             return this;
@@ -416,10 +417,15 @@ public final class GryoMapper implements Mapper<Kryo> {
          * a class that is already registered will override that registration.
          */
         public Builder addCustom(final Class clazz, final Serializer serializer) {
-            if (null == serializer)
-                addCustom(clazz);
-            else
-                addCustom(clazz, kryo -> serializer);
+            addOrOverrideRegistration(GryoTypeReg.of(clazz, currentSerializationId.getAndIncrement(), serializer));
+            return this;
+        }
+
+        /**
+         * Register custom class to serialize with a custom serialization shim.
+         */
+        public Builder addCustom(final Class clazz, final SerializerShim serializer) {
+            addOrOverrideRegistration(GryoTypeReg.of(clazz, currentSerializationId.getAndIncrement(), serializer));
             return this;
         }
 
@@ -427,15 +433,8 @@ public final class GryoMapper implements Mapper<Kryo> {
          * Register a custom class to serialize with a custom serializer as returned from a {@link Function}. Note
          * that calling this method for a class that is already registered will override that registration.
          */
-        public Builder addCustom(final Class clazz, final Function<Kryo, Serializer> serializer) {
-            final Optional<Triplet<Class, Function<Kryo, Serializer>, Integer>> found = findSerializer(clazz);
-            if (found.isPresent()) {
-                final Triplet<Class, Function<Kryo, Serializer>, Integer> t = found.get();
-                serializationList.remove(t);
-                serializationList.add(t.setAt1(serializer));
-            } else
-                serializationList.add(Triplet.with(clazz, serializer, currentSerializationId.getAndIncrement()));
-
+        public Builder addCustom(final Class clazz, final Function<Kryo, Serializer> functionOfKryo) {
+            addOrOverrideRegistration(GryoTypeReg.of(clazz, currentSerializationId.getAndIncrement(), functionOfKryo));
             return this;
         }
 
@@ -481,23 +480,123 @@ public final class GryoMapper implements Mapper<Kryo> {
                         addCustom(p.getValue0(), (Function<Kryo, Serializer>) p.getValue1());
                     else
                         throw new IllegalStateException(String.format(
-                                "Unexpected value provided by the %s for %s - expects [null, %s implementation or Function<%s, %s>]",
-                                IoRegistry.class.getSimpleName(), p.getValue0().getClass().getSimpleName(),
+                                "Unexpected value provided by %s for serializable class %s - expected a parameter in [null, %s implementation or Function<%s, %s>], but received %s",
+                                registry.getClass().getSimpleName(), p.getValue0().getClass().getCanonicalName(),
                                 Serializer.class.getName(), Kryo.class.getSimpleName(),
-                                Serializer.class.getSimpleName()));
+                                Serializer.class.getSimpleName(), p.getValue1()));
                 });
             });
 
             return new GryoMapper(this);
         }
 
-        private Optional<Triplet<Class, Function<Kryo, Serializer>, Integer>> findSerializer(final Class clazz) {
-            final Iterator<Triplet<Class, Function<Kryo, Serializer>, Integer>> itty = IteratorUtils.filter(
-                    serializationList, t -> t.getValue0().equals(clazz)).iterator();
-            if (itty.hasNext())
-                return Optional.of(itty.next());
-            else
-                return Optional.empty();
+        private <T> void addOrOverrideRegistration(TypeRegistration<T> newRegistration) {
+            Iterator<TypeRegistration<?>> iter = typeRegistrations.iterator();
+            while (iter.hasNext()) {
+                TypeRegistration<?> existingRegistration = iter.next();
+                if (existingRegistration.getTargetClass().equals(newRegistration.getTargetClass())) {
+                    iter.remove();
+                    break;
+                }
+            }
+            typeRegistrations.add(newRegistration);
+        }
+    }
+
+    private static class GryoTypeReg<T> implements TypeRegistration<T> {
+
+        private final Class<T> clazz;
+        private final Serializer<T> shadedSerializer;
+        private final SerializerShim<T> serializerShim;
+        private final Function<Kryo, Serializer> functionOfShadedKryo;
+        private final int id;
+
+        private GryoTypeReg(Class<T> clazz,
+                            Serializer<T> shadedSerializer,
+                            SerializerShim<T> serializerShim,
+                            Function<Kryo, Serializer> functionOfShadedKryo,
+                            int id) {
+            this.clazz = clazz;
+            this.shadedSerializer = shadedSerializer;
+            this.serializerShim = serializerShim;
+            this.functionOfShadedKryo = functionOfShadedKryo;
+            this.id = id;
+
+            int serializerCount = 0;
+            if (null != this.shadedSerializer)
+                serializerCount++;
+            if (null != this.serializerShim)
+                serializerCount++;
+            if (null != this.functionOfShadedKryo)
+                serializerCount++;
+
+            if (1 < serializerCount) {
+                String msg = String.format(
+                        "GryoTypeReg accepts at most one kind of serializer, but multiple " +
+                                "serializers were supplied for class %s (id %s).  " +
+                                "Shaded serializer: %s.  Shim serializer: %s.  Shaded serializer function: %s.",
+                        this.clazz.getCanonicalName(), id,
+                        this.shadedSerializer, this.serializerShim, this.functionOfShadedKryo);
+                throw new IllegalArgumentException(msg);
+            }
+        }
+
+        private static <T> GryoTypeReg<T> of(Class<T> clazz, int id) {
+            return new GryoTypeReg<>(clazz, null, null, null, id);
+        }
+
+        private static <T> GryoTypeReg<T> of(Class<T> clazz, int id, Serializer<T> shadedSerializer) {
+            return new GryoTypeReg<>(clazz, shadedSerializer, null, null, id);
+        }
+
+        private static <T> GryoTypeReg<T> of(Class<T> clazz, int id, SerializerShim<T> serializerShim) {
+            return new GryoTypeReg<>(clazz, null, serializerShim, null, id);
+        }
+
+        private static <T> GryoTypeReg<T> of(Class clazz, int id, Function<Kryo, Serializer> fct) {
+            return new GryoTypeReg<>(clazz, null, null, fct, id);
+        }
+
+        @Override
+        public Serializer<T> getShadedSerializer() {
+            return shadedSerializer;
+        }
+
+        @Override
+        public SerializerShim<T> getSerializerShim() {
+            return serializerShim;
+        }
+
+        @Override
+        public Function<Kryo, Serializer> getFunctionOfShadedKryo() {
+            return functionOfShadedKryo;
+        }
+
+        @Override
+        public Class<T> getTargetClass() {
+            return clazz;
+        }
+
+        @Override
+        public int getId() {
+            return id;
+        }
+
+        @Override
+        public Kryo registerWith(Kryo kryo) {
+            if (null != functionOfShadedKryo)
+                kryo.register(clazz, functionOfShadedKryo.apply(kryo), id);
+            else if (null != shadedSerializer)
+                kryo.register(clazz, shadedSerializer, id);
+            else if (null != serializerShim)
+                kryo.register(clazz, new ShadedSerializerAdapter<>(serializerShim), id);
+            else {
+                kryo.register(clazz, kryo.getDefaultSerializer(clazz), id);
+                // Suprisingly, the preceding call is not equivalent to
+                //   kryo.register(clazz, id);
+            }
+
+            return kryo;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java
new file mode 100644
index 0000000..5ca3f31
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/TypeRegistration.java
@@ -0,0 +1,70 @@
+/*
+ * 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.structure.io.gryo;
+
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.SerializerShim;
+import org.apache.tinkerpop.shaded.kryo.Kryo;
+import org.apache.tinkerpop.shaded.kryo.Serializer;
+
+/**
+ * Represents a class serializable with Gryo.
+ * <p>
+ * At most one of the {@link #getShadedSerializer()}, {@link #getSerializerShim()},
+ * and {@link #getFunctionOfShadedKryo()} will return a non-null value.  If all
+ * three methods return null, then there is no custom serialization logic associated
+ * with this class.  Gryo/Kryo will use its default serializer.
+ *
+ * @param <T> the serializable type
+ */
+public interface TypeRegistration<T> {
+
+    /**
+     * @return the serializable class this instance describes
+     */
+    Class<T> getTargetClass();
+
+    /**
+     * @return numeric identifier used as a shorthand for this type in Gryo's serialized form
+     */
+    int getId();
+
+    /**
+     * @return the shaded-Kryo serializer that handles this type, if one is defined
+     */
+    Serializer<T> getShadedSerializer();
+
+    /**
+     * @return the shim-Kryo serializer that handles this type, if one is defined
+     */
+    SerializerShim<T> getSerializerShim();
+
+    /**
+     * @return a function that accepts a shaded-Kryo instance and returns a serializer, if such a function is defined
+     */
+    java.util.function.Function<Kryo, Serializer> getFunctionOfShadedKryo();
+
+    /**
+     * Registers this type on the supplied {@link Kryo} instance, using whatever custom serializer
+     * may be present, then returns the same {@linkplain Kryo} instance supplied as the parameter.
+     *
+     * @param kryo Kryo instance into which this type is registered
+     * @return the sole parameter
+     */
+    Kryo registerWith(Kryo kryo);
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/URISerializer.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/URISerializer.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/URISerializer.java
index 7b6cfec..de08061 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/URISerializer.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/URISerializer.java
@@ -18,29 +18,32 @@
  */
 package org.apache.tinkerpop.gremlin.structure.io.gryo;
 
-import org.apache.tinkerpop.shaded.kryo.Kryo;
-import org.apache.tinkerpop.shaded.kryo.Serializer;
-import org.apache.tinkerpop.shaded.kryo.io.Input;
-import org.apache.tinkerpop.shaded.kryo.io.Output;
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.InputShim;
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShim;
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.OutputShim;
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.SerializerShim;
 
 import java.net.URI;
 
 /**
  * @author Stephen Mallette (http://stephen.genoprime.com)
  */
-final class URISerializer extends Serializer<URI> {
+final class URISerializer implements SerializerShim<URI> {
 
-    public URISerializer() {
-        setImmutable(true);
-    }
+    public URISerializer() { }
 
     @Override
-    public void write(final Kryo kryo, final Output output, final URI uri) {
+    public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final URI uri) {
         output.writeString(uri.toString());
     }
 
     @Override
-    public URI read(final Kryo kryo, final Input input, final Class<URI> uriClass) {
+    public <I extends InputShim> URI read(final KryoShim<I, ?> kryo, final I input, final Class<URI> uriClass) {
         return URI.create(input.readString());
     }
+
+    @Override
+    public boolean isImmutable() {
+        return true;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/UUIDSerializer.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/UUIDSerializer.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/UUIDSerializer.java
index e694e38..b86ddc2 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/UUIDSerializer.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/UUIDSerializer.java
@@ -18,6 +18,10 @@
  */
 package org.apache.tinkerpop.gremlin.structure.io.gryo;
 
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.InputShim;
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShim;
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.OutputShim;
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.SerializerShim;
 import org.apache.tinkerpop.shaded.kryo.Kryo;
 import org.apache.tinkerpop.shaded.kryo.Serializer;
 import org.apache.tinkerpop.shaded.kryo.io.Input;
@@ -28,19 +32,23 @@ import java.util.UUID;
 /**
  * @author Stephen Mallette (http://stephen.genoprime.com)
  */
-final class UUIDSerializer extends Serializer<UUID> {
-    public UUIDSerializer() {
-        setImmutable(true);
-    }
+final class UUIDSerializer implements SerializerShim<UUID> {
+
+    public UUIDSerializer() { }
 
     @Override
-    public void write(final Kryo kryo, final Output output, final UUID uuid) {
+    public <O extends OutputShim> void write(final KryoShim<?, O> kryo, final O output, final UUID uuid) {
         output.writeLong(uuid.getMostSignificantBits());
         output.writeLong(uuid.getLeastSignificantBits());
     }
 
     @Override
-    public UUID read(final Kryo kryo, final Input input, final Class<UUID> uuidClass) {
+    public <I extends InputShim> UUID read(final KryoShim<I, ?> kryo, final I input, final Class<UUID> uuidClass) {
         return new UUID(input.readLong(), input.readLong());
     }
+
+    @Override
+    public boolean isImmutable() {
+        return true;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/InputShim.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/InputShim.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/InputShim.java
new file mode 100644
index 0000000..6d00884
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/InputShim.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.structure.io.kryoshim;
+
+/**
+ * A minimal {@link org.apache.tinkerpop.shaded.kryo.io.Input}-like abstraction.
+ */
+public interface InputShim {
+
+    byte readByte();
+
+    byte[] readBytes(int size);
+
+    String readString();
+
+    long readLong();
+
+    int readInt();
+
+    double readDouble();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShim.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShim.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShim.java
new file mode 100644
index 0000000..e2a95e6
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShim.java
@@ -0,0 +1,40 @@
+/*
+ * 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.structure.io.kryoshim;
+
+/**
+ * A minimal {@link org.apache.tinkerpop.shaded.kryo.Kryo}-like abstraction.
+ *
+ * @param <I> this interface's complementary InputShim
+ * @param <O> this interface's complementary OutputShim
+ */
+public interface KryoShim<I extends InputShim, O extends OutputShim> {
+
+    <T> T readObject(I input, Class<T> type);
+
+    Object readClassAndObject(I input);
+
+    void writeObject(O output, Object object);
+
+    void writeClassAndObject(O output, Object object);
+
+    <T> T readObjectOrNull(I input, Class<T> type);
+
+    void writeObjectOrNull(O output, Object object, Class type);
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShimService.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShimService.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShimService.java
new file mode 100644
index 0000000..024d40c
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShimService.java
@@ -0,0 +1,83 @@
+/*
+ * 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.structure.io.kryoshim;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This interface exists to decouple HadoopPools from TinkerPop's shaded Kryo.
+ * <p>
+ * VertexWritable and ObjectWritable formerly implemented Serializable by
+ * resorting to statically-pooled shaded Kryo instances maintained by the HadoopPools class.
+ * This is awkward because those shaded Kryo instances require class registration by default.
+ * <p>
+ * Consider what happens with custom property datatypes reachable from the reference graph rooted at an ObjectWritable
+ * or VertexWritable instance.  It is not enough for these property classes to merely implement
+ * Serializable, though one think that from skimming ObjectWritable/VertexWritable.  Those classes
+ * must also register with TinkerPop's internal, shaded Kryo instances as maintained by HadoopPools,
+ * or else configure those instances to accept unregistered classes.
+ * Otherwise, TinkerPop's shaded Kryo will refuse to serialize those properties (even though
+ * they implement Serializable, and even though the user might think they are only using
+ * Java's standard Serialization mechanism!).
+ * <p>
+ * By hiding the mechanics of serialization behind this interface instead of hardcoding it in
+ * HadoopPools, the user can decide how to implement serialization for ObjectWritable/VertexWritable
+ * (and whatever other classes in TinkerPop decide to implement Serializable but then delegate
+ * all of the implementation details, like ObjectWritable/VertexWritable do now).
+ */
+public interface KryoShimService {
+
+    /**
+     * Deserializes an object from an input stream.
+     *
+     * @param source the stream from which to read an object's serialized form
+     * @return the first deserialized object available from {@code source}
+     */
+    Object readClassAndObject(InputStream source);
+
+    /**
+     * Serializes an object to an output stream.  This may flush the output stream.
+     *
+     * @param o the object to serialize
+     * @param sink the stream into which the serialized object is written
+     */
+    void writeClassAndObject(Object o, OutputStream sink);
+
+    /**
+     * Returns this service's relative priority number.  Unless explicitly overridden through a
+     * system property ({@link KryoShimServiceLoader#SHIM_CLASS_SYSTEM_PROPERTY}),
+     * the service implementation with the numerically highest priority will be used
+     * and all others ignored.  In other words, the highest priority wins (in the absence of a
+     * system property override).
+     * <p>
+     * TinkerPop's current default implementation uses priority value zero.
+     * <p>
+     * Third-party implementations of this interface should (but are not technically required)
+     * to use a priority value with absolute value greater than 100.
+     * <p>
+     * The implementation currently breaks priority ties by lexicographical comparison of
+     * fully-qualified package-and-classname, but this tie-breaking behavior should be
+     * considered undefined and subject to future change.  Ties are ignored if the service
+     * is explicitly set through the system property mentioned above.
+     *
+     * @return this implementation's priority value
+     */
+    int getPriority();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShimServiceLoader.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShimServiceLoader.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShimServiceLoader.java
new file mode 100644
index 0000000..1d5413d
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/KryoShimServiceLoader.java
@@ -0,0 +1,137 @@
+/*
+ * 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.structure.io.kryoshim;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.ServiceLoader;
+
+public class KryoShimServiceLoader {
+
+    private static volatile KryoShimService CACHED_SHIM_SERVICE;
+
+    private static final Logger log = LoggerFactory.getLogger(KryoShimServiceLoader.class);
+
+    /**
+     * Set this system property to the fully-qualified name of a {@link KryoShimService}
+     * package-and-classname to force it into service.  Setting this property causes the
+     * priority-selection mechanism ({@link KryoShimService#getPriority()}) to be ignored.
+     */
+    public static final String SHIM_CLASS_SYSTEM_PROPERTY = "tinkerpop.kryo.shim";
+
+    public static KryoShimService load(boolean forceReload) {
+
+        if (null != CACHED_SHIM_SERVICE && !forceReload) {
+            return CACHED_SHIM_SERVICE;
+        }
+
+        ArrayList<KryoShimService> services = new ArrayList<>();
+
+        ServiceLoader<KryoShimService> sl = ServiceLoader.load(KryoShimService.class);
+
+        KryoShimService result = null;
+
+        synchronized (KryoShimServiceLoader.class) {
+            if (forceReload) {
+                sl.reload();
+            }
+
+            for (KryoShimService kss : sl) {
+                services.add(kss);
+            }
+        }
+
+        String shimClass = System.getProperty(SHIM_CLASS_SYSTEM_PROPERTY);
+
+        if (null != shimClass) {
+            for (KryoShimService kss : services) {
+                if (kss.getClass().getCanonicalName().equals(shimClass)) {
+                    log.info("Set {} provider to {} ({}) from system property {}={}",
+                            KryoShimService.class.getSimpleName(), kss, kss.getClass(),
+                            SHIM_CLASS_SYSTEM_PROPERTY, shimClass);
+                    result = kss;
+                }
+            }
+        } else {
+            Collections.sort(services, KryoShimServiceComparator.INSTANCE);
+
+            for (KryoShimService kss : services) {
+                log.debug("Found Kryo shim service class {} (priority {})", kss.getClass(), kss.getPriority());
+            }
+
+            if (0 != services.size()) {
+                result = services.get(services.size() - 1);
+            }
+        }
+
+
+        if (null == result) {
+            throw new IllegalStateException("Unable to load KryoShimService");
+        }
+
+        log.info("Set {} provider to {} ({}) because its priority value ({}) is the highest available",
+                KryoShimService.class.getSimpleName(), result, result.getClass(), result.getPriority());
+
+        return CACHED_SHIM_SERVICE = result;
+    }
+
+    public static KryoShimService load() {
+        return load(false);
+    }
+
+    public static byte[] writeClassAndObjectToBytes(Object o) {
+        KryoShimService shimService = load();
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+        shimService.writeClassAndObject(o, baos);
+
+        return baos.toByteArray();
+    }
+
+    public static <T> T readClassAndObject(InputStream source) {
+        KryoShimService shimService = load();
+
+        return (T)shimService.readClassAndObject(source);
+    }
+
+    private enum KryoShimServiceComparator implements Comparator<KryoShimService> {
+        INSTANCE;
+
+        @Override
+        public int compare(KryoShimService a, KryoShimService b) {
+            int ap = a.getPriority();
+            int bp = b.getPriority();
+
+            if (ap < bp) {
+                return -1;
+            } else if (bp < ap) {
+                return 1;
+            } else {
+                return a.getClass().getCanonicalName().compareTo(b.getClass().getCanonicalName());
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/OutputShim.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/OutputShim.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/OutputShim.java
new file mode 100644
index 0000000..4468434
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/OutputShim.java
@@ -0,0 +1,41 @@
+/*
+ * 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.structure.io.kryoshim;
+
+import java.io.IOException;
+
+/**
+ * A minimal {@link org.apache.tinkerpop.shaded.kryo.io.Output}-like abstraction.
+ */
+public interface OutputShim {
+
+    void writeByte(byte b);
+
+    void writeBytes(byte[] array, int offset, int count);
+
+    void writeString(String s);
+
+    void writeLong(long l);
+
+    void writeInt(int i);
+
+    void writeDouble(double d);
+
+    void flush();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/SerializerShim.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/SerializerShim.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/SerializerShim.java
new file mode 100644
index 0000000..686350d
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/SerializerShim.java
@@ -0,0 +1,35 @@
+/*
+ * 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.structure.io.kryoshim;
+
+/**
+ * A minimal {@link org.apache.tinkerpop.shaded.kryo.Serializer}-like abstraction.
+ *
+ * @param <T> the class this serializer reads/writes from/to bytes.
+ */
+public interface SerializerShim<T> {
+
+    <O extends OutputShim> void write(KryoShim<?, O> kryo, O output, T starGraph);
+
+    <I extends InputShim> T read(KryoShim<I, ?> kryo, I input, Class<T> clazz);
+
+    default boolean isImmutable() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/package-info.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/package-info.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/package-info.java
new file mode 100644
index 0000000..436f117
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/package-info.java
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+/**
+ * Abstracts a minimal subset of Kryo types and methods.
+ * <p>
+ * Kryo is often shaded.  For instance, TinkerPop's Gryo
+ * serializer relies on a shaded Kryo package.
+ * TinkerPop serializers written against a particular shaded
+ * Kryo package (or the unshaded Kryo package) are compatible
+ * only with that package.  In contrast, TinkerPop serializers written
+ * against this abstraction can be used with any shaded or
+ * unshaded Kryo package, so long as the signatures and behavior
+ * of the methods in this package remain stable.
+ * <p>
+ * To show how this is useful, consider
+ * {@link org.apache.tinkerpop.gremlin.structure.util.star.StarGraphSerializer}.
+ * This class has logic unique to TinkerPop that performs
+ * efficient and forward-compatible serialization of
+ * {@link org.apache.tinkerpop.gremlin.structure.util.star.StarGraph}
+ * instances.  It takes advantage of package-level visibility
+ * and the fact that it shares a package with its target,
+ * so it would be challenging to cleanly and naturally replicate
+ * (i.e. without package spoofing or runtime visibility overrides).
+ * By implementing
+ * {@link org.apache.tinkerpop.gremlin.structure.io.kryoshim.SerializerShim}
+ * instead of, say, Gryo's shaded
+ * {@link org.apache.tinkerpop.shaded.kryo.Serializer},
+ * it can be used with anybody's Kryo package, regardless of whether
+ * that package is shaded or not.  This lets third-parties reuse
+ * TinkerPop's efficient, internals-aware StarGraph serializer on
+ * their own serialization platform (and without altering
+ * TinkerPop's bytecode, let alone its source).
+ * <p>
+ * The number of types and methods in this
+ * package is deliberately small to reduce the likelihood of a
+ * new Kryo release introducing an incompatible change.
+ */
+package org.apache.tinkerpop.gremlin.structure.io.kryoshim;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedInputAdapter.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedInputAdapter.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedInputAdapter.java
new file mode 100644
index 0000000..2872326
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedInputAdapter.java
@@ -0,0 +1,66 @@
+/*
+ * 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.structure.io.kryoshim.shaded;
+
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.InputShim;
+import org.apache.tinkerpop.shaded.kryo.io.Input;
+
+public class ShadedInputAdapter implements InputShim {
+
+    private final Input shadedInput;
+
+    public ShadedInputAdapter(Input shadedInput) {
+        this.shadedInput = shadedInput;
+    }
+
+    Input getShadedInput() {
+        return shadedInput;
+    }
+
+    @Override
+    public byte readByte()
+    {
+        return shadedInput.readByte();
+    }
+
+    @Override
+    public byte[] readBytes(int size) {
+        return shadedInput.readBytes(size);
+    }
+
+    @Override
+    public String readString() {
+        return shadedInput.readString();
+    }
+
+    @Override
+    public long readLong() {
+        return shadedInput.readLong();
+    }
+
+    @Override
+    public int readInt() {
+        return shadedInput.readInt();
+    }
+
+    @Override
+    public double readDouble() {
+        return shadedInput.readDouble();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedKryoAdapter.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedKryoAdapter.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedKryoAdapter.java
new file mode 100644
index 0000000..0e85f6f
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedKryoAdapter.java
@@ -0,0 +1,67 @@
+/*
+ * 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.structure.io.kryoshim.shaded;
+
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.KryoShim;
+import org.apache.tinkerpop.shaded.kryo.Kryo;
+
+public class ShadedKryoAdapter implements KryoShim<ShadedInputAdapter, ShadedOutputAdapter> {
+
+    private final Kryo shadedKryo;
+
+    public ShadedKryoAdapter(Kryo shadedKryo) {
+        this.shadedKryo = shadedKryo;
+    }
+
+    @Override
+    public <T> T readObject(ShadedInputAdapter input, Class<T> type)
+    {
+        return shadedKryo.readObject(input.getShadedInput(), type);
+    }
+
+    @Override
+    public Object readClassAndObject(ShadedInputAdapter input)
+    {
+        return shadedKryo.readClassAndObject(input.getShadedInput());
+    }
+
+    @Override
+    public void writeObject(ShadedOutputAdapter output, Object object)
+    {
+        shadedKryo.writeObject(output.getShadedOutput(), object);
+    }
+
+    @Override
+    public void writeClassAndObject(ShadedOutputAdapter output, Object object)
+    {
+        shadedKryo.writeClassAndObject(output.getShadedOutput(), object);
+    }
+
+    @Override
+    public <T> T readObjectOrNull(ShadedInputAdapter input, Class<T> type)
+    {
+        return shadedKryo.readObjectOrNull(input.getShadedInput(), type);
+    }
+
+    @Override
+    public void writeObjectOrNull(ShadedOutputAdapter output, Object object, Class type)
+    {
+        shadedKryo.writeObjectOrNull(output.getShadedOutput(), object, type);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedOutputAdapter.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedOutputAdapter.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedOutputAdapter.java
new file mode 100644
index 0000000..5ca0ecb
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedOutputAdapter.java
@@ -0,0 +1,72 @@
+/*
+ * 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.structure.io.kryoshim.shaded;
+
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.OutputShim;
+import org.apache.tinkerpop.shaded.kryo.io.Output;
+
+public class ShadedOutputAdapter implements OutputShim {
+
+    private final Output shadedOutput;
+
+    public ShadedOutputAdapter(Output shadedOutput) {
+        this.shadedOutput = shadedOutput;
+    }
+
+    @Override
+    public void writeByte(byte b)
+    {
+        shadedOutput.writeByte(b);
+    }
+
+    @Override
+    public void writeBytes(byte[] array, int offset, int count) {
+        shadedOutput.writeBytes(array, offset, count);
+    }
+
+    @Override
+    public void writeString(String s) {
+        shadedOutput.writeString(s);
+    }
+
+    @Override
+    public void writeLong(long l) {
+        shadedOutput.writeLong(l);
+    }
+
+    @Override
+    public void writeInt(int i) {
+        shadedOutput.writeInt(i);
+    }
+
+    @Override
+    public void writeDouble(double d) {
+        shadedOutput.writeDouble(d);
+    }
+
+    @Override
+    public void flush() {
+        shadedOutput.flush();
+    }
+
+    Output getShadedOutput()
+    {
+        return shadedOutput;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedSerializerAdapter.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedSerializerAdapter.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedSerializerAdapter.java
new file mode 100644
index 0000000..ebfd0fb
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/ShadedSerializerAdapter.java
@@ -0,0 +1,54 @@
+/*
+ * 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.structure.io.kryoshim.shaded;
+
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.SerializerShim;
+import org.apache.tinkerpop.shaded.kryo.Kryo;
+import org.apache.tinkerpop.shaded.kryo.Serializer;
+import org.apache.tinkerpop.shaded.kryo.io.Input;
+import org.apache.tinkerpop.shaded.kryo.io.Output;
+
+public class ShadedSerializerAdapter<T> extends Serializer<T> {
+
+    SerializerShim<T> serializer;
+
+    public ShadedSerializerAdapter(SerializerShim<T> serializer) {
+        this.serializer = serializer;
+        setImmutable(this.serializer.isImmutable());
+    }
+
+    @Override
+    public void write(Kryo kryo, Output output, T t) {
+        /* These adapters could be cached pretty efficiently in instance fields if it were guaranteed that this
+         * class was never subject to concurrent use.  That's true of Kryo instances, but it is not clear that
+         * it is true of Serializer instances.
+         */
+        ShadedKryoAdapter shadedKryoAdapter = new ShadedKryoAdapter(kryo);
+        ShadedOutputAdapter shadedOutputAdapter = new ShadedOutputAdapter(output);
+        serializer.write(shadedKryoAdapter, shadedOutputAdapter, t);
+    }
+
+    @Override
+    public T read(Kryo kryo, Input input, Class<T> aClass) {
+        // Same caching opportunity as in write(...)
+        ShadedKryoAdapter shadedKryoAdapter = new ShadedKryoAdapter(kryo);
+        ShadedInputAdapter shadedInputAdapter = new ShadedInputAdapter(input);
+        return serializer.read(shadedKryoAdapter, shadedInputAdapter, aClass);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/package-info.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/package-info.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/package-info.java
new file mode 100644
index 0000000..ea70003
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/kryoshim/shaded/package-info.java
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+/**
+ * Implementations of the interfaces in
+ * {@link org.apache.tinkerpop.gremlin.structure.io.kryoshim}
+ * using TinkerPop's shaded copy of Kryo.
+ */
+package org.apache.tinkerpop.gremlin.structure.io.kryoshim.shaded;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/ef528697/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphGryoSerializer.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphGryoSerializer.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphGryoSerializer.java
index 3805e2c..2acd770 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphGryoSerializer.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/star/StarGraphGryoSerializer.java
@@ -18,20 +18,13 @@
  */
 package org.apache.tinkerpop.gremlin.structure.util.star;
 
-import org.apache.tinkerpop.gremlin.process.computer.GraphFilter;
-import org.apache.tinkerpop.gremlin.structure.Direction;
-import org.apache.tinkerpop.gremlin.structure.Edge;
-import org.apache.tinkerpop.gremlin.structure.T;
-import org.apache.tinkerpop.gremlin.structure.VertexProperty;
-import org.apache.tinkerpop.shaded.kryo.Kryo;
-import org.apache.tinkerpop.shaded.kryo.Serializer;
-import org.apache.tinkerpop.shaded.kryo.io.Input;
-import org.apache.tinkerpop.shaded.kryo.io.Output;
-
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
+import org.apache.tinkerpop.gremlin.process.computer.GraphFilter;
+import org.apache.tinkerpop.gremlin.structure.Direction;
+import org.apache.tinkerpop.gremlin.structure.io.kryoshim.shaded.ShadedSerializerAdapter;
+
 /**
  * Kryo serializer for {@link StarGraph}.  Implements an internal versioning capability for backward compatibility.
  * The single byte at the front of the serialization stream denotes the version.  That version can be used to choose
@@ -42,15 +35,10 @@ import java.util.Map;
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  * @author Stephen Mallette (http://stephen.genoprime.com)
  */
-public final class StarGraphGryoSerializer extends Serializer<StarGraph> {
+public final class StarGraphGryoSerializer extends ShadedSerializerAdapter<StarGraph>  {
 
     private static final Map<Direction, StarGraphGryoSerializer> CACHE = new HashMap<>();
 
-    private final Direction edgeDirectionToSerialize;
-    private GraphFilter graphFilter = new GraphFilter(); // will allow all vertices/edges
-
-    private final static byte VERSION_1 = Byte.MIN_VALUE;
-
     static {
         CACHE.put(Direction.BOTH, new StarGraphGryoSerializer(Direction.BOTH));
         CACHE.put(Direction.IN, new StarGraphGryoSerializer(Direction.IN));
@@ -58,8 +46,12 @@ public final class StarGraphGryoSerializer extends Serializer<StarGraph> {
         CACHE.put(null, new StarGraphGryoSerializer(null));
     }
 
+    private StarGraphGryoSerializer(final Direction edgeDirectionToSerialize, final GraphFilter graphFilter) {
+        super(new StarGraphSerializer(edgeDirectionToSerialize, graphFilter));
+    }
+
     private StarGraphGryoSerializer(final Direction edgeDirectionToSerialize) {
-        this.edgeDirectionToSerialize = edgeDirectionToSerialize;
+        this(edgeDirectionToSerialize, new GraphFilter());
     }
 
     /**
@@ -71,100 +63,7 @@ public final class StarGraphGryoSerializer extends Serializer<StarGraph> {
     }
 
     public static StarGraphGryoSerializer withGraphFilter(final GraphFilter graphFilter) {
-        final StarGraphGryoSerializer serializer = new StarGraphGryoSerializer(Direction.BOTH);
-        serializer.graphFilter = graphFilter.clone();
+        final StarGraphGryoSerializer serializer = new StarGraphGryoSerializer(Direction.BOTH, graphFilter.clone());
         return serializer;
     }
-
-    @Override
-    public void write(final Kryo kryo, final Output output, final StarGraph starGraph) {
-        output.writeByte(VERSION_1);
-        kryo.writeObjectOrNull(output, starGraph.edgeProperties, HashMap.class);
-        kryo.writeObjectOrNull(output, starGraph.metaProperties, HashMap.class);
-        kryo.writeClassAndObject(output, starGraph.starVertex.id);
-        kryo.writeObject(output, starGraph.starVertex.label);
-        writeEdges(kryo, output, starGraph, Direction.IN);
-        writeEdges(kryo, output, starGraph, Direction.OUT);
-        kryo.writeObject(output, null != starGraph.starVertex.vertexProperties);
-        if (null != starGraph.starVertex.vertexProperties) {
-            kryo.writeObject(output, starGraph.starVertex.vertexProperties.size());
-            for (final Map.Entry<String, List<VertexProperty>> vertexProperties : starGraph.starVertex.vertexProperties.entrySet()) {
-                kryo.writeObject(output, vertexProperties.getKey());
-                kryo.writeObject(output, vertexProperties.getValue().size());
-                for (final VertexProperty vertexProperty : vertexProperties.getValue()) {
-                    kryo.writeClassAndObject(output, vertexProperty.id());
-                    kryo.writeClassAndObject(output, vertexProperty.value());
-                }
-            }
-        }
-    }
-
-    /**
-     * If the returned {@link StarGraph} is null, that means that the {@link GraphFilter} filtered the vertex.
-     */
-    @Override
-    public StarGraph read(final Kryo kryo, final Input input, final Class<StarGraph> aClass) {
-        final StarGraph starGraph = StarGraph.open();
-        input.readByte();  // version field ignored for now - for future use with backward compatibility
-        starGraph.edgeProperties = kryo.readObjectOrNull(input, HashMap.class);
-        starGraph.metaProperties = kryo.readObjectOrNull(input, HashMap.class);
-        starGraph.addVertex(T.id, kryo.readClassAndObject(input), T.label, kryo.readObject(input, String.class));
-        readEdges(kryo, input, starGraph, Direction.IN);
-        readEdges(kryo, input, starGraph, Direction.OUT);
-        if (kryo.readObject(input, Boolean.class)) {
-            final int numberOfUniqueKeys = kryo.readObject(input, Integer.class);
-            for (int i = 0; i < numberOfUniqueKeys; i++) {
-                final String vertexPropertyKey = kryo.readObject(input, String.class);
-                final int numberOfVertexPropertiesWithKey = kryo.readObject(input, Integer.class);
-                for (int j = 0; j < numberOfVertexPropertiesWithKey; j++) {
-                    final Object id = kryo.readClassAndObject(input);
-                    final Object value = kryo.readClassAndObject(input);
-                    starGraph.starVertex.property(VertexProperty.Cardinality.list, vertexPropertyKey, value, T.id, id);
-                }
-            }
-        }
-        return this.graphFilter.hasFilter() ? starGraph.applyGraphFilter(this.graphFilter).orElse(null) : starGraph;
-    }
-
-    private void writeEdges(final Kryo kryo, final Output output, final StarGraph starGraph, final Direction direction) {
-        // only write edges if there are some AND if the user requested them to be serialized AND if they match
-        // the direction being serialized by the format
-        final Map<String, List<Edge>> starEdges = direction.equals(Direction.OUT) ? starGraph.starVertex.outEdges : starGraph.starVertex.inEdges;
-        final boolean writeEdges = null != starEdges && edgeDirectionToSerialize != null
-                && (edgeDirectionToSerialize == direction || edgeDirectionToSerialize == Direction.BOTH);
-        kryo.writeObject(output, writeEdges);
-        if (writeEdges) {
-            kryo.writeObject(output, starEdges.size());
-            for (final Map.Entry<String, List<Edge>> edges : starEdges.entrySet()) {
-                kryo.writeObject(output, edges.getKey());
-                kryo.writeObject(output, edges.getValue().size());
-                for (final Edge edge : edges.getValue()) {
-                    kryo.writeClassAndObject(output, edge.id());
-                    kryo.writeClassAndObject(output, direction.equals(Direction.OUT) ? edge.inVertex().id() : edge.outVertex().id());
-                }
-            }
-        }
-    }
-
-    private void readEdges(final Kryo kryo, final Input input, final StarGraph starGraph, final Direction direction) {
-        if (kryo.readObject(input, Boolean.class)) {
-            final int numberOfUniqueLabels = kryo.readObject(input, Integer.class);
-            for (int i = 0; i < numberOfUniqueLabels; i++) {
-                final String edgeLabel = kryo.readObject(input, String.class);
-                final int numberOfEdgesWithLabel = kryo.readObject(input, Integer.class);
-                for (int j = 0; j < numberOfEdgesWithLabel; j++) {
-                    final Object edgeId = kryo.readClassAndObject(input);
-                    final Object adjacentVertexId = kryo.readClassAndObject(input);
-                    if (this.graphFilter.checkEdgeLegality(direction, edgeLabel).positive()) {
-                        if (direction.equals(Direction.OUT))
-                            starGraph.starVertex.addOutEdge(edgeLabel, starGraph.addVertex(T.id, adjacentVertexId), T.id, edgeId);
-                        else
-                            starGraph.starVertex.addInEdge(edgeLabel, starGraph.addVertex(T.id, adjacentVertexId), T.id, edgeId);
-                    } else if (null != starGraph.edgeProperties) {
-                        starGraph.edgeProperties.remove(edgeId);
-                    }
-                }
-            }
-        }
-    }
 }


[22/34] incubator-tinkerpop git commit: Moved metrics init to Gremlin Server constructor.

Posted by ok...@apache.org.
Moved metrics init to Gremlin Server constructor.

By moving it to the constructor, the metric init does not get bypassed if programmatically starting Gremlin Server. CTR


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

Branch: refs/heads/TINKERPOP-1278
Commit: f7e7df093af7a488e52dd96440c25d08c29dfe73
Parents: 6e17d66
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Mon Jun 6 16:16:39 2016 -0400
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Mon Jun 6 16:16:39 2016 -0400

----------------------------------------------------------------------
 .../java/org/apache/tinkerpop/gremlin/server/GremlinServer.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f7e7df09/gremlin-server/src/main/java/org/apache/tinkerpop/gremlin/server/GremlinServer.java
----------------------------------------------------------------------
diff --git a/gremlin-server/src/main/java/org/apache/tinkerpop/gremlin/server/GremlinServer.java b/gremlin-server/src/main/java/org/apache/tinkerpop/gremlin/server/GremlinServer.java
index 49b2375..49ce3bc 100644
--- a/gremlin-server/src/main/java/org/apache/tinkerpop/gremlin/server/GremlinServer.java
+++ b/gremlin-server/src/main/java/org/apache/tinkerpop/gremlin/server/GremlinServer.java
@@ -80,6 +80,7 @@ public class GremlinServer {
      * Construct a Gremlin Server instance from {@link Settings}.
      */
     public GremlinServer(final Settings settings) {
+        settings.optionalMetrics().ifPresent(GremlinServer::configureMetrics);
         this.settings = settings;
         this.isEpollEnabled = settings.useEpollEventLoop && SystemUtils.IS_OS_LINUX;
         if(settings.useEpollEventLoop && !SystemUtils.IS_OS_LINUX){
@@ -341,7 +342,6 @@ public class GremlinServer {
         }
 
         logger.info("Configuring Gremlin Server from {}", file);
-        settings.optionalMetrics().ifPresent(GremlinServer::configureMetrics);
         final GremlinServer server = new GremlinServer(settings);
         server.start().exceptionally(t -> {
             logger.error("Gremlin Server was unable to start and will now begin shutdown: {}", t.getMessage());


[25/34] incubator-tinkerpop git commit: VertexProgramHelper now uses Gryo to serialize objects if the standard Java Serializer fails. Moving forward (3.3.x), this should all be accomplished by KryoShimServiceLoader.

Posted by ok...@apache.org.
VertexProgramHelper now uses Gryo to serialize objects if the standard Java Serializer fails. Moving forward (3.3.x), this should all be accomplished by KryoShimServiceLoader.


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

Branch: refs/heads/TINKERPOP-1278
Commit: 797364cb4b5c4d0bd18d59a3a7c6cdb5603e136c
Parents: e700363
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Mon Jun 6 16:37:38 2016 -0600
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Mon Jun 6 16:37:38 2016 -0600

----------------------------------------------------------------------
 .../computer/util/VertexProgramHelper.java      | 33 +++++++++++++++-----
 .../gremlin/structure/io/gryo/GryoMapper.java   | 17 +++++-----
 .../gremlin/hadoop/HadoopGraphProvider.java     |  4 +++
 .../structure/io/gryo/GryoRegistrator.java      | 24 +++-----------
 4 files changed, 45 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/797364cb/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/util/VertexProgramHelper.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/util/VertexProgramHelper.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/util/VertexProgramHelper.java
index bc67866..2b3a0b2 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/util/VertexProgramHelper.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/computer/util/VertexProgramHelper.java
@@ -25,8 +25,13 @@ import org.apache.tinkerpop.gremlin.process.traversal.Step;
 import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
 import org.apache.tinkerpop.gremlin.process.traversal.step.map.EdgeVertexStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.map.VertexStep;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoPool;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader;
 import org.apache.tinkerpop.gremlin.util.Serializer;
+import org.apache.tinkerpop.shaded.kryo.io.Input;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashSet;
@@ -37,6 +42,8 @@ import java.util.Set;
  */
 public final class VertexProgramHelper {
 
+    private static final GryoPool GRYO_POOL = GryoPool.build().create();
+
     private VertexProgramHelper() {
     }
 
@@ -67,21 +74,33 @@ public final class VertexProgramHelper {
             final String byteString = Arrays.toString(Serializer.serializeObject(object));
             configuration.setProperty(key, byteString.substring(1, byteString.length() - 1));
         } catch (final IOException e) {
-            throw new IllegalArgumentException(e.getMessage(), e);
+            try {
+                final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+                GRYO_POOL.doWithWriter(kryo -> kryo.writeObject(outputStream, object));
+                String byteString = Arrays.toString(outputStream.toByteArray());
+                configuration.setProperty(key, byteString.substring(1, byteString.length() - 1));
+            } catch (final Exception e1) {
+                throw new IllegalArgumentException(e1.getMessage(), e1);
+            }
         }
     }
 
     public static <T> T deserialize(final Configuration configuration, final String key) {
+        final String[] stringBytes = configuration.getString(key).split(",");
+        byte[] bytes = new byte[stringBytes.length];
+        for (int i = 0; i < stringBytes.length; i++) {
+            bytes[i] = Byte.valueOf(stringBytes[i].trim());
+        }
         try {
-            final String[] stringBytes = configuration.getString(key).split(",");
-            byte[] bytes = new byte[stringBytes.length];
-            for (int i = 0; i < stringBytes.length; i++) {
-                bytes[i] = Byte.valueOf(stringBytes[i].trim());
-            }
             return (T) Serializer.deserializeObject(bytes);
         } catch (final IOException | ClassNotFoundException e) {
-            throw new IllegalArgumentException(e.getMessage(), e);
+            try {
+                return (T) GRYO_POOL.readWithKryo(kryo -> kryo.readClassAndObject(new Input(new ByteArrayInputStream(bytes))));
+            } catch (final Exception e1) {
+                throw new IllegalArgumentException(e1.getMessage(), e1);
+            }
         }
+
     }
 
     public static <S, E> Traversal.Admin<S, E> reverse(final Traversal.Admin<S, E> traversal) {

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/797364cb/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
index 41ca44d..7bf9b7d 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoMapper.java
@@ -48,6 +48,7 @@ import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.TraverserSe
 import org.apache.tinkerpop.gremlin.process.traversal.util.DefaultTraversalMetrics;
 import org.apache.tinkerpop.gremlin.process.traversal.util.ImmutableMetrics;
 import org.apache.tinkerpop.gremlin.process.traversal.util.MutableMetrics;
+import org.apache.tinkerpop.gremlin.process.traversal.util.PureTraversal;
 import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalExplanation;
 import org.apache.tinkerpop.gremlin.structure.Direction;
 import org.apache.tinkerpop.gremlin.structure.Edge;
@@ -330,6 +331,8 @@ public final class GryoMapper implements Mapper<Kryo> {
             add(GryoTypeReg.of(AtomicLong.class, 79));
             add(GryoTypeReg.of(Pair.class, 88, new PairSerializer()));
             add(GryoTypeReg.of(TraversalExplanation.class, 106, new JavaSerializer()));
+            add(GryoTypeReg.of(GraphFilter.class, 120, new JavaSerializer())); // ***LAST ID***
+            //add(GryoTypeReg.of(PureTraversal.class, 121, new JavaSerializer()));
 
             add(GryoTypeReg.of(Duration.class, 93, new JavaTimeSerializers.DurationSerializer()));
             add(GryoTypeReg.of(Instant.class, 94, new JavaTimeSerializers.InstantSerializer()));
@@ -355,7 +358,7 @@ public final class GryoMapper implements Mapper<Kryo> {
             add(GryoTypeReg.of(GroupStepV3d0.GroupBiOperatorV3d0.class, 113));
             add(GryoTypeReg.of(RangeGlobalStep.RangeBiOperator.class, 114));
             add(GryoTypeReg.of(OrderGlobalStep.OrderBiOperator.class, 118, new JavaSerializer())); // because they contain traversals
-            add(GryoTypeReg.of(ProfileStep.ProfileBiOperator.class, 119)); // ***LAST ID***
+            add(GryoTypeReg.of(ProfileStep.ProfileBiOperator.class, 119));
         }};
 
         private final List<IoRegistry> registries = new ArrayList<>();
@@ -374,12 +377,12 @@ public final class GryoMapper implements Mapper<Kryo> {
             // For justification of these default registration rules, see TinkerPopKryoRegistrator
             for (TypeRegistration<?> tr : typeRegistrations) {
                 if (tr.hasSerializer() /* no serializer is acceptable */ &&
-                    null == tr.getSerializerShim() /* a shim serializer is acceptable */ &&
-                    !(tr.getShadedSerializer() instanceof JavaSerializer) /* shaded JavaSerializer is acceptable */) {
+                        null == tr.getSerializerShim() /* a shim serializer is acceptable */ &&
+                        !(tr.getShadedSerializer() instanceof JavaSerializer) /* shaded JavaSerializer is acceptable */) {
                     // everything else is invalid
                     String msg = String.format("The default GryoMapper type registration %s is invalid.  " +
-                            "It must supply either an implementation of %s or %s, but supplies neither.  " +
-                            "This is probably a bug in GryoMapper's default serialization class registrations.", tr,
+                                    "It must supply either an implementation of %s or %s, but supplies neither.  " +
+                                    "This is probably a bug in GryoMapper's default serialization class registrations.", tr,
                             SerializerShim.class.getCanonicalName(), JavaSerializer.class.getCanonicalName());
                     throw new IllegalStateException(msg);
                 }
@@ -553,8 +556,8 @@ public final class GryoMapper implements Mapper<Kryo> {
             if (1 < serializerCount) {
                 String msg = String.format(
                         "GryoTypeReg accepts at most one kind of serializer, but multiple " +
-                        "serializers were supplied for class %s (id %s).  " +
-                        "Shaded serializer: %s.  Shim serializer: %s.  Shaded serializer function: %s.",
+                                "serializers were supplied for class %s (id %s).  " +
+                                "Shaded serializer: %s.  Shim serializer: %s.  Shaded serializer function: %s.",
                         this.clazz.getCanonicalName(), id,
                         this.shadedSerializer, this.serializerShim, this.functionOfShadedKryo);
                 throw new IllegalArgumentException(msg);

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/797364cb/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java
index da00463..57157db 100644
--- a/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java
+++ b/hadoop-gremlin/src/test/java/org/apache/tinkerpop/gremlin/hadoop/HadoopGraphProvider.java
@@ -28,6 +28,7 @@ import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopGraph;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopProperty;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopVertex;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopVertexProperty;
+import org.apache.tinkerpop.gremlin.hadoop.structure.io.HadoopPoolShimService;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.graphson.GraphSONInputFormat;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.gryo.GryoInputFormat;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.gryo.GryoOutputFormat;
@@ -45,6 +46,8 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 
+import static org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.KryoShimServiceLoader.SHIM_CLASS_SYSTEM_PROPERTY;
+
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  * @author Stephen Mallette (http://stephen.genoprime.com)
@@ -109,6 +112,7 @@ public class HadoopGraphProvider extends AbstractGraphProvider {
 
     @Override
     public Map<String, Object> getBaseConfiguration(final String graphName, final Class<?> test, final String testMethodName, final LoadGraphWith.GraphData loadGraphWith) {
+        System.setProperty(SHIM_CLASS_SYSTEM_PROPERTY, HadoopPoolShimService.class.getCanonicalName());
         this.graphSONInput = RANDOM.nextBoolean();
         return new HashMap<String, Object>() {{
             put(Graph.GRAPH, HadoopGraph.class.getName());

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/797364cb/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java
----------------------------------------------------------------------
diff --git a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java
index 9563408..68112d7 100644
--- a/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java
+++ b/spark-gremlin/src/main/java/org/apache/tinkerpop/gremlin/spark/structure/io/gryo/GryoRegistrator.java
@@ -22,8 +22,6 @@ import com.esotericsoftware.kryo.Kryo;
 import com.esotericsoftware.kryo.Serializer;
 import com.esotericsoftware.kryo.serializers.JavaSerializer;
 import org.apache.spark.serializer.KryoRegistrator;
-import org.apache.spark.util.SerializableConfiguration;
-import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopConfiguration;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopEdge;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopProperty;
 import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopVertex;
@@ -31,37 +29,27 @@ import org.apache.tinkerpop.gremlin.hadoop.structure.HadoopVertexProperty;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.ObjectWritable;
 import org.apache.tinkerpop.gremlin.hadoop.structure.io.VertexWritable;
 import org.apache.tinkerpop.gremlin.process.computer.util.ComputerGraph;
-import org.apache.tinkerpop.gremlin.process.traversal.Path;
 import org.apache.tinkerpop.gremlin.process.traversal.step.util.ImmutablePath;
 import org.apache.tinkerpop.gremlin.process.traversal.step.util.MutablePath;
-import org.apache.tinkerpop.gremlin.process.traversal.traverser.B_LP_O_P_S_SE_SL_Traverser;
-import org.apache.tinkerpop.gremlin.process.traversal.traverser.B_LP_O_S_SE_SL_Traverser;
-import org.apache.tinkerpop.gremlin.process.traversal.traverser.B_O_S_SE_SL_Traverser;
-import org.apache.tinkerpop.gremlin.process.traversal.traverser.B_O_Traverser;
-import org.apache.tinkerpop.gremlin.process.traversal.traverser.LP_O_OB_P_S_SE_SL_Traverser;
-import org.apache.tinkerpop.gremlin.process.traversal.traverser.LP_O_OB_S_SE_SL_Traverser;
-import org.apache.tinkerpop.gremlin.process.traversal.traverser.O_OB_S_SE_SL_Traverser;
-import org.apache.tinkerpop.gremlin.process.traversal.traverser.O_Traverser;
 import org.apache.tinkerpop.gremlin.spark.process.computer.payload.MessagePayload;
 import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewIncomingPayload;
 import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewOutgoingPayload;
 import org.apache.tinkerpop.gremlin.spark.process.computer.payload.ViewPayload;
 import org.apache.tinkerpop.gremlin.spark.structure.io.gryo.kryoshim.unshaded.UnshadedSerializerAdapter;
-import org.apache.tinkerpop.gremlin.structure.Edge;
-import org.apache.tinkerpop.gremlin.structure.Property;
-import org.apache.tinkerpop.gremlin.structure.Vertex;
-import org.apache.tinkerpop.gremlin.structure.VertexProperty;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoMapper;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoSerializers;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.TypeRegistration;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.kryoshim.SerializerShim;
 import org.apache.tinkerpop.gremlin.structure.util.star.StarGraph;
-import org.apache.tinkerpop.gremlin.structure.util.star.StarGraphSerializer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.collection.mutable.WrappedArray;
 
-import java.util.*;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
 
 /**
  * A spark.kryo.registrator implementation that installs TinkerPop types.
@@ -205,8 +193,6 @@ public class GryoRegistrator implements KryoRegistrator {
         m.put(VertexWritable.class, new UnshadedSerializerAdapter<>(new VertexWritableSerializer()));
         m.put(ObjectWritable.class, new UnshadedSerializerAdapter<>(new ObjectWritableSerializer<>()));
         //
-        m.put(HadoopConfiguration.class, null);
-        //
         m.put(HadoopVertex.class, new UnshadedSerializerAdapter<>(new GryoSerializers.VertexSerializer()));
         m.put(HadoopVertexProperty.class, new UnshadedSerializerAdapter<>(new GryoSerializers.VertexPropertySerializer()));
         m.put(HadoopProperty.class, new UnshadedSerializerAdapter<>(new GryoSerializers.PropertySerializer()));