Added gryo "lite" serializer.

Serializes graph elements to "reference" rather than "detached". Altered all 
the configuration files that required the change. This serializer is just an 
option - the default configuration from the client side is to use the existing 
serializer.


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

Branch: refs/heads/TINKERPOP-1308
Commit: f811a8aecbfa82b1747503a88b13af71b23e3ae9
Parents: 2e03050
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Fri May 20 12:43:12 2016 -0400
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu May 26 13:06:07 2016 -0400

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |   3 +-
 .../structure/io/gryo/GryoSerializers.java      |  12 -
 .../ser/AbstractGryoMessageSerializerV1d0.java  | 311 ++++++++++
 .../ser/GryoLiteMessageSerializerV1d0.java      | 156 +++++
 .../driver/ser/GryoMessageSerializerV1d0.java   | 290 +--------
 .../tinkerpop/gremlin/driver/ser/SerTokens.java |   1 +
 .../gremlin/driver/ser/Serializers.java         |   5 +-
 .../ser/GryoLiteMessageSerializerV1d0Test.java  | 589 +++++++++++++++++++
 .../ser/GryoMessageSerializerV1d0Test.java      |   2 +-
 gremlin-server/conf/gremlin-server-classic.yaml |   1 +
 .../conf/gremlin-server-modern-readonly.yaml    |   1 +
 gremlin-server/conf/gremlin-server-modern.yaml  |   1 +
 gremlin-server/conf/gremlin-server-neo4j.yaml   |   1 +
 .../conf/gremlin-server-rest-secure.yaml        |   2 -
 gremlin-server/conf/gremlin-server-secure.yaml  |   1 +
 gremlin-server/conf/gremlin-server-spark.yaml   |   1 +
 gremlin-server/conf/gremlin-server.yaml         |   1 +
 .../server/GremlinResultSetIntegrateTest.java   |  16 +
 .../remote/gremlin-server-integration.yaml      |   1 +
 .../server/gremlin-server-integration.yaml      |   1 +
 20 files changed, 1100 insertions(+), 296 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index d2a8e3e..6efcdc6 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -26,7 +26,6 @@ 
image::https://raw.githubusercontent.com/apache/incubator-tinkerpop/master/docs/
 TinkerPop 3.2.1 (NOT OFFICIALLY RELEASED YET)
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
-* `GryoMapper` allows overrides of existing serializers on calls to 
`addCustom` on the builder.
 * Fixed a `NullPointerException` bug around nested `group()`-steps in OLAP.
 * Fixed a severe bug around halted traversers in a multi-job OLAP traversal 
chain.
 * Ensure a separation of `GraphComputer` and `VertexProgram` configurations in 
`SparkGraphComputer` and `GiraphGraphComputer`.
@@ -34,6 +33,8 @@ 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`.
+* 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.
 * Fixed a bug in master-traversal traverser propagation.
 * Added useful methods for custom `VertexPrograms` to be used with 
`program()`-step.

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/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 a38de03..ae99ac6 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
@@ -64,9 +64,6 @@ final class GryoSerializers {
      * Serializes any {@link Vertex} implementation encountered to an {@link 
DetachedVertex}.
      */
     final static class VertexSerializer extends Serializer<Vertex> {
-        public VertexSerializer() {
-        }
-
         @Override
         public void write(final Kryo kryo, final Output output, final Vertex 
vertex) {
             kryo.writeClassAndObject(output, DetachedFactory.detach(vertex, 
true));
@@ -82,9 +79,6 @@ final class GryoSerializers {
      * Serializes any {@link Property} implementation encountered to an {@link 
DetachedProperty}.
      */
     final static class PropertySerializer extends Serializer<Property> {
-        public PropertySerializer() {
-        }
-
         @Override
         public void write(final Kryo kryo, final Output output, final Property 
property) {
             kryo.writeClassAndObject(output, property instanceof 
VertexProperty ? DetachedFactory.detach((VertexProperty) property, true) : 
DetachedFactory.detach(property));
@@ -100,9 +94,6 @@ final class GryoSerializers {
      * Serializes any {@link VertexProperty} implementation encountered to an 
{@link DetachedVertexProperty}.
      */
     final static class VertexPropertySerializer extends 
Serializer<VertexProperty> {
-        public VertexPropertySerializer() {
-        }
-
         @Override
         public void write(final Kryo kryo, final Output output, final 
VertexProperty vertexProperty) {
             kryo.writeClassAndObject(output, 
DetachedFactory.detach(vertexProperty, true));
@@ -118,9 +109,6 @@ final class GryoSerializers {
      * Serializes any {@link Path} implementation encountered to an {@link 
DetachedPath}.
      */
     final static class PathSerializer extends Serializer<Path> {
-        public PathSerializer() {
-        }
-
         @Override
         public void write(final Kryo kryo, final Output output, final Path 
path) {
             kryo.writeClassAndObject(output, DetachedFactory.detach(path, 
false));

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/AbstractGryoMessageSerializerV1d0.java
----------------------------------------------------------------------
diff --git 
a/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/AbstractGryoMessageSerializerV1d0.java
 
b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/AbstractGryoMessageSerializerV1d0.java
new file mode 100644
index 0000000..4190d96
--- /dev/null
+++ 
b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/AbstractGryoMessageSerializerV1d0.java
@@ -0,0 +1,311 @@
+/*
+ * 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.driver.ser;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.util.ReferenceCountUtil;
+import org.apache.tinkerpop.gremlin.driver.message.RequestMessage;
+import org.apache.tinkerpop.gremlin.driver.message.ResponseMessage;
+import org.apache.tinkerpop.gremlin.driver.message.ResponseStatusCode;
+import org.apache.tinkerpop.gremlin.structure.Graph;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoIo;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoMapper;
+import org.apache.tinkerpop.shaded.kryo.ClassResolver;
+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.io.ByteArrayOutputStream;
+import java.lang.reflect.Method;
+import java.nio.charset.Charset;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+/**
+ * @author Stephen Mallette (http://stephen.genoprime.com)
+ */
+public abstract class AbstractGryoMessageSerializerV1d0 extends 
AbstractMessageSerializer {
+    private GryoMapper gryoMapper;
+    private ThreadLocal<Kryo> kryoThreadLocal = new ThreadLocal<Kryo>() {
+        @Override
+        protected Kryo initialValue() {
+            return gryoMapper.createMapper();
+        }
+    };
+
+    private static final Charset UTF8 = Charset.forName("UTF-8");
+
+    public static final String TOKEN_CUSTOM = "custom";
+    public static final String TOKEN_SERIALIZE_RESULT_TO_STRING = 
"serializeResultToString";
+    public static final String TOKEN_USE_MAPPER_FROM_GRAPH = 
"useMapperFromGraph";
+    public static final String TOKEN_BUFFER_SIZE = "bufferSize";
+    public static final String TOKEN_CLASS_RESOLVER_SUPPLIER = 
"classResolverSupplier";
+
+    protected boolean serializeToString = false;
+    private int bufferSize = 4096;
+
+    /**
+     * Creates an instance with a provided mapper configured {@link 
GryoMapper} instance. Note that this instance
+     * will be overridden by {@link #configure} is called.
+     */
+    public AbstractGryoMessageSerializerV1d0(final GryoMapper kryo) {
+        this.gryoMapper = kryo;
+    }
+
+    /**
+     * Called from the {@link #configure(Map, Map)} method right before the 
call to create the builder. Sub-classes
+     * can choose to alter the builder or completely replace it.
+     */
+    public GryoMapper.Builder beforeCreateMapper(final GryoMapper.Builder 
builder, final Map<String, Object> config,
+                                                 final Map<String, Graph> 
graphs) {
+        return builder;
+    }
+
+    @Override
+    public final void configure(final Map<String, Object> config, final 
Map<String, Graph> graphs) {
+        final GryoMapper.Builder builder;
+        final Object graphToUseForMapper = 
config.get(TOKEN_USE_MAPPER_FROM_GRAPH);
+        if (graphToUseForMapper != null) {
+            if (null == graphs) throw new IllegalStateException(String.format(
+                    "No graphs have been provided to the serializer and 
therefore %s is not a valid configuration", TOKEN_USE_MAPPER_FROM_GRAPH));
+
+            final Graph g = graphs.get(graphToUseForMapper.toString());
+            if (null == g) throw new IllegalStateException(String.format(
+                    "There is no graph named [%s] configured to be used in the 
%s setting",
+                    graphToUseForMapper, TOKEN_USE_MAPPER_FROM_GRAPH));
+
+            // a graph was found so use the mapper it constructs.  this allows 
gryo to be auto-configured with any
+            // custom classes that the implementation allows for
+            builder = g.io(GryoIo.build()).mapper();
+        } else {
+            // no graph was supplied so just use the default - this will 
likely be the case when using a graph
+            // with no custom classes or a situation where the user needs 
complete control like when using two
+            // distinct implementations each with their own custom classes.
+            builder = GryoMapper.build();
+        }
+
+        addIoRegistries(config, builder);
+        addClassResolverSupplier(config, builder);
+        addCustomClasses(config, builder);
+
+        this.serializeToString = 
Boolean.parseBoolean(config.getOrDefault(TOKEN_SERIALIZE_RESULT_TO_STRING, 
"false").toString());
+        this.bufferSize = 
Integer.parseInt(config.getOrDefault(TOKEN_BUFFER_SIZE, "4096").toString());
+
+        this.gryoMapper = beforeCreateMapper(builder, config, graphs).create();
+    }
+
+    private void addClassResolverSupplier(final Map<String, Object> config, 
final GryoMapper.Builder builder) {
+        final String className = (String) 
config.getOrDefault(TOKEN_CLASS_RESOLVER_SUPPLIER, null);
+        if (className != null && !className.isEmpty()) {
+            try {
+                final Class<?> clazz = Class.forName(className);
+                try {
+                    final Method instanceMethod = 
clazz.getDeclaredMethod("getInstance");
+                    builder.classResolver((Supplier<ClassResolver>) 
instanceMethod.invoke(null));
+                } catch (Exception methodex) {
+                    // tried getInstance() and that failed so try 
newInstance() no-arg constructor
+                    builder.classResolver((Supplier<ClassResolver>) 
clazz.newInstance());
+                }
+            } catch (Exception ex) {
+                throw new IllegalStateException(ex);
+            }
+        }
+    }
+
+    private void addCustomClasses(final Map<String, Object> config, final 
GryoMapper.Builder builder) {
+        final List<String> classNameList = 
getListStringFromConfig(TOKEN_CUSTOM, config);
+
+        classNameList.stream().forEach(serializerDefinition -> {
+            String className;
+            Optional<String> serializerName;
+            if (serializerDefinition.contains(";")) {
+                final String[] split = serializerDefinition.split(";");
+                if (split.length != 2)
+                    throw new IllegalStateException(String.format("Invalid 
format for serializer definition [%s] - expected <class>;<serializer-class>", 
serializerDefinition));
+
+                className = split[0];
+                serializerName = Optional.of(split[1]);
+            } else {
+                serializerName = Optional.empty();
+                className = serializerDefinition;
+            }
+
+            try {
+                final Class clazz = Class.forName(className);
+                final Serializer serializer;
+                if (serializerName.isPresent()) {
+                    final Class serializerClazz = 
Class.forName(serializerName.get());
+                    serializer = (Serializer) serializerClazz.newInstance();
+                    builder.addCustom(clazz, kryo -> serializer);
+                } else
+                    builder.addCustom(clazz);
+            } catch (Exception ex) {
+                throw new IllegalStateException("Class could not be found", 
ex);
+            }
+        });
+    }
+
+    @Override
+    public ResponseMessage deserializeResponse(final ByteBuf msg) throws 
SerializationException {
+        try {
+            final Kryo kryo = kryoThreadLocal.get();
+            final byte[] payload = new byte[msg.capacity()];
+            msg.readBytes(payload);
+            try (final Input input = new Input(payload)) {
+                final UUID requestId = kryo.readObjectOrNull(input, 
UUID.class);
+                final int status = input.readShort();
+                final String statusMsg = input.readString();
+                final Map<String,Object> statusAttributes = 
(Map<String,Object>) kryo.readClassAndObject(input);
+                final Object result = kryo.readClassAndObject(input);
+                final Map<String,Object> metaAttributes = (Map<String,Object>) 
kryo.readClassAndObject(input);
+
+                return ResponseMessage.build(requestId)
+                        .code(ResponseStatusCode.getFromValue(status))
+                        .statusMessage(statusMsg)
+                        .statusAttributes(statusAttributes)
+                        .result(result)
+                        .responseMetaData(metaAttributes)
+                        .create();
+            }
+        } catch (Exception ex) {
+            logger.warn("Response [{}] could not be deserialized by {}.", msg, 
GryoMessageSerializerV1d0.class.getName());
+            throw new SerializationException(ex);
+        }
+    }
+
+    @Override
+    public ByteBuf serializeResponseAsBinary(final ResponseMessage 
responseMessage, final ByteBufAllocator allocator) throws 
SerializationException {
+        ByteBuf encodedMessage = null;
+        try {
+            final Kryo kryo = kryoThreadLocal.get();
+            try (final ByteArrayOutputStream baos = new 
ByteArrayOutputStream()) {
+                final Output output = new Output(baos, bufferSize);
+
+                // request id - if present
+                kryo.writeObjectOrNull(output, responseMessage.getRequestId() 
!= null ? responseMessage.getRequestId() : null, UUID.class);
+
+                // status
+                
output.writeShort(responseMessage.getStatus().getCode().getValue());
+                output.writeString(responseMessage.getStatus().getMessage());
+                kryo.writeClassAndObject(output, 
responseMessage.getStatus().getAttributes());
+
+                // result
+                kryo.writeClassAndObject(output, serializeToString ? 
serializeResultToString(responseMessage) : 
responseMessage.getResult().getData());
+                kryo.writeClassAndObject(output, 
responseMessage.getResult().getMeta());
+
+                final long size = output.total();
+                if (size > Integer.MAX_VALUE)
+                    throw new SerializationException(String.format("Message 
size of %s exceeds allocatable space", size));
+
+                output.flush();
+                encodedMessage = allocator.buffer((int) size);
+                encodedMessage.writeBytes(baos.toByteArray());
+            }
+
+            return encodedMessage;
+        } catch (Exception ex) {
+            if (encodedMessage != null) 
ReferenceCountUtil.release(encodedMessage);
+
+            logger.warn("Response [{}] could not be serialized by {}.", 
responseMessage.toString(), GryoMessageSerializerV1d0.class.getName());
+            throw new SerializationException(ex);
+        }
+    }
+
+    @Override
+    public RequestMessage deserializeRequest(final ByteBuf msg) throws 
SerializationException {
+        try {
+            final Kryo kryo = kryoThreadLocal.get();
+            final byte[] payload = new byte[msg.readableBytes()];
+            msg.readBytes(payload);
+            try (final Input input = new Input(payload)) {
+                // by the time the message gets here, the mime length/type 
have been already read, so this part just
+                // needs to process the payload.
+                final UUID id = kryo.readObject(input, UUID.class);
+                final String processor = input.readString();
+                final String op = input.readString();
+
+                final RequestMessage.Builder builder = RequestMessage.build(op)
+                        .overrideRequestId(id)
+                        .processor(processor);
+
+                final Map<String, Object> args = kryo.readObject(input, 
HashMap.class);
+                args.forEach(builder::addArg);
+                return builder.create();
+            }
+        } catch (Exception ex) {
+            logger.warn("Request [{}] could not be deserialized by {}.", msg, 
GryoMessageSerializerV1d0.class.getName());
+            throw new SerializationException(ex);
+        }
+    }
+
+    @Override
+    public ByteBuf serializeRequestAsBinary(final RequestMessage 
requestMessage, final ByteBufAllocator allocator) throws SerializationException 
{
+        ByteBuf encodedMessage = null;
+        try {
+            final Kryo kryo = kryoThreadLocal.get();
+            try (final ByteArrayOutputStream baos = new 
ByteArrayOutputStream()) {
+                final Output output = new Output(baos, bufferSize);
+                final String mimeType = mimeTypesSupported()[0];
+                output.writeByte(mimeType.length());
+                output.write(mimeType.getBytes(UTF8));
+
+                kryo.writeObject(output, requestMessage.getRequestId());
+                output.writeString(requestMessage.getProcessor());
+                output.writeString(requestMessage.getOp());
+                kryo.writeObject(output, requestMessage.getArgs());
+
+                final long size = output.total();
+                if (size > Integer.MAX_VALUE)
+                    throw new SerializationException(String.format("Message 
size of %s exceeds allocatable space", size));
+
+                output.flush();
+                encodedMessage = allocator.buffer((int) size);
+                encodedMessage.writeBytes(baos.toByteArray());
+            }
+
+            return encodedMessage;
+        } catch (Exception ex) {
+            if (encodedMessage != null) 
ReferenceCountUtil.release(encodedMessage);
+
+            logger.warn("Request [{}] could not be serialized by {}.", 
requestMessage.toString(), GryoMessageSerializerV1d0.class.getName());
+            throw new SerializationException(ex);
+        }
+    }
+
+    private Object serializeResultToString(final ResponseMessage msg) {
+        if (msg.getResult() == null) return "null";
+        if (msg.getResult().getData() == null) return "null";
+
+        // the IteratorHandler should return a collection so keep it as such
+        final Object o = msg.getResult().getData();
+        if (o instanceof Collection) {
+            return ((Collection) o).stream().map(d -> null == d ? "null" : 
d.toString()).collect(Collectors.toList());
+        } else {
+            return o.toString();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GryoLiteMessageSerializerV1d0.java
----------------------------------------------------------------------
diff --git 
a/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GryoLiteMessageSerializerV1d0.java
 
b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GryoLiteMessageSerializerV1d0.java
new file mode 100644
index 0000000..a4ff38b
--- /dev/null
+++ 
b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GryoLiteMessageSerializerV1d0.java
@@ -0,0 +1,156 @@
+/*
+ * 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.driver.ser;
+
+import org.apache.tinkerpop.gremlin.process.traversal.Path;
+import org.apache.tinkerpop.gremlin.structure.Edge;
+import org.apache.tinkerpop.gremlin.structure.Graph;
+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.util.reference.ReferenceEdge;
+import org.apache.tinkerpop.gremlin.structure.util.reference.ReferenceFactory;
+import org.apache.tinkerpop.gremlin.structure.util.reference.ReferencePath;
+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.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.Map;
+
+/**
+ * An alternative Gryo serializer that uses "referenced" graph elements during 
serialization. Referenced elements such
+ * as {@link ReferenceVertex} exclude the label and the properties associated 
with it and only return the identifier.
+ *
+ * @author Stephen Mallette (http://stephen.genoprime.com)
+ */
+public class GryoLiteMessageSerializerV1d0 extends 
AbstractGryoMessageSerializerV1d0 {
+
+    private static final String MIME_TYPE = SerTokens.MIME_GRYO_LITE_V1D0;
+    private static final String MIME_TYPE_STRINGD = 
SerTokens.MIME_GRYO_LITE_V1D0 + "-stringd";
+
+    /**
+     * Creates an instance with a standard {@link GryoMapper} instance. Note 
that this instance
+     * will be overridden by {@link #configure} is called.
+     */
+    public GryoLiteMessageSerializerV1d0() {
+        super(overrideWithLite(GryoMapper.build()).create());
+    }
+
+    @Override
+    public String[] mimeTypesSupported() {
+        return new String[]{serializeToString ? MIME_TYPE_STRINGD : MIME_TYPE};
+    }
+
+    @Override
+    public GryoMapper.Builder beforeCreateMapper(final GryoMapper.Builder 
builder, final Map<String, Object> config,
+                                                 final Map<String, Graph> 
graphs) {
+        return overrideWithLite(builder);
+    }
+
+    private static GryoMapper.Builder overrideWithLite(final 
GryoMapper.Builder builder) {
+        // override the core graph Elements so as to serialize with 
"reference" as opposed to "detached"
+        builder.addCustom(Edge.class, new EdgeLiteSerializer());
+        builder.addCustom(Vertex.class, new VertexLiteSerializer());
+        builder.addCustom(VertexProperty.class, new 
VertexPropertyLiteSerializer());
+        builder.addCustom(Property.class, new PropertyLiteSerializer());
+        builder.addCustom(Path.class, new PathLiteSerializer());
+        return builder;
+    }
+
+    /**
+     * Serializes any {@link Edge} implementation encountered to a {@link 
ReferenceEdge}.
+     */
+    final static class EdgeLiteSerializer extends Serializer<Edge> {
+        @Override
+        public void write(final Kryo kryo, final Output output, final Edge 
edge) {
+            kryo.writeClassAndObject(output, ReferenceFactory.detach(edge));
+        }
+
+        @Override
+        public Edge read(final Kryo kryo, final Input input, final Class<Edge> 
edgeClass) {
+            final Object o = kryo.readClassAndObject(input);
+            return (Edge) o;
+        }
+    }
+
+    /**
+     * Serializes any {@link Vertex} implementation encountered to an {@link 
ReferenceVertex}.
+     */
+    final static class VertexLiteSerializer extends Serializer<Vertex> {
+        @Override
+        public void write(final Kryo kryo, final Output output, final Vertex 
vertex) {
+            kryo.writeClassAndObject(output, ReferenceFactory.detach(vertex));
+        }
+
+        @Override
+        public Vertex read(final Kryo kryo, final Input input, final 
Class<Vertex> vertexClass) {
+            return (Vertex) kryo.readClassAndObject(input);
+        }
+    }
+
+    /**
+     * Serializes any {@link Property} implementation encountered to an {@link 
ReferenceProperty}.
+     */
+    final static class PropertyLiteSerializer extends Serializer<Property> {
+        @Override
+        public void write(final Kryo kryo, final Output output, final Property 
property) {
+            kryo.writeClassAndObject(output, property instanceof 
VertexProperty ? ReferenceFactory.detach((VertexProperty) property) : 
ReferenceFactory.detach(property));
+        }
+
+        @Override
+        public Property read(final Kryo kryo, final Input input, final 
Class<Property> propertyClass) {
+            return (Property) kryo.readClassAndObject(input);
+        }
+    }
+
+    /**
+     * Serializes any {@link VertexProperty} implementation encountered to an 
{@link ReferenceVertexProperty}.
+     */
+    final static class VertexPropertyLiteSerializer extends 
Serializer<VertexProperty> {
+        @Override
+        public void write(final Kryo kryo, final Output output, final 
VertexProperty vertexProperty) {
+            kryo.writeClassAndObject(output, 
ReferenceFactory.detach(vertexProperty));
+        }
+
+        @Override
+        public VertexProperty read(final Kryo kryo, final Input input, final 
Class<VertexProperty> vertexPropertyClass) {
+            return (VertexProperty) kryo.readClassAndObject(input);
+        }
+    }
+
+    /**
+     * Serializes any {@link Path} implementation encountered to an {@link 
ReferencePath}.
+     */
+    final static class PathLiteSerializer extends Serializer<Path> {
+        @Override
+        public void write(final Kryo kryo, final Output output, final Path 
path) {
+            kryo.writeClassAndObject(output, ReferenceFactory.detach(path));
+        }
+
+        @Override
+        public Path read(final Kryo kryo, final Input input, final Class<Path> 
pathClass) {
+            return (Path) kryo.readClassAndObject(input);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GryoMessageSerializerV1d0.java
----------------------------------------------------------------------
diff --git 
a/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GryoMessageSerializerV1d0.java
 
b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GryoMessageSerializerV1d0.java
index ce46dfc..e9b51d1 100644
--- 
a/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GryoMessageSerializerV1d0.java
+++ 
b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GryoMessageSerializerV1d0.java
@@ -18,301 +18,33 @@
  */
 package org.apache.tinkerpop.gremlin.driver.ser;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.util.ReferenceCountUtil;
-import org.apache.tinkerpop.gremlin.driver.message.RequestMessage;
-import org.apache.tinkerpop.gremlin.driver.message.ResponseMessage;
-import org.apache.tinkerpop.gremlin.driver.message.ResponseStatusCode;
-import org.apache.tinkerpop.gremlin.structure.Graph;
-import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoIo;
 import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoMapper;
-import org.apache.tinkerpop.shaded.kryo.ClassResolver;
-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.io.ByteArrayOutputStream;
-import java.lang.reflect.Method;
-import java.nio.charset.Charset;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.UUID;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
+import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedVertex;
 
 /**
+ * The standard Gryo serializer that uses "detached" graph elements during 
serialization. Detached elements such as
+ * {@link DetachedVertex} include the label and the properties associated with 
it which could be more costly for
+ * network serialization purposes.
+ *
+ * @see GryoLiteMessageSerializerV1d0
+ *
  * @author Stephen Mallette (http://stephen.genoprime.com)
  */
-public final class GryoMessageSerializerV1d0 extends AbstractMessageSerializer 
{
-    private GryoMapper gryoMapper;
-    private ThreadLocal<Kryo> kryoThreadLocal = new ThreadLocal<Kryo>() {
-        @Override
-        protected Kryo initialValue() {
-            return gryoMapper.createMapper();
-        }
-    };
-
-    private static final Charset UTF8 = Charset.forName("UTF-8");
+public final class GryoMessageSerializerV1d0 extends 
AbstractGryoMessageSerializerV1d0 {
 
     private static final String MIME_TYPE = SerTokens.MIME_GRYO_V1D0;
     private static final String MIME_TYPE_STRINGD = SerTokens.MIME_GRYO_V1D0 + 
"-stringd";
 
-    public static final String TOKEN_CUSTOM = "custom";
-    public static final String TOKEN_SERIALIZE_RESULT_TO_STRING = 
"serializeResultToString";
-    public static final String TOKEN_USE_MAPPER_FROM_GRAPH = 
"useMapperFromGraph";
-    public static final String TOKEN_BUFFER_SIZE = "bufferSize";
-    public static final String TOKEN_CLASS_RESOLVER_SUPPLIER = 
"classResolverSupplier";
-
-    private boolean serializeToString = false;
-    private int bufferSize = 4096;
-
     /**
      * Creates an instance with a standard {@link GryoMapper} instance. Note 
that this instance
-     * will be overriden by {@link #configure} is called.
-     */
-    public GryoMessageSerializerV1d0() {
-        gryoMapper = GryoMapper.build().create();
-    }
-
-    /**
-     * Creates an instance with a provided mapper configured {@link 
GryoMapper} instance. Note that this instance
      * will be overridden by {@link #configure} is called.
      */
-    public GryoMessageSerializerV1d0(final GryoMapper kryo) {
-        this.gryoMapper = kryo;
-    }
-
-    @Override
-    public void configure(final Map<String, Object> config, final Map<String, 
Graph> graphs) {
-        final GryoMapper.Builder builder;
-        final Object graphToUseForMapper = 
config.get(TOKEN_USE_MAPPER_FROM_GRAPH);
-        if (graphToUseForMapper != null) {
-            if (null == graphs) throw new IllegalStateException(String.format(
-                    "No graphs have been provided to the serializer and 
therefore %s is not a valid configuration", TOKEN_USE_MAPPER_FROM_GRAPH));
-
-            final Graph g = graphs.get(graphToUseForMapper.toString());
-            if (null == g) throw new IllegalStateException(String.format(
-                    "There is no graph named [%s] configured to be used in the 
%s setting",
-                    graphToUseForMapper, TOKEN_USE_MAPPER_FROM_GRAPH));
-
-            // a graph was found so use the mapper it constructs.  this allows 
gryo to be auto-configured with any
-            // custom classes that the implementation allows for
-            builder = g.io(GryoIo.build()).mapper();
-        } else {
-            // no graph was supplied so just use the default - this will 
likely be the case when using a graph
-            // with no custom classes or a situation where the user needs 
complete control like when using two
-            // distinct implementations each with their own custom classes.
-            builder = GryoMapper.build();
-        }
-
-        addIoRegistries(config, builder);
-        addClassResolverSupplier(config, builder);
-        addCustomClasses(config, builder);
-
-        this.serializeToString = 
Boolean.parseBoolean(config.getOrDefault(TOKEN_SERIALIZE_RESULT_TO_STRING, 
"false").toString());
-        this.bufferSize = 
Integer.parseInt(config.getOrDefault(TOKEN_BUFFER_SIZE, "4096").toString());
-
-        this.gryoMapper = builder.create();
-    }
-
-    private void addClassResolverSupplier(final Map<String, Object> config, 
final GryoMapper.Builder builder) {
-        final String className = (String) 
config.getOrDefault(TOKEN_CLASS_RESOLVER_SUPPLIER, null);
-        if (className != null && !className.isEmpty()) {
-            try {
-                final Class<?> clazz = Class.forName(className);
-                try {
-                    final Method instanceMethod = 
clazz.getDeclaredMethod("getInstance");
-                    builder.classResolver((Supplier<ClassResolver>) 
instanceMethod.invoke(null));
-                } catch (Exception methodex) {
-                    // tried getInstance() and that failed so try 
newInstance() no-arg constructor
-                    builder.classResolver((Supplier<ClassResolver>) 
clazz.newInstance());
-                }
-            } catch (Exception ex) {
-                throw new IllegalStateException(ex);
-            }
-        }
-    }
-
-    private void addCustomClasses(final Map<String, Object> config, final 
GryoMapper.Builder builder) {
-        final List<String> classNameList = 
getListStringFromConfig(TOKEN_CUSTOM, config);
-
-        classNameList.stream().forEach(serializerDefinition -> {
-            String className;
-            Optional<String> serializerName;
-            if (serializerDefinition.contains(";")) {
-                final String[] split = serializerDefinition.split(";");
-                if (split.length != 2)
-                    throw new IllegalStateException(String.format("Invalid 
format for serializer definition [%s] - expected <class>;<serializer-class>", 
serializerDefinition));
-
-                className = split[0];
-                serializerName = Optional.of(split[1]);
-            } else {
-                serializerName = Optional.empty();
-                className = serializerDefinition;
-            }
-
-            try {
-                final Class clazz = Class.forName(className);
-                final Serializer serializer;
-                if (serializerName.isPresent()) {
-                    final Class serializerClazz = 
Class.forName(serializerName.get());
-                    serializer = (Serializer) serializerClazz.newInstance();
-                    builder.addCustom(clazz, kryo -> serializer);
-                } else
-                    builder.addCustom(clazz);
-            } catch (Exception ex) {
-                throw new IllegalStateException("Class could not be found", 
ex);
-            }
-        });
+    public GryoMessageSerializerV1d0() {
+        super(GryoMapper.build().create());
     }
 
     @Override
     public String[] mimeTypesSupported() {
-        return new String[]{this.serializeToString ? MIME_TYPE_STRINGD : 
MIME_TYPE};
-    }
-
-    @Override
-    public ResponseMessage deserializeResponse(final ByteBuf msg) throws 
SerializationException {
-        try {
-            final Kryo kryo = kryoThreadLocal.get();
-            final byte[] payload = new byte[msg.capacity()];
-            msg.readBytes(payload);
-            try (final Input input = new Input(payload)) {
-                final UUID requestId = kryo.readObjectOrNull(input, 
UUID.class);
-                final int status = input.readShort();
-                final String statusMsg = input.readString();
-                final Map<String,Object> statusAttributes = 
(Map<String,Object>) kryo.readClassAndObject(input);
-                final Object result = kryo.readClassAndObject(input);
-                final Map<String,Object> metaAttributes = (Map<String,Object>) 
kryo.readClassAndObject(input);
-
-                return ResponseMessage.build(requestId)
-                        .code(ResponseStatusCode.getFromValue(status))
-                        .statusMessage(statusMsg)
-                        .statusAttributes(statusAttributes)
-                        .result(result)
-                        .responseMetaData(metaAttributes)
-                        .create();
-            }
-        } catch (Exception ex) {
-            logger.warn("Response [{}] could not be deserialized by {}.", msg, 
GryoMessageSerializerV1d0.class.getName());
-            throw new SerializationException(ex);
-        }
-    }
-
-    @Override
-    public ByteBuf serializeResponseAsBinary(final ResponseMessage 
responseMessage, final ByteBufAllocator allocator) throws 
SerializationException {
-        ByteBuf encodedMessage = null;
-        try {
-            final Kryo kryo = kryoThreadLocal.get();
-            try (final ByteArrayOutputStream baos = new 
ByteArrayOutputStream()) {
-                final Output output = new Output(baos, bufferSize);
-
-                // request id - if present
-                kryo.writeObjectOrNull(output, responseMessage.getRequestId() 
!= null ? responseMessage.getRequestId() : null, UUID.class);
-
-                // status
-                
output.writeShort(responseMessage.getStatus().getCode().getValue());
-                output.writeString(responseMessage.getStatus().getMessage());
-                kryo.writeClassAndObject(output, 
responseMessage.getStatus().getAttributes());
-
-                // result
-                kryo.writeClassAndObject(output, serializeToString ? 
serializeResultToString(responseMessage) : 
responseMessage.getResult().getData());
-                kryo.writeClassAndObject(output, 
responseMessage.getResult().getMeta());
-
-                final long size = output.total();
-                if (size > Integer.MAX_VALUE)
-                    throw new SerializationException(String.format("Message 
size of %s exceeds allocatable space", size));
-
-                output.flush();
-                encodedMessage = allocator.buffer((int) size);
-                encodedMessage.writeBytes(baos.toByteArray());
-            }
-
-            return encodedMessage;
-        } catch (Exception ex) {
-            if (encodedMessage != null) 
ReferenceCountUtil.release(encodedMessage);
-
-            logger.warn("Response [{}] could not be serialized by {}.", 
responseMessage.toString(), GryoMessageSerializerV1d0.class.getName());
-            throw new SerializationException(ex);
-        }
-    }
-
-    @Override
-    public RequestMessage deserializeRequest(final ByteBuf msg) throws 
SerializationException {
-        try {
-            final Kryo kryo = kryoThreadLocal.get();
-            final byte[] payload = new byte[msg.readableBytes()];
-            msg.readBytes(payload);
-            try (final Input input = new Input(payload)) {
-                // by the time the message gets here, the mime length/type 
have been already read, so this part just
-                // needs to process the payload.
-                final UUID id = kryo.readObject(input, UUID.class);
-                final String processor = input.readString();
-                final String op = input.readString();
-
-                final RequestMessage.Builder builder = RequestMessage.build(op)
-                        .overrideRequestId(id)
-                        .processor(processor);
-
-                final Map<String, Object> args = kryo.readObject(input, 
HashMap.class);
-                args.forEach(builder::addArg);
-                return builder.create();
-            }
-        } catch (Exception ex) {
-            logger.warn("Request [{}] could not be deserialized by {}.", msg, 
GryoMessageSerializerV1d0.class.getName());
-            throw new SerializationException(ex);
-        }
-    }
-
-    @Override
-    public ByteBuf serializeRequestAsBinary(final RequestMessage 
requestMessage, final ByteBufAllocator allocator) throws SerializationException 
{
-        ByteBuf encodedMessage = null;
-        try {
-            final Kryo kryo = kryoThreadLocal.get();
-            try (final ByteArrayOutputStream baos = new 
ByteArrayOutputStream()) {
-                final Output output = new Output(baos, bufferSize);
-                final String mimeType = serializeToString ? MIME_TYPE_STRINGD 
: MIME_TYPE;
-                output.writeByte(mimeType.length());
-                output.write(mimeType.getBytes(UTF8));
-
-                kryo.writeObject(output, requestMessage.getRequestId());
-                output.writeString(requestMessage.getProcessor());
-                output.writeString(requestMessage.getOp());
-                kryo.writeObject(output, requestMessage.getArgs());
-
-                final long size = output.total();
-                if (size > Integer.MAX_VALUE)
-                    throw new SerializationException(String.format("Message 
size of %s exceeds allocatable space", size));
-
-                output.flush();
-                encodedMessage = allocator.buffer((int) size);
-                encodedMessage.writeBytes(baos.toByteArray());
-            }
-
-            return encodedMessage;
-        } catch (Exception ex) {
-            if (encodedMessage != null) 
ReferenceCountUtil.release(encodedMessage);
-
-            logger.warn("Request [{}] could not be serialized by {}.", 
requestMessage.toString(), GryoMessageSerializerV1d0.class.getName());
-            throw new SerializationException(ex);
-        }
-    }
-
-    private Object serializeResultToString(final ResponseMessage msg) {
-        if (msg.getResult() == null) return "null";
-        if (msg.getResult().getData() == null) return "null";
-
-        // the IteratorHandler should return a collection so keep it as such
-        final Object o = msg.getResult().getData();
-        if (o instanceof Collection) {
-            return ((Collection) o).stream().map(d -> null == d ? "null" : 
d.toString()).collect(Collectors.toList());
-        } else {
-            return o.toString();
-        }
+        return new String[]{serializeToString ? MIME_TYPE_STRINGD : MIME_TYPE};
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/SerTokens.java
----------------------------------------------------------------------
diff --git 
a/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/SerTokens.java
 
b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/SerTokens.java
index 779355a..c857116 100644
--- 
a/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/SerTokens.java
+++ 
b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/SerTokens.java
@@ -39,4 +39,5 @@ public final  class SerTokens {
     public static final String MIME_JSON = "application/json";
     public static final String MIME_GRAPHSON_V1D0 = 
"application/vnd.gremlin-v1.0+json";
     public static final String MIME_GRYO_V1D0 = 
"application/vnd.gremlin-v1.0+gryo";
+    public static final String MIME_GRYO_LITE_V1D0 = 
"application/vnd.gremlin-v1.0+gryo-lite";
 }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/Serializers.java
----------------------------------------------------------------------
diff --git 
a/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/Serializers.java
 
b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/Serializers.java
index 11a180e..45ae301 100644
--- 
a/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/Serializers.java
+++ 
b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/Serializers.java
@@ -28,7 +28,8 @@ import org.apache.tinkerpop.gremlin.driver.MessageSerializer;
 public enum Serializers {
     GRAPHSON(SerTokens.MIME_JSON),
     GRAPHSON_V1D0(SerTokens.MIME_GRAPHSON_V1D0),
-    GRYO_V1D0(SerTokens.MIME_GRYO_V1D0);
+    GRYO_V1D0(SerTokens.MIME_GRYO_V1D0),
+    GRYO_LITE_V1D0(SerTokens.MIME_GRYO_LITE_V1D0);
 
     private String value;
 
@@ -62,6 +63,8 @@ public enum Serializers {
                 return new GraphSONMessageSerializerGremlinV1d0();
             case SerTokens.MIME_GRYO_V1D0:
                 return new GryoMessageSerializerV1d0();
+            case SerTokens.MIME_GRYO_LITE_V1D0:
+                return new GryoLiteMessageSerializerV1d0();
             default:
                 throw new RuntimeException("Could not create a simple 
MessageSerializer instance of " + value);
         }

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GryoLiteMessageSerializerV1d0Test.java
----------------------------------------------------------------------
diff --git 
a/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GryoLiteMessageSerializerV1d0Test.java
 
b/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GryoLiteMessageSerializerV1d0Test.java
new file mode 100644
index 0000000..0880f5f
--- /dev/null
+++ 
b/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GryoLiteMessageSerializerV1d0Test.java
@@ -0,0 +1,589 @@
+/*
+ * 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.driver.ser;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.UnpooledByteBufAllocator;
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.tinkerpop.gremlin.driver.MessageSerializer;
+import org.apache.tinkerpop.gremlin.driver.message.RequestMessage;
+import org.apache.tinkerpop.gremlin.driver.message.ResponseMessage;
+import org.apache.tinkerpop.gremlin.driver.message.ResponseStatusCode;
+import 
org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversalSource;
+import org.apache.tinkerpop.gremlin.process.traversal.step.util.Tree;
+import org.apache.tinkerpop.gremlin.structure.Edge;
+import org.apache.tinkerpop.gremlin.structure.Graph;
+import org.apache.tinkerpop.gremlin.structure.Vertex;
+import org.apache.tinkerpop.gremlin.structure.VertexProperty;
+import org.apache.tinkerpop.gremlin.structure.io.AbstractIoRegistry;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoClassResolver;
+import org.apache.tinkerpop.gremlin.structure.io.gryo.GryoIo;
+import org.apache.tinkerpop.gremlin.structure.util.reference.ReferenceEdge;
+import org.apache.tinkerpop.gremlin.structure.util.reference.ReferenceVertex;
+import org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerFactory;
+import org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerGraph;
+import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
+import org.apache.tinkerpop.shaded.kryo.ClassResolver;
+import org.apache.tinkerpop.shaded.kryo.Kryo;
+import org.apache.tinkerpop.shaded.kryo.KryoException;
+import org.apache.tinkerpop.shaded.kryo.Registration;
+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.junit.Test;
+
+import java.awt.Color;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Serializer tests that cover non-lossy serialization/deserialization methods.
+ *
+ * @author Stephen Mallette (http://stephen.genoprime.com)
+ */
+public class GryoLiteMessageSerializerV1d0Test {
+    private static final Map<String, Object> config = new HashMap<String, 
Object>() {{
+        
put(AbstractGryoMessageSerializerV1d0.TOKEN_SERIALIZE_RESULT_TO_STRING, true);
+    }};
+
+    private UUID requestId = 
UUID.fromString("6457272A-4018-4538-B9AE-08DD5DDC0AA1");
+    private ResponseMessage.Builder responseMessageBuilder = 
ResponseMessage.build(requestId);
+    private static ByteBufAllocator allocator = 
UnpooledByteBufAllocator.DEFAULT;
+
+    public MessageSerializer binarySerializer = new 
GryoLiteMessageSerializerV1d0();
+
+    public MessageSerializer textSerializer = new 
GryoLiteMessageSerializerV1d0();
+
+    public GryoLiteMessageSerializerV1d0Test() {
+        textSerializer.configure(config, null);
+    }
+
+    @Test
+    public void shouldConfigureIoRegistry() throws Exception {
+        final MessageSerializer serializer = new 
GryoLiteMessageSerializerV1d0();
+        final Map<String, Object> config = new HashMap<String, Object>() {{
+            put(AbstractGryoMessageSerializerV1d0.TOKEN_IO_REGISTRIES, 
Arrays.asList(ColorIoRegistry.class.getName()));
+        }};
+
+        serializer.configure(config, null);
+
+        final ResponseMessage toSerialize = 
ResponseMessage.build(requestId).result(Color.RED).create();
+        final ByteBuf bb = serializer.serializeResponseAsBinary(toSerialize, 
allocator);
+        final ResponseMessage deserialized = 
serializer.deserializeResponse(bb);
+
+        assertCommon(deserialized);
+        assertEquals(Color.RED, deserialized.getResult().getData());
+    }
+
+    @Test
+    public void shouldConfigureCustomClassResolver() {
+        final MessageSerializer serializer = new 
GryoLiteMessageSerializerV1d0();
+        final Map<String, Object> config = new HashMap<String, Object>() {{
+            
put(AbstractGryoMessageSerializerV1d0.TOKEN_CLASS_RESOLVER_SUPPLIER, 
ErrorOnlyClassResolverSupplier.class.getName());
+        }};
+
+        serializer.configure(config, null);
+
+        try {
+            
serializer.serializeResponseAsBinary(responseMessageBuilder.create(), 
allocator);
+            fail("Should fail because the ClassResolver used here always 
generates an error");
+        } catch (Exception ex) {
+            assertEquals("java.lang.RuntimeException: Registration is not 
allowed with this ClassResolver - it is not a good implementation", 
ex.getMessage());
+        }
+    }
+
+    @Test
+    public void shouldConfigureCustomClassResolverFromInstance() {
+        final MessageSerializer serializer = new 
GryoLiteMessageSerializerV1d0();
+        final Map<String, Object> config = new HashMap<String, Object>() {{
+            
put(AbstractGryoMessageSerializerV1d0.TOKEN_CLASS_RESOLVER_SUPPLIER, 
ErrorOnlyClassResolverSupplierAsInstance.class.getName());
+        }};
+
+        serializer.configure(config, null);
+
+        try {
+            
serializer.serializeResponseAsBinary(responseMessageBuilder.create(), 
allocator);
+            fail("Should fail because the ClassResolver used here always 
generates an error");
+        } catch (Exception ex) {
+            assertEquals("java.lang.RuntimeException: Registration is not 
allowed with this ClassResolver - it is not a good implementation", 
ex.getMessage());
+        }
+    }
+
+    @Test
+    public void shouldSerializeIterable() throws Exception {
+        final ArrayList<Integer> list = new ArrayList<>();
+        list.add(1);
+        list.add(100);
+
+        final ResponseMessage response = convertBinary(list);
+        assertCommon(response);
+
+        final List<Integer> deserializedFunList = (List<Integer>) 
response.getResult().getData();
+        assertEquals(2, deserializedFunList.size());
+        assertEquals(new Integer(1), deserializedFunList.get(0));
+        assertEquals(new Integer(100), deserializedFunList.get(1));
+    }
+
+    @Test
+    public void shouldSerializeIterableToString() throws Exception {
+        final ArrayList<Integer> list = new ArrayList<>();
+        list.add(1);
+        list.add(100);
+
+        final ResponseMessage response = convertText(list);
+        assertCommon(response);
+
+        final List deserializedFunList = (List) response.getResult().getData();
+        assertEquals(2, deserializedFunList.size());
+        assertEquals("1", deserializedFunList.get(0));
+        assertEquals("100", deserializedFunList.get(1));
+    }
+
+    @Test
+    public void shouldSerializeIterableToStringWithNull() throws Exception {
+        final ArrayList<Integer> list = new ArrayList<>();
+        list.add(1);
+        list.add(null);
+        list.add(100);
+
+        final ResponseMessage response = convertText(list);
+        assertCommon(response);
+
+        final List deserializedFunList = (List) response.getResult().getData();
+        assertEquals(3, deserializedFunList.size());
+        assertEquals("1", deserializedFunList.get(0).toString());
+        assertEquals("null", deserializedFunList.get(1).toString());
+        assertEquals("100", deserializedFunList.get(2).toString());
+    }
+
+    @Test
+    public void shouldSerializeIterableWithNull() throws Exception {
+        final ArrayList<Integer> list = new ArrayList<>();
+        list.add(1);
+        list.add(null);
+        list.add(100);
+
+        final ResponseMessage response = convertBinary(list);
+        assertCommon(response);
+
+        final List<Integer> deserializedFunList = (List<Integer>) 
response.getResult().getData();
+        assertEquals(3, deserializedFunList.size());
+        assertEquals(new Integer(1), deserializedFunList.get(0));
+        assertNull(deserializedFunList.get(1));
+        assertEquals(new Integer(100), deserializedFunList.get(2));
+    }
+
+    @Test
+    public void shouldSerializeMap() throws Exception {
+        final Map<String, Object> map = new HashMap<>();
+        final Map<String, String> innerMap = new HashMap<>();
+        innerMap.put("a", "b");
+
+        map.put("x", 1);
+        map.put("y", "some");
+        map.put("z", innerMap);
+
+        final ResponseMessage response = convertBinary(map);
+        assertCommon(response);
+
+        final Map<String, Object> deserializedMap = (Map<String, Object>) 
response.getResult().getData();
+        assertEquals(3, deserializedMap.size());
+        assertEquals(1, deserializedMap.get("x"));
+        assertEquals("some", deserializedMap.get("y"));
+
+        final Map<String, String> deserializedInnerMap = (Map<String, String>) 
deserializedMap.get("z");
+        assertEquals(1, deserializedInnerMap.size());
+        assertEquals("b", deserializedInnerMap.get("a"));
+    }
+
+    @Test
+    public void shouldSerializeMapEntry() throws Exception {
+        final Graph graph = TinkerGraph.open();
+        final Vertex v1 = graph.addVertex();
+        final Date d = new Date();
+
+        final Map<Object, Object> map = new HashMap<>();
+        map.put("x", 1);
+        map.put(v1, 100);
+        map.put(d, "test");
+
+        final ResponseMessage response = 
convertBinary(IteratorUtils.asList(map.entrySet()));
+        assertCommon(response);
+
+        final List<Map.Entry<Object, Object>> deserializedEntries = 
(List<Map.Entry<Object, Object>>) response.getResult().getData();
+        assertEquals(3, deserializedEntries.size());
+        deserializedEntries.forEach(e -> {
+            if (e.getKey().equals("x"))
+                assertEquals(1, e.getValue());
+            else if (e.getKey().equals(v1))
+                assertEquals(100, e.getValue());
+            else if (e.getKey().equals(d))
+                assertEquals("test", e.getValue());
+            else
+                fail("Map entries contains a key that is not part of what was 
serialized");
+        });
+    }
+
+    @Test
+    public void shouldSerializeEdge() throws Exception {
+        final Graph g = TinkerGraph.open();
+        final Vertex v1 = g.addVertex();
+        final Vertex v2 = g.addVertex();
+        final Edge e = v1.addEdge("test", v2);
+        e.property("abc", 123);
+
+        final Iterable<Edge> iterable = IteratorUtils.list(g.edges());
+
+        final ResponseMessage response = convertBinary(iterable);
+        assertCommon(response);
+
+        final List<ReferenceEdge> edgeList = (List<ReferenceEdge>) 
response.getResult().getData();
+        assertEquals(1, edgeList.size());
+
+        final ReferenceEdge deserializedEdge = edgeList.get(0);
+        assertEquals(e.id(), deserializedEdge.id());
+        assertEquals("test", deserializedEdge.label());
+
+        assertEquals(0, IteratorUtils.count(deserializedEdge.properties()));
+        assertEquals(v1.id(), deserializedEdge.outVertex().id());
+        assertEquals("", deserializedEdge.outVertex().label());
+        assertEquals(v2.id(), deserializedEdge.inVertex().id());
+        assertEquals("", deserializedEdge.inVertex().label());
+    }
+
+    @Test
+    public void shouldSerializeTree() throws Exception {
+        final Graph g = TinkerFactory.createModern();
+        final Tree t = g.traversal().V().out().out().tree().by("name").next();
+
+        final ResponseMessage response = convertBinary(t);
+        assertCommon(response);
+
+        final Tree deserialized = (Tree) response.getResult().getData();
+        assertEquals(t, deserialized);
+
+        assertThat(deserialized.containsKey("marko"), is(true));
+        assertEquals(1, deserialized.size());
+
+        final Tree markoChildren = (Tree) deserialized.get("marko");
+        assertThat(markoChildren.containsKey("josh"), is(true));
+        assertEquals(1, markoChildren.size());
+
+        final Tree joshChildren = (Tree) markoChildren.get("josh");
+        assertThat(joshChildren.containsKey("lop"), is(true));
+        assertThat(joshChildren.containsKey("ripple"), is(true));
+        assertEquals(2, joshChildren.size());
+    }
+
+    @Test
+    public void shouldSerializeVertexWithEmbeddedMap() throws Exception {
+        final Graph g = TinkerGraph.open();
+        final Vertex v = g.addVertex();
+        final Map<String, Object> map = new HashMap<>();
+        map.put("x", 500);
+        map.put("y", "some");
+
+        final ArrayList<Object> friends = new ArrayList<>();
+        friends.add("x");
+        friends.add(5);
+        friends.add(map);
+
+        v.property(VertexProperty.Cardinality.single, "friends", friends);
+
+        final List list = IteratorUtils.list(g.vertices());
+
+        final ResponseMessage response = convertBinary(list);
+        assertCommon(response);
+
+        final List<ReferenceVertex> vertexList = (List<ReferenceVertex>) 
response.getResult().getData();
+        assertEquals(1, vertexList.size());
+
+        final ReferenceVertex deserializedVertex = vertexList.get(0);
+        assertEquals(0L, deserializedVertex.id());
+        assertEquals("", deserializedVertex.label());
+
+        assertEquals(0, IteratorUtils.count(deserializedVertex.properties()));
+    }
+
+    @Test
+    public void shouldSerializeToMapWithElementForKey() throws Exception {
+        final TinkerGraph graph = TinkerFactory.createClassic();
+        final GraphTraversalSource g = graph.traversal();
+        final Map<Vertex, Integer> map = new HashMap<>();
+        map.put(g.V().has("name", "marko").next(), 1000);
+
+        final ResponseMessage response = convertBinary(map);
+        assertCommon(response);
+
+        final Map<Vertex, Integer> deserializedMap = (Map<Vertex, Integer>) 
response.getResult().getData();
+        assertEquals(1, deserializedMap.size());
+
+        final Vertex deserializedMarko = 
deserializedMap.keySet().iterator().next();
+        assertEquals(0, IteratorUtils.count(deserializedMarko.properties()));
+        assertEquals(1, deserializedMarko.id());
+        assertEquals("", deserializedMarko.label());
+
+        assertEquals(new Integer(1000), 
deserializedMap.values().iterator().next());
+    }
+
+    @Test
+    public void shouldSerializeFullResponseMessage() throws Exception {
+        final UUID id = UUID.randomUUID();
+
+        final Map<String, Object> metaData = new HashMap<>();
+        metaData.put("test", "this");
+        metaData.put("one", 1);
+
+        final Map<String, Object> attributes = new HashMap<>();
+        attributes.put("test", "that");
+        attributes.put("two", 2);
+
+        final ResponseMessage response = ResponseMessage.build(id)
+                .responseMetaData(metaData)
+                .code(ResponseStatusCode.SUCCESS)
+                .result("some-result")
+                .statusAttributes(attributes)
+                .statusMessage("worked")
+                .create();
+
+        final ByteBuf bb = 
binarySerializer.serializeResponseAsBinary(response, allocator);
+        final ResponseMessage deserialized = 
binarySerializer.deserializeResponse(bb);
+
+        assertEquals(id, deserialized.getRequestId());
+        assertEquals("this", deserialized.getResult().getMeta().get("test"));
+        assertEquals(1, deserialized.getResult().getMeta().get("one"));
+        assertEquals("some-result", deserialized.getResult().getData());
+        assertEquals("that", 
deserialized.getStatus().getAttributes().get("test"));
+        assertEquals(2, deserialized.getStatus().getAttributes().get("two"));
+        assertEquals(ResponseStatusCode.SUCCESS.getValue(), 
deserialized.getStatus().getCode().getValue());
+        assertEquals("worked", deserialized.getStatus().getMessage());
+    }
+
+    @Test
+    public void shouldHaveTooSmallBufferToSerializeResponseMessage() throws 
Exception {
+        final UUID id = UUID.randomUUID();
+
+        final Map<String, Object> metaData = new HashMap<>();
+        metaData.put("test", "this");
+        metaData.put("one", 1);
+
+        final Map<String, Object> attributes = new HashMap<>();
+        attributes.put("test", "that");
+        attributes.put("two", 2);
+
+        final ResponseMessage response = ResponseMessage.build(id)
+                .responseMetaData(metaData)
+                .code(ResponseStatusCode.SUCCESS)
+                .result("some-result")
+                .statusAttributes(attributes)
+                .statusMessage("worked")
+                .create();
+
+        final MessageSerializer binarySerializerWithSmallBuffer = new 
GryoLiteMessageSerializerV1d0();
+        final Map<String, Object> configWithSmallBuffer = new HashMap<String, 
Object>() {{
+            put("bufferSize", 1);
+        }};
+        binarySerializerWithSmallBuffer.configure(configWithSmallBuffer, null);
+
+        try {
+            
binarySerializerWithSmallBuffer.serializeResponseAsBinary(response, allocator);
+            fail("Should have a buffer size that is too small");
+        } catch (Exception ex) {
+            final Throwable root = ExceptionUtils.getRootCause(ex);
+            assertThat(root, instanceOf(KryoException.class));
+        }
+    }
+
+    @Test
+    public void shouldReturnAllBytesInResponse() throws Exception {
+        final UUID id = UUID.randomUUID();
+
+        final Map<String, Object> metaData = new HashMap<>();
+        metaData.put("test", "this");
+        metaData.put("one", 1);
+
+        final Map<String, Object> attributes = new HashMap<>();
+        attributes.put("test", "that");
+        attributes.put("two", 2);
+
+        final ResponseMessage response = ResponseMessage.build(id)
+                .responseMetaData(metaData)
+                .code(ResponseStatusCode.SUCCESS)
+                .result("some-result")
+                .statusAttributes(attributes)
+                .statusMessage("worked")
+                .create();
+
+        final MessageSerializer binarySerializerWithSmallBuffer = new 
GryoLiteMessageSerializerV1d0();
+        final Map<String, Object> configWithSmallBuffer = new HashMap<String, 
Object>() {{
+            // set to bufferSize < total message size but still greater than 
any individual object requires
+            put("bufferSize", 50);
+        }};
+        binarySerializerWithSmallBuffer.configure(configWithSmallBuffer, null);
+
+        ByteBuf buf = 
binarySerializerWithSmallBuffer.serializeResponseAsBinary(response, allocator);
+        assertTrue(buf.isReadable());
+        assertEquals(82, buf.readableBytes());
+    }
+
+    @Test
+    public void shouldSerializeFullRequestMessage() throws Exception {
+        final UUID id = UUID.randomUUID();
+
+        final RequestMessage request = RequestMessage.build("try")
+                .overrideRequestId(id)
+                .processor("pro")
+                .addArg("test", "this")
+                .create();
+        final ByteBuf bb = binarySerializer.serializeRequestAsBinary(request, 
allocator);
+        final int mimeLen = bb.readByte();
+        bb.readBytes(new byte[mimeLen]);
+        final RequestMessage deserialized = 
binarySerializer.deserializeRequest(bb);
+
+        assertEquals(id, deserialized.getRequestId());
+        assertEquals("pro", deserialized.getProcessor());
+        assertEquals("try", deserialized.getOp());
+        assertEquals("this", deserialized.getArgs().get("test"));
+    }
+
+    @Test
+    public void shouldHaveTooSmallBufferToSerializeRequestMessage() throws 
Exception {
+        final UUID id = UUID.randomUUID();
+
+        final RequestMessage request = RequestMessage.build("try")
+                .overrideRequestId(id)
+                .processor("pro")
+                .addArg("test", "this")
+                .create();
+
+        final MessageSerializer binarySerializerWithSmallBuffer = new 
GryoLiteMessageSerializerV1d0();
+        final Map<String, Object> configWithSmallBuffer = new HashMap<String, 
Object>() {{
+            put("bufferSize", 1);
+        }};
+        binarySerializerWithSmallBuffer.configure(configWithSmallBuffer, null);
+
+        try {
+            binarySerializerWithSmallBuffer.serializeRequestAsBinary(request, 
allocator);
+            fail("Should have a buffer size that is too small");
+        } catch (Exception ex) {
+            final Throwable root = ExceptionUtils.getRootCause(ex);
+            assertThat(root, instanceOf(KryoException.class));
+        }
+    }
+
+    @Test
+    public void shouldReturnAllBytesInRequest() throws Exception {
+        final UUID id = UUID.randomUUID();
+
+        final RequestMessage request = RequestMessage.build("try")
+                .overrideRequestId(id)
+                .processor("pro")
+                .addArg("test", "this")
+                .create();
+
+        final MessageSerializer binarySerializerWithSmallBuffer = new 
GryoLiteMessageSerializerV1d0();
+        final Map<String, Object> configWithSmallBuffer = new HashMap<String, 
Object>() {{
+            // set to bufferSize < total message size but still greater than 
any individual object requires
+            put("bufferSize", 50);
+        }};
+        binarySerializerWithSmallBuffer.configure(configWithSmallBuffer, null);
+
+        ByteBuf buf = 
binarySerializerWithSmallBuffer.serializeRequestAsBinary(request, allocator);
+        assertTrue(buf.isReadable());
+        assertEquals(76, buf.readableBytes());
+    }
+
+    private void assertCommon(final ResponseMessage response) {
+        assertEquals(requestId, response.getRequestId());
+        assertEquals(ResponseStatusCode.SUCCESS, 
response.getStatus().getCode());
+    }
+
+    private ResponseMessage convertBinary(final Object toSerialize) throws 
SerializationException {
+        final ByteBuf bb = 
binarySerializer.serializeResponseAsBinary(responseMessageBuilder.result(toSerialize).create(),
 allocator);
+        return binarySerializer.deserializeResponse(bb);
+    }
+
+    private ResponseMessage convertText(final Object toSerialize) throws 
SerializationException {
+        final ByteBuf bb = 
textSerializer.serializeResponseAsBinary(responseMessageBuilder.result(toSerialize).create(),
 allocator);
+        return textSerializer.deserializeResponse(bb);
+    }
+
+    public static class ErrorOnlyClassResolverSupplierAsInstance implements 
Supplier<ClassResolver> {
+
+        private static final ErrorOnlyClassResolverSupplierAsInstance instance 
= new ErrorOnlyClassResolverSupplierAsInstance();
+
+        private ErrorOnlyClassResolverSupplierAsInstance() {}
+
+        public static ErrorOnlyClassResolverSupplierAsInstance getInstance() {
+            return instance;
+        }
+
+        @Override
+        public ClassResolver get() {
+            return new ErrorOnlyClassResolver();
+        }
+    }
+
+    public static class ErrorOnlyClassResolverSupplier implements 
Supplier<ClassResolver> {
+        @Override
+        public ClassResolver get() {
+            return new ErrorOnlyClassResolver();
+        }
+    }
+
+    public static class ErrorOnlyClassResolver extends GryoClassResolver {
+        @Override
+        public Registration getRegistration(Class clazz) {
+            throw new RuntimeException("Registration is not allowed with this 
ClassResolver - it is not a good implementation");
+        }
+    }
+
+    public static class ColorIoRegistry extends AbstractIoRegistry {
+        public ColorIoRegistry() {
+            register(GryoIo.class, Color.class, new ColorSerializer());
+        }
+    }
+
+    public static class ColorSerializer extends Serializer<Color> {
+        @Override
+        public void write(final Kryo kryo, final Output output, final Color 
color) {
+            output.write(color.equals(Color.RED) ? 1 : 0);
+        }
+
+        @Override
+        public Color read(final Kryo kryo, final Input input, final 
Class<Color> aClass) {
+            return input.read() == 1 ? Color.RED : Color.BLACK;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GryoMessageSerializerV1d0Test.java
----------------------------------------------------------------------
diff --git 
a/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GryoMessageSerializerV1d0Test.java
 
b/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GryoMessageSerializerV1d0Test.java
index bb731a9..4064f49 100644
--- 
a/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GryoMessageSerializerV1d0Test.java
+++ 
b/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GryoMessageSerializerV1d0Test.java
@@ -49,7 +49,7 @@ import org.apache.tinkerpop.shaded.kryo.io.Input;
 import org.apache.tinkerpop.shaded.kryo.io.Output;
 import org.junit.Test;
 
-import java.awt.*;
+import java.awt.Color;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Date;

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/gremlin-server/conf/gremlin-server-classic.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/conf/gremlin-server-classic.yaml 
b/gremlin-server/conf/gremlin-server-classic.yaml
index 0ac4a58..388a27e 100644
--- a/gremlin-server/conf/gremlin-server-classic.yaml
+++ b/gremlin-server/conf/gremlin-server-classic.yaml
@@ -31,6 +31,7 @@ scriptEngines: {
     scripts: [scripts/complex-lifecycle.groovy]}}
 serializers:
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { 
useMapperFromGraph: graph }}       # application/vnd.gremlin-v1.0+gryo
+  - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoLiteMessageSerializerV1d0, config: 
{ useMapperFromGraph: graph }}        # application/vnd.gremlin-v1.0+gryo-lite
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { 
serializeResultToString: true }}   # application/vnd.gremlin-v1.0+gryo-stringd
 metrics: {
   slf4jReporter: {enabled: true, interval: 180000}}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/gremlin-server/conf/gremlin-server-modern-readonly.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/conf/gremlin-server-modern-readonly.yaml 
b/gremlin-server/conf/gremlin-server-modern-readonly.yaml
index cc0b0ad..5214b50 100644
--- a/gremlin-server/conf/gremlin-server-modern-readonly.yaml
+++ b/gremlin-server/conf/gremlin-server-modern-readonly.yaml
@@ -31,6 +31,7 @@ scriptEngines: {
     scripts: [scripts/generate-modern-readonly.groovy]}}
 serializers:
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { 
useMapperFromGraph: graph }}       # application/vnd.gremlin-v1.0+gryo
+  - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoLiteMessageSerializerV1d0, config: 
{ useMapperFromGraph: graph }}   # application/vnd.gremlin-v1.0+gryo-lite
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { 
serializeResultToString: true }}   # application/vnd.gremlin-v1.0+gryo-stringd
 metrics: {
   slf4jReporter: {enabled: true, interval: 180000}}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/gremlin-server/conf/gremlin-server-modern.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/conf/gremlin-server-modern.yaml 
b/gremlin-server/conf/gremlin-server-modern.yaml
index c97ae14..cec7f81 100644
--- a/gremlin-server/conf/gremlin-server-modern.yaml
+++ b/gremlin-server/conf/gremlin-server-modern.yaml
@@ -31,6 +31,7 @@ scriptEngines: {
     scripts: [scripts/generate-modern.groovy]}}
 serializers:
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { 
useMapperFromGraph: graph }}       # application/vnd.gremlin-v1.0+gryo
+  - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoLiteMessageSerializerV1d0, config: 
{ useMapperFromGraph: graph }}   # application/vnd.gremlin-v1.0+gryo-lite
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { 
serializeResultToString: true }}   # application/vnd.gremlin-v1.0+gryo-stringd
 metrics: {
   slf4jReporter: {enabled: true, interval: 180000}}

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/gremlin-server/conf/gremlin-server-neo4j.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/conf/gremlin-server-neo4j.yaml 
b/gremlin-server/conf/gremlin-server-neo4j.yaml
index d32ce8c..55aa170 100644
--- a/gremlin-server/conf/gremlin-server-neo4j.yaml
+++ b/gremlin-server/conf/gremlin-server-neo4j.yaml
@@ -45,6 +45,7 @@ scriptEngines: {
       staticImports: [java.lang.Math.PI]}}
 serializers:
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { 
useMapperFromGraph: graph }}            # application/vnd.gremlin-v1.0+gryo
+  - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoLiteMessageSerializerV1d0, config: 
{ useMapperFromGraph: graph }}        # application/vnd.gremlin-v1.0+gryo-lite
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { 
serializeResultToString: true }}        # 
application/vnd.gremlin-v1.0+gryo-stringd
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, 
config: { useMapperFromGraph: graph }} # application/vnd.gremlin-v1.0+json
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: 
{ useMapperFromGraph: graph }}        # application/json

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/gremlin-server/conf/gremlin-server-rest-secure.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/conf/gremlin-server-rest-secure.yaml 
b/gremlin-server/conf/gremlin-server-rest-secure.yaml
index ff5c550..fd04855 100644
--- a/gremlin-server/conf/gremlin-server-rest-secure.yaml
+++ b/gremlin-server/conf/gremlin-server-rest-secure.yaml
@@ -44,8 +44,6 @@ scriptEngines: {
               
"org.apache.tinkerpop.gremlin.groovy.jsr223.customizer.TimedInterruptCustomizerProvider":[10000],
               
"org.apache.tinkerpop.gremlin.groovy.jsr223.customizer.CompileStaticCustomizerProvider":["org.apache.tinkerpop.gremlin.groovy.jsr223.customizer.SimpleSandboxExtension"]}}}}
 serializers:
-  - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { 
useMapperFromGraph: graph }}            # application/vnd.gremlin-v1.0+gryo
-  - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { 
serializeResultToString: true }}        # 
application/vnd.gremlin-v1.0+gryo-stringd
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, 
config: { useMapperFromGraph: graph }} # application/vnd.gremlin-v1.0+json
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: 
{ useMapperFromGraph: graph }}        # application/json
 processors:

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/gremlin-server/conf/gremlin-server-secure.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/conf/gremlin-server-secure.yaml 
b/gremlin-server/conf/gremlin-server-secure.yaml
index 8189fef..4597187 100644
--- a/gremlin-server/conf/gremlin-server-secure.yaml
+++ b/gremlin-server/conf/gremlin-server-secure.yaml
@@ -45,6 +45,7 @@ scriptEngines: {
               
"org.apache.tinkerpop.gremlin.groovy.jsr223.customizer.CompileStaticCustomizerProvider":["org.apache.tinkerpop.gremlin.groovy.jsr223.customizer.SimpleSandboxExtension"]}}}}
 serializers:
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { 
useMapperFromGraph: graph }}            # application/vnd.gremlin-v1.0+gryo
+  - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoLiteMessageSerializerV1d0, config: 
{ useMapperFromGraph: graph }}        # application/vnd.gremlin-v1.0+gryo-lite
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { 
serializeResultToString: true }}        # 
application/vnd.gremlin-v1.0+gryo-stringd
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, 
config: { useMapperFromGraph: graph }} # application/vnd.gremlin-v1.0+json
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: 
{ useMapperFromGraph: graph }}        # application/json

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/gremlin-server/conf/gremlin-server-spark.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/conf/gremlin-server-spark.yaml 
b/gremlin-server/conf/gremlin-server-spark.yaml
index a0467fc..9340611 100644
--- a/gremlin-server/conf/gremlin-server-spark.yaml
+++ b/gremlin-server/conf/gremlin-server-spark.yaml
@@ -58,6 +58,7 @@ scriptEngines: {
       staticImports: [java.lang.Math.PI]}}
 serializers:
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { 
useMapperFromGraph: graph }}            # application/vnd.gremlin-v1.0+gryo
+  - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoLiteMessageSerializerV1d0, config: 
{ useMapperFromGraph: graph }}        # application/vnd.gremlin-v1.0+gryo-lite
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { 
serializeResultToString: true }}        # 
application/vnd.gremlin-v1.0+gryo-stringd
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, 
config: { useMapperFromGraph: graph }} # application/vnd.gremlin-v1.0+json
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: 
{ useMapperFromGraph: graph }}        # application/json

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/gremlin-server/conf/gremlin-server.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/conf/gremlin-server.yaml 
b/gremlin-server/conf/gremlin-server.yaml
index 0723553..c05ac27 100644
--- a/gremlin-server/conf/gremlin-server.yaml
+++ b/gremlin-server/conf/gremlin-server.yaml
@@ -35,6 +35,7 @@ scriptEngines: {
       staticImports: [java.lang.Math.PI]}}
 serializers:
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { 
useMapperFromGraph: graph }}            # application/vnd.gremlin-v1.0+gryo
+  - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoLiteMessageSerializerV1d0, config: 
{ useMapperFromGraph: graph }}        # application/vnd.gremlin-v1.0+gryo-lite
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { 
serializeResultToString: true }}        # 
application/vnd.gremlin-v1.0+gryo-stringd
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, 
config: { useMapperFromGraph: graph }} # application/vnd.gremlin-v1.0+json
   - { className: 
org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: 
{ useMapperFromGraph: graph }}        # application/json

http://git-wip-us.apache.org/repos/asf/incubator-tinkerpop/blob/f811a8ae/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinResultSetIntegrateTest.java
----------------------------------------------------------------------
diff --git 
a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinResultSetIntegrateTest.java
 
b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinResultSetIntegrateTest.java
index 129f1a0..3a4ab27 100644
--- 
a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinResultSetIntegrateTest.java
+++ 
b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinResultSetIntegrateTest.java
@@ -24,6 +24,7 @@ import org.apache.tinkerpop.gremlin.driver.MessageSerializer;
 import org.apache.tinkerpop.gremlin.driver.Result;
 import org.apache.tinkerpop.gremlin.driver.ResultSet;
 import org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0;
+import org.apache.tinkerpop.gremlin.driver.ser.Serializers;
 import org.apache.tinkerpop.gremlin.process.traversal.Path;
 import org.apache.tinkerpop.gremlin.process.traversal.Traverser;
 import 
org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversalSource;
@@ -38,8 +39,10 @@ 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.gremlin.structure.util.reference.ReferenceVertex;
 import org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerGraph;
 import org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry;
+import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 import org.hamcrest.CoreMatchers;
 import org.junit.After;
 import org.junit.Before;
@@ -164,6 +167,19 @@ public class GremlinResultSetIntegrateTest extends 
AbstractGremlinServerIntegrat
     }
 
     @Test
+    public void shouldHandleVertexResultWithLiteSerialization() throws 
Exception {
+        final Cluster cluster = 
Cluster.build().serializer(Serializers.GRYO_LITE_V1D0).create();
+        final Client clientLite = cluster.connect();
+        final ResultSet results = clientLite.submit("g.V(1).next()");
+        final Vertex v = results.all().get().get(0).getVertex();
+        assertThat(v, instanceOf(ReferenceVertex.class));
+
+        assertEquals(1L, v.id());
+        assertEquals("", v.label());
+        assertEquals(0, IteratorUtils.count(v.properties()));
+    }
+
+    @Test
     public void shouldHandleVertexPropertyResult() throws Exception {
         final ResultSet results = 
client.submit("g.V().properties('name').next()");
         final VertexProperty<String> v = 
results.all().get().get(0).getVertexProperty();

Reply via email to