You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tinkerpop.apache.org by sp...@apache.org on 2017/01/19 22:55:49 UTC

[01/31] tinkerpop git commit: Created OrdertedTraverser which allows us to move beyond the star graph for OrderGlobalStep by()-projections. Moreover, it reduces the complexity of ordering as the objects of comparison are already determined. Going to gene [Forced Update!]

Repository: tinkerpop
Updated Branches:
  refs/heads/TINKERPOP-1565 8b9ac6d6f -> 8ffd0b8fb (forced update)


Created OrdertedTraverser which allows us to move beyond the star graph for OrderGlobalStep by()-projections. Moreover, it reduces the complexity of ordering as the objects of comparison are already determined. Going to generalize fully to a ProjectedTraverser which will allow us to do the same for SampleGlobalStep, DedupGlobalStep, and down the road maintain order even as the computation is re-distributed to workers.


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

Branch: refs/heads/TINKERPOP-1565
Commit: 501e97a1ecb23f76b2fddba8eaed1dba4a5a839e
Parents: e80a4cd
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Jan 18 09:08:24 2017 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Wed Jan 18 09:08:24 2017 -0700

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |   1 +
 .../traversal/step/map/OrderGlobalStep.java     |  61 ++---
 .../step/util/CollectingBarrierStep.java        |   4 +-
 .../ComputerVerificationStrategy.java           |   5 +-
 .../traversal/traverser/OrderedTraverser.java   | 235 +++++++++++++++++++
 .../gremlin/structure/io/gryo/GryoVersion.java  |   4 +-
 6 files changed, 277 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/501e97a1/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index 4f3f9ce..86c6b4f 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -26,6 +26,7 @@ image::https://raw.githubusercontent.com/apache/tinkerpop/master/docs/static/ima
 TinkerPop 3.2.4 (Release Date: NOT OFFICIALLY RELEASED YET)
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
+* `OrderGlobalStep` now emits traversers with their `by()`-projections and thus, can move beyond the local star graph.
 * SASL negotiation supports both a byte array and Base64 encoded bytes as a string for authentication to Gremlin Server.
 * Deprecated `TinkerIoRegistry` replacing it with the more consistently named `TinkerIoRegistryV1d0`.
 * Made error messaging more consistent during result iteration timeouts in Gremlin Server.

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/501e97a1/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
index a7d21b2..ac5df90 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
@@ -27,10 +27,10 @@ import org.apache.tinkerpop.gremlin.process.traversal.step.ByModulating;
 import org.apache.tinkerpop.gremlin.process.traversal.step.ComparatorHolder;
 import org.apache.tinkerpop.gremlin.process.traversal.step.TraversalParent;
 import org.apache.tinkerpop.gremlin.process.traversal.step.util.CollectingBarrierStep;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.OrderedTraverser;
 import org.apache.tinkerpop.gremlin.process.traversal.traverser.TraverserRequirement;
 import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.TraverserSet;
 import org.apache.tinkerpop.gremlin.structure.util.StringFactory;
-import org.apache.tinkerpop.gremlin.util.function.ChainedComparator;
 import org.javatuples.Pair;
 
 import java.io.Serializable;
@@ -49,8 +49,8 @@ import java.util.stream.Collectors;
 public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBarrierStep<S> implements ComparatorHolder<S, C>, TraversalParent, ByModulating {
 
     private List<Pair<Traversal.Admin<S, C>, Comparator<C>>> comparators = new ArrayList<>();
-    private ChainedComparator<S, C> chainedComparator = null;
     private long limit = Long.MAX_VALUE;
+    private boolean isShuffle = false;
 
     public OrderGlobalStep(final Traversal.Admin traversal) {
         super(traversal);
@@ -58,12 +58,30 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
 
     @Override
     public void barrierConsumer(final TraverserSet<S> traverserSet) {
-        if (null == this.chainedComparator)
-            this.chainedComparator = new ChainedComparator<>(true, this.comparators);
-        if (this.chainedComparator.isShuffle())
+        if (this.isShuffle)
             traverserSet.shuffle();
         else
-            traverserSet.sort((Comparator) this.chainedComparator);
+            traverserSet.sort(Comparator.naturalOrder());
+    }
+
+    @Override
+    public void processAllStarts() {
+        if (this.starts.hasNext()) {
+            while (this.starts.hasNext()) {
+                this.traverserSet.add(new OrderedTraverser<S>(this.starts.next(), (List) this.comparators));
+            }
+            this.barrierConsumer(this.traverserSet);
+        }
+    }
+
+    @Override
+    public Traverser.Admin<S> processNextStart() {
+        if (!this.traverserSet.isEmpty()) {
+            return this.traverserSet.remove();
+        } else if (this.starts.hasNext()) {
+            this.processAllStarts();
+        }
+        return ((OrderedTraverser) this.traverserSet.remove()).getInternal();
     }
 
     public void setLimit(final long limit) {
@@ -76,6 +94,7 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
 
     @Override
     public void addComparator(final Traversal.Admin<S, C> traversal, final Comparator<C> comparator) {
+        this.isShuffle = Order.shuffle == (Comparator) comparator;
         this.comparators.add(new Pair<>(this.integrateChild(traversal), comparator));
     }
 
@@ -125,7 +144,6 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
         for (final Pair<Traversal.Admin<S, C>, Comparator<C>> comparator : this.comparators) {
             clone.comparators.add(new Pair<>(comparator.getValue0().clone(), comparator.getValue1()));
         }
-        clone.chainedComparator = null;
         return clone;
     }
 
@@ -137,47 +155,34 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
 
     @Override
     public MemoryComputeKey<TraverserSet<S>> getMemoryComputeKey() {
-        if (null == this.chainedComparator)
-            this.chainedComparator = new ChainedComparator<>(true, this.comparators);
-        return MemoryComputeKey.of(this.getId(), new OrderBiOperator<>(this.chainedComparator, this.limit), false, true);
+        return MemoryComputeKey.of(this.getId(), new OrderBiOperator<>(this.limit, this.isShuffle), false, true);
     }
 
     ////////////////
 
-    public static final class OrderBiOperator<S> implements BinaryOperator<TraverserSet<S>>, Serializable, Cloneable {
+    public static final class OrderBiOperator<S> implements BinaryOperator<TraverserSet<S>>, Serializable {
 
-        private ChainedComparator chainedComparator;
         private long limit;
+        private boolean isShuffle;
 
         private OrderBiOperator() {
             // for serializers that need a no-arg constructor
         }
 
-        public OrderBiOperator(final ChainedComparator<S, ?> chainedComparator, final long limit) {
-            this.chainedComparator = chainedComparator;
+        public OrderBiOperator(final long limit, final boolean isShuffle) {
             this.limit = limit;
-        }
-
-        @Override
-        public OrderBiOperator<S> clone() {
-            try {
-                final OrderBiOperator<S> clone = (OrderBiOperator<S>) super.clone();
-                clone.chainedComparator = this.chainedComparator.clone();
-                return clone;
-            } catch (final CloneNotSupportedException e) {
-                throw new IllegalStateException(e.getMessage(), e);
-            }
+            this.isShuffle = isShuffle;
         }
 
         @Override
         public TraverserSet<S> apply(final TraverserSet<S> setA, final TraverserSet<S> setB) {
             setA.addAll(setB);
             if (Long.MAX_VALUE != this.limit && setA.bulkSize() > this.limit) {
-                if (this.chainedComparator.isShuffle())
+                if (this.isShuffle)
                     setA.shuffle();
                 else
-                    setA.sort(this.chainedComparator);
-                long counter = 0l;
+                    setA.sort(Comparator.naturalOrder());
+                long counter = 0L;
                 final Iterator<Traverser.Admin<S>> traversers = setA.iterator();
                 while (traversers.hasNext()) {
                     final Traverser.Admin<S> traverser = traversers.next();

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/501e97a1/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/CollectingBarrierStep.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/CollectingBarrierStep.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/CollectingBarrierStep.java
index f9c85a2..b0cce80 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/CollectingBarrierStep.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/CollectingBarrierStep.java
@@ -39,8 +39,8 @@ import java.util.function.BinaryOperator;
  */
 public abstract class CollectingBarrierStep<S> extends AbstractStep<S, S> implements Barrier<TraverserSet<S>> {
 
-    private TraverserSet<S> traverserSet = new TraverserSet<>();
-    private int maxBarrierSize;
+    protected TraverserSet<S> traverserSet = new TraverserSet<>();
+    protected int maxBarrierSize;
 
     public CollectingBarrierStep(final Traversal.Admin traversal) {
         this(traversal, Integer.MAX_VALUE);

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/501e97a1/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategy.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategy.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategy.java
index fc73fc3..5777adb 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategy.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategy.java
@@ -28,6 +28,7 @@ import org.apache.tinkerpop.gremlin.process.traversal.step.GraphComputing;
 import org.apache.tinkerpop.gremlin.process.traversal.step.Mutating;
 import org.apache.tinkerpop.gremlin.process.traversal.step.PathProcessor;
 import org.apache.tinkerpop.gremlin.process.traversal.step.TraversalParent;
+import org.apache.tinkerpop.gremlin.process.traversal.step.filter.SampleGlobalStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.map.GraphStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.InjectStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.SubgraphStep;
@@ -86,8 +87,8 @@ public final class ComputerVerificationStrategy extends AbstractTraversalStrateg
                     throw new VerificationException("Local traversals may not traverse past the local star-graph on GraphComputer: " + traversalOptional.get(), traversal);
             }
 
-            // collecting barriers and dedup global use can only operate on the element and its properties (no incidences)
-            if (step instanceof CollectingBarrierStep && step instanceof TraversalParent) {
+            // sample step use can only operate on the element and its properties (no incidences)
+            if (step instanceof SampleGlobalStep) {
                 if (((TraversalParent) step).getLocalChildren().stream().filter(t -> !TraversalHelper.isLocalProperties(t)).findAny().isPresent())
                     throw new VerificationException("The following barrier step can not process the incident edges of a vertex on GraphComputer: " + step, traversal);
             }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/501e97a1/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/OrderedTraverser.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/OrderedTraverser.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/OrderedTraverser.java
new file mode 100644
index 0000000..4dddaa3
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/OrderedTraverser.java
@@ -0,0 +1,235 @@
+/*
+ *  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.process.traversal.traverser;
+
+import org.apache.tinkerpop.gremlin.process.traversal.Order;
+import org.apache.tinkerpop.gremlin.process.traversal.Path;
+import org.apache.tinkerpop.gremlin.process.traversal.Step;
+import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
+import org.apache.tinkerpop.gremlin.process.traversal.TraversalSideEffects;
+import org.apache.tinkerpop.gremlin.process.traversal.Traverser;
+import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalUtil;
+import org.apache.tinkerpop.gremlin.structure.util.Attachable;
+import org.javatuples.Pair;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Function;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public final class OrderedTraverser<T> implements Traverser.Admin<T> {
+
+    private Traverser.Admin<T> internal;
+    private int order;
+    private final List<Pair<Object, Comparator<?>>> orderChecks = new ArrayList<>();
+
+    private OrderedTraverser() {
+        // for serialization
+    }
+
+    public OrderedTraverser(final Traverser.Admin<T> internal, final List<Pair<Traversal.Admin<T, ?>, Comparator>> checks) {
+        this(internal, 0);
+        for (final Pair<Traversal.Admin<T, ?>, Comparator> pairs : checks) {
+            this.orderChecks.add(Pair.with(TraversalUtil.apply(this.internal, pairs.getValue0()), pairs.getValue1()));
+        }
+    }
+
+    public OrderedTraverser(final Traverser.Admin<T> internal, final int order) {
+        this.internal = internal instanceof OrderedTraverser ? ((OrderedTraverser) internal).internal : internal;
+        this.order = order;
+    }
+
+    public Traverser.Admin<T> getInternal() {
+        return this.internal;
+    }
+
+    public int order() {
+        return this.order;
+    }
+
+    @Override
+    public void merge(final Admin<?> other) {
+        this.internal.merge(other);
+    }
+
+    @Override
+    public <R> Admin<R> split(R r, Step<T, R> step) {
+        return new OrderedTraverser<>(this.internal.split(r, step), this.order);
+    }
+
+    @Override
+    public Admin<T> split() {
+        return new OrderedTraverser<>(this.internal.split(), this.order);
+    }
+
+    @Override
+    public void addLabels(final Set<String> labels) {
+        this.internal.addLabels(labels);
+    }
+
+    @Override
+    public void keepLabels(final Set<String> labels) {
+        this.internal.keepLabels(labels);
+    }
+
+    @Override
+    public void dropLabels(final Set<String> labels) {
+        this.internal.dropLabels(labels);
+    }
+
+    @Override
+    public void dropPath() {
+        this.internal.dropPath();
+    }
+
+    @Override
+    public void set(final T t) {
+        this.internal.set(t);
+    }
+
+    @Override
+    public void incrLoops(final String stepLabel) {
+        this.internal.incrLoops(stepLabel);
+    }
+
+    @Override
+    public void resetLoops() {
+        this.internal.resetLoops();
+    }
+
+    @Override
+    public String getStepId() {
+        return this.internal.getStepId();
+    }
+
+    @Override
+    public void setStepId(final String stepId) {
+        this.internal.setStepId(stepId);
+    }
+
+    @Override
+    public void setBulk(final long count) {
+        this.internal.setBulk(count);
+    }
+
+    @Override
+    public Admin<T> detach() {
+        this.internal = this.internal.detach();
+        return this;
+    }
+
+    @Override
+    public T attach(final Function<Attachable<T>, T> method) {
+        return this.internal.attach(method);
+    }
+
+    @Override
+    public void setSideEffects(final TraversalSideEffects sideEffects) {
+        this.internal.setSideEffects(sideEffects);
+    }
+
+    @Override
+    public TraversalSideEffects getSideEffects() {
+        return this.internal.getSideEffects();
+    }
+
+    @Override
+    public Set<String> getTags() {
+        return this.internal.getTags();
+    }
+
+    @Override
+    public T get() {
+        return this.internal.get();
+    }
+
+    @Override
+    public <S> S sack() {
+        return this.internal.sack();
+    }
+
+    @Override
+    public <S> void sack(final S object) {
+        this.internal.sack(object);
+    }
+
+    @Override
+    public Path path() {
+        return this.internal.path();
+    }
+
+    @Override
+    public int loops() {
+        return this.internal.loops();
+    }
+
+    @Override
+    public long bulk() {
+        return this.internal.bulk();
+    }
+
+    @Override
+    public int hashCode() {
+        return this.internal.hashCode();
+    }
+
+    @Override
+    public boolean equals(final Object object) {
+        return object instanceof OrderedTraverser && ((OrderedTraverser) object).internal.equals(this.internal);
+    }
+
+    @Override
+    public OrderedTraverser<T> clone() {
+        try {
+            final OrderedTraverser<T> clone = (OrderedTraverser<T>) super.clone();
+            clone.internal = (Traverser.Admin<T>) this.internal.clone();
+            return clone;
+        } catch (final CloneNotSupportedException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public int compareTo(final Traverser<T> o) {
+        if (!(o instanceof OrderedTraverser))
+            return 0;
+        else {
+            if (this.orderChecks.isEmpty()) {
+                return Order.incr.compare(this.get(), o.get());
+            } else {
+                final OrderedTraverser<T> other = (OrderedTraverser<T>) o;
+                for (int i = 0; i < this.orderChecks.size(); i++) {
+                    final Comparator comparator = this.orderChecks.get(i).getValue1();
+                    final Object thisObject = this.orderChecks.get(i).getValue0();
+                    final Object otherObject = other.orderChecks.get(i).getValue0();
+                    final int comparison = comparator.compare(thisObject, otherObject);
+                    if (comparison != 0)
+                        return comparison;
+
+                }
+                return 0;
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/501e97a1/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
index 0bd9e87..a04f2d9 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
@@ -51,6 +51,7 @@ import org.apache.tinkerpop.gremlin.process.traversal.traverser.LP_O_OB_P_S_SE_S
 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.process.traversal.traverser.OrderedTraverser;
 import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.TraverserSet;
 import org.apache.tinkerpop.gremlin.process.traversal.util.AndP;
 import org.apache.tinkerpop.gremlin.process.traversal.util.DefaultTraversalMetrics;
@@ -221,7 +222,7 @@ public enum GryoVersion {
             add(GryoTypeReg.of(AbstractMap.SimpleImmutableEntry.class, 121));
             add(GryoTypeReg.of(java.sql.Timestamp.class, 161));
             add(GryoTypeReg.of(InetAddress.class, 162, new UtilSerializers.InetAddressSerializer()));
-            add(GryoTypeReg.of(ByteBuffer.class, 163, new UtilSerializers.ByteBufferSerializer()));  // ***LAST ID***
+            add(GryoTypeReg.of(ByteBuffer.class, 163, new UtilSerializers.ByteBufferSerializer()));
 
             add(GryoTypeReg.of(ReferenceEdge.class, 81));
             add(GryoTypeReg.of(ReferenceVertexProperty.class, 82));
@@ -245,6 +246,7 @@ public enum GryoVersion {
             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(OrderedTraverser.class, 164));   // ***LAST ID***
             add(GryoTypeReg.of(DefaultRemoteTraverser.class, 123, new GryoSerializers.DefaultRemoteTraverserSerializer()));
 
             add(GryoTypeReg.of(Bytecode.class, 122, new GryoSerializers.BytecodeSerializer()));


[17/31] tinkerpop git commit: minor nothing tweak.

Posted by sp...@apache.org.
minor nothing tweak.


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

Branch: refs/heads/TINKERPOP-1565
Commit: bb97c9420840fbc7cfe1493c3a7abfa1b86b46e6
Parents: 5809700
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Nov 30 07:55:45 2016 -0700
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Jan 19 15:15:32 2017 -0500

----------------------------------------------------------------------
 .../structure/util/reference/ReferenceVertexProperty.java | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/bb97c942/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/reference/ReferenceVertexProperty.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/reference/ReferenceVertexProperty.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/reference/ReferenceVertexProperty.java
index 93b4c48..472a63b 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/reference/ReferenceVertexProperty.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/reference/ReferenceVertexProperty.java
@@ -34,7 +34,7 @@ import java.util.NoSuchElementException;
 public class ReferenceVertexProperty<V> extends ReferenceElement<VertexProperty<V>> implements VertexProperty<V> {
 
     private ReferenceVertex vertex;
-    private String key;
+    private String label;
     private V value;
 
     private ReferenceVertexProperty() {
@@ -43,8 +43,8 @@ public class ReferenceVertexProperty<V> extends ReferenceElement<VertexProperty<
 
     public ReferenceVertexProperty(final VertexProperty<V> vertexProperty) {
         super(vertexProperty);
-        this.vertex = ReferenceFactory.detach(vertexProperty.element());
-        this.key = vertexProperty.key();
+        this.vertex = new ReferenceVertex(vertexProperty.element());
+        this.label = vertexProperty.key();
         this.value = vertexProperty.value();
     }
 
@@ -55,12 +55,12 @@ public class ReferenceVertexProperty<V> extends ReferenceElement<VertexProperty<
 
     @Override
     public String key() {
-        return this.key;
+        return this.label;
     }
 
     @Override
     public String label() {
-        return this.key;
+        return this.label;
     }
 
     @Override


[02/31] tinkerpop git commit: We now have ProjectedTraveser which is a Traverser with List of projections. OrderGlobalStep uses this and thus, can order for everything within the local star graph. Added MultiComparator which is like ChainedCompar Posted by sp...@apache.org.
We now have ProjectedTraveser which is a Traverser with List<Object> of projections. OrderGlobalStep uses this and thus, can order for everything within the local star graph. Added MultiComparator which is like ChainedComparator but doesn't contain traversal projections -- just comparators.


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

Branch: refs/heads/TINKERPOP-1565
Commit: 5045f67f469e163d1363f953672a3f38b4ff2a3f
Parents: 501e97a
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Jan 18 09:38:21 2017 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Wed Jan 18 09:38:21 2017 -0700

----------------------------------------------------------------------
 .../traversal/step/map/OrderGlobalStep.java     |  39 ++-
 .../traversal/traverser/OrderedTraverser.java   | 235 -------------------
 .../traversal/traverser/ProjectedTraverser.java | 199 ++++++++++++++++
 .../gremlin/structure/io/gryo/GryoVersion.java  |   8 +-
 .../gremlin/util/function/MultiComparator.java  |  60 +++++
 5 files changed, 295 insertions(+), 246 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/5045f67f/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
index ac5df90..9c071f1 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
@@ -27,10 +27,12 @@ import org.apache.tinkerpop.gremlin.process.traversal.step.ByModulating;
 import org.apache.tinkerpop.gremlin.process.traversal.step.ComparatorHolder;
 import org.apache.tinkerpop.gremlin.process.traversal.step.TraversalParent;
 import org.apache.tinkerpop.gremlin.process.traversal.step.util.CollectingBarrierStep;
-import org.apache.tinkerpop.gremlin.process.traversal.traverser.OrderedTraverser;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.ProjectedTraverser;
 import org.apache.tinkerpop.gremlin.process.traversal.traverser.TraverserRequirement;
 import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.TraverserSet;
+import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalUtil;
 import org.apache.tinkerpop.gremlin.structure.util.StringFactory;
+import org.apache.tinkerpop.gremlin.util.function.MultiComparator;
 import org.javatuples.Pair;
 
 import java.io.Serializable;
@@ -49,6 +51,7 @@ import java.util.stream.Collectors;
 public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBarrierStep<S> implements ComparatorHolder<S, C>, TraversalParent, ByModulating {
 
     private List<Pair<Traversal.Admin<S, C>, Comparator<C>>> comparators = new ArrayList<>();
+    private MultiComparator<C> multiComparator = null;
     private long limit = Long.MAX_VALUE;
     private boolean isShuffle = false;
 
@@ -61,14 +64,16 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
         if (this.isShuffle)
             traverserSet.shuffle();
         else
-            traverserSet.sort(Comparator.naturalOrder());
+            traverserSet.sort((Comparator) this.multiComparator);
     }
 
     @Override
     public void processAllStarts() {
+        if (null == this.multiComparator)
+            this.multiComparator = this.createMultiComparator();
         if (this.starts.hasNext()) {
             while (this.starts.hasNext()) {
-                this.traverserSet.add(new OrderedTraverser<S>(this.starts.next(), (List) this.comparators));
+                this.traverserSet.add(this.createOrderedTraverser(this.starts.next()));
             }
             this.barrierConsumer(this.traverserSet);
         }
@@ -81,7 +86,7 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
         } else if (this.starts.hasNext()) {
             this.processAllStarts();
         }
-        return ((OrderedTraverser) this.traverserSet.remove()).getInternal();
+        return ((ProjectedTraverser) this.traverserSet.remove()).getInternal();
     }
 
     public void setLimit(final long limit) {
@@ -155,7 +160,25 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
 
     @Override
     public MemoryComputeKey<TraverserSet<S>> getMemoryComputeKey() {
-        return MemoryComputeKey.of(this.getId(), new OrderBiOperator<>(this.limit, this.isShuffle), false, true);
+        if (null == this.multiComparator)
+            this.multiComparator = this.createMultiComparator();
+        return MemoryComputeKey.of(this.getId(), new OrderBiOperator<>(this.limit, this.isShuffle, this.multiComparator), false, true);
+    }
+
+    private ProjectedTraverser<S> createOrderedTraverser(final Traverser.Admin<S> traverser) {
+        final List<Object> projections = new ArrayList<>(this.comparators.size());
+        for (final Pair<Traversal.Admin<S, C>, Comparator<C>> pair : this.comparators) {
+            projections.add(TraversalUtil.apply(traverser, pair.getValue0()));
+        }
+        return new ProjectedTraverser<S>(traverser, projections);
+    }
+
+    private MultiComparator<C> createMultiComparator() {
+        final List<Comparator<C>> list = new ArrayList<>(this.comparators.size());
+        for (final Pair<Traversal.Admin<S, C>, Comparator<C>> pair : this.comparators) {
+            list.add(pair.getValue1());
+        }
+        return new MultiComparator<>(list);
     }
 
     ////////////////
@@ -164,14 +187,16 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
 
         private long limit;
         private boolean isShuffle;
+        private MultiComparator comparator;
 
         private OrderBiOperator() {
             // for serializers that need a no-arg constructor
         }
 
-        public OrderBiOperator(final long limit, final boolean isShuffle) {
+        public OrderBiOperator(final long limit, final boolean isShuffle, final MultiComparator multiComparator) {
             this.limit = limit;
             this.isShuffle = isShuffle;
+            this.comparator = multiComparator;
         }
 
         @Override
@@ -181,7 +206,7 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
                 if (this.isShuffle)
                     setA.shuffle();
                 else
-                    setA.sort(Comparator.naturalOrder());
+                    setA.sort(this.comparator);
                 long counter = 0L;
                 final Iterator<Traverser.Admin<S>> traversers = setA.iterator();
                 while (traversers.hasNext()) {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/5045f67f/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/OrderedTraverser.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/OrderedTraverser.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/OrderedTraverser.java
deleted file mode 100644
index 4dddaa3..0000000
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/OrderedTraverser.java
+++ /dev/null
@@ -1,235 +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.process.traversal.traverser;
-
-import org.apache.tinkerpop.gremlin.process.traversal.Order;
-import org.apache.tinkerpop.gremlin.process.traversal.Path;
-import org.apache.tinkerpop.gremlin.process.traversal.Step;
-import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
-import org.apache.tinkerpop.gremlin.process.traversal.TraversalSideEffects;
-import org.apache.tinkerpop.gremlin.process.traversal.Traverser;
-import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalUtil;
-import org.apache.tinkerpop.gremlin.structure.util.Attachable;
-import org.javatuples.Pair;
-
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Set;
-import java.util.function.Function;
-
-/**
- * @author Marko A. Rodriguez (http://markorodriguez.com)
- */
-public final class OrderedTraverser<T> implements Traverser.Admin<T> {
-
-    private Traverser.Admin<T> internal;
-    private int order;
-    private final List<Pair<Object, Comparator<?>>> orderChecks = new ArrayList<>();
-
-    private OrderedTraverser() {
-        // for serialization
-    }
-
-    public OrderedTraverser(final Traverser.Admin<T> internal, final List<Pair<Traversal.Admin<T, ?>, Comparator>> checks) {
-        this(internal, 0);
-        for (final Pair<Traversal.Admin<T, ?>, Comparator> pairs : checks) {
-            this.orderChecks.add(Pair.with(TraversalUtil.apply(this.internal, pairs.getValue0()), pairs.getValue1()));
-        }
-    }
-
-    public OrderedTraverser(final Traverser.Admin<T> internal, final int order) {
-        this.internal = internal instanceof OrderedTraverser ? ((OrderedTraverser) internal).internal : internal;
-        this.order = order;
-    }
-
-    public Traverser.Admin<T> getInternal() {
-        return this.internal;
-    }
-
-    public int order() {
-        return this.order;
-    }
-
-    @Override
-    public void merge(final Admin<?> other) {
-        this.internal.merge(other);
-    }
-
-    @Override
-    public <R> Admin<R> split(R r, Step<T, R> step) {
-        return new OrderedTraverser<>(this.internal.split(r, step), this.order);
-    }
-
-    @Override
-    public Admin<T> split() {
-        return new OrderedTraverser<>(this.internal.split(), this.order);
-    }
-
-    @Override
-    public void addLabels(final Set<String> labels) {
-        this.internal.addLabels(labels);
-    }
-
-    @Override
-    public void keepLabels(final Set<String> labels) {
-        this.internal.keepLabels(labels);
-    }
-
-    @Override
-    public void dropLabels(final Set<String> labels) {
-        this.internal.dropLabels(labels);
-    }
-
-    @Override
-    public void dropPath() {
-        this.internal.dropPath();
-    }
-
-    @Override
-    public void set(final T t) {
-        this.internal.set(t);
-    }
-
-    @Override
-    public void incrLoops(final String stepLabel) {
-        this.internal.incrLoops(stepLabel);
-    }
-
-    @Override
-    public void resetLoops() {
-        this.internal.resetLoops();
-    }
-
-    @Override
-    public String getStepId() {
-        return this.internal.getStepId();
-    }
-
-    @Override
-    public void setStepId(final String stepId) {
-        this.internal.setStepId(stepId);
-    }
-
-    @Override
-    public void setBulk(final long count) {
-        this.internal.setBulk(count);
-    }
-
-    @Override
-    public Admin<T> detach() {
-        this.internal = this.internal.detach();
-        return this;
-    }
-
-    @Override
-    public T attach(final Function<Attachable<T>, T> method) {
-        return this.internal.attach(method);
-    }
-
-    @Override
-    public void setSideEffects(final TraversalSideEffects sideEffects) {
-        this.internal.setSideEffects(sideEffects);
-    }
-
-    @Override
-    public TraversalSideEffects getSideEffects() {
-        return this.internal.getSideEffects();
-    }
-
-    @Override
-    public Set<String> getTags() {
-        return this.internal.getTags();
-    }
-
-    @Override
-    public T get() {
-        return this.internal.get();
-    }
-
-    @Override
-    public <S> S sack() {
-        return this.internal.sack();
-    }
-
-    @Override
-    public <S> void sack(final S object) {
-        this.internal.sack(object);
-    }
-
-    @Override
-    public Path path() {
-        return this.internal.path();
-    }
-
-    @Override
-    public int loops() {
-        return this.internal.loops();
-    }
-
-    @Override
-    public long bulk() {
-        return this.internal.bulk();
-    }
-
-    @Override
-    public int hashCode() {
-        return this.internal.hashCode();
-    }
-
-    @Override
-    public boolean equals(final Object object) {
-        return object instanceof OrderedTraverser && ((OrderedTraverser) object).internal.equals(this.internal);
-    }
-
-    @Override
-    public OrderedTraverser<T> clone() {
-        try {
-            final OrderedTraverser<T> clone = (OrderedTraverser<T>) super.clone();
-            clone.internal = (Traverser.Admin<T>) this.internal.clone();
-            return clone;
-        } catch (final CloneNotSupportedException e) {
-            throw new IllegalStateException(e.getMessage(), e);
-        }
-    }
-
-    @Override
-    public int compareTo(final Traverser<T> o) {
-        if (!(o instanceof OrderedTraverser))
-            return 0;
-        else {
-            if (this.orderChecks.isEmpty()) {
-                return Order.incr.compare(this.get(), o.get());
-            } else {
-                final OrderedTraverser<T> other = (OrderedTraverser<T>) o;
-                for (int i = 0; i < this.orderChecks.size(); i++) {
-                    final Comparator comparator = this.orderChecks.get(i).getValue1();
-                    final Object thisObject = this.orderChecks.get(i).getValue0();
-                    final Object otherObject = other.orderChecks.get(i).getValue0();
-                    final int comparison = comparator.compare(thisObject, otherObject);
-                    if (comparison != 0)
-                        return comparison;
-
-                }
-                return 0;
-            }
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/5045f67f/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java
new file mode 100644
index 0000000..67e723a
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.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.process.traversal.traverser;
+
+import org.apache.tinkerpop.gremlin.process.traversal.Path;
+import org.apache.tinkerpop.gremlin.process.traversal.Step;
+import org.apache.tinkerpop.gremlin.process.traversal.TraversalSideEffects;
+import org.apache.tinkerpop.gremlin.process.traversal.Traverser;
+import org.apache.tinkerpop.gremlin.structure.util.Attachable;
+
+import java.util.List;
+import java.util.Set;
+import java.util.function.Function;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public final class ProjectedTraverser<T> implements Traverser.Admin<T> {
+
+    private Traverser.Admin<T> internal;
+    private List<Object> projections;
+
+    private ProjectedTraverser() {
+        // for serialization
+    }
+
+    public ProjectedTraverser(final Traverser.Admin<T> internal, final List<Object> projections) {
+        this.internal = internal;
+        this.projections = projections;
+    }
+
+
+    public Traverser.Admin<T> getInternal() {
+        return this.internal;
+    }
+
+    public List<Object> getProjections() {
+        return this.projections;
+    }
+
+    @Override
+    public void merge(final Admin<?> other) {
+        this.internal.merge(other);
+    }
+
+    @Override
+    public <R> Admin<R> split(R r, Step<T, R> step) {
+        return new ProjectedTraverser<>(this.internal.split(r, step), this.projections);
+    }
+
+    @Override
+    public Admin<T> split() {
+        return new ProjectedTraverser<>(this.internal.split(), this.projections);
+    }
+
+    @Override
+    public void addLabels(final Set<String> labels) {
+        this.internal.addLabels(labels);
+    }
+
+    @Override
+    public void keepLabels(final Set<String> labels) {
+        this.internal.keepLabels(labels);
+    }
+
+    @Override
+    public void dropLabels(final Set<String> labels) {
+        this.internal.dropLabels(labels);
+    }
+
+    @Override
+    public void dropPath() {
+        this.internal.dropPath();
+    }
+
+    @Override
+    public void set(final T t) {
+        this.internal.set(t);
+    }
+
+    @Override
+    public void incrLoops(final String stepLabel) {
+        this.internal.incrLoops(stepLabel);
+    }
+
+    @Override
+    public void resetLoops() {
+        this.internal.resetLoops();
+    }
+
+    @Override
+    public String getStepId() {
+        return this.internal.getStepId();
+    }
+
+    @Override
+    public void setStepId(final String stepId) {
+        this.internal.setStepId(stepId);
+    }
+
+    @Override
+    public void setBulk(final long count) {
+        this.internal.setBulk(count);
+    }
+
+    @Override
+    public Admin<T> detach() {
+        this.internal = this.internal.detach();
+        return this;
+    }
+
+    @Override
+    public T attach(final Function<Attachable<T>, T> method) {
+        return this.internal.attach(method);
+    }
+
+    @Override
+    public void setSideEffects(final TraversalSideEffects sideEffects) {
+        this.internal.setSideEffects(sideEffects);
+    }
+
+    @Override
+    public TraversalSideEffects getSideEffects() {
+        return this.internal.getSideEffects();
+    }
+
+    @Override
+    public Set<String> getTags() {
+        return this.internal.getTags();
+    }
+
+    @Override
+    public T get() {
+        return this.internal.get();
+    }
+
+    @Override
+    public <S> S sack() {
+        return this.internal.sack();
+    }
+
+    @Override
+    public <S> void sack(final S object) {
+        this.internal.sack(object);
+    }
+
+    @Override
+    public Path path() {
+        return this.internal.path();
+    }
+
+    @Override
+    public int loops() {
+        return this.internal.loops();
+    }
+
+    @Override
+    public long bulk() {
+        return this.internal.bulk();
+    }
+
+    @Override
+    public int hashCode() {
+        return this.internal.hashCode();
+    }
+
+    @Override
+    public boolean equals(final Object object) {
+        return object instanceof ProjectedTraverser && ((ProjectedTraverser) object).internal.equals(this.internal);
+    }
+
+    @Override
+    public ProjectedTraverser<T> clone() {
+        try {
+            final ProjectedTraverser<T> clone = (ProjectedTraverser<T>) super.clone();
+            clone.internal = (Traverser.Admin<T>) this.internal.clone();
+            return clone;
+        } catch (final CloneNotSupportedException e) {
+            throw new IllegalStateException(e.getMessage(), e);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/5045f67f/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
index a04f2d9..f4e31fd 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
@@ -51,13 +51,11 @@ import org.apache.tinkerpop.gremlin.process.traversal.traverser.LP_O_OB_P_S_SE_S
 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.process.traversal.traverser.OrderedTraverser;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.ProjectedTraverser;
 import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.TraverserSet;
-import org.apache.tinkerpop.gremlin.process.traversal.util.AndP;
 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.OrP;
 import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalExplanation;
 import org.apache.tinkerpop.gremlin.structure.Column;
 import org.apache.tinkerpop.gremlin.structure.Direction;
@@ -81,6 +79,7 @@ import org.apache.tinkerpop.gremlin.structure.util.star.StarGraph;
 import org.apache.tinkerpop.gremlin.structure.util.star.StarGraphSerializer;
 import org.apache.tinkerpop.gremlin.util.function.HashSetSupplier;
 import org.apache.tinkerpop.gremlin.util.function.Lambda;
+import org.apache.tinkerpop.gremlin.util.function.MultiComparator;
 import org.apache.tinkerpop.shaded.kryo.KryoSerializable;
 import org.apache.tinkerpop.shaded.kryo.serializers.JavaSerializer;
 import org.javatuples.Pair;
@@ -246,7 +245,7 @@ public enum GryoVersion {
             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(OrderedTraverser.class, 164));   // ***LAST ID***
+            add(GryoTypeReg.of(ProjectedTraverser.class, 164));
             add(GryoTypeReg.of(DefaultRemoteTraverser.class, 123, new GryoSerializers.DefaultRemoteTraverserSerializer()));
 
             add(GryoTypeReg.of(Bytecode.class, 122, new GryoSerializers.BytecodeSerializer()));
@@ -261,6 +260,7 @@ public enum GryoVersion {
             add(GryoTypeReg.of(SackFunctions.Barrier.class, 135));
             add(GryoTypeReg.of(TraversalOptionParent.Pick.class, 137));
             add(GryoTypeReg.of(HashSetSupplier.class, 136, new UtilSerializers.HashSetSupplierSerializer()));
+            add(GryoTypeReg.of(MultiComparator.class, 165));   // ***LAST ID***
 
             add(GryoTypeReg.of(TraverserSet.class, 58));
             add(GryoTypeReg.of(Tree.class, 61));

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/5045f67f/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java
new file mode 100644
index 0000000..427aa3d
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java
@@ -0,0 +1,60 @@
+/*
+ *  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.util.function;
+
+import org.apache.tinkerpop.gremlin.process.traversal.Order;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.ProjectedTraverser;
+
+import java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public final class MultiComparator<C> implements Comparator<C>, Serializable {
+
+    private final List<Comparator> comparators;
+
+    public MultiComparator(final List<Comparator<C>> comparators) {
+        this.comparators = (List) comparators;
+    }
+
+    @Override
+    public int compare(final C objectA, final C objectB) {
+        if (this.comparators.isEmpty()) {
+            return Order.incr.compare(objectA, objectB);
+        } else {
+            for (int i = 0; i < this.comparators.size(); i++) {
+                final int comparison = this.comparators.get(i).compare(this.getObject(objectA, i), this.getObject(objectB, i));
+                if (comparison != 0)
+                    return comparison;
+            }
+            return 0;
+        }
+    }
+
+    private final Object getObject(final C object, final int index) {
+        if (object instanceof ProjectedTraverser)
+            return ((ProjectedTraverser) object).getProjections().get(index);
+        else
+            return object;
+    }
+}


[25/31] tinkerpop git commit: TINKERPOP-1565 Added a 3.0 TraversalSerializer class

Posted by sp...@apache.org.
TINKERPOP-1565 Added a 3.0 TraversalSerializer class


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

Branch: refs/heads/TINKERPOP-1565
Commit: 9dc30cd5d29f98681f4d33abbf991574ed7a97d8
Parents: 7ca3a29
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Wed Jan 18 13:49:18 2017 -0500
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Jan 19 15:15:33 2017 -0500

----------------------------------------------------------------------
 .../structure/io/graphson/GraphSONModule.java   |  30 +-
 .../io/graphson/TraversalSerializersV3d0.java   | 390 +++++++++++++++++++
 2 files changed, 405 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/9dc30cd5/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java
index f4acf12..91df42a 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java
@@ -207,8 +207,8 @@ abstract class GraphSONModule extends TinkerPopJacksonModule {
             addSerializer(Double.class, new GraphSONSerializersV3d0.DoubleGraphSONSerializer());
 
             // traversal
-            addSerializer(Traversal.class, new GraphSONTraversalSerializersV2d0.TraversalJacksonSerializer());
-            addSerializer(Bytecode.class, new GraphSONTraversalSerializersV2d0.BytecodeJacksonSerializer());
+            addSerializer(Traversal.class, new TraversalSerializersV3d0.TraversalJacksonSerializer());
+            addSerializer(Bytecode.class, new TraversalSerializersV3d0.BytecodeJacksonSerializer());
             Stream.of(VertexProperty.Cardinality.class,
                     Column.class,
                     Direction.class,
@@ -218,12 +218,12 @@ abstract class GraphSONModule extends TinkerPopJacksonModule {
                     SackFunctions.Barrier.class,
                     Scope.class,
                     TraversalOptionParent.Pick.class,
-                    T.class).forEach(e -> addSerializer(e, new GraphSONTraversalSerializersV2d0.EnumJacksonSerializer()));
-            addSerializer(P.class, new GraphSONTraversalSerializersV2d0.PJacksonSerializer());
-            addSerializer(Lambda.class, new GraphSONTraversalSerializersV2d0.LambdaJacksonSerializer());
-            addSerializer(Bytecode.Binding.class, new GraphSONTraversalSerializersV2d0.BindingJacksonSerializer());
-            addSerializer(Traverser.class, new GraphSONTraversalSerializersV2d0.TraverserJacksonSerializer());
-            addSerializer(TraversalStrategy.class, new GraphSONTraversalSerializersV2d0.TraversalStrategyJacksonSerializer());
+                    T.class).forEach(e -> addSerializer(e, new TraversalSerializersV3d0.EnumJacksonSerializer()));
+            addSerializer(P.class, new TraversalSerializersV3d0.PJacksonSerializer());
+            addSerializer(Lambda.class, new TraversalSerializersV3d0.LambdaJacksonSerializer());
+            addSerializer(Bytecode.Binding.class, new TraversalSerializersV3d0.BindingJacksonSerializer());
+            addSerializer(Traverser.class, new TraversalSerializersV3d0.TraverserJacksonSerializer());
+            addSerializer(TraversalStrategy.class, new TraversalSerializersV3d0.TraversalStrategyJacksonSerializer());
 
             /////////////////////// DESERIALIZERS ////////////////////////////
 
@@ -242,8 +242,8 @@ abstract class GraphSONModule extends TinkerPopJacksonModule {
             addDeserializer(Double.class, new GraphSONSerializersV3d0.DoubleJackonsDeserializer());
 
             // traversal
-            addDeserializer(Bytecode.class, new GraphSONTraversalSerializersV2d0.BytecodeJacksonDeserializer());
-            addDeserializer(Bytecode.Binding.class, new GraphSONTraversalSerializersV2d0.BindingJacksonDeserializer());
+            addDeserializer(Bytecode.class, new TraversalSerializersV3d0.BytecodeJacksonDeserializer());
+            addDeserializer(Bytecode.Binding.class, new TraversalSerializersV3d0.BindingJacksonDeserializer());
             Stream.of(VertexProperty.Cardinality.values(),
                     Column.values(),
                     Direction.values(),
@@ -253,10 +253,10 @@ abstract class GraphSONModule extends TinkerPopJacksonModule {
                     SackFunctions.Barrier.values(),
                     Scope.values(),
                     TraversalOptionParent.Pick.values(),
-                    T.values()).flatMap(Stream::of).forEach(e -> addDeserializer(e.getClass(), new GraphSONTraversalSerializersV2d0.EnumJacksonDeserializer(e.getDeclaringClass())));
-            addDeserializer(P.class, new GraphSONTraversalSerializersV2d0.PJacksonDeserializer());
-            addDeserializer(Lambda.class, new GraphSONTraversalSerializersV2d0.LambdaJacksonDeserializer());
-            addDeserializer(Traverser.class, new GraphSONTraversalSerializersV2d0.TraverserJacksonDeserializer());
+                    T.values()).flatMap(Stream::of).forEach(e -> addDeserializer(e.getClass(), new TraversalSerializersV3d0.EnumJacksonDeserializer(e.getDeclaringClass())));
+            addDeserializer(P.class, new TraversalSerializersV3d0.PJacksonDeserializer());
+            addDeserializer(Lambda.class, new TraversalSerializersV3d0.LambdaJacksonDeserializer());
+            addDeserializer(Traverser.class, new TraversalSerializersV3d0.TraverserJacksonDeserializer());
             Arrays.asList(
                     ConnectiveStrategy.class,
                     ElementIdStrategy.class,
@@ -284,7 +284,7 @@ abstract class GraphSONModule extends TinkerPopJacksonModule {
                     //
                     GraphFilterStrategy.class,
                     VertexProgramStrategy.class
-            ).forEach(strategy -> addDeserializer(strategy, new GraphSONTraversalSerializersV2d0.TraversalStrategyProxyJacksonDeserializer(strategy)));
+            ).forEach(strategy -> addDeserializer(strategy, new TraversalSerializersV3d0.TraversalStrategyProxyJacksonDeserializer(strategy)));
         }
 
         public static Builder build() {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/9dc30cd5/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/TraversalSerializersV3d0.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/TraversalSerializersV3d0.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/TraversalSerializersV3d0.java
new file mode 100644
index 0000000..e4f6df9
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/TraversalSerializersV3d0.java
@@ -0,0 +1,390 @@
+/*
+ *  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.graphson;
+
+import org.apache.commons.configuration.ConfigurationConverter;
+import org.apache.commons.configuration.MapConfiguration;
+import org.apache.tinkerpop.gremlin.process.remote.traversal.DefaultRemoteTraverser;
+import org.apache.tinkerpop.gremlin.process.traversal.Bytecode;
+import org.apache.tinkerpop.gremlin.process.traversal.P;
+import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
+import org.apache.tinkerpop.gremlin.process.traversal.TraversalStrategy;
+import org.apache.tinkerpop.gremlin.process.traversal.Traverser;
+import org.apache.tinkerpop.gremlin.process.traversal.strategy.TraversalStrategyProxy;
+import org.apache.tinkerpop.gremlin.process.traversal.util.AndP;
+import org.apache.tinkerpop.gremlin.process.traversal.util.ConnectiveP;
+import org.apache.tinkerpop.gremlin.process.traversal.util.OrP;
+import org.apache.tinkerpop.gremlin.util.function.Lambda;
+import org.apache.tinkerpop.shaded.jackson.core.JsonGenerator;
+import org.apache.tinkerpop.shaded.jackson.core.JsonParser;
+import org.apache.tinkerpop.shaded.jackson.core.JsonProcessingException;
+import org.apache.tinkerpop.shaded.jackson.databind.DeserializationContext;
+import org.apache.tinkerpop.shaded.jackson.databind.SerializerProvider;
+import org.apache.tinkerpop.shaded.jackson.databind.deser.std.StdDeserializer;
+import org.apache.tinkerpop.shaded.jackson.databind.jsontype.TypeSerializer;
+import org.apache.tinkerpop.shaded.jackson.databind.ser.std.StdScalarSerializer;
+import org.apache.tinkerpop.shaded.jackson.databind.ser.std.StdSerializer;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+final class TraversalSerializersV3d0 {
+
+    private TraversalSerializersV3d0() {
+    }
+
+    /////////////////
+    // SERIALIZERS //
+    ////////////////
+
+    final static class TraversalJacksonSerializer extends StdSerializer<Traversal> {
+
+        public TraversalJacksonSerializer() {
+            super(Traversal.class);
+        }
+
+        @Override
+        public void serialize(final Traversal traversal, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+                throws IOException {
+            jsonGenerator.writeObject(traversal.asAdmin().getBytecode());
+        }
+
+        @Override
+        public void serializeWithType(final Traversal traversal, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider, final TypeSerializer typeSerializer)
+                throws IOException {
+            serialize(traversal, jsonGenerator, serializerProvider);
+        }
+
+    }
+
+    final static class BytecodeJacksonSerializer extends StdScalarSerializer<Bytecode> {
+
+        public BytecodeJacksonSerializer() {
+            super(Bytecode.class);
+        }
+
+        @Override
+        public void serialize(final Bytecode bytecode, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+                throws IOException {
+            jsonGenerator.writeStartObject();
+            if (bytecode.getSourceInstructions().iterator().hasNext()) {
+                jsonGenerator.writeArrayFieldStart(GraphSONTokens.SOURCE);
+                for (final Bytecode.Instruction instruction : bytecode.getSourceInstructions()) {
+                    jsonGenerator.writeStartArray();
+                    jsonGenerator.writeString(instruction.getOperator());
+                    for (final Object argument : instruction.getArguments()) {
+                        jsonGenerator.writeObject(argument);
+                    }
+                    jsonGenerator.writeEndArray();
+                }
+                jsonGenerator.writeEndArray();
+            }
+            if (bytecode.getStepInstructions().iterator().hasNext()) {
+                jsonGenerator.writeArrayFieldStart(GraphSONTokens.STEP);
+                for (final Bytecode.Instruction instruction : bytecode.getStepInstructions()) {
+                    jsonGenerator.writeStartArray();
+                    jsonGenerator.writeString(instruction.getOperator());
+                    for (final Object argument : instruction.getArguments()) {
+                        jsonGenerator.writeObject(argument);
+                    }
+                    jsonGenerator.writeEndArray();
+                }
+                jsonGenerator.writeEndArray();
+            }
+
+            jsonGenerator.writeEndObject();
+        }
+    }
+
+    static class EnumJacksonSerializer extends StdScalarSerializer<Enum> {
+
+        public EnumJacksonSerializer() {
+            super(Enum.class);
+        }
+
+        @Override
+        public void serialize(final Enum enumInstance, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+                throws IOException {
+            jsonGenerator.writeString(enumInstance.name());
+        }
+
+    }
+
+    final static class PJacksonSerializer extends StdScalarSerializer<P> {
+
+        public PJacksonSerializer() {
+            super(P.class);
+        }
+
+        @Override
+        public void serialize(final P p, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+                throws IOException {
+            jsonGenerator.writeStartObject();
+            jsonGenerator.writeStringField(GraphSONTokens.PREDICATE,
+                    p instanceof ConnectiveP ?
+                            p instanceof AndP ?
+                                    GraphSONTokens.AND :
+                                    GraphSONTokens.OR :
+                            p.getBiPredicate().toString());
+            if (p instanceof ConnectiveP) {
+                jsonGenerator.writeArrayFieldStart(GraphSONTokens.VALUE);
+                for (final P<?> predicate : ((ConnectiveP<?>) p).getPredicates()) {
+                    jsonGenerator.writeObject(predicate);
+                }
+                jsonGenerator.writeEndArray();
+            } else {
+                if (p.getValue() instanceof Collection) {
+                    jsonGenerator.writeArrayFieldStart(GraphSONTokens.VALUE);
+                    for (final Object object : (Collection) p.getValue()) {
+                        jsonGenerator.writeObject(object);
+                    }
+                    jsonGenerator.writeEndArray();
+                } else
+                    jsonGenerator.writeObjectField(GraphSONTokens.VALUE, p.getValue());
+            }
+            jsonGenerator.writeEndObject();
+        }
+
+    }
+
+    final static class LambdaJacksonSerializer extends StdScalarSerializer<Lambda> {
+
+        public LambdaJacksonSerializer() {
+            super(Lambda.class);
+        }
+
+        @Override
+        public void serialize(final Lambda lambda, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+                throws IOException {
+            jsonGenerator.writeStartObject();
+            jsonGenerator.writeStringField(GraphSONTokens.SCRIPT, lambda.getLambdaScript());
+            jsonGenerator.writeStringField(GraphSONTokens.LANGUAGE, lambda.getLambdaLanguage());
+            jsonGenerator.writeNumberField(GraphSONTokens.ARGUMENTS, lambda.getLambdaArguments());
+            jsonGenerator.writeEndObject();
+        }
+
+    }
+
+    final static class BindingJacksonSerializer extends StdScalarSerializer<Bytecode.Binding> {
+
+        public BindingJacksonSerializer() {
+            super(Bytecode.Binding.class);
+        }
+
+        @Override
+        public void serialize(final Bytecode.Binding binding, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+                throws IOException {
+            jsonGenerator.writeStartObject();
+            jsonGenerator.writeStringField(GraphSONTokens.KEY, binding.variable());
+            jsonGenerator.writeObjectField(GraphSONTokens.VALUE, binding.value());
+            jsonGenerator.writeEndObject();
+        }
+
+    }
+
+    final static class TraverserJacksonSerializer extends StdScalarSerializer<Traverser> {
+
+        public TraverserJacksonSerializer() {
+            super(Traverser.class);
+        }
+
+        @Override
+        public void serialize(final Traverser traverserInstance, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+                throws IOException {
+            jsonGenerator.writeStartObject();
+            jsonGenerator.writeObjectField(GraphSONTokens.BULK, traverserInstance.bulk());
+            jsonGenerator.writeObjectField(GraphSONTokens.VALUE, traverserInstance.get());
+            jsonGenerator.writeEndObject();
+        }
+    }
+
+    final static class TraversalStrategyJacksonSerializer extends StdScalarSerializer<TraversalStrategy> {
+
+        public TraversalStrategyJacksonSerializer() {
+            super(TraversalStrategy.class);
+        }
+
+        @Override
+        public void serialize(final TraversalStrategy traversalStrategy, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+                throws IOException {
+            jsonGenerator.writeStartObject();
+            for (final Map.Entry<Object, Object> entry : ConfigurationConverter.getMap(traversalStrategy.getConfiguration()).entrySet()) {
+                jsonGenerator.writeObjectField((String) entry.getKey(), entry.getValue());
+            }
+            jsonGenerator.writeEndObject();
+        }
+    }
+
+    ///////////////////
+    // DESERIALIZERS //
+    //////////////////
+
+    final static class BytecodeJacksonDeserializer extends AbstractObjectDeserializer<Bytecode> {
+
+        public BytecodeJacksonDeserializer() {
+            super(Bytecode.class);
+        }
+
+        @Override
+        public Bytecode createObject(final Map<String, Object> data) {
+            final Bytecode bytecode = new Bytecode();
+            if (data.containsKey(GraphSONTokens.SOURCE)) {
+                final List<List<Object>> instructions = (List) data.get(GraphSONTokens.SOURCE);
+                for (final List<Object> instruction : instructions) {
+                    bytecode.addSource((String) instruction.get(0), Arrays.copyOfRange(instruction.toArray(), 1, instruction.size()));
+                }
+            }
+            if (data.containsKey(GraphSONTokens.STEP)) {
+                final List<List<Object>> instructions = (List) data.get(GraphSONTokens.STEP);
+                for (final List<Object> instruction : instructions) {
+                    bytecode.addStep((String) instruction.get(0), Arrays.copyOfRange(instruction.toArray(), 1, instruction.size()));
+                }
+            }
+            return bytecode;
+        }
+    }
+
+    final static class EnumJacksonDeserializer<A extends Enum> extends StdDeserializer<A> {
+
+        public EnumJacksonDeserializer(final Class<A> enumClass) {
+            super(enumClass);
+        }
+
+        @Override
+        public A deserialize(final JsonParser jsonParser, final DeserializationContext deserializationContext) throws IOException, JsonProcessingException {
+            final Class<A> enumClass = (Class<A>) this._valueClass;
+            final String enumName = jsonParser.getText();
+            for (final Enum a : enumClass.getEnumConstants()) {
+                if (a.name().equals(enumName))
+                    return (A) a;
+            }
+            throw new IOException("Unknown enum type: " + enumClass);
+        }
+    }
+
+    final static class PJacksonDeserializer extends AbstractObjectDeserializer<P> {
+
+        public PJacksonDeserializer() {
+            super(P.class);
+        }
+
+        @Override
+        public P createObject(final Map<String, Object> data) {
+            final String predicate = (String) data.get(GraphSONTokens.PREDICATE);
+            final Object value = data.get(GraphSONTokens.VALUE);
+            if (predicate.equals(GraphSONTokens.AND) || predicate.equals(GraphSONTokens.OR)) {
+                return predicate.equals(GraphSONTokens.AND) ? new AndP((List<P>) value) : new OrP((List<P>) value);
+            } else {
+                try {
+                    if (value instanceof Collection) {
+                        if (predicate.equals("between"))
+                            return P.between(((List) value).get(0), ((List) value).get(1));
+                        else if (predicate.equals("inside"))
+                            return P.between(((List) value).get(0), ((List) value).get(1));
+                        else if (predicate.equals("outside"))
+                            return P.outside(((List) value).get(0), ((List) value).get(1));
+                        else if (predicate.equals("within"))
+                            return P.within((Collection) value);
+                        else if (predicate.equals("without"))
+                            return P.without((Collection) value);
+                        else
+                            return (P) P.class.getMethod(predicate, Collection.class).invoke(null, (Collection) value);
+                    } else {
+                        try {
+                            return (P) P.class.getMethod(predicate, Object.class).invoke(null, value);
+                        } catch (final NoSuchMethodException e) {
+                            return (P) P.class.getMethod(predicate, Object[].class).invoke(null, (Object) new Object[]{value});
+                        }
+                    }
+                } catch (final Exception e) {
+                    throw new IllegalStateException(e.getMessage(), e);
+                }
+            }
+        }
+    }
+
+    final static class LambdaJacksonDeserializer extends AbstractObjectDeserializer<Lambda> {
+
+        public LambdaJacksonDeserializer() {
+            super(Lambda.class);
+        }
+
+        @Override
+        public Lambda createObject(final Map<String, Object> data) {
+            final String script = (String) data.get(GraphSONTokens.SCRIPT);
+            final String language = (String) data.get(GraphSONTokens.LANGUAGE);
+            final int arguments = ((Number) data.getOrDefault(GraphSONTokens.ARGUMENTS, -1)).intValue();
+            //
+            if (-1 == arguments || arguments > 2)
+                return new Lambda.UnknownArgLambda(script, language, arguments);
+            else if (0 == arguments)
+                return new Lambda.ZeroArgLambda<>(script, language);
+            else if (1 == arguments)
+                return new Lambda.OneArgLambda<>(script, language);
+            else
+                return new Lambda.TwoArgLambda<>(script, language);
+        }
+    }
+
+    final static class BindingJacksonDeserializer extends AbstractObjectDeserializer<Bytecode.Binding> {
+
+        public BindingJacksonDeserializer() {
+            super(Bytecode.Binding.class);
+        }
+
+        @Override
+        public Bytecode.Binding createObject(final Map<String, Object> data) {
+            return new Bytecode.Binding<>((String) data.get(GraphSONTokens.KEY), data.get(GraphSONTokens.VALUE));
+        }
+    }
+
+    static class TraverserJacksonDeserializer extends AbstractObjectDeserializer<Traverser> {
+
+        public TraverserJacksonDeserializer() {
+            super(Traverser.class);
+        }
+
+        @Override
+        public Traverser createObject(final Map<String, Object> data) {
+            return new DefaultRemoteTraverser<>(data.get(GraphSONTokens.VALUE), (Long) data.get(GraphSONTokens.BULK));
+        }
+    }
+
+    final static class TraversalStrategyProxyJacksonDeserializer<T extends TraversalStrategy> extends AbstractObjectDeserializer<TraversalStrategyProxy> {
+
+        private final Class<T> clazz;
+
+        public TraversalStrategyProxyJacksonDeserializer(final Class<T> clazz) {
+            super(TraversalStrategyProxy.class);
+            this.clazz = clazz;
+        }
+
+        @Override
+        public TraversalStrategyProxy<T> createObject(final Map<String, Object> data) {
+            return new TraversalStrategyProxy<>(this.clazz, new MapConfiguration(data));
+        }
+    }
+}


[07/31] tinkerpop git commit: moved all the GroupStep work against tp32/

Posted by sp...@apache.org.
moved all the GroupStep work against tp32/


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

Branch: refs/heads/TINKERPOP-1565
Commit: 3496402a4e0c2803031d3b88086aabd5c6a2cfd8
Parents: 97cc07d
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Thu Jan 19 04:16:56 2017 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Thu Jan 19 04:16:56 2017 -0700

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |   1 +
 .../process/traversal/step/map/GroupStep.java   | 263 +++----------------
 .../step/sideEffect/GroupSideEffectStep.java    |  50 ++--
 .../step/sideEffect/GroovyGroupTest.groovy      |   5 +
 .../traversal/step/sideEffect/GroupTest.java    |  30 ++-
 5 files changed, 91 insertions(+), 258 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/3496402a/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index 4f3f9ce..74751fa 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -26,6 +26,7 @@ image::https://raw.githubusercontent.com/apache/tinkerpop/master/docs/static/ima
 TinkerPop 3.2.4 (Release Date: NOT OFFICIALLY RELEASED YET)
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
+* `GroupBiOperator` no longer maintains state and thus, no more side-effect related OLAP inconsistencies.
 * SASL negotiation supports both a byte array and Base64 encoded bytes as a string for authentication to Gremlin Server.
 * Deprecated `TinkerIoRegistry` replacing it with the more consistently named `TinkerIoRegistryV1d0`.
 * Made error messaging more consistent during result iteration timeouts in Gremlin Server.

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/3496402a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/GroupStep.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/GroupStep.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/GroupStep.java
index d6ce421..07ca4ae 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/GroupStep.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/GroupStep.java
@@ -19,7 +19,7 @@
 
 package org.apache.tinkerpop.gremlin.process.traversal.step.map;
 
-import org.apache.tinkerpop.gremlin.process.traversal.Step;
+import org.apache.tinkerpop.gremlin.process.traversal.Operator;
 import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
 import org.apache.tinkerpop.gremlin.process.traversal.Traverser;
 import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__;
@@ -29,22 +29,14 @@ import org.apache.tinkerpop.gremlin.process.traversal.lambda.IdentityTraversal;
 import org.apache.tinkerpop.gremlin.process.traversal.lambda.TokenTraversal;
 import org.apache.tinkerpop.gremlin.process.traversal.step.Barrier;
 import org.apache.tinkerpop.gremlin.process.traversal.step.ByModulating;
-import org.apache.tinkerpop.gremlin.process.traversal.step.LambdaHolder;
 import org.apache.tinkerpop.gremlin.process.traversal.step.TraversalParent;
-import org.apache.tinkerpop.gremlin.process.traversal.step.util.EmptyStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.util.ReducingBarrierStep;
 import org.apache.tinkerpop.gremlin.process.traversal.traverser.TraverserRequirement;
-import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.TraverserSet;
 import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalHelper;
 import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalUtil;
 import org.apache.tinkerpop.gremlin.structure.util.StringFactory;
 import org.apache.tinkerpop.gremlin.util.function.HashMapSupplier;
-import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
-import org.javatuples.Pair;
 
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -60,14 +52,14 @@ public final class GroupStep<S, K, V> extends ReducingBarrierStep<S, Map<K, V>>
 
     private char state = 'k';
     private Traversal.Admin<S, K> keyTraversal;
-    private Traversal.Admin<S, ?> preTraversal;
     private Traversal.Admin<S, V> valueTraversal;
+    private Barrier barrierStep;
 
     public GroupStep(final Traversal.Admin traversal) {
         super(traversal);
         this.valueTraversal = this.integrateChild(__.fold().asAdmin());
-        this.preTraversal = this.integrateChild(generatePreTraversal(this.valueTraversal));
-        this.setReducingBiOperator(new GroupBiOperator<>(this.valueTraversal));
+        this.barrierStep = TraversalHelper.getFirstStepOfAssignableClass(Barrier.class, this.valueTraversal).orElse(null);
+        this.setReducingBiOperator(new GroupBiOperator<>(null == this.barrierStep ? Operator.assign : this.barrierStep.getMemoryComputeKey().getReducer()));
         this.setSeedSupplier(HashMapSupplier.instance());
     }
 
@@ -78,8 +70,8 @@ public final class GroupStep<S, K, V> extends ReducingBarrierStep<S, Map<K, V>>
             this.state = 'v';
         } else if ('v' == this.state) {
             this.valueTraversal = this.integrateChild(convertValueTraversal(kvTraversal));
-            this.preTraversal = this.integrateChild(generatePreTraversal(this.valueTraversal));
-            this.setReducingBiOperator(new GroupBiOperator<>(this.valueTraversal));
+            this.barrierStep = TraversalHelper.getFirstStepOfAssignableClass(Barrier.class, this.valueTraversal).orElse(null);
+            this.setReducingBiOperator(new GroupBiOperator<>(null == this.barrierStep ? Operator.assign : this.barrierStep.getMemoryComputeKey().getReducer()));
             this.state = 'x';
         } else {
             throw new IllegalStateException("The key and value traversals for group()-step have already been set: " + this);
@@ -89,17 +81,13 @@ public final class GroupStep<S, K, V> extends ReducingBarrierStep<S, Map<K, V>>
     @Override
     public Map<K, V> projectTraverser(final Traverser.Admin<S> traverser) {
         final Map<K, V> map = new HashMap<>(1);
-        if (null == this.preTraversal) {
-            map.put(TraversalUtil.applyNullable(traverser, this.keyTraversal), (V) traverser);
-        } else {
-            final TraverserSet traverserSet = new TraverserSet<>();
-            this.preTraversal.reset();
-            this.preTraversal.addStart(traverser);
-            while (this.preTraversal.hasNext()) {
-                traverserSet.add(this.preTraversal.nextTraverser());
-            }
-            map.put(TraversalUtil.applyNullable(traverser, this.keyTraversal), (V) traverserSet);
-        }
+        this.valueTraversal.reset();
+        this.valueTraversal.addStart(traverser);
+        if (null == this.barrierStep) {
+            if (this.valueTraversal.hasNext())
+                map.put(TraversalUtil.applyNullable(traverser, this.keyTraversal), (V) this.valueTraversal.next());
+        } else if (this.barrierStep.hasNextBarrier())
+            map.put(TraversalUtil.applyNullable(traverser, this.keyTraversal), (V) this.barrierStep.nextBarrier());
         return map;
     }
 
@@ -110,12 +98,10 @@ public final class GroupStep<S, K, V> extends ReducingBarrierStep<S, Map<K, V>>
 
     @Override
     public List<Traversal.Admin<?, ?>> getLocalChildren() {
-        final List<Traversal.Admin<?, ?>> children = new ArrayList<>(3);
+        final List<Traversal.Admin<?, ?>> children = new ArrayList<>(2);
         if (null != this.keyTraversal)
             children.add(this.keyTraversal);
         children.add(this.valueTraversal);
-        if (null != this.preTraversal)
-            children.add(this.preTraversal);
         return children;
     }
 
@@ -130,8 +116,7 @@ public final class GroupStep<S, K, V> extends ReducingBarrierStep<S, Map<K, V>>
         if (null != this.keyTraversal)
             clone.keyTraversal = this.keyTraversal.clone();
         clone.valueTraversal = this.valueTraversal.clone();
-        clone.preTraversal = (Traversal.Admin<S, ?>) GroupStep.generatePreTraversal(clone.valueTraversal);
-        clone.setReducingBiOperator(new GroupBiOperator<>(clone.valueTraversal));
+        clone.barrierStep = TraversalHelper.getFirstStepOfAssignableClass(Barrier.class, clone.valueTraversal).orElse(null);
         return clone;
     }
 
@@ -140,7 +125,6 @@ public final class GroupStep<S, K, V> extends ReducingBarrierStep<S, Map<K, V>>
         super.setTraversal(parentTraversal);
         integrateChild(this.keyTraversal);
         integrateChild(this.valueTraversal);
-        integrateChild(this.preTraversal);
     }
 
     @Override
@@ -158,180 +142,31 @@ public final class GroupStep<S, K, V> extends ReducingBarrierStep<S, Map<K, V>>
 
     ///////////////////////
 
-    public static final class GroupBiOperator<K, V> implements BinaryOperator<Map<K, V>>, Serializable, Cloneable {
-
-        // size limit before Barrier.processAllStarts() to lazy reduce
-        private static final int SIZE_LIMIT = 1000;
-
-        private Traversal.Admin<?, V> valueTraversal;
-        private Barrier barrierStep;
+    public static final class GroupBiOperator<K, V> implements BinaryOperator<Map<K, V>>, Serializable {
 
-        public GroupBiOperator(final Traversal.Admin<?, V> valueTraversal) {
-            // if there is a lambda that can not be serialized, then simply use TraverserSets
-            if (TraversalHelper.hasStepOfAssignableClassRecursively(LambdaHolder.class, valueTraversal)) {
-                this.valueTraversal = null;
-                this.barrierStep = null;
-            } else {
-                this.valueTraversal = valueTraversal.clone();
-                this.barrierStep = TraversalHelper.getFirstStepOfAssignableClass(Barrier.class, this.valueTraversal).orElse(null);
-            }
-        }
+        private BinaryOperator<V> barrierAggregator;
 
         public GroupBiOperator() {
             // no-arg constructor for serialization
         }
 
-        @Override
-        public GroupBiOperator<K, V> clone() {
-            try {
-                final GroupBiOperator<K, V> clone = (GroupBiOperator<K, V>) super.clone();
-                if (null != this.valueTraversal) {
-                    clone.valueTraversal = this.valueTraversal.clone();
-                    clone.barrierStep = TraversalHelper.getFirstStepOfAssignableClass(Barrier.class, clone.valueTraversal).orElse(null);
-                }
-                return clone;
-            } catch (final CloneNotSupportedException e) {
-                throw new IllegalStateException(e.getMessage(), e);
-            }
+        public GroupBiOperator(final BinaryOperator<V> barrierAggregator) {
+            this.barrierAggregator = barrierAggregator;
         }
 
         @Override
         public Map<K, V> apply(final Map<K, V> mapA, final Map<K, V> mapB) {
             for (final K key : mapB.keySet()) {
-                Object objectA = mapA.get(key);
-                final Object objectB = mapB.get(key);
-                assert null != objectB;
-                if (null == objectA) {
+                V objectA = mapA.get(key);
+                final V objectB = mapB.get(key);
+                if (null == objectA)
                     objectA = objectB;
-                } else {
-                    // TRAVERSER
-                    if (objectA instanceof Traverser.Admin) {
-                        if (objectB instanceof Traverser.Admin) {
-                            final TraverserSet set = new TraverserSet();
-                            set.add((Traverser.Admin) objectA);
-                            set.add((Traverser.Admin) objectB);
-                            objectA = set;
-                        } else if (objectB instanceof TraverserSet) {
-                            final TraverserSet set = (TraverserSet) objectB;
-                            set.add((Traverser.Admin) objectA);
-                            if (null != this.barrierStep && set.size() > SIZE_LIMIT) {
-                                this.valueTraversal.reset();
-                                ((Step) this.barrierStep).addStarts(set.iterator());
-                                objectA = this.barrierStep.nextBarrier();
-                            } else
-                                objectA = objectB;
-                        } else if (objectB instanceof Pair) {
-                            final TraverserSet set = (TraverserSet) ((Pair) objectB).getValue0();
-                            set.add((Traverser.Admin) objectA);
-                            if (set.size() > SIZE_LIMIT) {    // barrier step can never be null -- no need to check
-                                this.valueTraversal.reset();
-                                ((Step) this.barrierStep).addStarts(set.iterator());
-                                this.barrierStep.addBarrier(((Pair) objectB).getValue1());
-                                objectA = this.barrierStep.nextBarrier();
-                            } else
-                                objectA = Pair.with(set, ((Pair) objectB).getValue1());
-                        } else
-                            objectA = Pair.with(new TraverserSet((Traverser.Admin) objectA), objectB);
-                        // TRAVERSER SET
-                    } else if (objectA instanceof TraverserSet) {
-                        if (objectB instanceof Traverser.Admin) {
-                            final TraverserSet set = (TraverserSet) objectA;
-                            set.add((Traverser.Admin) objectB);
-                            if (null != this.barrierStep && set.size() > SIZE_LIMIT) {
-                                this.valueTraversal.reset();
-                                ((Step) this.barrierStep).addStarts(set.iterator());
-                                objectA = this.barrierStep.nextBarrier();
-                            }
-                        } else if (objectB instanceof TraverserSet) {
-                            final TraverserSet set = (TraverserSet) objectA;
-                            set.addAll((TraverserSet) objectB);
-                            if (null != this.barrierStep && set.size() > SIZE_LIMIT) {
-                                this.valueTraversal.reset();
-                                ((Step) this.barrierStep).addStarts(set.iterator());
-                                objectA = this.barrierStep.nextBarrier();
-                            }
-                        } else if (objectB instanceof Pair) {
-                            final TraverserSet set = (TraverserSet) objectA;
-                            set.addAll((TraverserSet) ((Pair) objectB).getValue0());
-                            if (set.size() > SIZE_LIMIT) {  // barrier step can never be null -- no need to check
-                                this.valueTraversal.reset();
-                                ((Step) this.barrierStep).addStarts(set.iterator());
-                                this.barrierStep.addBarrier(((Pair) objectB).getValue1());
-                                objectA = this.barrierStep.nextBarrier();
-                            } else
-                                objectA = Pair.with(set, ((Pair) objectB).getValue1());
-                        } else
-                            objectA = Pair.with(objectA, objectB);
-                        // TRAVERSER SET + BARRIER
-                    } else if (objectA instanceof Pair) {
-                        if (objectB instanceof Traverser.Admin) {
-                            final TraverserSet set = ((TraverserSet) ((Pair) objectA).getValue0());
-                            set.add((Traverser.Admin) objectB);
-                            if (set.size() > SIZE_LIMIT) { // barrier step can never be null -- no need to check
-                                this.valueTraversal.reset();
-                                ((Step) this.barrierStep).addStarts(set.iterator());
-                                this.barrierStep.addBarrier(((Pair) objectA).getValue1());
-                                objectA = this.barrierStep.nextBarrier();
-                            }
-                        } else if (objectB instanceof TraverserSet) {
-                            final TraverserSet set = (TraverserSet) ((Pair) objectA).getValue0();
-                            set.addAll((TraverserSet) objectB);
-                            if (set.size() > SIZE_LIMIT) {   // barrier step can never be null -- no need to check
-                                this.valueTraversal.reset();
-                                ((Step) this.barrierStep).addStarts(set.iterator());
-                                this.barrierStep.addBarrier(((Pair) objectA).getValue1());
-                                objectA = this.barrierStep.nextBarrier();
-                            }
-                        } else if (objectB instanceof Pair) {
-                            this.valueTraversal.reset();
-                            this.barrierStep.addBarrier(((Pair) objectA).getValue1());
-                            this.barrierStep.addBarrier(((Pair) objectB).getValue1());
-                            ((Step) this.barrierStep).addStarts(((TraverserSet) ((Pair) objectA).getValue0()).iterator());
-                            ((Step) this.barrierStep).addStarts(((TraverserSet) ((Pair) objectB).getValue0()).iterator());
-                            objectA = this.barrierStep.nextBarrier();
-                        } else {
-                            this.valueTraversal.reset();
-                            this.barrierStep.addBarrier(((Pair) objectA).getValue1());
-                            this.barrierStep.addBarrier(objectB);
-                            ((Step) this.barrierStep).addStarts(((TraverserSet) ((Pair) objectA).getValue0()).iterator());
-                            objectA = this.barrierStep.nextBarrier();
-                        }
-                        // BARRIER
-                    } else {
-                        if (objectB instanceof Traverser.Admin) {
-                            objectA = Pair.with(new TraverserSet<>((Traverser.Admin) objectB), objectA);
-                        } else if (objectB instanceof TraverserSet) {
-                            objectA = Pair.with(objectB, objectA);
-                        } else if (objectB instanceof Pair) {
-                            this.valueTraversal.reset();
-                            this.barrierStep.addBarrier(objectA);
-                            this.barrierStep.addBarrier(((Pair) objectB).getValue1());
-                            ((Step) this.barrierStep).addStarts(((TraverserSet) ((Pair) objectB).getValue0()).iterator());
-                            objectA = this.barrierStep.nextBarrier();
-                        } else {
-                            this.valueTraversal.reset();
-                            this.barrierStep.addBarrier(objectA);
-                            this.barrierStep.addBarrier(objectB);
-                            objectA = this.barrierStep.nextBarrier();
-                        }
-                    }
-                }
-                mapA.put(key, (V) objectA);
+                else if (null != objectB)
+                    objectA = this.barrierAggregator.apply(objectA, objectB);
+                mapA.put(key, objectA);
             }
             return mapA;
         }
-
-        // necessary to control Java Serialization to ensure proper clearing of internal traverser data
-        private void writeObject(final ObjectOutputStream outputStream) throws IOException {
-            // necessary as a non-root child is being sent over the wire
-            if (null != this.valueTraversal) this.valueTraversal.setParent(EmptyStep.instance());
-            outputStream.writeObject(null == this.valueTraversal ? null : this.valueTraversal.clone()); // todo: reset() instead?
-        }
-
-        private void readObject(final ObjectInputStream inputStream) throws IOException, ClassNotFoundException {
-            this.valueTraversal = (Traversal.Admin<?, V>) inputStream.readObject();
-            this.barrierStep = null == this.valueTraversal ? null : TraversalHelper.getFirstStepOfAssignableClass(Barrier.class, this.valueTraversal).orElse(null);
-        }
     }
 
 
@@ -343,55 +178,19 @@ public final class GroupStep<S, K, V> extends ReducingBarrierStep<S, Map<K, V>>
                 valueTraversal instanceof IdentityTraversal ||
                 valueTraversal.getStartStep() instanceof LambdaMapStep && ((LambdaMapStep) valueTraversal.getStartStep()).getMapFunction() instanceof FunctionTraverser) {
             return (Traversal.Admin<S, E>) __.map(valueTraversal).fold();
-        } else {
+        } else
             return valueTraversal;
-        }
-    }
-
-    public static Traversal.Admin<?, ?> generatePreTraversal(final Traversal.Admin<?, ?> valueTraversal) {
-        if (!TraversalHelper.hasStepOfAssignableClass(Barrier.class, valueTraversal))
-            return valueTraversal.clone();
-        final Traversal.Admin<?, ?> first = __.identity().asAdmin();
-        boolean updated = false;
-        for (final Step step : valueTraversal.getSteps()) {
-            if (step instanceof Barrier)
-                break;
-            first.addStep(step.clone());
-            updated = true;
-        }
-        return updated ? first : null;
     }
 
     public static <K, V> Map<K, V> doFinalReduction(final Map<K, Object> map, final Traversal.Admin<?, V> valueTraversal) {
-        final Map<K, V> reducedMap = new HashMap<>(map.size());
-        final Barrier reducingBarrierStep = TraversalHelper.getFirstStepOfAssignableClass(Barrier.class, valueTraversal).orElse(null);
-        IteratorUtils.removeOnNext(map.entrySet().iterator()).forEachRemaining(entry -> {
-            if (null == reducingBarrierStep) {
-                if (entry.getValue() instanceof TraverserSet) {
-                    if (!((TraverserSet) entry.getValue()).isEmpty())
-                        reducedMap.put(entry.getKey(), ((TraverserSet<V>) entry.getValue()).peek().get());
-                } else
-                    reducedMap.put(entry.getKey(), (V) entry.getValue());
-            } else {
+        TraversalHelper.getFirstStepOfAssignableClass(Barrier.class, valueTraversal).ifPresent(barrierStep -> {
+            for (final K key : map.keySet()) {
                 valueTraversal.reset();
-                if (entry.getValue() instanceof Traverser.Admin)
-                    ((Step) reducingBarrierStep).addStart((Traverser.Admin) entry.getValue());
-                else if (entry.getValue() instanceof TraverserSet)
-                    ((Step) reducingBarrierStep).addStarts(((TraverserSet) entry.getValue()).iterator());
-                else if (entry.getValue() instanceof Pair) {
-                    ((Step) reducingBarrierStep).addStarts(((TraverserSet) (((Pair) entry.getValue()).getValue0())).iterator());
-                    reducingBarrierStep.addBarrier((((Pair) entry.getValue()).getValue1()));
-                } else
-                    reducingBarrierStep.addBarrier(entry.getValue());
+                barrierStep.addBarrier(map.get(key));
                 if (valueTraversal.hasNext())
-                    reducedMap.put(entry.getKey(), valueTraversal.next());
+                    map.put(key, valueTraversal.next());
             }
         });
-        assert map.isEmpty();
-        map.clear();
-        map.putAll(reducedMap);
         return (Map<K, V>) map;
     }
-}
-
-
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/3496402a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroupSideEffectStep.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroupSideEffectStep.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroupSideEffectStep.java
index 0e8a4f5..9847a53 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroupSideEffectStep.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroupSideEffectStep.java
@@ -18,15 +18,17 @@
  */
 package org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect;
 
+import org.apache.tinkerpop.gremlin.process.traversal.Operator;
 import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
 import org.apache.tinkerpop.gremlin.process.traversal.Traverser;
 import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__;
+import org.apache.tinkerpop.gremlin.process.traversal.step.Barrier;
 import org.apache.tinkerpop.gremlin.process.traversal.step.ByModulating;
 import org.apache.tinkerpop.gremlin.process.traversal.step.SideEffectCapable;
 import org.apache.tinkerpop.gremlin.process.traversal.step.TraversalParent;
 import org.apache.tinkerpop.gremlin.process.traversal.step.map.GroupStep;
 import org.apache.tinkerpop.gremlin.process.traversal.traverser.TraverserRequirement;
-import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.TraverserSet;
+import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalHelper;
 import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalUtil;
 import org.apache.tinkerpop.gremlin.structure.util.StringFactory;
 import org.apache.tinkerpop.gremlin.util.function.HashMapSupplier;
@@ -44,8 +46,8 @@ public final class GroupSideEffectStep<S, K, V> extends SideEffectStep<S> implem
 
     private char state = 'k';
     private Traversal.Admin<S, K> keyTraversal;
-    private Traversal.Admin<S, ?> preTraversal;
     private Traversal.Admin<S, V> valueTraversal;
+    private Barrier barrierStep;
     ///
     private String sideEffectKey;
 
@@ -53,8 +55,11 @@ public final class GroupSideEffectStep<S, K, V> extends SideEffectStep<S> implem
         super(traversal);
         this.sideEffectKey = sideEffectKey;
         this.valueTraversal = this.integrateChild(__.fold().asAdmin());
-        this.preTraversal = this.integrateChild(GroupStep.generatePreTraversal(this.valueTraversal));
-        this.getTraversal().getSideEffects().registerIfAbsent(this.sideEffectKey, HashMapSupplier.instance(), new GroupStep.GroupBiOperator<>(this.valueTraversal));
+        this.barrierStep = TraversalHelper.getFirstStepOfAssignableClass(Barrier.class, this.valueTraversal).orElse(null);
+        this.getTraversal().getSideEffects().registerIfAbsent(this.sideEffectKey, HashMapSupplier.instance(),
+                new GroupStep.GroupBiOperator<>(null == this.barrierStep ?
+                        Operator.assign :
+                        this.barrierStep.getMemoryComputeKey().getReducer()));
     }
 
     @Override
@@ -64,8 +69,11 @@ public final class GroupSideEffectStep<S, K, V> extends SideEffectStep<S> implem
             this.state = 'v';
         } else if ('v' == this.state) {
             this.valueTraversal = this.integrateChild(GroupStep.convertValueTraversal(kvTraversal));
-            this.preTraversal = this.integrateChild(GroupStep.generatePreTraversal(this.valueTraversal));
-            this.getTraversal().getSideEffects().register(this.sideEffectKey, null, new GroupStep.GroupBiOperator<>(this.valueTraversal));
+            this.barrierStep = TraversalHelper.getFirstStepOfAssignableClass(Barrier.class, this.valueTraversal).orElse(null);
+            this.getTraversal().getSideEffects().register(this.sideEffectKey, null,
+                    new GroupStep.GroupBiOperator<>(null == this.barrierStep ?
+                            Operator.assign :
+                            this.barrierStep.getMemoryComputeKey().getReducer()));
             this.state = 'x';
         } else {
             throw new IllegalStateException("The key and value traversals for group()-step have already been set: " + this);
@@ -75,18 +83,15 @@ public final class GroupSideEffectStep<S, K, V> extends SideEffectStep<S> implem
     @Override
     protected void sideEffect(final Traverser.Admin<S> traverser) {
         final Map<K, V> map = new HashMap<>(1);
-        if (null == this.preTraversal) {
-            map.put(TraversalUtil.applyNullable(traverser, this.keyTraversal), (V) traverser.split());
-        } else {
-            final TraverserSet traverserSet = new TraverserSet<>();
-            this.preTraversal.reset();
-            this.preTraversal.addStart(traverser.split());
-            while(this.preTraversal.hasNext()) {
-                traverserSet.add(this.preTraversal.nextTraverser());
-            }
-            map.put(TraversalUtil.applyNullable(traverser, this.keyTraversal), (V) traverserSet);
-        }
-        this.getTraversal().getSideEffects().add(this.sideEffectKey, map);
+        this.valueTraversal.reset();
+        this.valueTraversal.addStart(traverser);
+        if (null == this.barrierStep) {
+            if (this.valueTraversal.hasNext())
+                map.put(TraversalUtil.applyNullable(traverser, this.keyTraversal), (V) this.valueTraversal.next());
+        } else if (this.barrierStep.hasNextBarrier())
+            map.put(TraversalUtil.applyNullable(traverser, this.keyTraversal), (V) this.barrierStep.nextBarrier());
+        if (!map.isEmpty())
+            this.getTraversal().getSideEffects().add(this.sideEffectKey, map);
     }
 
     @Override
@@ -101,12 +106,10 @@ public final class GroupSideEffectStep<S, K, V> extends SideEffectStep<S> implem
 
     @Override
     public List<Traversal.Admin<?, ?>> getLocalChildren() {
-        final List<Traversal.Admin<?, ?>> children = new ArrayList<>(3);
+        final List<Traversal.Admin<?, ?>> children = new ArrayList<>(2);
         if (null != this.keyTraversal)
             children.add(this.keyTraversal);
         children.add(this.valueTraversal);
-        if (null != this.preTraversal)
-            children.add(this.preTraversal);
         return children;
     }
 
@@ -121,7 +124,7 @@ public final class GroupSideEffectStep<S, K, V> extends SideEffectStep<S> implem
         if (null != this.keyTraversal)
             clone.keyTraversal = this.keyTraversal.clone();
         clone.valueTraversal = this.valueTraversal.clone();
-        clone.preTraversal = (Traversal.Admin<S, ?>) GroupStep.generatePreTraversal(clone.valueTraversal);
+        clone.barrierStep = TraversalHelper.getFirstStepOfAssignableClass(Barrier.class, clone.valueTraversal).orElse(null);
         return clone;
     }
 
@@ -130,7 +133,6 @@ public final class GroupSideEffectStep<S, K, V> extends SideEffectStep<S> implem
         super.setTraversal(parentTraversal);
         this.integrateChild(this.keyTraversal);
         this.integrateChild(this.valueTraversal);
-        this.integrateChild(this.preTraversal);
     }
 
     @Override
@@ -145,4 +147,4 @@ public final class GroupSideEffectStep<S, K, V> extends SideEffectStep<S> implem
     public Map<K, V> generateFinalResult(final Map<K, ?> object) {
         return GroupStep.doFinalReduction((Map<K, Object>) object, this.valueTraversal);
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/3496402a/gremlin-groovy-test/src/main/groovy/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroovyGroupTest.groovy
----------------------------------------------------------------------
diff --git a/gremlin-groovy-test/src/main/groovy/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroovyGroupTest.groovy b/gremlin-groovy-test/src/main/groovy/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroovyGroupTest.groovy
index 84da296..3ce9efe 100644
--- a/gremlin-groovy-test/src/main/groovy/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroovyGroupTest.groovy
+++ b/gremlin-groovy-test/src/main/groovy/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroovyGroupTest.groovy
@@ -123,5 +123,10 @@ public abstract class GroovyGroupTest {
         public Traversal<Vertex, Map<String, String>> get_g_V_groupXmX_byXnameX_byXinXknowsX_nameX_capXmX() {
             new ScriptTraversal<>(g, "gremlin-groovy", "g.V.group('m').by('name').by(__.in('knows').name).cap('m')")
         }
+
+        @Override
+        public Traversal<Vertex, Map<String, Number>> get_g_V_group_byXlabelX_byXbothE_groupXaX_byXlabelX_byXweight_sumX_weight_sumX() {
+            new ScriptTraversal<>(g, "gremlin-groovy", "g.V.group().by(label).by(bothE().group('a').by(label).by(values('weight').sum).weight.sum)")
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/3496402a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroupTest.java
----------------------------------------------------------------------
diff --git a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroupTest.java b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroupTest.java
index 036c8c8..71b15a5 100644
--- a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroupTest.java
+++ b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroupTest.java
@@ -37,10 +37,12 @@ import java.util.Map;
 import static org.apache.tinkerpop.gremlin.LoadGraphWith.GraphData.GRATEFUL;
 import static org.apache.tinkerpop.gremlin.LoadGraphWith.GraphData.MODERN;
 import static org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__.both;
+import static org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__.bothE;
 import static org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__.constant;
 import static org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__.count;
 import static org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__.out;
 import static org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__.outE;
+import static org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.__.values;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -90,6 +92,8 @@ public abstract class GroupTest extends AbstractGremlinProcessTest {
 
     public abstract Traversal<Vertex, Map<String, String>> get_g_V_groupXmX_byXnameX_byXinXknowsX_nameX_capXmX();
 
+    public abstract Traversal<Vertex, Map<String, Number>> get_g_V_group_byXlabelX_byXbothE_groupXaX_byXlabelX_byXweight_sumX_weight_sumX();
+
     @Test
     @LoadGraphWith(MODERN)
     public void g_V_group_byXnameX() {
@@ -441,6 +445,23 @@ public abstract class GroupTest extends AbstractGremlinProcessTest {
         checkSideEffects(traversal.asAdmin().getSideEffects(), "m", HashMap.class);
     }
 
+    @Test
+    @LoadGraphWith(MODERN)
+    public void g_V_group_byXlabelX_byXbothE_groupXaX_byXlabelX_byXweight_sumX_weight_sumX() {
+        final Traversal<Vertex, Map<String, Number>> traversal = get_g_V_group_byXlabelX_byXbothE_groupXaX_byXlabelX_byXweight_sumX_weight_sumX();
+        printTraversalForm(traversal);
+        final Map<String, Number> map = traversal.next();
+        assertFalse(traversal.hasNext());
+        assertEquals(2, map.size());
+        assertEquals(2.0d, map.get("software").doubleValue(), 0.01d);
+        assertEquals(5.0d, map.get("person").doubleValue(), 0.01d);
+        checkSideEffects(traversal.asAdmin().getSideEffects(), "a", HashMap.class);
+        final Map<String, Number> sideEffect = traversal.asAdmin().getSideEffects().get("a");
+        assertEquals(2, sideEffect.size());
+        assertEquals(4.0d, sideEffect.get("created").doubleValue(), 0.01d);
+        assertEquals(3.0d, sideEffect.get("knows").doubleValue(), 0.01d);
+    }
+
     public static class Traversals extends GroupTest {
 
         @Override
@@ -525,17 +546,22 @@ public abstract class GroupTest extends AbstractGremlinProcessTest {
 
         @Override
         public Traversal<Vertex, Map<Long, Map<String, List<Vertex>>>> get_g_V_group_byXbothE_countX_byXgroup_byXlabelXX() {
-            return g.V().<Long, Map<String, List<Vertex>>>group().by(__.bothE().count()).by(__.group().by(T.label));
+            return g.V().<Long, Map<String, List<Vertex>>>group().by(bothE().count()).by(__.group().by(T.label));
         }
 
         @Override
         public Traversal<Vertex, Map<String, Map<String, Number>>> get_g_V_outXfollowedByX_group_byXsongTypeX_byXbothE_group_byXlabelX_byXweight_sumXX() {
-            return g.V().out("followedBy").<String, Map<String, Number>>group().by("songType").by(__.bothE().group().by(T.label).by(__.values("weight").sum()));
+            return g.V().out("followedBy").<String, Map<String, Number>>group().by("songType").by(bothE().group().by(T.label).by(values("weight").sum()));
         }
 
         @Override
         public Traversal<Vertex, Map<String, String>> get_g_V_groupXmX_byXnameX_byXinXknowsX_nameX_capXmX() {
             return g.V().group("m").by("name").by(__.in("knows").values("name")).cap("m");
         }
+
+        @Override
+        public Traversal<Vertex, Map<String, Number>> get_g_V_group_byXlabelX_byXbothE_groupXaX_byXlabelX_byXweight_sumX_weight_sumX() {
+            return g.V().<String, Number>group().by(T.label).by(bothE().group("a").by(T.label).by(values("weight").sum()).values("weight").sum());
+        }
     }
 }


[03/31] tinkerpop git commit: Removed a PathProcessor.ID constraint from ComputerVerificationStrategy. Moreover, sampling and ordering is more efficient as the projected data is co-located with the traverser in the new ProjectedTraverser wrapper. Going t

Posted by sp...@apache.org.
Removed a PathProcessor.ID constraint from ComputerVerificationStrategy. Moreover, sampling and ordering is more efficient as the projected data is co-located with the traverser in the new ProjectedTraverser wrapper. Going to leave it at this for tp32/... Moving forward, we can make it so we don't need to DetachFactory.detach(true) for CollectingBarrierStep by maintaining 'future data.' Its complicated and I don't want to introduce potential bugs.


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

Branch: refs/heads/TINKERPOP-1565
Commit: b2f0c57df6fd9191904213622ae718a0790d7a03
Parents: 5045f67
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Jan 18 11:07:32 2017 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Wed Jan 18 11:07:32 2017 -0700

----------------------------------------------------------------------
 .../traversal/step/filter/SampleGlobalStep.java | 19 ++++++++++++--
 .../traversal/step/map/OrderGlobalStep.java     | 27 +++++---------------
 .../step/util/CollectingBarrierStep.java        | 24 ++++++++++-------
 .../ComputerVerificationStrategy.java           |  8 ------
 .../traversal/traverser/ProjectedTraverser.java | 16 +++++++-----
 .../gremlin/util/function/MultiComparator.java  | 14 +++++++---
 6 files changed, 60 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/b2f0c57d/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/SampleGlobalStep.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/SampleGlobalStep.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/SampleGlobalStep.java
index 0a4da58..2b2cf20 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/SampleGlobalStep.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/SampleGlobalStep.java
@@ -24,6 +24,7 @@ import org.apache.tinkerpop.gremlin.process.traversal.lambda.ConstantTraversal;
 import org.apache.tinkerpop.gremlin.process.traversal.step.ByModulating;
 import org.apache.tinkerpop.gremlin.process.traversal.step.TraversalParent;
 import org.apache.tinkerpop.gremlin.process.traversal.step.util.CollectingBarrierStep;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.ProjectedTraverser;
 import org.apache.tinkerpop.gremlin.process.traversal.traverser.TraverserRequirement;
 import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.TraverserSet;
 import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalUtil;
@@ -64,6 +65,15 @@ public final class SampleGlobalStep<S> extends CollectingBarrierStep<S> implemen
     }
 
     @Override
+    public void processAllStarts() {
+        if (this.starts.hasNext()) {
+            while (this.starts.hasNext()) {
+                this.traverserSet.add(this.createProjectedTraverser(this.starts.next()));
+            }
+        }
+    }
+
+    @Override
     public void barrierConsumer(final TraverserSet<S> traverserSet) {
         // return the entire traverser set if the set is smaller than the amount to sample
         if (traverserSet.bulkSize() <= this.amountToSample)
@@ -71,7 +81,7 @@ public final class SampleGlobalStep<S> extends CollectingBarrierStep<S> implemen
         //////////////// else sample the set
         double totalWeight = 0.0d;
         for (final Traverser.Admin<S> s : traverserSet) {
-            totalWeight = totalWeight + TraversalUtil.apply(s, this.probabilityTraversal).doubleValue() * s.bulk();
+            totalWeight = totalWeight + (((ProjectedTraverser<S, Number>) s).getProjections().get(0).doubleValue() * s.bulk());
         }
         ///////
         final TraverserSet<S> sampledSet = new TraverserSet<>();
@@ -82,7 +92,7 @@ public final class SampleGlobalStep<S> extends CollectingBarrierStep<S> implemen
             for (final Traverser.Admin<S> s : traverserSet) {
                 long sampleBulk = sampledSet.contains(s) ? sampledSet.get(s).bulk() : 0;
                 if (sampleBulk < s.bulk()) {
-                    final double currentWeight = TraversalUtil.apply(s, this.probabilityTraversal).doubleValue();
+                    final double currentWeight = ((ProjectedTraverser<S, Number>) s).getProjections().get(0).doubleValue();
                     for (int i = 0; i < (s.bulk() - sampleBulk); i++) {
                         runningWeight = runningWeight + currentWeight;
                         if (RANDOM.nextDouble() <= ((runningWeight / totalWeight))) {
@@ -104,6 +114,11 @@ public final class SampleGlobalStep<S> extends CollectingBarrierStep<S> implemen
         traverserSet.addAll(sampledSet);
     }
 
+
+    private final ProjectedTraverser<S, Number> createProjectedTraverser(final Traverser.Admin<S> traverser) {
+        return new ProjectedTraverser<>(traverser, Collections.singletonList(TraversalUtil.apply(traverser, this.probabilityTraversal)));
+    }
+
     @Override
     public Set<TraverserRequirement> getRequirements() {
         return this.getSelfAndChildRequirements(TraverserRequirement.BULK);

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/b2f0c57d/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
index 9c071f1..55d8650 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
@@ -73,22 +73,11 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
             this.multiComparator = this.createMultiComparator();
         if (this.starts.hasNext()) {
             while (this.starts.hasNext()) {
-                this.traverserSet.add(this.createOrderedTraverser(this.starts.next()));
+                this.traverserSet.add(this.createProjectedTraverser(this.starts.next()));
             }
-            this.barrierConsumer(this.traverserSet);
         }
     }
 
-    @Override
-    public Traverser.Admin<S> processNextStart() {
-        if (!this.traverserSet.isEmpty()) {
-            return this.traverserSet.remove();
-        } else if (this.starts.hasNext()) {
-            this.processAllStarts();
-        }
-        return ((ProjectedTraverser) this.traverserSet.remove()).getInternal();
-    }
-
     public void setLimit(final long limit) {
         this.limit = limit;
     }
@@ -162,18 +151,18 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
     public MemoryComputeKey<TraverserSet<S>> getMemoryComputeKey() {
         if (null == this.multiComparator)
             this.multiComparator = this.createMultiComparator();
-        return MemoryComputeKey.of(this.getId(), new OrderBiOperator<>(this.limit, this.isShuffle, this.multiComparator), false, true);
+        return MemoryComputeKey.of(this.getId(), new OrderBiOperator<>(this.limit, this.multiComparator), false, true);
     }
 
-    private ProjectedTraverser<S> createOrderedTraverser(final Traverser.Admin<S> traverser) {
+    private final ProjectedTraverser<S,Object> createProjectedTraverser(final Traverser.Admin<S> traverser) {
         final List<Object> projections = new ArrayList<>(this.comparators.size());
         for (final Pair<Traversal.Admin<S, C>, Comparator<C>> pair : this.comparators) {
             projections.add(TraversalUtil.apply(traverser, pair.getValue0()));
         }
-        return new ProjectedTraverser<S>(traverser, projections);
+        return new ProjectedTraverser<>(traverser, projections);
     }
 
-    private MultiComparator<C> createMultiComparator() {
+    private final MultiComparator<C> createMultiComparator() {
         final List<Comparator<C>> list = new ArrayList<>(this.comparators.size());
         for (final Pair<Traversal.Admin<S, C>, Comparator<C>> pair : this.comparators) {
             list.add(pair.getValue1());
@@ -186,16 +175,14 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
     public static final class OrderBiOperator<S> implements BinaryOperator<TraverserSet<S>>, Serializable {
 
         private long limit;
-        private boolean isShuffle;
         private MultiComparator comparator;
 
         private OrderBiOperator() {
             // for serializers that need a no-arg constructor
         }
 
-        public OrderBiOperator(final long limit, final boolean isShuffle, final MultiComparator multiComparator) {
+        public OrderBiOperator(final long limit, final MultiComparator multiComparator) {
             this.limit = limit;
-            this.isShuffle = isShuffle;
             this.comparator = multiComparator;
         }
 
@@ -203,7 +190,7 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
         public TraverserSet<S> apply(final TraverserSet<S> setA, final TraverserSet<S> setB) {
             setA.addAll(setB);
             if (Long.MAX_VALUE != this.limit && setA.bulkSize() > this.limit) {
-                if (this.isShuffle)
+                if (this.comparator.isShuffle())
                     setA.shuffle();
                 else
                     setA.sort(this.comparator);

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/b2f0c57d/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/CollectingBarrierStep.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/CollectingBarrierStep.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/CollectingBarrierStep.java
index b0cce80..f99201d 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/CollectingBarrierStep.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/CollectingBarrierStep.java
@@ -23,11 +23,13 @@ import org.apache.tinkerpop.gremlin.process.traversal.Operator;
 import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
 import org.apache.tinkerpop.gremlin.process.traversal.Traverser;
 import org.apache.tinkerpop.gremlin.process.traversal.step.Barrier;
+import org.apache.tinkerpop.gremlin.process.traversal.traverser.ProjectedTraverser;
 import org.apache.tinkerpop.gremlin.process.traversal.traverser.TraverserRequirement;
 import org.apache.tinkerpop.gremlin.process.traversal.traverser.util.TraverserSet;
 import org.apache.tinkerpop.gremlin.process.traversal.util.FastNoSuchElementException;
 import org.apache.tinkerpop.gremlin.structure.util.StringFactory;
 import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedFactory;
+import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 
 import java.util.Collections;
 import java.util.NoSuchElementException;
@@ -40,7 +42,8 @@ import java.util.function.BinaryOperator;
 public abstract class CollectingBarrierStep<S> extends AbstractStep<S, S> implements Barrier<TraverserSet<S>> {
 
     protected TraverserSet<S> traverserSet = new TraverserSet<>();
-    protected int maxBarrierSize;
+    private int maxBarrierSize;
+    private boolean barrierConsumed = false;
 
     public CollectingBarrierStep(final Traversal.Admin traversal) {
         this(traversal, Integer.MAX_VALUE);
@@ -68,7 +71,6 @@ public abstract class CollectingBarrierStep<S> extends AbstractStep<S, S> implem
                     this.traverserSet.add(this.starts.next());
                 }
             }
-            this.barrierConsumer(this.traverserSet);
         }
     }
 
@@ -85,11 +87,10 @@ public abstract class CollectingBarrierStep<S> extends AbstractStep<S, S> implem
             throw FastNoSuchElementException.instance();
         else {
             final TraverserSet<S> temp = new TraverserSet<>();
-            this.traverserSet.iterator().forEachRemaining(t -> {
+            IteratorUtils.removeOnNext(this.traverserSet.iterator()).forEachRemaining(t -> {
                 DetachedFactory.detach(t, true); // this should be dynamic
                 temp.add(t);
             });
-            this.traverserSet.clear();
             return temp;
         }
     }
@@ -98,23 +99,28 @@ public abstract class CollectingBarrierStep<S> extends AbstractStep<S, S> implem
     public void addBarrier(final TraverserSet<S> barrier) {
         this.traverserSet = barrier;
         this.traverserSet.forEach(traverser -> traverser.setSideEffects(this.getTraversal().getSideEffects()));
-        this.barrierConsumer(this.traverserSet);
+        this.barrierConsumed = false;
     }
 
     @Override
     public Traverser.Admin<S> processNextStart() {
-        if (!this.traverserSet.isEmpty()) {
-            return this.traverserSet.remove();
-        } else if (this.starts.hasNext()) {
+        if (this.traverserSet.isEmpty() && this.starts.hasNext()) {
             this.processAllStarts();
+            this.barrierConsumed = false;
+        }
+        //
+        if (!this.barrierConsumed) {
+            this.barrierConsumer(this.traverserSet);
+            this.barrierConsumed = true;
         }
-        return this.traverserSet.remove();
+        return ProjectedTraverser.tryUnwrap(this.traverserSet.remove());
     }
 
     @Override
     public CollectingBarrierStep<S> clone() {
         final CollectingBarrierStep<S> clone = (CollectingBarrierStep<S>) super.clone();
         clone.traverserSet = new TraverserSet<>();
+        clone.barrierConsumed = false;
         return clone;
     }
 

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/b2f0c57d/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategy.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategy.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategy.java
index 5777adb..ef9b95c 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategy.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/strategy/verification/ComputerVerificationStrategy.java
@@ -28,11 +28,9 @@ import org.apache.tinkerpop.gremlin.process.traversal.step.GraphComputing;
 import org.apache.tinkerpop.gremlin.process.traversal.step.Mutating;
 import org.apache.tinkerpop.gremlin.process.traversal.step.PathProcessor;
 import org.apache.tinkerpop.gremlin.process.traversal.step.TraversalParent;
-import org.apache.tinkerpop.gremlin.process.traversal.step.filter.SampleGlobalStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.map.GraphStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.InjectStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.sideEffect.SubgraphStep;
-import org.apache.tinkerpop.gremlin.process.traversal.step.util.CollectingBarrierStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.util.EmptyStep;
 import org.apache.tinkerpop.gremlin.process.traversal.step.util.ProfileStep;
 import org.apache.tinkerpop.gremlin.process.traversal.strategy.AbstractTraversalStrategy;
@@ -87,12 +85,6 @@ public final class ComputerVerificationStrategy extends AbstractTraversalStrateg
                     throw new VerificationException("Local traversals may not traverse past the local star-graph on GraphComputer: " + traversalOptional.get(), traversal);
             }
 
-            // sample step use can only operate on the element and its properties (no incidences)
-            if (step instanceof SampleGlobalStep) {
-                if (((TraversalParent) step).getLocalChildren().stream().filter(t -> !TraversalHelper.isLocalProperties(t)).findAny().isPresent())
-                    throw new VerificationException("The following barrier step can not process the incident edges of a vertex on GraphComputer: " + step, traversal);
-            }
-
             // this is a problem because sideEffect.merge() is transient on the OLAP reduction
             if (TraversalHelper.getRootTraversal(traversal).getTraverserRequirements().contains(TraverserRequirement.ONE_BULK))
                 throw new VerificationException("One bulk is currently not supported on GraphComputer: " + step, traversal);

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/b2f0c57d/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java
index 67e723a..5cecdc4 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java
@@ -32,16 +32,16 @@ import java.util.function.Function;
 /**
  * @author Marko A. Rodriguez (http://markorodriguez.com)
  */
-public final class ProjectedTraverser<T> implements Traverser.Admin<T> {
+public final class ProjectedTraverser<T, P> implements Traverser.Admin<T> {
 
     private Traverser.Admin<T> internal;
-    private List<Object> projections;
+    private List<P> projections;
 
     private ProjectedTraverser() {
         // for serialization
     }
 
-    public ProjectedTraverser(final Traverser.Admin<T> internal, final List<Object> projections) {
+    public ProjectedTraverser(final Traverser.Admin<T> internal, final List<P> projections) {
         this.internal = internal;
         this.projections = projections;
     }
@@ -51,7 +51,7 @@ public final class ProjectedTraverser<T> implements Traverser.Admin<T> {
         return this.internal;
     }
 
-    public List<Object> getProjections() {
+    public List<P> getProjections() {
         return this.projections;
     }
 
@@ -187,13 +187,17 @@ public final class ProjectedTraverser<T> implements Traverser.Admin<T> {
     }
 
     @Override
-    public ProjectedTraverser<T> clone() {
+    public ProjectedTraverser<T, P> clone() {
         try {
-            final ProjectedTraverser<T> clone = (ProjectedTraverser<T>) super.clone();
+            final ProjectedTraverser<T, P> clone = (ProjectedTraverser<T, P>) super.clone();
             clone.internal = (Traverser.Admin<T>) this.internal.clone();
             return clone;
         } catch (final CloneNotSupportedException e) {
             throw new IllegalStateException(e.getMessage(), e);
         }
     }
+
+    public static <T> Traverser.Admin<T> tryUnwrap(final Traverser.Admin<T> traverser) {
+        return traverser instanceof ProjectedTraverser ? ((ProjectedTraverser) traverser).getInternal() : traverser;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/b2f0c57d/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java
index 427aa3d..b7176ab 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java
@@ -32,9 +32,11 @@ import java.util.List;
 public final class MultiComparator<C> implements Comparator<C>, Serializable {
 
     private final List<Comparator> comparators;
+    private final boolean isShuffle;
 
     public MultiComparator(final List<Comparator<C>> comparators) {
         this.comparators = (List) comparators;
+        this.isShuffle = !this.comparators.isEmpty() && Order.shuffle == this.comparators.get(this.comparators.size() - 1);
     }
 
     @Override
@@ -43,14 +45,20 @@ public final class MultiComparator<C> implements Comparator<C>, Serializable {
             return Order.incr.compare(objectA, objectB);
         } else {
             for (int i = 0; i < this.comparators.size(); i++) {
-                final int comparison = this.comparators.get(i).compare(this.getObject(objectA, i), this.getObject(objectB, i));
-                if (comparison != 0)
-                    return comparison;
+                if (Order.shuffle != this.comparators.get(i)) {
+                    final int comparison = this.comparators.get(i).compare(this.getObject(objectA, i), this.getObject(objectB, i));
+                    if (comparison != 0)
+                        return comparison;
+                }
             }
             return 0;
         }
     }
 
+    public boolean isShuffle() {
+        return this.isShuffle;
+    }
+
     private final Object getObject(final C object, final int index) {
         if (object instanceof ProjectedTraverser)
             return ((ProjectedTraverser) object).getProjections().get(index);


[30/31] tinkerpop git commit: TINKERPOP-1414 Changed Gremlin Server and TinkerGraph to default GraphSON 2.0

Posted by sp...@apache.org.
TINKERPOP-1414 Changed Gremlin Server and TinkerGraph to default GraphSON 2.0

This work is done as part of TINKERPOP-1565 which is related to GraphSON 2.0 changes.


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

Branch: refs/heads/TINKERPOP-1565
Commit: a00416291b769b007fc34f06dc16e1a2eb9246a2
Parents: bb97c94
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Wed Dec 7 10:37:04 2016 -0500
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Jan 19 15:15:33 2017 -0500

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |   1 +
 docs/src/upgrade/release-3.3.x.asciidoc         |  65 +++++-
 .../structure/io/graphson/GraphSONIo.java       |   2 +-
 .../structure/io/graphson/GraphSONMapper.java   |   4 +-
 gremlin-server/conf/gremlin-server-classic.yaml |  12 +-
 .../conf/gremlin-server-modern-py.yaml          |  12 +-
 .../conf/gremlin-server-modern-readonly.yaml    |  12 +-
 gremlin-server/conf/gremlin-server-modern.yaml  |  12 +-
 gremlin-server/conf/gremlin-server-neo4j.yaml   |  12 +-
 .../conf/gremlin-server-rest-modern.yaml        |   6 +-
 .../conf/gremlin-server-rest-secure.yaml        |   6 +-
 gremlin-server/conf/gremlin-server-secure.yaml  |  12 +-
 gremlin-server/conf/gremlin-server-spark.yaml   |  12 +-
 gremlin-server/conf/gremlin-server.yaml         |  12 +-
 .../gremlin/server/AbstractChannelizer.java     |   5 +-
 .../server/GremlinServerAuthIntegrateTest.java  |  16 +-
 .../GremlinServerAuthOldIntegrateTest.java      |  20 +-
 .../server/GremlinServerHttpIntegrateTest.java  |  31 +--
 .../server/GremlinServerIntegrateTest.java      |   2 +-
 .../remote/gremlin-server-integration.yaml      |   6 +-
 .../server/gremlin-server-integration.yaml      |   6 +-
 .../gremlin/structure/SerializationTest.java    | 229 ++++++++++++++++++-
 .../tinkerpop/gremlin/structure/io/IoTest.java  | 143 +++++++++++-
 .../tinkergraph/structure/TinkerGraph.java      |   2 +-
 24 files changed, 521 insertions(+), 119 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index f0d1c4f..934022f 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -29,6 +29,7 @@ TinkerPop 3.3.0 (Release Date: NOT OFFICIALLY RELEASED YET)
 * Added more specific typing to various `__` traversal steps. E.g. `<A,Vertex>out()` is `<Vertex,Vertex>out()`.
 * Updated Docker build scripts to include Python dependencies (NOTE: users should remove any previously generated TinkerPop Docker images).
 * Added "attachment requisite" `VertexProperty.element()` and `Property.element()` data in GraphSON serialization.
+* GraphSON 2.0 is now the default serialization format in TinkerGraph and Gremlin Server.
 * Added `Vertex`, `Edge`, `VertexProperty`, and `Property` serializers to Gremlin-Python and exposed tests that use graph object arguments.
 * `Bytecode.getSourceInstructions()` and `Bytecode.getStepInstructions()` now returns `List<Instruction>` instead of `Iterable<Instruction>`.
 * Added various `TraversalStrategy` registrations with `GryoMapper`.

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/docs/src/upgrade/release-3.3.x.asciidoc
----------------------------------------------------------------------
diff --git a/docs/src/upgrade/release-3.3.x.asciidoc b/docs/src/upgrade/release-3.3.x.asciidoc
index 9233d57..4e43077 100644
--- a/docs/src/upgrade/release-3.3.x.asciidoc
+++ b/docs/src/upgrade/release-3.3.x.asciidoc
@@ -32,6 +32,69 @@ Please see the link:https://github.com/apache/tinkerpop/blob/3.3.3/CHANGELOG.asc
 Upgrading for Users
 ~~~~~~~~~~~~~~~~~~~
 
+GraphSON 2.0
+^^^^^^^^^^^^
+
+Both TinkerGraph and Gremlin Server have been defaulted to work with GraphSON 2.0. For TinkerGraph this means that
+the following commands:
+
+[source,java]
+----
+Graph graph = TinkerFactory.createModern();
+graph.io(IoCore.graphson()).writeGraph("tinkerpop-modern.json");
+
+final Graph newGraph = TinkerGraph.open();
+newGraph.io(IoCore.graphson()).readGraph("tinkerpop-modern.json");
+----
+
+will write and read GraphSON 2.0 format rather than 1.0. To use 1.0 format simply set the `version()` on the
+appropriate builder methods:
+
+[source,java]
+----
+Graph graph = TinkerFactory.createModern();
+GraphSONMapper mapper = graph.io(IoCore.graphson()).mapper().version(GraphSONVersion.V1_0).create()
+try (OutputStream os = new FileOutputStream("tinkerpop-modern.json")) {
+    graph.io(IoCore.graphson()).writer().mapper(mapper).create().writeGraph(os, graph)
+}
+
+final Graph newGraph = TinkerGraph.open();
+try (InputStream stream = new FileInputStream("tinkerpop-modern.json")) {
+    newGraph.io(IoCore.graphson()).reader().mapper(mapper).vertexIdKey("name").create().readGraph(stream, newGraph);
+}
+----
+
+For Gremlin Server, this change means that the `application/json` mime type no longer returns GraphSON 1.0 without
+type embedding. Instead, Gremlin Server will return GraphSON 2.0 with partial types enabled (i.e. which is equivalent
+to `application/vnd.gremlin-v2.0+json`). The `serializers` section the sample Gremlin Server YAML files now typically
+look like this:
+
+[source,yaml]
+----
+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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
+----
+
+It is possible to bring back the original configuration for `application/json` by changing the last entry as follows:
+
+[source,yaml]
+----
+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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: { useMapperFromGraph: graph }}         # application/json
+----
+
+See: link:https://issues.apache.org/jira/browse/TINKERPOP-1414[TINKERPOP-1414]
+
 GraphTraversal Has-Methods Re-Organized
 ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
 
@@ -176,4 +239,4 @@ If the old `GryoSerializer` model is desired, then the properties file should si
 spark.serializer=org.apache.tinkerpop.gremlin.spark.structure.io.gryo.GryoSerializer
 ```
 
-See: link:https://issues.apache.org/jira/browse/TINKERPOP-1389
\ No newline at end of file
+See: link:https://issues.apache.org/jira/browse/TINKERPOP-1389

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONIo.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONIo.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONIo.java
index f56afb9..0c92e1c 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONIo.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONIo.java
@@ -101,7 +101,7 @@ public final class GraphSONIo implements Io<GraphSONReader.Builder, GraphSONWrit
      * Create a new builder using the default version of GraphSON.
      */
     public static Io.Builder<GraphSONIo> build() {
-        return build(GraphSONVersion.V1_0);
+        return build(GraphSONVersion.V2_0);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONMapper.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONMapper.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONMapper.java
index 7e46c98..f82ebb7 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONMapper.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONMapper.java
@@ -181,7 +181,7 @@ public class GraphSONMapper implements Mapper<ObjectMapper> {
         private boolean normalize = false;
         private boolean embedTypes = false;
         private List<IoRegistry> registries = new ArrayList<>();
-        private GraphSONVersion version = GraphSONVersion.V1_0;
+        private GraphSONVersion version = GraphSONVersion.V2_0;
         // GraphSON 2.0 should have types activated by default, otherwise use there's no point in using it instead of 1.0.
         private TypeInfo typeInfo = TypeInfo.PARTIAL_TYPES;
 
@@ -198,7 +198,7 @@ public class GraphSONMapper implements Mapper<ObjectMapper> {
         }
 
         /**
-         * Set the version of GraphSON to use. The default is {@link GraphSONVersion#V1_0}.
+         * Set the version of GraphSON to use. The default is {@link GraphSONVersion#V2_0}.
          */
         public Builder version(final GraphSONVersion version) {
             this.version = version;

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/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 ac78da0..d1cbbf3 100644
--- a/gremlin-server/conf/gremlin-server-classic.yaml
+++ b/gremlin-server/conf/gremlin-server-classic.yaml
@@ -28,12 +28,12 @@ scriptEngines: {
     staticImports: [java.lang.Math.PI],
     scripts: [scripts/generate-classic.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
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { useMapperFromGraph: graph }} # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0] }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: { useMapperFromGraph: graph }}        # application/json
+  - { 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
 metrics: {
   slf4jReporter: {enabled: true, interval: 180000}}
 strictTransactionManagement: false

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/gremlin-server/conf/gremlin-server-modern-py.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/conf/gremlin-server-modern-py.yaml b/gremlin-server/conf/gremlin-server-modern-py.yaml
index e00eb1d..ff8228f 100644
--- a/gremlin-server/conf/gremlin-server-modern-py.yaml
+++ b/gremlin-server/conf/gremlin-server-modern-py.yaml
@@ -43,12 +43,12 @@ scriptEngines: {
   gremlin-python: {}
 }
 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.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0] }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: { useMapperFromGraph: graph }}        # application/json
+  - { 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
 metrics: {
   slf4jReporter: {enabled: true, interval: 180000}}
 strictTransactionManagement: false

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/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 379b358..18f7ca4 100644
--- a/gremlin-server/conf/gremlin-server-modern-readonly.yaml
+++ b/gremlin-server/conf/gremlin-server-modern-readonly.yaml
@@ -28,12 +28,12 @@ scriptEngines: {
     staticImports: [java.lang.Math.PI],
     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
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { useMapperFromGraph: graph }} # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0] }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: { useMapperFromGraph: graph }}        # application/json
+  - { 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
 metrics: {
   slf4jReporter: {enabled: true, interval: 180000}}
 strictTransactionManagement: false

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/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 4ac5587..b063868 100644
--- a/gremlin-server/conf/gremlin-server-modern.yaml
+++ b/gremlin-server/conf/gremlin-server-modern.yaml
@@ -28,12 +28,12 @@ scriptEngines: {
     staticImports: [java.lang.Math.PI],
     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
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { useMapperFromGraph: graph }} # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0] }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: { useMapperFromGraph: graph }}        # application/json
+  - { 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
 metrics: {
   slf4jReporter: {enabled: true, interval: 180000}}
 strictTransactionManagement: false

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/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 8d68c7f..6aac0f2 100644
--- a/gremlin-server/conf/gremlin-server-neo4j.yaml
+++ b/gremlin-server/conf/gremlin-server-neo4j.yaml
@@ -39,12 +39,12 @@ scriptEngines: {
     staticImports: [java.lang.Math.PI],
     scripts: [scripts/empty-sample.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
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { useMapperFromGraph: graph }} # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0] }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: { useMapperFromGraph: graph }}        # application/json
+  - { 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
 processors:
   - { className: org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor, config: { sessionTimeout: 28800000 }}
   - { className: org.apache.tinkerpop.gremlin.server.op.traversal.TraversalOpProcessor, config: { cacheExpirationTime: 600000, cacheMaxSize: 1000 }}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/gremlin-server/conf/gremlin-server-rest-modern.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/conf/gremlin-server-rest-modern.yaml b/gremlin-server/conf/gremlin-server-rest-modern.yaml
index 8c41ee7..1fa23f9 100644
--- a/gremlin-server/conf/gremlin-server-rest-modern.yaml
+++ b/gremlin-server/conf/gremlin-server-rest-modern.yaml
@@ -29,9 +29,9 @@ scriptEngines: {
     staticImports: [java.lang.Math.PI],
     scripts: [scripts/generate-modern.groovy]}}
 serializers:
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { useMapperFromGraph: graph }}  # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0] }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: { useMapperFromGraph: graph }}         # application/json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
 metrics: {
   slf4jReporter: {enabled: true, interval: 180000}}
 strictTransactionManagement: false

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/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 646a1e5..8ed8f41 100644
--- a/gremlin-server/conf/gremlin-server-rest-secure.yaml
+++ b/gremlin-server/conf/gremlin-server-rest-secure.yaml
@@ -42,9 +42,9 @@ 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.GraphSONMessageSerializerGremlinV1d0, config: { useMapperFromGraph: graph }} # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0] }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: { useMapperFromGraph: graph }}        # application/json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
 processors:
   - { className: org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor, config: { sessionTimeout: 28800000 }}
   - { className: org.apache.tinkerpop.gremlin.server.op.traversal.TraversalOpProcessor, config: { cacheExpirationTime: 600000, cacheMaxSize: 1000 }}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/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 5b5e91e..14f7034 100644
--- a/gremlin-server/conf/gremlin-server-secure.yaml
+++ b/gremlin-server/conf/gremlin-server-secure.yaml
@@ -42,12 +42,12 @@ 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.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.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0] }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: { useMapperFromGraph: graph }}        # application/json
+  - { 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
 processors:
   - { className: org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor, config: { sessionTimeout: 28800000 }}
   - { className: org.apache.tinkerpop.gremlin.server.op.traversal.TraversalOpProcessor, config: { cacheExpirationTime: 600000, cacheMaxSize: 1000 }}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/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 d79185f..3c198f5 100644
--- a/gremlin-server/conf/gremlin-server-spark.yaml
+++ b/gremlin-server/conf/gremlin-server-spark.yaml
@@ -52,12 +52,12 @@ scriptEngines: {
     staticImports: [java.lang.Math.PI],
     scripts: [scripts/spark.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
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { useMapperFromGraph: graph }} # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0] }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: { useMapperFromGraph: graph }}        # application/json
+  - { 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
 processors:
   - { className: org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor, config: { sessionTimeout: 28800000 }}
   - { className: org.apache.tinkerpop.gremlin.server.op.traversal.TraversalOpProcessor, config: { cacheExpirationTime: 600000, cacheMaxSize: 1000 }}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/gremlin-server/conf/gremlin-server.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/conf/gremlin-server.yaml b/gremlin-server/conf/gremlin-server.yaml
index 8cf2947..d4e2849 100644
--- a/gremlin-server/conf/gremlin-server.yaml
+++ b/gremlin-server/conf/gremlin-server.yaml
@@ -29,12 +29,12 @@ scriptEngines: {
     staticImports: [java.lang.Math.PI],
     scripts: [scripts/empty-sample.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
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { useMapperFromGraph: graph }} # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0] }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: { useMapperFromGraph: graph }}        # application/json
+  - { 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
 processors:
   - { className: org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor, config: { sessionTimeout: 28800000 }}
   - { className: org.apache.tinkerpop.gremlin.server.op.traversal.TraversalOpProcessor, config: { cacheExpirationTime: 600000, cacheMaxSize: 1000 }}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/gremlin-server/src/main/java/org/apache/tinkerpop/gremlin/server/AbstractChannelizer.java
----------------------------------------------------------------------
diff --git a/gremlin-server/src/main/java/org/apache/tinkerpop/gremlin/server/AbstractChannelizer.java b/gremlin-server/src/main/java/org/apache/tinkerpop/gremlin/server/AbstractChannelizer.java
index 57c6994..59238fc 100644
--- a/gremlin-server/src/main/java/org/apache/tinkerpop/gremlin/server/AbstractChannelizer.java
+++ b/gremlin-server/src/main/java/org/apache/tinkerpop/gremlin/server/AbstractChannelizer.java
@@ -24,9 +24,8 @@ import io.netty.handler.ssl.SslContextBuilder;
 import io.netty.handler.ssl.SslProvider;
 import io.netty.handler.ssl.util.SelfSignedCertificate;
 import org.apache.tinkerpop.gremlin.driver.MessageSerializer;
-import org.apache.tinkerpop.gremlin.driver.ser.AbstractGraphSONMessageSerializerV1d0;
 import org.apache.tinkerpop.gremlin.driver.ser.AbstractGryoMessageSerializerV1d0;
-import org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0;
+import org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0;
 import org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0;
 import org.apache.tinkerpop.gremlin.groovy.engine.GremlinExecutor;
 import org.apache.tinkerpop.gremlin.server.auth.Authenticator;
@@ -72,7 +71,7 @@ public abstract class AbstractChannelizer extends ChannelInitializer<SocketChann
             new Settings.SerializerSettings(GryoMessageSerializerV1d0.class.getName(), new HashMap<String,Object>(){{
                 put(AbstractGryoMessageSerializerV1d0.TOKEN_SERIALIZE_RESULT_TO_STRING, true);
             }}),
-            new Settings.SerializerSettings(GraphSONMessageSerializerV1d0.class.getName(), Collections.emptyMap())
+            new Settings.SerializerSettings(GraphSONMessageSerializerV2d0.class.getName(), Collections.emptyMap())
     );
 
     protected Settings settings;

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthIntegrateTest.java
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthIntegrateTest.java b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthIntegrateTest.java
index 1c5188d..5d0f82e 100644
--- a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthIntegrateTest.java
+++ b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthIntegrateTest.java
@@ -173,9 +173,9 @@ public class GremlinServerAuthIntegrateTest extends AbstractGremlinServerIntegra
         final Client client = cluster.connect();
 
         try {
-            assertEquals(2, client.submit("1+1").all().get().get(0).getInt());
-            assertEquals(3, client.submit("1+2").all().get().get(0).getInt());
-            assertEquals(4, client.submit("1+3").all().get().get(0).getInt());
+            assertEquals(3, client.submit("1+2").all().get().get(0).get(Map.class).get("@value"));
+            assertEquals(2, client.submit("1+1").all().get().get(0).get(Map.class).get("@value"));
+            assertEquals(4, client.submit("1+3").all().get().get(0).get(Map.class).get("@value"));
         } finally {
             cluster.close();
         }
@@ -203,12 +203,12 @@ public class GremlinServerAuthIntegrateTest extends AbstractGremlinServerIntegra
         final Client client = cluster.connect(name.getMethodName());
 
         try {
-            Map vertex = (Map) client.submit("v=graph.addVertex(\"name\", \"stephen\")").all().get().get(0).getObject();
-            Map<String, List<Map>> properties = (Map) vertex.get("properties");
+            final Map vertex = (Map) client.submit("v=graph.addVertex(\"name\", \"stephen\")").all().get().get(0).getObject();
+            final Map<String, List<Map>> properties = (Map) ((Map) vertex.get("@value")).get("properties");
             assertEquals("stephen", properties.get("name").get(0).get("value"));
 
             final Map vpName = (Map)client.submit("v.property('name')").all().get().get(0).getObject();
-            assertEquals("stephen", vpName.get("value"));
+            assertEquals("stephen", ((Map) vpName.get("@value")).get("value"));
         } finally {
             cluster.close();
         }
@@ -221,8 +221,8 @@ public class GremlinServerAuthIntegrateTest extends AbstractGremlinServerIntegra
         final Client client = cluster.connect(name.getMethodName());
 
         try {
-            Map vertex = (Map) client.submit("v=graph.addVertex('name', 'stephen')").all().get().get(0).getObject();
-            Map<String, List<Map>> properties = (Map) vertex.get("properties");
+            final Map vertex = (Map) client.submit("v=graph.addVertex('name', 'stephen')").all().get().get(0).getObject();
+            final Map<String, List<Map>> properties = (Map) vertex.get("properties");
             assertEquals("stephen", properties.get("name").get(0).get("value"));
 
             final Map vpName = (Map)client.submit("v.property('name')").all().get().get(0).getObject();

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthOldIntegrateTest.java
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthOldIntegrateTest.java b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthOldIntegrateTest.java
index f0e2104..c8312ae 100644
--- a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthOldIntegrateTest.java
+++ b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthOldIntegrateTest.java
@@ -179,9 +179,9 @@ public class GremlinServerAuthOldIntegrateTest extends AbstractGremlinServerInte
         final Client client = cluster.connect();
 
         try {
-            assertEquals(2, client.submit("1+1").all().get().get(0).getInt());
-            assertEquals(3, client.submit("1+2").all().get().get(0).getInt());
-            assertEquals(4, client.submit("1+3").all().get().get(0).getInt());
+            assertEquals(3, client.submit("1+2").all().get().get(0).get(Map.class).get("@value"));
+            assertEquals(2, client.submit("1+1").all().get().get(0).get(Map.class).get("@value"));
+            assertEquals(4, client.submit("1+3").all().get().get(0).get(Map.class).get("@value"));
         } finally {
             cluster.close();
         }
@@ -209,12 +209,12 @@ public class GremlinServerAuthOldIntegrateTest extends AbstractGremlinServerInte
         final Client client = cluster.connect(name.getMethodName());
 
         try {
-            Map vertex = (Map) client.submit("v=graph.addVertex(\"name\", \"stephen\")").all().get().get(0).getObject();
-            Map<String, List<Map>> properties = (Map) vertex.get("properties");
+            final Map vertex = (Map) client.submit("v=graph.addVertex(\"name\", \"stephen\")").all().get().get(0).getObject();
+            final Map<String, List<Map>> properties = (Map) ((Map) vertex.get("@value")).get("properties");
             assertEquals("stephen", properties.get("name").get(0).get("value"));
-            
+
             final Map vpName = (Map)client.submit("v.property('name')").all().get().get(0).getObject();
-            assertEquals("stephen", vpName.get("value"));
+            assertEquals("stephen", ((Map) vpName.get("@value")).get("value"));
         } finally {
             cluster.close();
         }
@@ -227,10 +227,10 @@ public class GremlinServerAuthOldIntegrateTest extends AbstractGremlinServerInte
         final Client client = cluster.connect(name.getMethodName());
 
         try {
-            Map vertex = (Map) client.submit("v=graph.addVertex('name', 'stephen')").all().get().get(0).getObject();
-            Map<String, List<Map>> properties = (Map) vertex.get("properties");
+            final Map vertex = (Map) client.submit("v=graph.addVertex('name', 'stephen')").all().get().get(0).getObject();
+            final Map<String, List<Map>> properties = (Map) vertex.get("properties");
             assertEquals("stephen", properties.get("name").get(0).get("value"));
-            
+
             final Map vpName = (Map)client.submit("v.property('name')").all().get().get(0).getObject();
             assertEquals("stephen", vpName.get("value"));
         } finally {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java
index 78109e6..b2a1568 100644
--- a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java
+++ b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.tinkerpop.gremlin.server;
 
+import org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0;
 import org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0;
 import org.apache.tinkerpop.gremlin.server.auth.SimpleAuthenticator;
 import org.apache.tinkerpop.gremlin.server.channel.HttpChannelizer;
@@ -77,10 +78,10 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
                 deleteDirectory(new File("/tmp/neo4j"));
                 settings.graphs.put("graph", "conf/neo4j-empty.properties");
                 break;
-            case "should200OnPOSTWithGraphSON2d0AcceptHeaderDefaultResultToJson":
+            case "should200OnPOSTWithGraphSON1d0AcceptHeaderDefaultResultToJson":
                 settings.serializers.clear();
                 final Settings.SerializerSettings serializerSettings = new Settings.SerializerSettings();
-                serializerSettings.className = GraphSONMessageSerializerV2d0.class.getName();
+                serializerSettings.className = GraphSONMessageSerializerV1d0.class.getName();
                 settings.serializers.add(serializerSettings);
                 break;
             case "should401OnGETWithNoAuthorizationHeader":
@@ -297,7 +298,7 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
             assertEquals("application/json", response.getEntity().getContentType().getValue());
             final String json = EntityUtils.toString(response.getEntity());
             final JsonNode node = mapper.readTree(json);
-            assertEquals(20, node.get("result").get("data").get(0).intValue());
+            assertEquals(20, node.get("result").get("data").get(0).get("@value").intValue());
         }
     }
 
@@ -365,7 +366,7 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
             assertEquals("application/json", response.getEntity().getContentType().getValue());
             final String json = EntityUtils.toString(response.getEntity());
             final JsonNode node = mapper.readTree(json);
-            assertEquals("stephen", node.get("result").get("data").get(0).get("properties").get("name").get(0).get(GraphSONTokens.VALUE).asText());
+            assertEquals("stephen", node.get("result").get("data").get(0).get("@value").get("properties").get("name").get(0).get(GraphSONTokens.VALUE).asText());
         }
     }
 
@@ -379,7 +380,7 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
             assertEquals("application/json", response.getEntity().getContentType().getValue());
             final String json = EntityUtils.toString(response.getEntity());
             final JsonNode node = mapper.readTree(json);
-            assertEquals(20, node.get("result").get("data").get(0).intValue());
+            assertEquals(20, node.get("result").get("data").get(0).get("@value").intValue());
         }
     }
 
@@ -449,7 +450,7 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
             assertEquals("application/json", response.getEntity().getContentType().getValue());
             final String json = EntityUtils.toString(response.getEntity());
             final JsonNode node = mapper.readTree(json);
-            assertEquals(Instant.MAX, Instant.parse(node.get("result").get("data").get(0).asText()));
+            assertEquals(Instant.MAX, Instant.parse(node.get("result").get("data").get(0).get("@value").asText()));
         }
     }
 
@@ -467,7 +468,7 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
             assertEquals("application/json", response.getEntity().getContentType().getValue());
             final String json = EntityUtils.toString(response.getEntity());
             final JsonNode node = mapper.readTree(json);
-            assertEquals(1, node.get("result").get("data").get(0).intValue());
+            assertEquals(1, node.get("result").get("data").get(0).get("@value").intValue());
         }
 
         final HttpGet httpget = new HttpGet(TestClientFactory.createURLString("?gremlin=g.V().count()"));
@@ -481,7 +482,7 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
                 assertEquals("application/json", response.getEntity().getContentType().getValue());
                 final String json = EntityUtils.toString(response.getEntity());
                 final JsonNode node = mapper.readTree(json);
-                assertEquals(1, node.get("result").get("data").get(0).intValue());
+                assertEquals(1, node.get("result").get("data").get(0).get("@value").intValue());
             }
         }
     }
@@ -599,7 +600,7 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
             assertEquals("application/json", response.getEntity().getContentType().getValue());
             final String json = EntityUtils.toString(response.getEntity());
             final JsonNode node = mapper.readTree(json);
-            assertEquals(20, node.get("result").get("data").get(0).intValue());
+            assertEquals(20, node.get("result").get("data").get(0).get("@value").intValue());
         }
     }
 
@@ -615,7 +616,7 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
             assertEquals("application/json", response.getEntity().getContentType().getValue());
             final String json = EntityUtils.toString(response.getEntity());
             final JsonNode node = mapper.readTree(json);
-            assertEquals(10, node.get("result").get("data").get(0).intValue());
+            assertEquals(10, node.get("result").get("data").get(0).get("@value").intValue());
         }
     }
 
@@ -631,7 +632,7 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
             assertEquals("application/json", response.getEntity().getContentType().getValue());
             final String json = EntityUtils.toString(response.getEntity());
             final JsonNode node = mapper.readTree(json);
-            assertEquals(10.5d, node.get("result").get("data").get(0).doubleValue(), 0.0001);
+            assertEquals(10.5d, node.get("result").get("data").get(0).get("@value").doubleValue(), 0.0001);
         }
     }
 
@@ -696,9 +697,9 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
             final String json = EntityUtils.toString(response.getEntity());
             final JsonNode node = mapper.readTree(json);
             assertEquals(true, node.get("result").get("data").isArray());
-            assertEquals(1, node.get("result").get("data").get(0).intValue());
-            assertEquals(2, node.get("result").get("data").get(1).intValue());
-            assertEquals(3, node.get("result").get("data").get(2).intValue());
+            assertEquals(1, node.get("result").get("data").get(0).get("@value").intValue());
+            assertEquals(2, node.get("result").get("data").get(1).get("@value").intValue());
+            assertEquals(3, node.get("result").get("data").get(2).get("@value").intValue());
         }
     }
 
@@ -715,7 +716,7 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
             final String json = EntityUtils.toString(response.getEntity());
             final JsonNode node = mapper.readTree(json);
             assertEquals(true, node.get("result").get("data").get(0).isObject());
-            assertEquals(1, node.get("result").get("data").get(0).get("y").asInt());
+            assertEquals(1, node.get("result").get("data").get(0).get("y").get("@value").asInt());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/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 1743e89..96cc132 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
@@ -675,7 +675,7 @@ public class GremlinServerIntegrateTest extends AbstractGremlinServerIntegration
 
     @Test
     public void shouldReceiveFailureOnBadGraphSONSerialization() throws Exception {
-        final Cluster cluster = TestClientFactory.build().serializer(Serializers.GRAPHSON_V1D0).create();
+        final Cluster cluster = TestClientFactory.build().serializer(Serializers.GRAPHSON_V2D0).create();
         final Client client = cluster.connect();
 
         try {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/driver/remote/gremlin-server-integration.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/driver/remote/gremlin-server-integration.yaml b/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/driver/remote/gremlin-server-integration.yaml
index a25c6b1..7d23fb9 100644
--- a/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/driver/remote/gremlin-server-integration.yaml
+++ b/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/driver/remote/gremlin-server-integration.yaml
@@ -35,9 +35,9 @@ serializers:
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { useMapperFromGraph: graph, custom: [groovy.json.JsonBuilder;org.apache.tinkerpop.gremlin.driver.ser.JsonBuilderGryoSerializer]}}
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GryoLiteMessageSerializerV1d0, config: { useMapperFromGraph: graph, custom: [groovy.json.JsonBuilder;org.apache.tinkerpop.gremlin.driver.ser.JsonBuilderGryoSerializer]}}
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { serializeResultToString: true}}
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { useMapperFromGraph: graph }}
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0] }}
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: { useMapperFromGraph: graph }}
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }}
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }}
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}
 processors:
   - { className: org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor, config: { sessionTimeout: 28800000 }}
 metrics: {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/server/gremlin-server-integration.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/server/gremlin-server-integration.yaml b/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/server/gremlin-server-integration.yaml
index d0f7b32..a467111 100644
--- a/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/server/gremlin-server-integration.yaml
+++ b/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/server/gremlin-server-integration.yaml
@@ -31,9 +31,9 @@ serializers:
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { useMapperFromGraph: graph, custom: [groovy.json.JsonBuilder;org.apache.tinkerpop.gremlin.driver.ser.JsonBuilderGryoSerializer]}}
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GryoLiteMessageSerializerV1d0, config: { useMapperFromGraph: graph, custom: [groovy.json.JsonBuilder;org.apache.tinkerpop.gremlin.driver.ser.JsonBuilderGryoSerializer]}}
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { serializeResultToString: true}}
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { useMapperFromGraph: graph }}
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0] }}
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: { useMapperFromGraph: graph }}
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }}
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }}
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}
 processors:
   - { className: org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor, config: { sessionTimeout: 28800000 }}
 metrics: {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/SerializationTest.java
----------------------------------------------------------------------
diff --git a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/SerializationTest.java b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/SerializationTest.java
index 4dacf48..a112777 100644
--- a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/SerializationTest.java
+++ b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/SerializationTest.java
@@ -21,9 +21,11 @@ package org.apache.tinkerpop.gremlin.structure;
 import org.apache.tinkerpop.gremlin.AbstractGremlinTest;
 import org.apache.tinkerpop.gremlin.LoadGraphWith;
 import org.apache.tinkerpop.gremlin.process.traversal.Path;
+import org.apache.tinkerpop.gremlin.process.traversal.util.Metrics;
 import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalMetrics;
 import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONIo;
 import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONTokens;
+import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONVersion;
 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.GryoWriter;
@@ -45,10 +47,12 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 import org.apache.tinkerpop.gremlin.process.traversal.step.util.Tree;
 import static org.hamcrest.CoreMatchers.instanceOf;
 
+import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.*;
 
 /**
@@ -245,7 +249,7 @@ public class SerializationTest {
             assertEquals("The objects differ", after, before);
         }
     }
-    
+
     public static class GraphSONTest extends AbstractGremlinTest {
         private final TypeReference<HashMap<String, Object>> mapTypeReference = new TypeReference<HashMap<String, Object>>() {
         };
@@ -253,7 +257,7 @@ public class SerializationTest {
         @Test
         @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
         public void shouldSerializeVertex() throws Exception {
-            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().create().createMapper();
+            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().version(GraphSONVersion.V1_0).create().createMapper();
             final Vertex v = graph.vertices(convertToVertexId("marko")).next();
             final String json = mapper.writeValueAsString(v);
             final Map<String, Object> m = mapper.readValue(json, mapTypeReference);
@@ -272,7 +276,7 @@ public class SerializationTest {
         @Test
         @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
         public void shouldSerializeEdge() throws Exception {
-            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().create().createMapper();
+            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().version(GraphSONVersion.V1_0).create().createMapper();
             final Edge e = g.E(convertToEdgeId("marko", "knows", "vadas")).next();
             final String json = mapper.writeValueAsString(e);
             final Map<String, Object> m = mapper.readValue(json, mapTypeReference);
@@ -286,7 +290,7 @@ public class SerializationTest {
         @Test
         @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
         public void shouldSerializeProperty() throws Exception {
-            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().create().createMapper();
+            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().version(GraphSONVersion.V1_0).create().createMapper();
             final Property p = g.E(convertToEdgeId("marko", "knows", "vadas")).next().property("weight");
             final String json = mapper.writeValueAsString(p);
             final Map<String, Object> m = mapper.readValue(json, mapTypeReference);
@@ -298,7 +302,7 @@ public class SerializationTest {
         @Test
         @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
         public void shouldSerializeVertexProperty() throws Exception {
-            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().create().createMapper();
+            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().version(GraphSONVersion.V1_0).create().createMapper();
             final VertexProperty vp = graph.vertices(convertToVertexId("marko")).next().property("name");
             final String json = mapper.writeValueAsString(vp);
             final Map<String, Object> m = mapper.readValue(json, mapTypeReference);
@@ -311,7 +315,7 @@ public class SerializationTest {
         @Test
         @LoadGraphWith(LoadGraphWith.GraphData.CREW)
         public void shouldSerializeVertexPropertyWithProperties() throws Exception {
-            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().create().createMapper();
+            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().version(GraphSONVersion.V1_0).create().createMapper();
             final VertexProperty vp = IteratorUtils.filter(graph.vertices(convertToVertexId("marko")).next().properties("location"), p -> p.value().equals("brussels")).next();
             final String json = mapper.writeValueAsString(vp);
             final Map<String, Object> m = mapper.readValue(json, mapTypeReference);
@@ -326,7 +330,7 @@ public class SerializationTest {
         @Test
         @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
         public void shouldSerializePath() throws Exception {
-            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().create().createMapper();
+            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().version(GraphSONVersion.V1_0).create().createMapper();
             final Path p = g.V(convertToVertexId("marko")).as("a").outE().as("b").inV().as("c").path()
                     .filter(t -> ((Vertex) t.get().objects().get(2)).value("name").equals("lop")).next();
             final String json = mapper.writeValueAsString(p);
@@ -354,8 +358,8 @@ public class SerializationTest {
         @Test
         @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
         public void shouldSerializeTraversalMetrics() throws Exception {
-            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().create().createMapper();
-            final TraversalMetrics tm = (TraversalMetrics) g.V().both().profile().next();
+            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().version(GraphSONVersion.V1_0).create().createMapper();
+            final TraversalMetrics tm = g.V().both().profile().next();
             final String json = mapper.writeValueAsString(tm);
             final Map<String, Object> m = mapper.readValue(json, mapTypeReference);
 
@@ -375,7 +379,7 @@ public class SerializationTest {
         @Test
         @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
         public void shouldSerializeTree() throws Exception {
-            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().create().createMapper();
+            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().version(GraphSONVersion.V1_0).create().createMapper();
             final Tree t = g.V(convertToVertexId("marko")).out().properties("name").tree().next();
             final String json = mapper.writeValueAsString(t);
             
@@ -426,4 +430,209 @@ public class SerializationTest {
             assertEquals(entry.getKey().toString(), branch2Prop.get(GraphSONTokens.KEY).get(GraphSONTokens.ID).toString());
         }
     }
+
+    public static class GraphSONV2d0Test extends AbstractGremlinTest {
+        private final TypeReference<HashMap<String, Object>> mapTypeReference = new TypeReference<HashMap<String, Object>>() {
+        };
+
+        @Test
+        @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+        public void shouldSerializeVertex() throws Exception {
+            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().version(GraphSONVersion.V2_0).create().createMapper();
+            final Vertex v = graph.vertices(convertToVertexId("marko")).next();
+            final String json = mapper.writeValueAsString(v);
+            final Vertex detached = mapper.readValue(json, Vertex.class);
+
+            assertNotNull(detached);
+            assertEquals(v.label(), detached.label());
+            assertEquals(v.id(), detached.id());
+            assertEquals(v.value("name").toString(), detached.value("name"));
+            assertEquals((Integer) v.value("age"), detached.value("age"));
+        }
+
+        @Test
+        @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+        public void shouldSerializeEdge() throws Exception {
+            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().version(GraphSONVersion.V2_0).create().createMapper();
+            final Edge e = g.E(convertToEdgeId("marko", "knows", "vadas")).next();
+            final String json = mapper.writeValueAsString(e);
+            final Edge detached = mapper.readValue(json, Edge.class);
+
+            assertNotNull(detached);
+            assertEquals(e.label(), detached.label());
+            assertEquals(e.id(), detached.id());
+            assertEquals((Double) e.value("weight"), detached.value("weight"));
+        }
+
+        @Test
+        @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+        public void shouldSerializeProperty() throws Exception {
+            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().version(GraphSONVersion.V2_0).create().createMapper();
+            final Property p = g.E(convertToEdgeId("marko", "knows", "vadas")).next().property("weight");
+            final String json = mapper.writeValueAsString(p);
+            final Property detached = mapper.readValue(json, Property.class);
+
+            assertNotNull(detached);
+            assertEquals(p.key(), detached.key());
+            assertEquals(p.value(), detached.value());
+        }
+
+        @Test
+        @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+        public void shouldSerializeVertexProperty() throws Exception {
+            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().version(GraphSONVersion.V2_0).create().createMapper();
+            final VertexProperty vp = graph.vertices(convertToVertexId("marko")).next().property("name");
+            final String json = mapper.writeValueAsString(vp);
+            final VertexProperty detached = mapper.readValue(json, VertexProperty.class);
+
+            assertNotNull(detached);
+            assertEquals(vp.label(), detached.label());
+            assertEquals(vp.id(), detached.id());
+            assertEquals(vp.value(), detached.value());
+        }
+
+        @Test
+        @LoadGraphWith(LoadGraphWith.GraphData.CREW)
+        public void shouldSerializeVertexPropertyWithProperties() throws Exception {
+            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().version(GraphSONVersion.V2_0).create().createMapper();
+            final VertexProperty vp = IteratorUtils.filter(graph.vertices(convertToVertexId("marko")).next().properties("location"), p -> p.value().equals("brussels")).next();
+            final String json = mapper.writeValueAsString(vp);
+            final VertexProperty<?> detached = mapper.readValue(json, VertexProperty.class);
+
+            assertNotNull(detached);
+            assertEquals(vp.label(), detached.label());
+            assertEquals(vp.id(), detached.id());
+            assertEquals(vp.value(), detached.value());
+            assertEquals(vp.values("startTime").next(), detached.values("startTime").next());
+            assertEquals(((Property) vp.properties("startTime").next()).key(), ((Property) detached.properties("startTime").next()).key());
+            assertEquals(vp.values("endTime").next(), detached.values("endTime").next());
+            assertEquals(((Property) vp.properties("endTime").next()).key(), ((Property) detached.properties("endTime").next()).key());
+        }
+
+        @Test
+        @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+        public void shouldSerializePath() throws Exception {
+            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().version(GraphSONVersion.V2_0).create().createMapper();
+            final Path p = g.V(convertToVertexId("marko")).as("a").outE().as("b").inV().as("c").path()
+                    .filter(t -> ((Vertex) t.get().objects().get(2)).value("name").equals("lop")).next();
+            final String json = mapper.writeValueAsString(p);
+            final Path detached = mapper.readValue(json, Path.class);
+
+            assertNotNull(detached);
+            assertEquals(p.labels().size(), detached.labels().size());
+            assertEquals(p.labels().get(0).size(), detached.labels().get(0).size());
+            assertEquals(p.labels().get(1).size(), detached.labels().get(1).size());
+            assertEquals(p.labels().get(2).size(), detached.labels().get(2).size());
+            assertTrue(p.labels().stream().flatMap(Collection::stream).allMatch(detached::hasLabel));
+
+            final Vertex vOut = p.get("a");
+            final Vertex detachedVOut = detached.get("a");
+            assertEquals(vOut.label(), detachedVOut.label());
+            assertEquals(vOut.id(), detachedVOut.id());
+
+            // TODO: dunno GraphSON seems to return properties - will make this more consistent here
+            // this is a SimpleTraverser so no properties are present in detachment
+            //assertFalse(detachedVOut.properties().hasNext());
+
+            final Edge e = p.get("b");
+            final Edge detachedE = detached.get("b");
+            assertEquals(e.label(), detachedE.label());
+            assertEquals(e.id(), detachedE.id());
+
+            // TODO: dunno GraphSON seems to return properties - will make this more consistent here
+            // this is a SimpleTraverser so no properties are present in detachment
+            //assertFalse(detachedE.properties().hasNext());
+
+            final Vertex vIn = p.get("c");
+            final Vertex detachedVIn = detached.get("c");
+            assertEquals(vIn.label(), detachedVIn.label());
+            assertEquals(vIn.id(), detachedVIn.id());
+
+            // TODO: dunno GraphSON seems to return properties - will make this more consistent here
+            // this is a SimpleTraverser so no properties are present in detachment
+            //assertFalse(detachedVIn.properties().hasNext());
+        }
+
+        @Test
+        @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+        public void shouldSerializeTraversalMetrics() throws Exception {
+            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().version(GraphSONVersion.V2_0).create().createMapper();
+            final TraversalMetrics before = g.V().both().profile().next();
+            final String json = mapper.writeValueAsString(before);
+            final TraversalMetrics after = mapper.readValue(json, TraversalMetrics.class);
+
+            assertNotNull(after);
+            assertEquals(before.getMetrics().size(), after.getMetrics().size());
+            assertEquals(before.getDuration(TimeUnit.MILLISECONDS), after.getDuration(TimeUnit.MILLISECONDS));
+            assertEquals(before.getMetrics().size(), after.getMetrics().size());
+
+            before.getMetrics().forEach(b -> {
+                final Optional<? extends Metrics> mFromA = after.getMetrics().stream().filter(a -> b.getId().equals(a.getId())).findFirst();
+                if (mFromA.isPresent()) {
+                    final Metrics m = mFromA.get();
+                    assertEquals(b.getAnnotations(), m.getAnnotations());
+                    assertEquals(b.getCounts(), m.getCounts());
+                    assertEquals(b.getName(), m.getName());
+                    assertEquals(b.getDuration(TimeUnit.MILLISECONDS), m.getDuration(TimeUnit.MILLISECONDS));
+                } else {
+                    fail("Metrics were not present after deserialization");
+                }
+            });
+        }
+
+        @Test
+        @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+        @org.junit.Ignore("TINKERPOP-1509")
+        public void shouldSerializeTree() throws Exception {
+            final ObjectMapper mapper = graph.io(GraphSONIo.build()).mapper().version(GraphSONVersion.V2_0).create().createMapper();
+            final Tree t = g.V(convertToVertexId("marko")).out().properties("name").tree().next();
+            final String json = mapper.writeValueAsString(t);
+
+            final HashMap<String, Object> m = (HashMap<String, Object>) mapper.readValue(json, mapTypeReference);
+
+            // Check Structure
+            assertEquals(1, m.size());
+            assertTrue(m.containsKey(convertToVertexId("marko").toString()));
+
+            // Check Structure n+1
+            final HashMap<String, Object> branch = (HashMap<String, Object>) m.get(convertToVertexId("marko").toString());
+            assertEquals(2, branch.size());
+            assertTrue(branch.containsKey(GraphSONTokens.KEY));
+            assertTrue(branch.containsKey(GraphSONTokens.VALUE));
+
+            //Check n+1 key (traversed element)
+            final HashMap<String, Object> branchKey = (HashMap<String, Object>) branch.get(GraphSONTokens.KEY);
+            assertTrue(branchKey.containsKey(GraphSONTokens.ID));
+            assertTrue(branchKey.containsKey(GraphSONTokens.LABEL));
+            assertTrue(branchKey.containsKey(GraphSONTokens.TYPE));
+            assertTrue(branchKey.containsKey(GraphSONTokens.PROPERTIES));
+            assertEquals(convertToVertexId("marko").toString(), branchKey.get(GraphSONTokens.ID).toString());
+            assertEquals("person", branchKey.get(GraphSONTokens.LABEL));
+            assertEquals("vertex", branchKey.get(GraphSONTokens.TYPE));
+            final HashMap<String, List<HashMap<String, Object>>> branchKeyProps = (HashMap<String, List<HashMap<String, Object>>>) branchKey.get(GraphSONTokens.PROPERTIES);
+            assertEquals("marko", branchKeyProps.get("name").get(0).get("value"));
+            assertEquals(29, branchKeyProps.get("age").get(0).get("value"));
+
+            //Check n+1 value (traversed element)
+            final HashMap<String, Object> branchValue = (HashMap<String, Object>) branch.get(GraphSONTokens.VALUE);
+            assertEquals(3, branchValue.size());
+            assertTrue(branchValue.containsKey(convertToVertexId("vadas").toString()));
+            assertTrue(branchValue.containsKey(convertToVertexId("lop").toString()));
+            assertTrue(branchValue.containsKey(convertToVertexId("josh").toString()));
+
+            // Check that vp[] functioned properly
+            final HashMap<String, HashMap<String, Object>> branch2 = (HashMap<String, HashMap<String, Object>>) branchValue.get(convertToVertexId("vadas").toString());
+            assertTrue(branch2.containsKey(GraphSONTokens.KEY));
+            assertTrue(branch2.containsKey(GraphSONTokens.VALUE));
+
+            final Map.Entry entry = branch2.get(GraphSONTokens.VALUE).entrySet().iterator().next();
+            final HashMap<String, HashMap<String, Object>> branch2Prop = (HashMap<String, HashMap<String, Object>>) entry.getValue();
+            assertTrue(branch2Prop.get(GraphSONTokens.KEY).containsKey(GraphSONTokens.ID));
+            assertTrue(branch2Prop.get(GraphSONTokens.KEY).containsKey(GraphSONTokens.VALUE));
+            assertTrue(branch2Prop.get(GraphSONTokens.KEY).containsKey(GraphSONTokens.LABEL));
+            assertEquals("name", branch2Prop.get(GraphSONTokens.KEY).get(GraphSONTokens.LABEL));
+            assertEquals("vadas", branch2Prop.get(GraphSONTokens.KEY).get(GraphSONTokens.VALUE));
+            assertEquals(entry.getKey().toString(), branch2Prop.get(GraphSONTokens.KEY).get(GraphSONTokens.ID).toString());
+        }
+    }
 }


[10/31] tinkerpop git commit: A proof that TINKERPOP-1261 is fixed given the refactor of GroupBiOperator.

Posted by sp...@apache.org.
A proof that TINKERPOP-1261 is fixed given the refactor of GroupBiOperator.


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

Branch: refs/heads/TINKERPOP-1565
Commit: d54b490a66837e98077fa1e90f45ad20497c1a19
Parents: 341ebf9
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Thu Jan 19 08:57:36 2017 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Thu Jan 19 08:57:36 2017 -0700

----------------------------------------------------------------------
 .../step/sideEffect/GroovyGroupTest.groovy      |  5 ++++
 .../traversal/step/sideEffect/GroupTest.java    | 30 ++++++++++++++++++++
 2 files changed, 35 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/d54b490a/gremlin-groovy-test/src/main/groovy/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroovyGroupTest.groovy
----------------------------------------------------------------------
diff --git a/gremlin-groovy-test/src/main/groovy/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroovyGroupTest.groovy b/gremlin-groovy-test/src/main/groovy/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroovyGroupTest.groovy
index 3ce9efe..fc0c55d 100644
--- a/gremlin-groovy-test/src/main/groovy/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroovyGroupTest.groovy
+++ b/gremlin-groovy-test/src/main/groovy/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroovyGroupTest.groovy
@@ -128,5 +128,10 @@ public abstract class GroovyGroupTest {
         public Traversal<Vertex, Map<String, Number>> get_g_V_group_byXlabelX_byXbothE_groupXaX_byXlabelX_byXweight_sumX_weight_sumX() {
             new ScriptTraversal<>(g, "gremlin-groovy", "g.V.group().by(label).by(bothE().group('a').by(label).by(values('weight').sum).weight.sum)")
         }
+
+        @Override
+        public Traversal<Vertex, Map<String, List<Object>>> get_g_withSideEffectXa__marko_666_noone_blahX_V_groupXaX_byXnameX_byXoutE_label_foldX_capXaX() {
+            new ScriptTraversal<>(g, "gremlin-groovy", "g.withSideEffect('a', map).V().group('a').by('name').by(outE().label.fold).cap('a')", "map", new HashMap<>(["marko": [666], "noone": ["blah"]]));
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/d54b490a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroupTest.java
----------------------------------------------------------------------
diff --git a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroupTest.java b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroupTest.java
index 71b15a5..5f2504e 100644
--- a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroupTest.java
+++ b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/sideEffect/GroupTest.java
@@ -28,9 +28,12 @@ import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 
@@ -94,6 +97,8 @@ public abstract class GroupTest extends AbstractGremlinProcessTest {
 
     public abstract Traversal<Vertex, Map<String, Number>> get_g_V_group_byXlabelX_byXbothE_groupXaX_byXlabelX_byXweight_sumX_weight_sumX();
 
+    public abstract Traversal<Vertex, Map<String, List<Object>>> get_g_withSideEffectXa__marko_666_noone_blahX_V_groupXaX_byXnameX_byXoutE_label_foldX_capXaX();
+
     @Test
     @LoadGraphWith(MODERN)
     public void g_V_group_byXnameX() {
@@ -462,6 +467,23 @@ public abstract class GroupTest extends AbstractGremlinProcessTest {
         assertEquals(3.0d, sideEffect.get("knows").doubleValue(), 0.01d);
     }
 
+    @Test
+    @LoadGraphWith(MODERN)
+    public void g_withSideEffectXa__marko_666_noone_blahX_V_groupXaX_byXnameX_byXoutE_label_foldX_capXaX() {
+        final Traversal<Vertex, Map<String, List<Object>>> traversal = get_g_withSideEffectXa__marko_666_noone_blahX_V_groupXaX_byXnameX_byXoutE_label_foldX_capXaX();
+        printTraversalForm(traversal);
+        final Map<String, List<Object>> map = traversal.next();
+        assertEquals(7, map.size());
+        assertEquals(Collections.singleton("blah"), new HashSet<>(map.get("noone")));
+        assertEquals(new HashSet<>(Arrays.asList("created", "knows", 666)), new HashSet<>(map.get("marko")));
+        assertEquals(Collections.singleton("created"), new HashSet<>(map.get("josh")));
+        assertEquals(Collections.singleton("created"), new HashSet<>(map.get("peter")));
+        assertEquals(Collections.emptySet(), new HashSet<>(map.get("vadas")));
+        assertEquals(Collections.emptySet(), new HashSet<>(map.get("lop")));
+        assertEquals(Collections.emptySet(), new HashSet<>(map.get("ripple")));
+        checkSideEffects(traversal.asAdmin().getSideEffects(), "a", HashMap.class);
+    }
+
     public static class Traversals extends GroupTest {
 
         @Override
@@ -563,5 +585,13 @@ public abstract class GroupTest extends AbstractGremlinProcessTest {
         public Traversal<Vertex, Map<String, Number>> get_g_V_group_byXlabelX_byXbothE_groupXaX_byXlabelX_byXweight_sumX_weight_sumX() {
             return g.V().<String, Number>group().by(T.label).by(bothE().group("a").by(T.label).by(values("weight").sum()).values("weight").sum());
         }
+
+        @Override
+        public Traversal<Vertex, Map<String, List<Object>>> get_g_withSideEffectXa__marko_666_noone_blahX_V_groupXaX_byXnameX_byXoutE_label_foldX_capXaX() {
+            final Map<String, List<Object>> map = new HashMap<>();
+            map.put("marko", new ArrayList<>(Collections.singleton(666)));
+            map.put("noone", new ArrayList<>(Collections.singleton("blah")));
+            return g.withSideEffect("a", map).V().group("a").by("name").by(outE().label().fold()).cap("a");
+        }
     }
 }


[22/31] tinkerpop git commit: TINKERPOP-1565 Stub out GraphSON 3.0

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlessevalaliased-v2d0-partial.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlessevalaliased-v2d0-partial.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlessevalaliased-v2d0-partial.json
index 761b71a..0f6a54e 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlessevalaliased-v2d0-partial.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlessevalaliased-v2d0-partial.json
@@ -1,8 +1,5 @@
 {
-  "requestId" : {
-    "@type" : "g:UUID",
-    "@value" : "cb682578-9d92-4499-9ebc-5c6aa73c5397"
-  },
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
   "op" : "eval",
   "processor" : "",
   "args" : {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlessevalaliased-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlessevalaliased-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlessevalaliased-v3d0.json
new file mode 100644
index 0000000..0f6a54e
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlessevalaliased-v3d0.json
@@ -0,0 +1,18 @@
+{
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
+  "op" : "eval",
+  "processor" : "",
+  "args" : {
+    "gremlin" : "social.V(x)",
+    "language" : "gremlin-groovy",
+    "aliases" : {
+      "g" : "social"
+    },
+    "bindings" : {
+      "x" : {
+        "@type" : "g:Int32",
+        "@value" : 1
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/short-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/short-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/short-v3d0.json
new file mode 100644
index 0000000..c68f5cd
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/short-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:Int16",
+  "@value" : 100
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/standardresult-v2d0-no-types.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/standardresult-v2d0-no-types.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/standardresult-v2d0-no-types.json
index 6f65698..4fd2fd3 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/standardresult-v2d0-no-types.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/standardresult-v2d0-no-types.json
@@ -13,13 +13,11 @@
         "name" : [ {
           "id" : 0,
           "value" : "marko",
-          "vertex" : 1,
           "label" : "name"
         } ],
         "location" : [ {
           "id" : 6,
           "value" : "san diego",
-          "vertex" : 1,
           "label" : "location",
           "properties" : {
             "startTime" : 1997,
@@ -28,7 +26,6 @@
         }, {
           "id" : 7,
           "value" : "santa cruz",
-          "vertex" : 1,
           "label" : "location",
           "properties" : {
             "startTime" : 2001,
@@ -37,7 +34,6 @@
         }, {
           "id" : 8,
           "value" : "brussels",
-          "vertex" : 1,
           "label" : "location",
           "properties" : {
             "startTime" : 2004,
@@ -46,7 +42,6 @@
         }, {
           "id" : 9,
           "value" : "santa fe",
-          "vertex" : 1,
           "label" : "location",
           "properties" : {
             "startTime" : 2005

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/standardresult-v2d0-partial.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/standardresult-v2d0-partial.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/standardresult-v2d0-partial.json
index e4d6ce0..857c6db 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/standardresult-v2d0-partial.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/standardresult-v2d0-partial.json
@@ -23,10 +23,6 @@
                 "@value" : 0
               },
               "value" : "marko",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "name"
             }
           } ],
@@ -38,10 +34,6 @@
                 "@value" : 6
               },
               "value" : "san diego",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -62,10 +54,6 @@
                 "@value" : 7
               },
               "value" : "santa cruz",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -86,10 +74,6 @@
                 "@value" : 8
               },
               "value" : "brussels",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -110,10 +94,6 @@
                 "@value" : 9
               },
               "value" : "santa fe",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/standardresult-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/standardresult-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/standardresult-v3d0.json
new file mode 100644
index 0000000..2b48871
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/standardresult-v3d0.json
@@ -0,0 +1,91 @@
+{
+  "requestId" : "41d2e28a-20a4-4ab0-b379-d810dede3786",
+  "status" : {
+    "message" : "",
+    "code" : 200,
+    "attributes" : { }
+  },
+  "result" : {
+    "data" : [ {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "label" : "person",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 0
+            },
+            "value" : "marko"
+          } ],
+          "location" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 6
+            },
+            "value" : "san diego",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 1997
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 7
+            },
+            "value" : "santa cruz",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 8
+            },
+            "value" : "brussels",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 9
+            },
+            "value" : "santa fe",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          } ]
+        }
+      }
+    } ],
+    "meta" : { }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/stargraph-v2d0-no-types.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/stargraph-v2d0-no-types.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/stargraph-v2d0-no-types.json
index eb87f72..c8f8fdb 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/stargraph-v2d0-no-types.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/stargraph-v2d0-no-types.json
@@ -6,13 +6,11 @@
       "name" : [ {
         "id" : 0,
         "value" : "marko",
-        "vertex" : 1,
         "label" : "name"
       } ],
       "location" : [ {
         "id" : 6,
         "value" : "san diego",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 1997,
@@ -21,7 +19,6 @@
       }, {
         "id" : 7,
         "value" : "santa cruz",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 2001,
@@ -30,7 +27,6 @@
       }, {
         "id" : 8,
         "value" : "brussels",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 2004,
@@ -39,7 +35,6 @@
       }, {
         "id" : 9,
         "value" : "santa fe",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 2005

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/stargraph-v2d0-partial.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/stargraph-v2d0-partial.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/stargraph-v2d0-partial.json
index d7bb548..59ff13a 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/stargraph-v2d0-partial.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/stargraph-v2d0-partial.json
@@ -16,10 +16,6 @@
               "@value" : 0
             },
             "value" : "marko",
-            "vertex" : {
-              "@type" : "g:Int32",
-              "@value" : 1
-            },
             "label" : "name"
           }
         } ],
@@ -31,10 +27,6 @@
               "@value" : 6
             },
             "value" : "san diego",
-            "vertex" : {
-              "@type" : "g:Int32",
-              "@value" : 1
-            },
             "label" : "location",
             "properties" : {
               "startTime" : {
@@ -55,10 +47,6 @@
               "@value" : 7
             },
             "value" : "santa cruz",
-            "vertex" : {
-              "@type" : "g:Int32",
-              "@value" : 1
-            },
             "label" : "location",
             "properties" : {
               "startTime" : {
@@ -79,10 +67,6 @@
               "@value" : 8
             },
             "value" : "brussels",
-            "vertex" : {
-              "@type" : "g:Int32",
-              "@value" : 1
-            },
             "label" : "location",
             "properties" : {
               "startTime" : {
@@ -103,10 +87,6 @@
               "@value" : 9
             },
             "value" : "santa fe",
-            "vertex" : {
-              "@type" : "g:Int32",
-              "@value" : 1
-            },
             "label" : "location",
             "properties" : {
               "startTime" : {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/stargraph-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/stargraph-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/stargraph-v3d0.json
new file mode 100644
index 0000000..0d82b27
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/stargraph-v3d0.json
@@ -0,0 +1,82 @@
+{
+  "starVertex" : {
+    "@type" : "g:Vertex",
+    "@value" : {
+      "id" : {
+        "@type" : "g:Int32",
+        "@value" : 1
+      },
+      "label" : "person",
+      "properties" : {
+        "name" : [ {
+          "id" : {
+            "@type" : "g:Int64",
+            "@value" : 0
+          },
+          "value" : "marko"
+        } ],
+        "location" : [ {
+          "id" : {
+            "@type" : "g:Int64",
+            "@value" : 6
+          },
+          "value" : "san diego",
+          "properties" : {
+            "startTime" : {
+              "@type" : "g:Int32",
+              "@value" : 1997
+            },
+            "endTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2001
+            }
+          }
+        }, {
+          "id" : {
+            "@type" : "g:Int64",
+            "@value" : 7
+          },
+          "value" : "santa cruz",
+          "properties" : {
+            "startTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2001
+            },
+            "endTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2004
+            }
+          }
+        }, {
+          "id" : {
+            "@type" : "g:Int64",
+            "@value" : 8
+          },
+          "value" : "brussels",
+          "properties" : {
+            "startTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2004
+            },
+            "endTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2005
+            }
+          }
+        }, {
+          "id" : {
+            "@type" : "g:Int64",
+            "@value" : 9
+          },
+          "value" : "santa fe",
+          "properties" : {
+            "startTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2005
+            }
+          }
+        } ]
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/t-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/t-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/t-v3d0.json
new file mode 100644
index 0000000..9693983
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/t-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:T",
+  "@value" : "label"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/timestamp-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/timestamp-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/timestamp-v3d0.json
new file mode 100644
index 0000000..1ca0e17
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/timestamp-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:Timestamp",
+  "@value" : 1481750076295
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v2d0-no-types.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v2d0-no-types.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v2d0-no-types.json
index 8b9376a..94ad061 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v2d0-no-types.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v2d0-no-types.json
@@ -6,13 +6,11 @@
       "name" : [ {
         "id" : 0,
         "value" : "marko",
-        "vertex" : 1,
         "label" : "name"
       } ],
       "location" : [ {
         "id" : 6,
         "value" : "san diego",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 1997,
@@ -21,7 +19,6 @@
       }, {
         "id" : 7,
         "value" : "santa cruz",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 2001,
@@ -30,7 +27,6 @@
       }, {
         "id" : 8,
         "value" : "brussels",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 2004,
@@ -39,7 +35,6 @@
       }, {
         "id" : 9,
         "value" : "santa fe",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 2005
@@ -53,13 +48,11 @@
       "name" : [ {
         "id" : 1,
         "value" : "stephen",
-        "vertex" : 7,
         "label" : "name"
       } ],
       "location" : [ {
         "id" : 10,
         "value" : "centreville",
-        "vertex" : 7,
         "label" : "location",
         "properties" : {
           "startTime" : 1990,
@@ -68,7 +61,6 @@
       }, {
         "id" : 11,
         "value" : "dulles",
-        "vertex" : 7,
         "label" : "location",
         "properties" : {
           "startTime" : 2000,
@@ -77,7 +69,6 @@
       }, {
         "id" : 12,
         "value" : "purcellville",
-        "vertex" : 7,
         "label" : "location",
         "properties" : {
           "startTime" : 2006
@@ -91,13 +82,11 @@
       "name" : [ {
         "id" : 2,
         "value" : "matthias",
-        "vertex" : 8,
         "label" : "name"
       } ],
       "location" : [ {
         "id" : 13,
         "value" : "bremen",
-        "vertex" : 8,
         "label" : "location",
         "properties" : {
           "startTime" : 2004,
@@ -106,7 +95,6 @@
       }, {
         "id" : 14,
         "value" : "baltimore",
-        "vertex" : 8,
         "label" : "location",
         "properties" : {
           "startTime" : 2007,
@@ -115,7 +103,6 @@
       }, {
         "id" : 15,
         "value" : "oakland",
-        "vertex" : 8,
         "label" : "location",
         "properties" : {
           "startTime" : 2011,
@@ -124,7 +111,6 @@
       }, {
         "id" : 16,
         "value" : "seattle",
-        "vertex" : 8,
         "label" : "location",
         "properties" : {
           "startTime" : 2014
@@ -138,13 +124,11 @@
       "name" : [ {
         "id" : 3,
         "value" : "daniel",
-        "vertex" : 9,
         "label" : "name"
       } ],
       "location" : [ {
         "id" : 17,
         "value" : "spremberg",
-        "vertex" : 9,
         "label" : "location",
         "properties" : {
           "startTime" : 1982,
@@ -153,7 +137,6 @@
       }, {
         "id" : 18,
         "value" : "kaiserslautern",
-        "vertex" : 9,
         "label" : "location",
         "properties" : {
           "startTime" : 2005,
@@ -162,7 +145,6 @@
       }, {
         "id" : 19,
         "value" : "aachen",
-        "vertex" : 9,
         "label" : "location",
         "properties" : {
           "startTime" : 2009
@@ -176,7 +158,6 @@
       "name" : [ {
         "id" : 4,
         "value" : "gremlin",
-        "vertex" : 10,
         "label" : "name"
       } ]
     }
@@ -187,7 +168,6 @@
       "name" : [ {
         "id" : 5,
         "value" : "tinkergraph",
-        "vertex" : 11,
         "label" : "name"
       } ]
     }
@@ -200,7 +180,10 @@
     "inV" : 10,
     "outV" : 1,
     "properties" : {
-      "since" : 2009
+      "since" : {
+        "key" : "since",
+        "value" : 2009
+      }
     }
   }, {
     "id" : 14,
@@ -210,7 +193,10 @@
     "inV" : 11,
     "outV" : 1,
     "properties" : {
-      "since" : 2010
+      "since" : {
+        "key" : "since",
+        "value" : 2010
+      }
     }
   }, {
     "id" : 15,
@@ -220,7 +206,10 @@
     "inV" : 10,
     "outV" : 1,
     "properties" : {
-      "skill" : 4
+      "skill" : {
+        "key" : "skill",
+        "value" : 4
+      }
     }
   }, {
     "id" : 16,
@@ -230,7 +219,10 @@
     "inV" : 11,
     "outV" : 1,
     "properties" : {
-      "skill" : 5
+      "skill" : {
+        "key" : "skill",
+        "value" : 5
+      }
     }
   }, {
     "id" : 17,
@@ -240,7 +232,10 @@
     "inV" : 10,
     "outV" : 7,
     "properties" : {
-      "since" : 2010
+      "since" : {
+        "key" : "since",
+        "value" : 2010
+      }
     }
   }, {
     "id" : 18,
@@ -250,7 +245,10 @@
     "inV" : 11,
     "outV" : 7,
     "properties" : {
-      "since" : 2011
+      "since" : {
+        "key" : "since",
+        "value" : 2011
+      }
     }
   }, {
     "id" : 19,
@@ -260,7 +258,10 @@
     "inV" : 10,
     "outV" : 7,
     "properties" : {
-      "skill" : 5
+      "skill" : {
+        "key" : "skill",
+        "value" : 5
+      }
     }
   }, {
     "id" : 20,
@@ -270,7 +271,10 @@
     "inV" : 11,
     "outV" : 7,
     "properties" : {
-      "skill" : 4
+      "skill" : {
+        "key" : "skill",
+        "value" : 4
+      }
     }
   }, {
     "id" : 21,
@@ -280,7 +284,10 @@
     "inV" : 10,
     "outV" : 8,
     "properties" : {
-      "since" : 2012
+      "since" : {
+        "key" : "since",
+        "value" : 2012
+      }
     }
   }, {
     "id" : 22,
@@ -290,7 +297,10 @@
     "inV" : 10,
     "outV" : 8,
     "properties" : {
-      "skill" : 3
+      "skill" : {
+        "key" : "skill",
+        "value" : 3
+      }
     }
   }, {
     "id" : 23,
@@ -300,7 +310,10 @@
     "inV" : 11,
     "outV" : 8,
     "properties" : {
-      "skill" : 3
+      "skill" : {
+        "key" : "skill",
+        "value" : 3
+      }
     }
   }, {
     "id" : 24,
@@ -310,7 +323,10 @@
     "inV" : 10,
     "outV" : 9,
     "properties" : {
-      "skill" : 5
+      "skill" : {
+        "key" : "skill",
+        "value" : 5
+      }
     }
   }, {
     "id" : 25,
@@ -320,7 +336,10 @@
     "inV" : 11,
     "outV" : 9,
     "properties" : {
-      "skill" : 3
+      "skill" : {
+        "key" : "skill",
+        "value" : 3
+      }
     }
   }, {
     "id" : 26,

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v2d0-partial.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v2d0-partial.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v2d0-partial.json
index f2df53e..24e95ed 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v2d0-partial.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v2d0-partial.json
@@ -18,10 +18,6 @@
                 "@value" : 0
               },
               "value" : "marko",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "name"
             }
           } ],
@@ -33,10 +29,6 @@
                 "@value" : 6
               },
               "value" : "san diego",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -57,10 +49,6 @@
                 "@value" : 7
               },
               "value" : "santa cruz",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -81,10 +69,6 @@
                 "@value" : 8
               },
               "value" : "brussels",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -105,10 +89,6 @@
                 "@value" : 9
               },
               "value" : "santa fe",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -137,10 +117,6 @@
                 "@value" : 1
               },
               "value" : "stephen",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 7
-              },
               "label" : "name"
             }
           } ],
@@ -152,10 +128,6 @@
                 "@value" : 10
               },
               "value" : "centreville",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 7
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -176,10 +148,6 @@
                 "@value" : 11
               },
               "value" : "dulles",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 7
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -200,10 +168,6 @@
                 "@value" : 12
               },
               "value" : "purcellville",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 7
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -232,10 +196,6 @@
                 "@value" : 2
               },
               "value" : "matthias",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 8
-              },
               "label" : "name"
             }
           } ],
@@ -247,10 +207,6 @@
                 "@value" : 13
               },
               "value" : "bremen",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 8
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -271,10 +227,6 @@
                 "@value" : 14
               },
               "value" : "baltimore",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 8
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -295,10 +247,6 @@
                 "@value" : 15
               },
               "value" : "oakland",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 8
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -319,10 +267,6 @@
                 "@value" : 16
               },
               "value" : "seattle",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 8
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -351,10 +295,6 @@
                 "@value" : 3
               },
               "value" : "daniel",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 9
-              },
               "label" : "name"
             }
           } ],
@@ -366,10 +306,6 @@
                 "@value" : 17
               },
               "value" : "spremberg",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 9
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -390,10 +326,6 @@
                 "@value" : 18
               },
               "value" : "kaiserslautern",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 9
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -414,10 +346,6 @@
                 "@value" : 19
               },
               "value" : "aachen",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 9
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -446,10 +374,6 @@
                 "@value" : 4
               },
               "value" : "gremlin",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 10
-              },
               "label" : "name"
             }
           } ]
@@ -472,10 +396,6 @@
                 "@value" : 5
               },
               "value" : "tinkergraph",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 11
-              },
               "label" : "name"
             }
           } ]
@@ -502,8 +422,14 @@
         },
         "properties" : {
           "since" : {
-            "@type" : "g:Int32",
-            "@value" : 2009
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "since",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 2009
+              }
+            }
           }
         }
       }
@@ -527,8 +453,14 @@
         },
         "properties" : {
           "since" : {
-            "@type" : "g:Int32",
-            "@value" : 2010
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "since",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 2010
+              }
+            }
           }
         }
       }
@@ -552,8 +484,14 @@
         },
         "properties" : {
           "skill" : {
-            "@type" : "g:Int32",
-            "@value" : 4
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "skill",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 4
+              }
+            }
           }
         }
       }
@@ -577,8 +515,14 @@
         },
         "properties" : {
           "skill" : {
-            "@type" : "g:Int32",
-            "@value" : 5
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "skill",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 5
+              }
+            }
           }
         }
       }
@@ -602,8 +546,14 @@
         },
         "properties" : {
           "since" : {
-            "@type" : "g:Int32",
-            "@value" : 2010
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "since",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 2010
+              }
+            }
           }
         }
       }
@@ -627,8 +577,14 @@
         },
         "properties" : {
           "since" : {
-            "@type" : "g:Int32",
-            "@value" : 2011
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "since",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 2011
+              }
+            }
           }
         }
       }
@@ -652,8 +608,14 @@
         },
         "properties" : {
           "skill" : {
-            "@type" : "g:Int32",
-            "@value" : 5
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "skill",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 5
+              }
+            }
           }
         }
       }
@@ -677,8 +639,14 @@
         },
         "properties" : {
           "skill" : {
-            "@type" : "g:Int32",
-            "@value" : 4
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "skill",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 4
+              }
+            }
           }
         }
       }
@@ -702,8 +670,14 @@
         },
         "properties" : {
           "since" : {
-            "@type" : "g:Int32",
-            "@value" : 2012
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "since",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 2012
+              }
+            }
           }
         }
       }
@@ -727,8 +701,14 @@
         },
         "properties" : {
           "skill" : {
-            "@type" : "g:Int32",
-            "@value" : 3
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "skill",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 3
+              }
+            }
           }
         }
       }
@@ -752,8 +732,14 @@
         },
         "properties" : {
           "skill" : {
-            "@type" : "g:Int32",
-            "@value" : 3
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "skill",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 3
+              }
+            }
           }
         }
       }
@@ -777,8 +763,14 @@
         },
         "properties" : {
           "skill" : {
-            "@type" : "g:Int32",
-            "@value" : 5
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "skill",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 5
+              }
+            }
           }
         }
       }
@@ -802,8 +794,14 @@
         },
         "properties" : {
           "skill" : {
-            "@type" : "g:Int32",
-            "@value" : 3
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "skill",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 3
+              }
+            }
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v3d0.json
new file mode 100644
index 0000000..a97c8c1
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v3d0.json
@@ -0,0 +1,671 @@
+{
+  "@type" : "tinker:graph",
+  "@value" : {
+    "vertices" : [ {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "label" : "person",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 0
+            },
+            "value" : "marko"
+          } ],
+          "location" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 6
+            },
+            "value" : "san diego",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 1997
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 7
+            },
+            "value" : "santa cruz",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 8
+            },
+            "value" : "brussels",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 9
+            },
+            "value" : "santa fe",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          } ]
+        }
+      }
+    }, {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 7
+        },
+        "label" : "person",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 1
+            },
+            "value" : "stephen"
+          } ],
+          "location" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 10
+            },
+            "value" : "centreville",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 1990
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2000
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 11
+            },
+            "value" : "dulles",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2000
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2006
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 12
+            },
+            "value" : "purcellville",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2006
+              }
+            }
+          } ]
+        }
+      }
+    }, {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 8
+        },
+        "label" : "person",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 2
+            },
+            "value" : "matthias"
+          } ],
+          "location" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 13
+            },
+            "value" : "bremen",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2007
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 14
+            },
+            "value" : "baltimore",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2007
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2011
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 15
+            },
+            "value" : "oakland",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2011
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2014
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 16
+            },
+            "value" : "seattle",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2014
+              }
+            }
+          } ]
+        }
+      }
+    }, {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 9
+        },
+        "label" : "person",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 3
+            },
+            "value" : "daniel"
+          } ],
+          "location" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 17
+            },
+            "value" : "spremberg",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 1982
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 18
+            },
+            "value" : "kaiserslautern",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2009
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 19
+            },
+            "value" : "aachen",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2009
+              }
+            }
+          } ]
+        }
+      }
+    }, {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "label" : "software",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 4
+            },
+            "value" : "gremlin"
+          } ]
+        }
+      }
+    }, {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "label" : "software",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 5
+            },
+            "value" : "tinkergraph"
+          } ]
+        }
+      }
+    } ],
+    "edges" : [ {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 13
+        },
+        "label" : "develops",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "properties" : {
+          "since" : {
+            "@type" : "g:Int32",
+            "@value" : 2009
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 14
+        },
+        "label" : "develops",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "properties" : {
+          "since" : {
+            "@type" : "g:Int32",
+            "@value" : 2010
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 15
+        },
+        "label" : "uses",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "properties" : {
+          "skill" : {
+            "@type" : "g:Int32",
+            "@value" : 4
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 16
+        },
+        "label" : "uses",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "properties" : {
+          "skill" : {
+            "@type" : "g:Int32",
+            "@value" : 5
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 17
+        },
+        "label" : "develops",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 7
+        },
+        "properties" : {
+          "since" : {
+            "@type" : "g:Int32",
+            "@value" : 2010
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 18
+        },
+        "label" : "develops",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 7
+        },
+        "properties" : {
+          "since" : {
+            "@type" : "g:Int32",
+            "@value" : 2011
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 19
+        },
+        "label" : "uses",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 7
+        },
+        "properties" : {
+          "skill" : {
+            "@type" : "g:Int32",
+            "@value" : 5
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 20
+        },
+        "label" : "uses",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 7
+        },
+        "properties" : {
+          "skill" : {
+            "@type" : "g:Int32",
+            "@value" : 4
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 21
+        },
+        "label" : "develops",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 8
+        },
+        "properties" : {
+          "since" : {
+            "@type" : "g:Int32",
+            "@value" : 2012
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 22
+        },
+        "label" : "uses",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 8
+        },
+        "properties" : {
+          "skill" : {
+            "@type" : "g:Int32",
+            "@value" : 3
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 23
+        },
+        "label" : "uses",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 8
+        },
+        "properties" : {
+          "skill" : {
+            "@type" : "g:Int32",
+            "@value" : 3
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 24
+        },
+        "label" : "uses",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 9
+        },
+        "properties" : {
+          "skill" : {
+            "@type" : "g:Int32",
+            "@value" : 5
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 25
+        },
+        "label" : "uses",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 9
+        },
+        "properties" : {
+          "skill" : {
+            "@type" : "g:Int32",
+            "@value" : 3
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 26
+        },
+        "label" : "traverses",
+        "inVLabel" : "software",
+        "outVLabel" : "software",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        }
+      }
+    } ]
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traversalmetrics-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traversalmetrics-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traversalmetrics-v3d0.json
new file mode 100644
index 0000000..fdd18a4
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traversalmetrics-v3d0.json
@@ -0,0 +1,114 @@
+{
+  "@type" : "g:TraversalMetrics",
+  "@value" : {
+    "dur" : {
+      "@type" : "g:Double",
+      "@value" : 0.004
+    },
+    "metrics" : [ {
+      "@type" : "g:Metrics",
+      "@value" : {
+        "dur" : {
+          "@type" : "g:Double",
+          "@value" : 100.0
+        },
+        "counts" : {
+          "traverserCount" : {
+            "@type" : "g:Int64",
+            "@value" : 4
+          },
+          "elementCount" : {
+            "@type" : "g:Int64",
+            "@value" : 4
+          }
+        },
+        "name" : "TinkerGraphStep(vertex,[~label.eq(person)])",
+        "annotations" : {
+          "percentDur" : {
+            "@type" : "g:Double",
+            "@value" : 25.0
+          }
+        },
+        "id" : "7.0.0()"
+      }
+    }, {
+      "@type" : "g:Metrics",
+      "@value" : {
+        "dur" : {
+          "@type" : "g:Double",
+          "@value" : 100.0
+        },
+        "counts" : {
+          "traverserCount" : {
+            "@type" : "g:Int64",
+            "@value" : 13
+          },
+          "elementCount" : {
+            "@type" : "g:Int64",
+            "@value" : 13
+          }
+        },
+        "name" : "VertexStep(OUT,vertex)",
+        "annotations" : {
+          "percentDur" : {
+            "@type" : "g:Double",
+            "@value" : 25.0
+          }
+        },
+        "id" : "2.0.0()"
+      }
+    }, {
+      "@type" : "g:Metrics",
+      "@value" : {
+        "dur" : {
+          "@type" : "g:Double",
+          "@value" : 100.0
+        },
+        "counts" : {
+          "traverserCount" : {
+            "@type" : "g:Int64",
+            "@value" : 7
+          },
+          "elementCount" : {
+            "@type" : "g:Int64",
+            "@value" : 7
+          }
+        },
+        "name" : "VertexStep(OUT,vertex)",
+        "annotations" : {
+          "percentDur" : {
+            "@type" : "g:Double",
+            "@value" : 25.0
+          }
+        },
+        "id" : "3.0.0()"
+      }
+    }, {
+      "@type" : "g:Metrics",
+      "@value" : {
+        "dur" : {
+          "@type" : "g:Double",
+          "@value" : 100.0
+        },
+        "counts" : {
+          "traverserCount" : {
+            "@type" : "g:Int64",
+            "@value" : 1
+          },
+          "elementCount" : {
+            "@type" : "g:Int64",
+            "@value" : 1
+          }
+        },
+        "name" : "TreeStep",
+        "annotations" : {
+          "percentDur" : {
+            "@type" : "g:Double",
+            "@value" : 25.0
+          }
+        },
+        "id" : "4.0.0()"
+      }
+    } ]
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v2d0-no-types.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v2d0-no-types.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v2d0-no-types.json
index f254df0..6d1f029 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v2d0-no-types.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v2d0-no-types.json
@@ -7,13 +7,11 @@
       "name" : [ {
         "id" : 0,
         "value" : "marko",
-        "vertex" : 1,
         "label" : "name"
       } ],
       "location" : [ {
         "id" : 6,
         "value" : "san diego",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 1997,
@@ -22,7 +20,6 @@
       }, {
         "id" : 7,
         "value" : "santa cruz",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 2001,
@@ -31,7 +28,6 @@
       }, {
         "id" : 8,
         "value" : "brussels",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 2004,
@@ -40,7 +36,6 @@
       }, {
         "id" : 9,
         "value" : "santa fe",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 2005

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v2d0-partial.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v2d0-partial.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v2d0-partial.json
index d7bbfeb..a59a29e 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v2d0-partial.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v2d0-partial.json
@@ -22,10 +22,6 @@
                 "@value" : 0
               },
               "value" : "marko",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "name"
             }
           } ],
@@ -37,10 +33,6 @@
                 "@value" : 6
               },
               "value" : "san diego",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -61,10 +53,6 @@
                 "@value" : 7
               },
               "value" : "santa cruz",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -85,10 +73,6 @@
                 "@value" : 8
               },
               "value" : "brussels",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -109,10 +93,6 @@
                 "@value" : 9
               },
               "value" : "santa fe",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v3d0.json
new file mode 100644
index 0000000..1e6a2ce
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v3d0.json
@@ -0,0 +1,89 @@
+{
+  "@type" : "g:Traverser",
+  "@value" : {
+    "bulk" : {
+      "@type" : "g:Int64",
+      "@value" : 1
+    },
+    "value" : {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "label" : "person",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 0
+            },
+            "value" : "marko"
+          } ],
+          "location" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 6
+            },
+            "value" : "san diego",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 1997
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 7
+            },
+            "value" : "santa cruz",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 8
+            },
+            "value" : "brussels",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 9
+            },
+            "value" : "santa fe",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          } ]
+        }
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v2d0-no-types.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v2d0-no-types.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v2d0-no-types.json
index c23fc2f..79231e8 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v2d0-no-types.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v2d0-no-types.json
@@ -6,13 +6,11 @@
       "name" : [ {
         "id" : 0,
         "value" : "marko",
-        "vertex" : 1,
         "label" : "name"
       } ],
       "location" : [ {
         "id" : 6,
         "value" : "san diego",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 1997,
@@ -21,7 +19,6 @@
       }, {
         "id" : 7,
         "value" : "santa cruz",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 2001,
@@ -30,7 +27,6 @@
       }, {
         "id" : 8,
         "value" : "brussels",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 2004,
@@ -39,7 +35,6 @@
       }, {
         "id" : 9,
         "value" : "santa fe",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 2005
@@ -55,7 +50,6 @@
         "name" : [ {
           "id" : 4,
           "value" : "gremlin",
-          "vertex" : 10,
           "label" : "name"
         } ]
       }
@@ -68,7 +62,6 @@
           "name" : [ {
             "id" : 5,
             "value" : "tinkergraph",
-            "vertex" : 11,
             "label" : "name"
           } ]
         }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v2d0-partial.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v2d0-partial.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v2d0-partial.json
index 2b6c597..74dcffc 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v2d0-partial.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v2d0-partial.json
@@ -18,10 +18,6 @@
                 "@value" : 0
               },
               "value" : "marko",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "name"
             }
           } ],
@@ -33,10 +29,6 @@
                 "@value" : 6
               },
               "value" : "san diego",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -57,10 +49,6 @@
                 "@value" : 7
               },
               "value" : "santa cruz",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -81,10 +69,6 @@
                 "@value" : 8
               },
               "value" : "brussels",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -105,10 +89,6 @@
                 "@value" : 9
               },
               "value" : "santa fe",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -141,10 +121,6 @@
                     "@value" : 4
                   },
                   "value" : "gremlin",
-                  "vertex" : {
-                    "@type" : "g:Int32",
-                    "@value" : 10
-                  },
                   "label" : "name"
                 }
               } ]
@@ -171,10 +147,6 @@
                         "@value" : 5
                       },
                       "value" : "tinkergraph",
-                      "vertex" : {
-                        "@type" : "g:Int32",
-                        "@value" : 11
-                      },
                       "label" : "name"
                     }
                   } ]

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v3d0.json
new file mode 100644
index 0000000..8868929
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v3d0.json
@@ -0,0 +1,137 @@
+{
+  "@type" : "g:Tree",
+  "@value" : [ {
+    "key" : {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "label" : "person",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 0
+            },
+            "value" : "marko"
+          } ],
+          "location" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 6
+            },
+            "value" : "san diego",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 1997
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 7
+            },
+            "value" : "santa cruz",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 8
+            },
+            "value" : "brussels",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 9
+            },
+            "value" : "santa fe",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          } ]
+        }
+      }
+    },
+    "value" : {
+      "@type" : "g:Tree",
+      "@value" : [ {
+        "key" : {
+          "@type" : "g:Vertex",
+          "@value" : {
+            "id" : {
+              "@type" : "g:Int32",
+              "@value" : 10
+            },
+            "label" : "software",
+            "properties" : {
+              "name" : [ {
+                "id" : {
+                  "@type" : "g:Int64",
+                  "@value" : 4
+                },
+                "value" : "gremlin"
+              } ]
+            }
+          }
+        },
+        "value" : {
+          "@type" : "g:Tree",
+          "@value" : [ {
+            "key" : {
+              "@type" : "g:Vertex",
+              "@value" : {
+                "id" : {
+                  "@type" : "g:Int32",
+                  "@value" : 11
+                },
+                "label" : "software",
+                "properties" : {
+                  "name" : [ {
+                    "id" : {
+                      "@type" : "g:Int64",
+                      "@value" : 5
+                    },
+                    "value" : "tinkergraph"
+                  } ]
+                }
+              }
+            },
+            "value" : {
+              "@type" : "g:Tree",
+              "@value" : [ ]
+            }
+          } ]
+        }
+      } ]
+    }
+  } ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/uuid-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/uuid-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/uuid-v3d0.json
new file mode 100644
index 0000000..1cf09f0
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/uuid-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:UUID",
+  "@value" : "41d2e28a-20a4-4ab0-b379-d810dede3786"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v2d0-no-types.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v2d0-no-types.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v2d0-no-types.json
index 5e819ba..8e6155f 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v2d0-no-types.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v2d0-no-types.json
@@ -5,13 +5,11 @@
     "name" : [ {
       "id" : 0,
       "value" : "marko",
-      "vertex" : 1,
       "label" : "name"
     } ],
     "location" : [ {
       "id" : 6,
       "value" : "san diego",
-      "vertex" : 1,
       "label" : "location",
       "properties" : {
         "startTime" : 1997,
@@ -20,7 +18,6 @@
     }, {
       "id" : 7,
       "value" : "santa cruz",
-      "vertex" : 1,
       "label" : "location",
       "properties" : {
         "startTime" : 2001,
@@ -29,7 +26,6 @@
     }, {
       "id" : 8,
       "value" : "brussels",
-      "vertex" : 1,
       "label" : "location",
       "properties" : {
         "startTime" : 2004,
@@ -38,7 +34,6 @@
     }, {
       "id" : 9,
       "value" : "santa fe",
-      "vertex" : 1,
       "label" : "location",
       "properties" : {
         "startTime" : 2005

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v2d0-partial.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v2d0-partial.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v2d0-partial.json
index 8358d9c..f102230 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v2d0-partial.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v2d0-partial.json
@@ -15,10 +15,6 @@
             "@value" : 0
           },
           "value" : "marko",
-          "vertex" : {
-            "@type" : "g:Int32",
-            "@value" : 1
-          },
           "label" : "name"
         }
       } ],
@@ -30,10 +26,6 @@
             "@value" : 6
           },
           "value" : "san diego",
-          "vertex" : {
-            "@type" : "g:Int32",
-            "@value" : 1
-          },
           "label" : "location",
           "properties" : {
             "startTime" : {
@@ -54,10 +46,6 @@
             "@value" : 7
           },
           "value" : "santa cruz",
-          "vertex" : {
-            "@type" : "g:Int32",
-            "@value" : 1
-          },
           "label" : "location",
           "properties" : {
             "startTime" : {
@@ -78,10 +66,6 @@
             "@value" : 8
           },
           "value" : "brussels",
-          "vertex" : {
-            "@type" : "g:Int32",
-            "@value" : 1
-          },
           "label" : "location",
           "properties" : {
             "startTime" : {
@@ -102,10 +86,6 @@
             "@value" : 9
           },
           "value" : "santa fe",
-          "vertex" : {
-            "@type" : "g:Int32",
-            "@value" : 1
-          },
           "label" : "location",
           "properties" : {
             "startTime" : {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v3d0.json
new file mode 100644
index 0000000..86f305d
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v3d0.json
@@ -0,0 +1,80 @@
+{
+  "@type" : "g:Vertex",
+  "@value" : {
+    "id" : {
+      "@type" : "g:Int32",
+      "@value" : 1
+    },
+    "label" : "person",
+    "properties" : {
+      "name" : [ {
+        "id" : {
+          "@type" : "g:Int64",
+          "@value" : 0
+        },
+        "value" : "marko"
+      } ],
+      "location" : [ {
+        "id" : {
+          "@type" : "g:Int64",
+          "@value" : 6
+        },
+        "value" : "san diego",
+        "properties" : {
+          "startTime" : {
+            "@type" : "g:Int32",
+            "@value" : 1997
+          },
+          "endTime" : {
+            "@type" : "g:Int32",
+            "@value" : 2001
+          }
+        }
+      }, {
+        "id" : {
+          "@type" : "g:Int64",
+          "@value" : 7
+        },
+        "value" : "santa cruz",
+        "properties" : {
+          "startTime" : {
+            "@type" : "g:Int32",
+            "@value" : 2001
+          },
+          "endTime" : {
+            "@type" : "g:Int32",
+            "@value" : 2004
+          }
+        }
+      }, {
+        "id" : {
+          "@type" : "g:Int64",
+          "@value" : 8
+        },
+        "value" : "brussels",
+        "properties" : {
+          "startTime" : {
+            "@type" : "g:Int32",
+            "@value" : 2004
+          },
+          "endTime" : {
+            "@type" : "g:Int32",
+            "@value" : 2005
+          }
+        }
+      }, {
+        "id" : {
+          "@type" : "g:Int64",
+          "@value" : 9
+        },
+        "value" : "santa fe",
+        "properties" : {
+          "startTime" : {
+            "@type" : "g:Int32",
+            "@value" : 2005
+          }
+        }
+      } ]
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v2d0-no-types.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v2d0-no-types.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v2d0-no-types.json
index 7ab4f4d..74025a8 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v2d0-no-types.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v2d0-no-types.json
@@ -1,6 +1,5 @@
 {
   "id" : 0,
   "value" : "marko",
-  "vertex" : 1,
   "label" : "name"
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v2d0-partial.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v2d0-partial.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v2d0-partial.json
index 0319bd1..af184b1 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v2d0-partial.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v2d0-partial.json
@@ -6,10 +6,6 @@
       "@value" : 0
     },
     "value" : "marko",
-    "vertex" : {
-      "@type" : "g:Int32",
-      "@value" : 1
-    },
     "label" : "name"
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v3d0.json
new file mode 100644
index 0000000..0319bd1
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v3d0.json
@@ -0,0 +1,15 @@
+{
+  "@type" : "g:VertexProperty",
+  "@value" : {
+    "id" : {
+      "@type" : "g:Int64",
+      "@value" : 0
+    },
+    "value" : "marko",
+    "vertex" : {
+      "@type" : "g:Int32",
+      "@value" : 1
+    },
+    "label" : "name"
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/year-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/year-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/year-v3d0.json
new file mode 100644
index 0000000..ff420bc
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/year-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:Year",
+  "@value" : "2016"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/yearmonth-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/yearmonth-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/yearmonth-v3d0.json
new file mode 100644
index 0000000..98a5e27
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/yearmonth-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:YearMonth",
+  "@value" : "2016-06"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/zoneddatetime-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/zoneddatetime-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/zoneddatetime-v3d0.json
new file mode 100644
index 0000000..367fc47
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/zoneddatetime-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:ZonedDateTime",
+  "@value" : "2016-12-23T12:12:24.000000036+02:00[GMT+02:00]"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/zoneoffset-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/zoneoffset-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/zoneoffset-v3d0.json
new file mode 100644
index 0000000..8591794
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/zoneoffset-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:ZoneOffset",
+  "@value" : "+03:06:09"
+}
\ No newline at end of file


[18/31] tinkerpop git commit: TINKERPOP-1565 Added tests for GraphSON 3.0

Posted by sp...@apache.org.
TINKERPOP-1565 Added tests for GraphSON 3.0

And fixed some broken tests preventing a proper build.


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

Branch: refs/heads/TINKERPOP-1565
Commit: 729af57f0d3515ccff591259f529a999af0b717c
Parents: 2dc9b51
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Fri Jan 13 08:20:22 2017 -0500
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Jan 19 15:15:33 2017 -0500

----------------------------------------------------------------------
 .../ser/GraphSONMessageSerializerV2d0Test.java  |  13 +-
 .../gremlin/structure/io/IoCustomTest.java      |   3 +
 .../gremlin/structure/io/IoEdgeTest.java        |   3 +
 .../gremlin/structure/io/IoGraphTest.java       |   1 +
 .../gremlin/structure/io/IoPropertyTest.java    |  12 +-
 .../tinkerpop/gremlin/structure/io/IoTest.java  | 123 +++++++++++++++++++
 .../gremlin/structure/io/IoVertexTest.java      |   3 +
 .../tinkerpop-classic-normalized-v3d0.json      |   6 +
 8 files changed, 151 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/729af57f/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV2d0Test.java
----------------------------------------------------------------------
diff --git a/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV2d0Test.java b/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV2d0Test.java
index 4125946..0bdc08d 100644
--- a/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV2d0Test.java
+++ b/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV2d0Test.java
@@ -251,7 +251,7 @@ public class GraphSONMessageSerializerV2d0Test {
 
         final JsonNode properties = edgeAsJson.get(GraphSONTokens.PROPERTIES);
         assertNotNull(properties);
-        assertEquals(123, properties.get("abc").get(GraphSONTokens.VALUEPROP).asInt());
+        assertEquals(123, properties.get("abc").get(GraphSONTokens.VALUEPROP).get("value").get(GraphSONTokens.VALUEPROP).asInt());
     }
 
     @Test
@@ -315,7 +315,7 @@ public class GraphSONMessageSerializerV2d0Test {
 
         final JsonNode friendProperties = properties.get("friends");
         assertEquals(1, friendProperties.size());
-        final JsonNode friendsProperty = friendProperties.get(0).get(GraphSONTokens.VALUE);
+        final JsonNode friendsProperty = friendProperties.get(0).get(GraphSONTokens.VALUEPROP).get(GraphSONTokens.VALUE);
         assertNotNull(friendsProperty);
         assertEquals(3, friendsProperty.size());
 
@@ -416,7 +416,6 @@ public class GraphSONMessageSerializerV2d0Test {
         final GraphTraversalSource g = graph.traversal();
         final Tree t = g.V(1).out().properties("name").tree().next();
 
-        
         final String results = SERIALIZER.serializeResponseAsString(ResponseMessage.build(msg).result(t).create());
 
         final JsonNode json = mapper.readTree(results);
@@ -440,7 +439,7 @@ public class GraphSONMessageSerializerV2d0Test {
                 .get(GraphSONTokens.PROPERTIES)
                 .get("name")
                 .get(0)
-                .get(GraphSONTokens.VALUE).asText());
+                .get(GraphSONTokens.VALUEPROP).get(GraphSONTokens.VALUE).asText());
 
         //check the leafs
         assertEquals("vadas", converted.get(GraphSONTokens.VALUEPROP)
@@ -451,7 +450,7 @@ public class GraphSONMessageSerializerV2d0Test {
                 .get(GraphSONTokens.PROPERTIES)
                 .get("name")
                 .get(0)
-                .get(GraphSONTokens.VALUE).asText());
+                .get(GraphSONTokens.VALUEPROP).get(GraphSONTokens.VALUE).asText());
 
         assertEquals("lop", converted.get(GraphSONTokens.VALUEPROP)
                 .get(0)
@@ -461,7 +460,7 @@ public class GraphSONMessageSerializerV2d0Test {
                 .get(GraphSONTokens.PROPERTIES)
                 .get("name")
                 .get(0)
-                .get(GraphSONTokens.VALUE).asText());
+                .get(GraphSONTokens.VALUEPROP).get(GraphSONTokens.VALUE).asText());
 
         assertEquals("josh", converted.get(GraphSONTokens.VALUEPROP)
                 .get(0)
@@ -471,7 +470,7 @@ public class GraphSONMessageSerializerV2d0Test {
                 .get(GraphSONTokens.PROPERTIES)
                 .get("name")
                 .get(0)
-                .get(GraphSONTokens.VALUE).asText());
+                .get(GraphSONTokens.VALUEPROP).get(GraphSONTokens.VALUE).asText());
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/729af57f/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoCustomTest.java
----------------------------------------------------------------------
diff --git a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoCustomTest.java b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoCustomTest.java
index d89f97e..f7340f8 100644
--- a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoCustomTest.java
+++ b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoCustomTest.java
@@ -67,6 +67,9 @@ public class IoCustomTest extends AbstractGremlinTest {
                 {"graphson-v2-embedded", true,
                         (Function<Graph, GraphReader>) g -> g.io(IoCore.graphson()).reader().mapper(g.io(GraphSONIo.build(GraphSONVersion.V2_0)).mapper().addCustomModule(moduleV2d0).typeInfo(TypeInfo.PARTIAL_TYPES).create()).create(),
                         (Function<Graph, GraphWriter>) g -> g.io(IoCore.graphson()).writer().mapper(g.io(GraphSONIo.build(GraphSONVersion.V2_0)).mapper().addCustomModule(moduleV2d0).typeInfo(TypeInfo.PARTIAL_TYPES).create()).create()},
+                {"graphson-v3", true,
+                        (Function<Graph, GraphReader>) g -> g.io(IoCore.graphson()).reader().mapper(g.io(GraphSONIo.build(GraphSONVersion.V3_0)).mapper().addCustomModule(moduleV2d0).create()).create(),
+                        (Function<Graph, GraphWriter>) g -> g.io(IoCore.graphson()).writer().mapper(g.io(GraphSONIo.build(GraphSONVersion.V3_0)).mapper().addCustomModule(moduleV2d0).create()).create()},
                 {"gryo", true,
                         (Function<Graph, GraphReader>) g -> g.io(IoCore.gryo()).reader().mapper(g.io(IoCore.gryo()).mapper().addCustom(CustomId.class).create()).create(),
                         (Function<Graph, GraphWriter>) g -> g.io(IoCore.gryo()).writer().mapper(g.io(IoCore.gryo()).mapper().addCustom(CustomId.class).create()).create()}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/729af57f/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoEdgeTest.java
----------------------------------------------------------------------
diff --git a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoEdgeTest.java b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoEdgeTest.java
index 8d2a58a..fa656a5 100644
--- a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoEdgeTest.java
+++ b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoEdgeTest.java
@@ -66,6 +66,9 @@ public class IoEdgeTest extends AbstractGremlinTest {
                 {"graphson-v2-embedded", true, true,
                         (Function<Graph, GraphReader>) g -> g.io(IoCore.graphson()).reader().mapper(g.io(GraphSONIo.build(GraphSONVersion.V2_0)).mapper().typeInfo(TypeInfo.PARTIAL_TYPES).create()).create(),
                         (Function<Graph, GraphWriter>) g -> g.io(IoCore.graphson()).writer().mapper(g.io(GraphSONIo.build(GraphSONVersion.V2_0)).mapper().typeInfo(TypeInfo.PARTIAL_TYPES).create()).create()},
+                {"graphson-v3", true, true,
+                        (Function<Graph, GraphReader>) g -> g.io(IoCore.graphson()).reader().mapper(g.io(GraphSONIo.build(GraphSONVersion.V3_0)).mapper().create()).create(),
+                        (Function<Graph, GraphWriter>) g -> g.io(IoCore.graphson()).writer().mapper(g.io(GraphSONIo.build(GraphSONVersion.V3_0)).mapper().create()).create()},
                 {"gryo", true, true,
                         (Function<Graph,GraphReader>) g -> g.io(IoCore.gryo()).reader().create(),
                         (Function<Graph, GraphWriter>) g -> g.io(IoCore.gryo()).writer().create()}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/729af57f/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoGraphTest.java
----------------------------------------------------------------------
diff --git a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoGraphTest.java b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoGraphTest.java
index 040849e..6e54377 100644
--- a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoGraphTest.java
+++ b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoGraphTest.java
@@ -53,6 +53,7 @@ public class IoGraphTest extends AbstractGremlinTest {
                 {"graphml", IoCore.graphml(), false, true, ".xml"},
                 {"graphsonv1d0", IoCore.graphson(), true, true, ".json"},
                 {"graphsonv2d0", GraphSONIo.build(GraphSONVersion.V2_0), true, true, ".json"},
+                {"graphsonv3d0", GraphSONIo.build(GraphSONVersion.V3_0), true, true, ".json"},
                 {"gryo", IoCore.gryo(), false, false, ".kryo"}
         });
     }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/729af57f/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoPropertyTest.java
----------------------------------------------------------------------
diff --git a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoPropertyTest.java b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoPropertyTest.java
index dec7230..ecde126 100644
--- a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoPropertyTest.java
+++ b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoPropertyTest.java
@@ -63,6 +63,9 @@ public class IoPropertyTest extends AbstractGremlinTest {
                 {"graphson-v2-embedded", true, true,
                         (Function<Graph, GraphReader>) g -> g.io(IoCore.graphson()).reader().mapper(g.io(GraphSONIo.build(GraphSONVersion.V2_0)).mapper().typeInfo(TypeInfo.PARTIAL_TYPES).create()).create(),
                         (Function<Graph, GraphWriter>) g -> g.io(IoCore.graphson()).writer().mapper(g.io(GraphSONIo.build(GraphSONVersion.V2_0)).mapper().typeInfo(TypeInfo.PARTIAL_TYPES).create()).create()},
+                {"graphson-v3", true, true,
+                        (Function<Graph, GraphReader>) g -> g.io(IoCore.graphson()).reader().mapper(g.io(GraphSONIo.build(GraphSONVersion.V3_0)).mapper().create()).create(),
+                        (Function<Graph, GraphWriter>) g -> g.io(IoCore.graphson()).writer().mapper(g.io(GraphSONIo.build(GraphSONVersion.V3_0)).mapper().create()).create()},
                 {"gryo", true, true,
                         (Function<Graph, GraphReader>) g -> g.io(IoCore.gryo()).reader().create(),
                         (Function<Graph, GraphWriter>) g -> g.io(IoCore.gryo()).writer().create()}
@@ -92,7 +95,6 @@ public class IoPropertyTest extends AbstractGremlinTest {
 
             // select any vertexproperty that has both start/end time
             final VertexProperty p = (VertexProperty) g.V(convertToVertexId("marko")).properties("location").as("p").has("endTime").select("p").next();
-            final Vertex v = p.element();
             writer.writeVertexProperty(os, p);
 
             final AtomicBoolean called = new AtomicBoolean(false);
@@ -104,7 +106,7 @@ public class IoPropertyTest extends AbstractGremlinTest {
                     assertEquals(IteratorUtils.count(p.properties()), IteratorUtils.count(propertyAttachable.get().properties()));
                     assertEquals(p.property("startTime").value(), ((Property) propertyAttachable.get().properties("startTime").next()).value());
                     assertEquals(p.property("endTime").value(), ((Property) propertyAttachable.get().properties("endTime").next()).value());
-                    if (ioType.equals("graphson-v2-embedded")) { // TODO: make this work with Gryo
+                    if (ioType.equals("graphson-v3")) { // TODO: make this work with Gryo
                         assertEquals(p, propertyAttachable.get());
                         assertEquals(p.element(), propertyAttachable.get().element());
                     }
@@ -123,7 +125,6 @@ public class IoPropertyTest extends AbstractGremlinTest {
         try (final ByteArrayOutputStream os = new ByteArrayOutputStream()) {
             final GraphWriter writer = writerMaker.apply(graph);
             final VertexProperty p = g.V(convertToVertexId("marko")).next().property("name");
-            final Vertex v = p.element();
             writer.writeVertexProperty(os, p);
 
             final AtomicBoolean called = new AtomicBoolean(false);
@@ -133,7 +134,7 @@ public class IoPropertyTest extends AbstractGremlinTest {
                     assertEquals(p.value(), propertyAttachable.get().value());
                     assertEquals(p.key(), propertyAttachable.get().key());
                     assertEquals(0, IteratorUtils.count(propertyAttachable.get().properties()));
-                    if (ioType.equals("graphson-v2-embedded")) { // TODO: make this work with Gryo
+                    if (ioType.equals("graphson-v3")) { // TODO: make this work with Gryo
                         assertEquals(p, propertyAttachable.get());
                         assertEquals(p.element(), propertyAttachable.get().element());
                     }
@@ -152,7 +153,6 @@ public class IoPropertyTest extends AbstractGremlinTest {
         try (final ByteArrayOutputStream os = new ByteArrayOutputStream()) {
             final GraphWriter writer = writerMaker.apply(graph);
             final Property p = g.E(convertToEdgeId("marko", "knows", "vadas")).next().property("weight");
-            final Edge e = (Edge) p.element();
             writer.writeProperty(os, p);
 
             final AtomicBoolean called = new AtomicBoolean(false);
@@ -161,7 +161,7 @@ public class IoPropertyTest extends AbstractGremlinTest {
                 reader.readProperty(bais, propertyAttachable -> {
                     assertEquals(p.value(), propertyAttachable.get().value());
                     assertEquals(p.key(), propertyAttachable.get().key());
-                    if (ioType.equals("graphson-v2-embedded")) { // TODO: make this work with Gryo
+                    if (ioType.equals("graphson-v3")) { // TODO: make this work with Gryo
                         assertEquals(p, propertyAttachable.get());
                         assertEquals(p.element(), propertyAttachable.get().element());
                     }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/729af57f/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoTest.java
----------------------------------------------------------------------
diff --git a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoTest.java b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoTest.java
index e16bbcc..6c188bd 100644
--- a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoTest.java
+++ b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoTest.java
@@ -607,6 +607,129 @@ public class IoTest {
         }
     }
 
+    public static final class GraphSONV3D0Test extends AbstractGremlinTest {
+        private Io.Builder<GraphSONIo> graphson;
+
+        @Before
+        public void setupBeforeEachTest() {
+            graphson = graphson();
+        }
+
+        /**
+         * Only need to execute this test with TinkerGraph or other graphs that support user supplied identifiers.
+         */
+        @Test
+        @FeatureRequirement(featureClass = VertexPropertyFeatures.class, feature = FEATURE_STRING_VALUES)
+        @FeatureRequirement(featureClass = VertexPropertyFeatures.class, feature = FEATURE_INTEGER_VALUES)
+        @FeatureRequirement(featureClass = EdgePropertyFeatures.class, feature = EdgePropertyFeatures.FEATURE_FLOAT_VALUES)
+        @FeatureRequirement(featureClass = Graph.Features.VertexFeatures.class, feature = Graph.Features.VertexFeatures.FEATURE_USER_SUPPLIED_IDS)
+        @FeatureRequirement(featureClass = Graph.Features.VertexFeatures.class, feature = Graph.Features.VertexFeatures.FEATURE_NUMERIC_IDS)
+        @FeatureRequirement(featureClass = Graph.Features.VertexPropertyFeatures.class, feature = Graph.Features.VertexPropertyFeatures.FEATURE_USER_SUPPLIED_IDS)
+        @FeatureRequirement(featureClass = Graph.Features.VariableFeatures.class, feature = FEATURE_VARIABLES)
+        @LoadGraphWith(LoadGraphWith.GraphData.CLASSIC)
+        public void shouldWriteNormalizedGraphSON() throws Exception {
+            try (ByteArrayOutputStream bos = new ByteArrayOutputStream()) {
+                final GraphSONMapper mapper = graph.io(graphson).mapper().version(GraphSONVersion.V3_0).normalize(true).create();
+                final GraphSONWriter w = graph.io(graphson).writer().mapper(mapper).create();
+                w.writeGraph(bos, graph);
+
+                final String expected = streamToString(IoTest.class.getResourceAsStream(TestHelper.convertPackageToResourcePath(GraphSONResourceAccess.class) + "tinkerpop-classic-normalized-v3d0.json"));
+                assertEquals(expected.replace("\n", "").replace("\r", ""), bos.toString().replace("\n", "").replace("\r", ""));
+            }
+        }
+
+        @Test
+        @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+        @FeatureRequirement(featureClass = Graph.Features.EdgeFeatures.class, feature = Graph.Features.EdgeFeatures.FEATURE_ADD_EDGES)
+        @FeatureRequirement(featureClass = Graph.Features.VertexFeatures.class, feature = Graph.Features.VertexFeatures.FEATURE_ADD_VERTICES)
+        public void shouldReadWriteModernWrappedInJsonObject() throws Exception {
+            final GraphSONMapper mapper = graph.io(graphson).mapper().version(GraphSONVersion.V3_0).create();
+            try (final ByteArrayOutputStream os = new ByteArrayOutputStream()) {
+                final GraphWriter writer = graph.io(graphson()).writer().wrapAdjacencyList(true).mapper(mapper).create();
+                writer.writeGraph(os, graph);
+
+                final Configuration configuration = graphProvider.newGraphConfiguration("readGraph", this.getClass(), name.getMethodName(), LoadGraphWith.GraphData.MODERN);
+                graphProvider.clear(configuration);
+                final Graph g1 = graphProvider.openTestGraph(configuration);
+                final GraphReader reader = graph.io(graphson()).reader().mapper(mapper).unwrapAdjacencyList(true).create();
+                try (final ByteArrayInputStream bais = new ByteArrayInputStream(os.toByteArray())) {
+                    reader.readGraph(bais, g1);
+                }
+
+                // modern uses double natively so always assert as such
+                IoTest.assertModernGraph(g1, true, true);
+
+                graphProvider.clear(g1, configuration);
+            }
+        }
+
+        /**
+         * This is just a serialization check for JSON.
+         */
+        @Test
+        @FeatureRequirement(featureClass = Graph.Features.VertexFeatures.class, feature = Graph.Features.VertexFeatures.FEATURE_ADD_VERTICES)
+        @FeatureRequirement(featureClass = Graph.Features.VertexFeatures.class, feature = FEATURE_USER_SUPPLIED_IDS)
+        @FeatureRequirement(featureClass = Graph.Features.VertexFeatures.class, feature = FEATURE_ANY_IDS)
+        public void shouldProperlySerializeCustomIdWithGraphSON() throws Exception {
+            final UUID id = UUID.fromString("AF4B5965-B176-4552-B3C1-FBBE2F52C305");
+            graph.addVertex(T.id, new CustomId("vertex", id));
+
+            final SimpleModule module = new CustomId.CustomIdTinkerPopJacksonModule();
+            final GraphWriter writer = graph.io(graphson).writer().mapper(
+                    graph.io(graphson).mapper().version(GraphSONVersion.V3_0).addCustomModule(module).create()).create();
+
+            try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
+                writer.writeGraph(baos, graph);
+
+                // reusing the same config used for creation of "g".
+                final Configuration configuration = graphProvider.newGraphConfiguration("g2", this.getClass(), name.getMethodName(), null);
+                graphProvider.clear(configuration);
+                final Graph g2 = graphProvider.openTestGraph(configuration);
+
+                try (final InputStream is = new ByteArrayInputStream(baos.toByteArray())) {
+                    final GraphReader reader = graph.io(graphson).reader()
+                            .mapper(graph.io(graphson).mapper().version(GraphSONVersion.V3_0).addCustomModule(module).create()).create();
+                    reader.readGraph(is, g2);
+                }
+
+                final Vertex v2 = g2.vertices().next();
+                final CustomId customId = (CustomId) v2.id();
+                assertEquals(id, customId.getElementId());
+                assertEquals("vertex", customId.getCluster());
+
+                // need to manually close the "g2" instance
+                graphProvider.clear(g2, configuration);
+            }
+        }
+
+        @Test
+        @FeatureRequirement(featureClass = Graph.Features.EdgeFeatures.class, feature = Graph.Features.EdgeFeatures.FEATURE_ADD_EDGES)
+        @FeatureRequirement(featureClass = EdgePropertyFeatures.class, feature = FEATURE_STRING_VALUES)
+        @FeatureRequirement(featureClass = Graph.Features.VertexFeatures.class, feature = Graph.Features.VertexFeatures.FEATURE_ADD_VERTICES)
+        public void shouldReadWriteSelfLoopingEdges() throws Exception {
+            final GraphSONMapper mapper = graph.io(graphson).mapper().version(GraphSONVersion.V3_0).create();
+            final Graph source = graph;
+            final Vertex v1 = source.addVertex();
+            final Vertex v2 = source.addVertex();
+            v1.addEdge("CONTROL", v2);
+            v1.addEdge("SELFLOOP", v1);
+
+            final Configuration targetConf = graphProvider.newGraphConfiguration("target", this.getClass(), name.getMethodName(), null);
+            final Graph target = graphProvider.openTestGraph(targetConf);
+            try (ByteArrayOutputStream os = new ByteArrayOutputStream()) {
+                source.io(IoCore.graphson()).writer().mapper(mapper).create().writeGraph(os, source);
+                try (ByteArrayInputStream is = new ByteArrayInputStream(os.toByteArray())) {
+                    target.io(IoCore.graphson()).reader().mapper(mapper).create().readGraph(is, target);
+                }
+            } catch (IOException ioe) {
+                throw new RuntimeException(ioe);
+            }
+
+            assertEquals(IteratorUtils.count(source.vertices()), IteratorUtils.count(target.vertices()));
+            assertEquals(IteratorUtils.count(source.edges()), IteratorUtils.count(target.edges()));
+        }
+    }
+
     public static void assertCrewGraph(final Graph g1, final boolean lossyForId) {
         assertEquals(new Long(6), g1.traversal().V().count().next());
         assertEquals(new Long(14), g1.traversal().E().count().next());

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/729af57f/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoVertexTest.java
----------------------------------------------------------------------
diff --git a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoVertexTest.java b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoVertexTest.java
index 6e692c4..d3aaa81 100644
--- a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoVertexTest.java
+++ b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoVertexTest.java
@@ -83,6 +83,9 @@ public class IoVertexTest extends AbstractGremlinTest {
                 {"graphson-v2-embedded", true, false,
                         (Function<Graph, GraphReader>) g -> g.io(IoCore.graphson()).reader().mapper(g.io(GraphSONIo.build(GraphSONVersion.V2_0)).mapper().typeInfo(TypeInfo.PARTIAL_TYPES).create()).create(),
                         (Function<Graph, GraphWriter>) g -> g.io(IoCore.graphson()).writer().mapper(g.io(GraphSONIo.build(GraphSONVersion.V2_0)).mapper().typeInfo(TypeInfo.PARTIAL_TYPES).create()).create()},
+                {"graphson-v3", true, false,
+                        (Function<Graph, GraphReader>) g -> g.io(IoCore.graphson()).reader().mapper(g.io(GraphSONIo.build(GraphSONVersion.V3_0)).mapper().create()).create(),
+                        (Function<Graph, GraphWriter>) g -> g.io(IoCore.graphson()).writer().mapper(g.io(GraphSONIo.build(GraphSONVersion.V3_0)).mapper().create()).create()},
                 {"gryo", true, true,
                         (Function<Graph, GraphReader>) g -> g.io(IoCore.gryo()).reader().create(),
                         (Function<Graph, GraphWriter>) g -> g.io(IoCore.gryo()).writer().create()}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/729af57f/gremlin-test/src/main/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/tinkerpop-classic-normalized-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-test/src/main/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/tinkerpop-classic-normalized-v3d0.json b/gremlin-test/src/main/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/tinkerpop-classic-normalized-v3d0.json
new file mode 100644
index 0000000..57d1777
--- /dev/null
+++ b/gremlin-test/src/main/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/tinkerpop-classic-normalized-v3d0.json
@@ -0,0 +1,6 @@
+{"id":{"@type":"g:Int32","@value":1},"label":"vertex","outE":{"created":[{"id":{"@type":"g:Int32","@value":9},"inV":{"@type":"g:Int32","@value":3},"properties":{"weight":{"@type":"g:Float","@value":0.4}}}],"knows":[{"id":{"@type":"g:Int32","@value":7},"inV":{"@type":"g:Int32","@value":2},"properties":{"weight":{"@type":"g:Float","@value":0.5}}},{"id":{"@type":"g:Int32","@value":8},"inV":{"@type":"g:Int32","@value":4},"properties":{"weight":{"@type":"g:Float","@value":1.0}}}]},"properties":{"age":[{"id":{"@type":"g:Int32","@value":2},"value":{"@type":"g:Int32","@value":29}}],"name":[{"id":{"@type":"g:Int32","@value":0},"value":"marko"}]}}
+{"id":{"@type":"g:Int32","@value":2},"label":"vertex","inE":{"knows":[{"id":{"@type":"g:Int32","@value":7},"outV":{"@type":"g:Int32","@value":1},"properties":{"weight":{"@type":"g:Float","@value":0.5}}}]},"properties":{"age":[{"id":{"@type":"g:Int32","@value":4},"value":{"@type":"g:Int32","@value":27}}],"name":[{"id":{"@type":"g:Int32","@value":3},"value":"vadas"}]}}
+{"id":{"@type":"g:Int32","@value":3},"label":"vertex","inE":{"created":[{"id":{"@type":"g:Int32","@value":11},"outV":{"@type":"g:Int32","@value":4},"properties":{"weight":{"@type":"g:Float","@value":0.4}}},{"id":{"@type":"g:Int32","@value":12},"outV":{"@type":"g:Int32","@value":6},"properties":{"weight":{"@type":"g:Float","@value":0.2}}},{"id":{"@type":"g:Int32","@value":9},"outV":{"@type":"g:Int32","@value":1},"properties":{"weight":{"@type":"g:Float","@value":0.4}}}]},"properties":{"lang":[{"id":{"@type":"g:Int32","@value":6},"value":"java"}],"name":[{"id":{"@type":"g:Int32","@value":5},"value":"lop"}]}}
+{"id":{"@type":"g:Int32","@value":4},"label":"vertex","inE":{"knows":[{"id":{"@type":"g:Int32","@value":8},"outV":{"@type":"g:Int32","@value":1},"properties":{"weight":{"@type":"g:Float","@value":1.0}}}]},"outE":{"created":[{"id":{"@type":"g:Int32","@value":10},"inV":{"@type":"g:Int32","@value":5},"properties":{"weight":{"@type":"g:Float","@value":1.0}}},{"id":{"@type":"g:Int32","@value":11},"inV":{"@type":"g:Int32","@value":3},"properties":{"weight":{"@type":"g:Float","@value":0.4}}}]},"properties":{"age":[{"id":{"@type":"g:Int32","@value":8},"value":{"@type":"g:Int32","@value":32}}],"name":[{"id":{"@type":"g:Int32","@value":7},"value":"josh"}]}}
+{"id":{"@type":"g:Int32","@value":5},"label":"vertex","inE":{"created":[{"id":{"@type":"g:Int32","@value":10},"outV":{"@type":"g:Int32","@value":4},"properties":{"weight":{"@type":"g:Float","@value":1.0}}}]},"properties":{"lang":[{"id":{"@type":"g:Int32","@value":10},"value":"java"}],"name":[{"id":{"@type":"g:Int32","@value":9},"value":"ripple"}]}}
+{"id":{"@type":"g:Int32","@value":6},"label":"vertex","outE":{"created":[{"id":{"@type":"g:Int32","@value":12},"inV":{"@type":"g:Int32","@value":3},"properties":{"weight":{"@type":"g:Float","@value":0.2}}}]},"properties":{"age":[{"id":{"@type":"g:Int32","@value":12},"value":{"@type":"g:Int32","@value":35}}],"name":[{"id":{"@type":"g:Int32","@value":11},"value":"peter"}]}}
\ No newline at end of file


[23/31] tinkerpop git commit: TINKERPOP-1565 Stub out GraphSON 3.0

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0Test.java
----------------------------------------------------------------------
diff --git a/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0Test.java b/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0Test.java
new file mode 100644
index 0000000..c1e43b5
--- /dev/null
+++ b/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0Test.java
@@ -0,0 +1,342 @@
+/*
+ * 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.tinkerpop.gremlin.driver.MessageSerializer;
+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.Property;
+import org.apache.tinkerpop.gremlin.structure.Vertex;
+import org.apache.tinkerpop.gremlin.structure.VertexProperty;
+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.jackson.databind.util.StdDateFormat;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
+/**
+ * Serializer tests that cover non-lossy serialization/deserialization methods.
+ *
+ * @author Stephen Mallette (http://stephen.genoprime.com)
+ */
+public class GraphSONMessageSerializerV3d0Test {
+
+    private final UUID requestId = UUID.fromString("6457272A-4018-4538-B9AE-08DD5DDC0AA1");
+    private final ResponseMessage.Builder responseMessageBuilder = ResponseMessage.build(requestId);
+    private final static ByteBufAllocator allocator = UnpooledByteBufAllocator.DEFAULT;
+
+    public final MessageSerializer serializer = new GraphSONMessageSerializerV3d0();
+
+    @Test
+    public void shouldSerializeIterable() throws Exception {
+        final ArrayList<Integer> list = new ArrayList<>();
+        list.add(1);
+        list.add(100);
+
+        final ResponseMessage response = convert(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 shouldSerializeIterableWithNull() throws Exception {
+        final ArrayList<Integer> list = new ArrayList<>();
+        list.add(1);
+        list.add(null);
+        list.add(100);
+
+        final ResponseMessage response = convert(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 = convert(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 shouldSerializeMapEntries() 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 = convert(IteratorUtils.asList(map.entrySet()));
+        assertCommon(response);
+
+        final List<Map<String, Object>> deserializedEntries = (List<Map<String, Object>>) response.getResult().getData();
+        assertEquals(3, deserializedEntries.size());
+        deserializedEntries.forEach(e -> {
+            if (e.containsKey("x"))
+                assertEquals(1, e.get("x"));
+            else if (e.containsKey(v1.id().toString()))
+                assertEquals(100, e.get(v1.id().toString()));
+            else if (e.containsKey(StdDateFormat.instance.format(d)))
+                assertEquals("test", e.get(StdDateFormat.instance.format(d)));
+            else
+                fail("Map entries contains a key that is not part of what was serialized");
+        });
+    }
+
+    @Test
+    public void shouldSerializeEdge() throws Exception {
+        final Graph graph = TinkerGraph.open();
+        final Vertex v1 = graph.addVertex();
+        final Vertex v2 = graph.addVertex();
+        final Edge e = v1.addEdge("test", v2);
+        e.property("abc", 123);
+
+        final Iterable<Edge> iterable = IteratorUtils.list(graph.edges());
+
+        final ResponseMessage response = convert(iterable);
+        assertCommon(response);
+
+        final List<Edge> edgeList = (List<Edge>) response.getResult().getData();
+        assertEquals(1, edgeList.size());
+
+        final Edge deserializedEdge = edgeList.get(0);
+        assertEquals(e.id(), deserializedEdge.id());
+        assertEquals(v1.id(), deserializedEdge.outVertex().id());
+        assertEquals(v2.id(), deserializedEdge.inVertex().id());
+        assertEquals(v1.label(), deserializedEdge.outVertex().label());
+        assertEquals(v2.label(), deserializedEdge.inVertex().label());
+        assertEquals(e.label(), deserializedEdge.label());
+
+        final List<Property> properties = new ArrayList<>();
+        deserializedEdge.properties().forEachRemaining(properties::add);
+        assertEquals(1, properties.size());
+
+        assertNotNull(properties);
+        assertEquals("abc", properties.get(0).key());
+        assertEquals(123, properties.get(0).value());
+
+    }
+
+    @Test
+    public void shouldSerializeEdgeProperty() throws Exception {
+        final Graph graph = TinkerGraph.open();
+        final Vertex v1 = graph.addVertex();
+        final Vertex v2 = graph.addVertex();
+        final Edge e = v1.addEdge("test", v2);
+        e.property("abc", 123);
+
+        final Iterable<Property<Object>> iterable = IteratorUtils.list(e.properties("abc"));
+        final ResponseMessage response = convert(iterable);
+        assertCommon(response);
+
+        final List<Property> propertyList = (List<Property>) response.getResult().getData();
+        assertEquals(1, propertyList.size());
+        assertEquals(123, propertyList.get(0).value());
+    }
+
+    @Test
+    public void shouldSerializeVertexWithEmbeddedMap() throws Exception {
+        final Graph graph = TinkerGraph.open();
+        final Vertex v = graph.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(graph.vertices());
+
+        final ResponseMessage response = convert(list);
+        assertCommon(response);
+
+        final List<Vertex> vertexList = (List<Vertex>) response.getResult().getData();
+        assertEquals(1, vertexList.size());
+
+        final Vertex deserializedVertex = vertexList.get(0);
+        assertEquals(v.id(), deserializedVertex.id());
+        assertEquals(Vertex.DEFAULT_LABEL, deserializedVertex.label());
+
+        final List<VertexProperty> properties = new ArrayList<>();
+        deserializedVertex.properties().forEachRemaining(properties::add);
+        assertEquals(1, properties.size());
+
+        final VertexProperty friendsProperty = properties.get(0);
+        final List<Object> deserializedInnerList = (List<Object>) friendsProperty.value();
+
+        assertEquals(3, deserializedInnerList.size());
+        assertEquals("x", deserializedInnerList.get(0));
+        assertEquals(5, deserializedInnerList.get(1));
+
+        final Map<String, Object> deserializedInnerInnerMap = (Map<String, Object>) deserializedInnerList.get(2);
+        assertEquals(2, deserializedInnerInnerMap.size());
+        assertEquals(500, deserializedInnerInnerMap.get("x"));
+        assertEquals("some", deserializedInnerInnerMap.get("y"));
+    }
+
+    @Test
+    public void shouldSerializeToJsonMapWithElementForKey() 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 = convert(map);
+        assertCommon(response);
+
+        final Map<String, Integer> deserializedMap = (Map<String, Integer>) response.getResult().getData();
+        assertEquals(1, deserializedMap.size());
+
+        // with no embedded types the key (which is a vertex) simply serializes out to an id
+        // {"result":{"1":1000},"code":200,"requestId":"2d62161b-9544-4f39-af44-62ec49f9a595","type":0}
+        assertEquals(new Integer(1000), deserializedMap.get("1"));
+    }
+
+    @Test
+    @org.junit.Ignore
+    public void shouldSerializeToTreeJson() throws Exception {
+        final TinkerGraph graph = TinkerFactory.createClassic();
+        final GraphTraversalSource g = graph.traversal();
+        final Map t = g.V(1).out().properties("name").tree().next();
+
+        final ResponseMessage response = convert(t);
+        assertCommon(response);
+
+        final Tree deserializedTree = (Tree)response.getResult().getData();
+
+        //check the first object and its key's properties
+        assertEquals(1, deserializedTree.size());
+        final Vertex v = ((Vertex) deserializedTree.keySet().iterator().next());
+        assertEquals(1, v.id());
+        assertEquals("marko", v.property("name").value());
+
+        final Tree firstTree = (Tree)deserializedTree.get(v);
+        assertEquals(3, firstTree.size());
+        Iterator<Vertex> vertexKeys = firstTree.keySet().iterator();
+
+        Tree t2 = (Tree)firstTree.get(vertexKeys.next());
+        VertexProperty vp = (VertexProperty)t2.keySet().iterator().next();
+        assertEquals(3, vp.id());
+        assertEquals("vadas", vp.value());
+
+        t2 = (Tree) firstTree.get(vertexKeys.next());
+        vp = (VertexProperty) t2.keySet().iterator().next();
+        assertEquals(5, vp.id());
+        assertEquals("lop", vp.value());
+
+        t2 = (Tree) firstTree.get(vertexKeys.next());
+        vp = (VertexProperty) t2.keySet().iterator().next();
+        assertEquals(7, vp.id());
+        assertEquals("josh", vp.value());
+    }
+
+    @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 = serializer.serializeResponseAsBinary(response, allocator);
+        final ResponseMessage deserialized = serializer.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());
+    }
+    
+    private void assertCommon(final ResponseMessage response) {
+        assertEquals(requestId, response.getRequestId());
+        assertEquals(ResponseStatusCode.SUCCESS, response.getStatus().getCode());
+    }
+
+    private ResponseMessage convert(final Object toSerialize) throws SerializationException {
+        final ByteBuf bb = serializer.serializeResponseAsBinary(responseMessageBuilder.result(toSerialize).create(), allocator);
+        return serializer.deserializeResponse(bb);
+    }
+}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-python/src/test/resources/org/apache/tinkerpop/gremlin/python/driver/gremlin-server-modern-secure-py.yaml
----------------------------------------------------------------------
diff --git a/gremlin-python/src/test/resources/org/apache/tinkerpop/gremlin/python/driver/gremlin-server-modern-secure-py.yaml b/gremlin-python/src/test/resources/org/apache/tinkerpop/gremlin/python/driver/gremlin-server-modern-secure-py.yaml
index a5b3480..fe32c6e 100644
--- a/gremlin-python/src/test/resources/org/apache/tinkerpop/gremlin/python/driver/gremlin-server-modern-secure-py.yaml
+++ b/gremlin-python/src/test/resources/org/apache/tinkerpop/gremlin/python/driver/gremlin-server-modern-secure-py.yaml
@@ -46,7 +46,7 @@ serializers:
   - { 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.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0, config: { useMapperFromGraph: graph }}        # application/json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}        # application/json,application/vnd.gremlin-v3.0+json
 metrics: {
   slf4jReporter: {enabled: true, interval: 180000}}
 strictTransactionManagement: false

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/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 d1cbbf3..da9068f 100644
--- a/gremlin-server/conf/gremlin-server-classic.yaml
+++ b/gremlin-server/conf/gremlin-server-classic.yaml
@@ -33,7 +33,7 @@ serializers:
   - { 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}         # application/json,application/vnd.gremlin-v3.0+json
 metrics: {
   slf4jReporter: {enabled: true, interval: 180000}}
 strictTransactionManagement: false

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-server/conf/gremlin-server-modern-py.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/conf/gremlin-server-modern-py.yaml b/gremlin-server/conf/gremlin-server-modern-py.yaml
index ff8228f..d8583ee 100644
--- a/gremlin-server/conf/gremlin-server-modern-py.yaml
+++ b/gremlin-server/conf/gremlin-server-modern-py.yaml
@@ -48,7 +48,7 @@ serializers:
   - { 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}         # application/json,application/vnd.gremlin-v3.0+json
 metrics: {
   slf4jReporter: {enabled: true, interval: 180000}}
 strictTransactionManagement: false

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/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 b063868..c67f20f 100644
--- a/gremlin-server/conf/gremlin-server-modern.yaml
+++ b/gremlin-server/conf/gremlin-server-modern.yaml
@@ -33,7 +33,7 @@ serializers:
   - { 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}         # application/json,application/vnd.gremlin-v3.0+json
 metrics: {
   slf4jReporter: {enabled: true, interval: 180000}}
 strictTransactionManagement: false

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-server/conf/gremlin-server.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/conf/gremlin-server.yaml b/gremlin-server/conf/gremlin-server.yaml
index d4e2849..3f3fc00 100644
--- a/gremlin-server/conf/gremlin-server.yaml
+++ b/gremlin-server/conf/gremlin-server.yaml
@@ -34,7 +34,7 @@ serializers:
   - { 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}         # application/json,application/vnd.gremlin-v3.0+json
 processors:
   - { className: org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor, config: { sessionTimeout: 28800000 }}
   - { className: org.apache.tinkerpop.gremlin.server.op.traversal.TraversalOpProcessor, config: { cacheExpirationTime: 600000, cacheMaxSize: 1000 }}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/pom.xml
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/pom.xml b/gremlin-tools/gremlin-io-test/pom.xml
index 691f637..2a3c6ed 100644
--- a/gremlin-tools/gremlin-io-test/pom.xml
+++ b/gremlin-tools/gremlin-io-test/pom.xml
@@ -180,7 +180,11 @@ toJsonV2d0NoTypes = { o, type, mapper, comment = "" ->
   toJson(o, type, mapper, comment, "v2d0-no-types")
 }
 
-writeSupportedV2Objects = { writer, mapper, toJsonFunction ->
+toJsonV3d0 = { o, type, mapper, comment = "" ->
+  toJson(o, type, mapper, comment, "v3d0")
+}
+
+writeSupportedV2V3Objects = { writer, mapper, toJsonFunction ->
   writer.write("Core\n")
   writer.write("~~~~\n\n")
   model.entries("Core").each {
@@ -241,7 +245,7 @@ mapper = GraphSONMapper.build().
 
 file = new File("${project.build.directory}/dev-docs/out-graphson-2d0-partial.txt")
 if (file.exists()) file.delete()
-file.withWriter { writeSupportedV2Objects(it, mapper, toJsonV2d0PartialTypes) }
+file.withWriter { writeSupportedV2V3Objects(it, mapper, toJsonV2d0PartialTypes) }
 
 mapper = GraphSONMapper.build().
                         addRegistry(TinkerIoRegistryV2d0.instance()).
@@ -252,7 +256,17 @@ mapper = GraphSONMapper.build().
 
 file = new File("${project.build.directory}/dev-docs/out-graphson-2d0-no-type.txt")
 if (file.exists()) file.delete()
-file.withWriter { writeSupportedV2Objects(it, mapper, toJsonV2d0NoTypes) }
+file.withWriter { writeSupportedV2V3Objects(it, mapper, toJsonV2d0NoTypes) }
+
+mapper = GraphSONMapper.build().
+                        addRegistry(TinkerIoRegistryV2d0.instance()).
+                        addCustomModule(GraphSONXModuleV2d0.build().create(false)).
+                        addCustomModule(new org.apache.tinkerpop.gremlin.driver.ser.AbstractGraphSONMessageSerializerV2d0.GremlinServerModule()).
+                        version(GraphSONVersion.V3_0).create().createMapper()
+
+file = new File("${project.build.directory}/dev-docs/out-graphson-3d0.txt")
+if (file.exists()) file.delete()
+file.withWriter { writeSupportedV2V3Objects(it, mapper, toJsonV3d0) }
 
 def ver = "_" + "${project.version}".replace(".","_").replace("-SNAPSHOT","")
 def target = "${project.basedir}/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/" + ver

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONCompatibility.java
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONCompatibility.java b/gremlin-tools/gremlin-io-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONCompatibility.java
index 522988a..69a12e7 100644
--- a/gremlin-tools/gremlin-io-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONCompatibility.java
+++ b/gremlin-tools/gremlin-io-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONCompatibility.java
@@ -36,7 +36,8 @@ public enum GraphSONCompatibility implements Compatibility {
     V2D0_NO_TYPE_3_2_4("3.2.4", "2.0", "v2d0-no-types"),
     V1D0_3_3_0("3.3.0", "2.0", "v1d0"),
     V2D0_PARTIAL_3_3_0("3.3.0", "2.0", "v2d0-partial"),
-    V2D0_NO_TYPE_3_3_0("3.3.0", "2.0", "v2d0-no-types");
+    V2D0_NO_TYPE_3_3_0("3.3.0", "2.0", "v2d0-no-types"),
+    V3D0_PARTIAL_3_3_0("3.3.0", "3.0", "v3d0");
 
     private static final String SEP = File.separator;
 

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTypedCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTypedCompatibilityTest.java b/gremlin-tools/gremlin-io-test/src/test/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTypedCompatibilityTest.java
index a3d2844..da200db 100644
--- a/gremlin-tools/gremlin-io-test/src/test/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTypedCompatibilityTest.java
+++ b/gremlin-tools/gremlin-io-test/src/test/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTypedCompatibilityTest.java
@@ -40,12 +40,19 @@ public class GraphSONTypedCompatibilityTest extends AbstractTypedCompatibilityTe
             addCustomModule(new org.apache.tinkerpop.gremlin.driver.ser.AbstractGraphSONMessageSerializerV2d0.GremlinServerModule()).
             version(GraphSONVersion.V2_0).create().createMapper();
 
+    private static ObjectMapper mapperV3 = GraphSONMapper.build().
+            addRegistry(TinkerIoRegistryV2d0.instance()).
+            addCustomModule(GraphSONXModuleV2d0.build().create(false)).
+            addCustomModule(new org.apache.tinkerpop.gremlin.driver.ser.AbstractGraphSONMessageSerializerV2d0.GremlinServerModule()).
+            version(GraphSONVersion.V3_0).create().createMapper();
+
     @Parameterized.Parameters(name = "expect({0})")
     public static Iterable<Object[]> data() {
         return Arrays.asList(new Object[][]{
                 {GraphSONCompatibility.V2D0_PARTIAL_3_2_3, mapperV2 },
                 {GraphSONCompatibility.V2D0_PARTIAL_3_2_4, mapperV2 },
-                {GraphSONCompatibility.V2D0_PARTIAL_3_3_0, mapperV2 }});
+                {GraphSONCompatibility.V2D0_PARTIAL_3_3_0, mapperV2 },
+                {GraphSONCompatibility.V3D0_PARTIAL_3_3_0, mapperV3 }});
     }
 
     @Parameterized.Parameter(value = 0)

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/authenticationchallenge-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/authenticationchallenge-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/authenticationchallenge-v3d0.json
new file mode 100644
index 0000000..8c5b82c
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/authenticationchallenge-v3d0.json
@@ -0,0 +1,12 @@
+{
+  "requestId" : "41d2e28a-20a4-4ab0-b379-d810dede3786",
+  "status" : {
+    "message" : "",
+    "code" : 407,
+    "attributes" : { }
+  },
+  "result" : {
+    "data" : null,
+    "meta" : { }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/authenticationresponse-v2d0-partial.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/authenticationresponse-v2d0-partial.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/authenticationresponse-v2d0-partial.json
index b893667..838e1fd 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/authenticationresponse-v2d0-partial.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/authenticationresponse-v2d0-partial.json
@@ -1,8 +1,5 @@
 {
-  "requestId" : {
-    "@type" : "g:UUID",
-    "@value" : "cb682578-9d92-4499-9ebc-5c6aa73c5397"
-  },
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
   "op" : "authentication",
   "processor" : "",
   "args" : {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/authenticationresponse-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/authenticationresponse-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/authenticationresponse-v3d0.json
new file mode 100644
index 0000000..838e1fd
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/authenticationresponse-v3d0.json
@@ -0,0 +1,9 @@
+{
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
+  "op" : "authentication",
+  "processor" : "",
+  "args" : {
+    "saslMechanism" : "PLAIN",
+    "sasl" : "AHN0ZXBocGhlbgBwYXNzd29yZA=="
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/barrier-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/barrier-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/barrier-v3d0.json
new file mode 100644
index 0000000..7ddccdd
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/barrier-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:Barrier",
+  "@value" : "normSack"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/bigdecimal-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/bigdecimal-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/bigdecimal-v3d0.json
new file mode 100644
index 0000000..475337c
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/bigdecimal-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:BigDecimal",
+  "@value" : 123456789987654321123456789987654321
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/biginteger-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/biginteger-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/biginteger-v3d0.json
new file mode 100644
index 0000000..58e6114
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/biginteger-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:BigInteger",
+  "@value" : 123456789987654321123456789987654321
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/binding-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/binding-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/binding-v3d0.json
new file mode 100644
index 0000000..579b8c7
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/binding-v3d0.json
@@ -0,0 +1,10 @@
+{
+  "@type" : "g:Binding",
+  "@value" : {
+    "key" : "x",
+    "value" : {
+      "@type" : "g:Int32",
+      "@value" : 1
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/byte-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/byte-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/byte-v3d0.json
new file mode 100644
index 0000000..979625b
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/byte-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:Byte",
+  "@value" : 1
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/bytebuffer-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/bytebuffer-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/bytebuffer-v3d0.json
new file mode 100644
index 0000000..5724115
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/bytebuffer-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:ByteBuffer",
+  "@value" : "c29tZSBieXRlcyBmb3IgeW91"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/bytecode-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/bytecode-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/bytecode-v3d0.json
new file mode 100644
index 0000000..269d277
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/bytecode-v3d0.json
@@ -0,0 +1,6 @@
+{
+  "@type" : "g:Bytecode",
+  "@value" : {
+    "step" : [ [ "V" ], [ "hasLabel", "person" ], [ "out" ], [ "in" ], [ "tree" ] ]
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/cardinality-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/cardinality-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/cardinality-v3d0.json
new file mode 100644
index 0000000..834e64e
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/cardinality-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:Cardinality",
+  "@value" : "list"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/char-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/char-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/char-v3d0.json
new file mode 100644
index 0000000..8f27e9d
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/char-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:Char",
+  "@value" : "x"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/class-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/class-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/class-v3d0.json
new file mode 100644
index 0000000..80f15a2
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/class-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:Class",
+  "@value" : "java.io.File"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/column-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/column-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/column-v3d0.json
new file mode 100644
index 0000000..0b3a56e
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/column-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:Column",
+  "@value" : "keys"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/date-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/date-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/date-v3d0.json
new file mode 100644
index 0000000..cf4007a
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/date-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:Date",
+  "@value" : 1481750076295
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/direction-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/direction-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/direction-v3d0.json
new file mode 100644
index 0000000..78cb7e4
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/direction-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:Direction",
+  "@value" : "OUT"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/double-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/double-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/double-v3d0.json
new file mode 100644
index 0000000..9ae4964
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/double-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:Double",
+  "@value" : 100.0
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/duration-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/duration-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/duration-v3d0.json
new file mode 100644
index 0000000..05c0ce9
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/duration-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:Duration",
+  "@value" : "PT120H"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v2d0-no-types.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v2d0-no-types.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v2d0-no-types.json
index 8f0ecab..a8e73db 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v2d0-no-types.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v2d0-no-types.json
@@ -6,6 +6,9 @@
   "inV" : 10,
   "outV" : 1,
   "properties" : {
-    "since" : 2009
+    "since" : {
+      "key" : "since",
+      "value" : 2009
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v2d0-partial.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v2d0-partial.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v2d0-partial.json
index 4c396f8..ba1c52e 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v2d0-partial.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v2d0-partial.json
@@ -18,8 +18,14 @@
     },
     "properties" : {
       "since" : {
-        "@type" : "g:Int32",
-        "@value" : 2009
+        "@type" : "g:Property",
+        "@value" : {
+          "key" : "since",
+          "value" : {
+            "@type" : "g:Int32",
+            "@value" : 2009
+          }
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v3d0.json
new file mode 100644
index 0000000..4c396f8
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v3d0.json
@@ -0,0 +1,26 @@
+{
+  "@type" : "g:Edge",
+  "@value" : {
+    "id" : {
+      "@type" : "g:Int32",
+      "@value" : 13
+    },
+    "label" : "develops",
+    "inVLabel" : "software",
+    "outVLabel" : "person",
+    "inV" : {
+      "@type" : "g:Int32",
+      "@value" : 10
+    },
+    "outV" : {
+      "@type" : "g:Int32",
+      "@value" : 1
+    },
+    "properties" : {
+      "since" : {
+        "@type" : "g:Int32",
+        "@value" : 2009
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/float-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/float-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/float-v3d0.json
new file mode 100644
index 0000000..7179aaf
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/float-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:Float",
+  "@value" : 100.0
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/inetaddress-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/inetaddress-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/inetaddress-v3d0.json
new file mode 100644
index 0000000..fba98c0
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/inetaddress-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:InetAddress",
+  "@value" : "localhost"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/instant-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/instant-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/instant-v3d0.json
new file mode 100644
index 0000000..3749741
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/instant-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:Instant",
+  "@value" : "2016-12-14T16:39:19.349Z"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/integer-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/integer-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/integer-v3d0.json
new file mode 100644
index 0000000..750ce7a
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/integer-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:Int32",
+  "@value" : 100
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/lambda-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/lambda-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/lambda-v3d0.json
new file mode 100644
index 0000000..5be179b
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/lambda-v3d0.json
@@ -0,0 +1,8 @@
+{
+  "@type" : "g:Lambda",
+  "@value" : {
+    "script" : "{ it.get() }",
+    "language" : "gremlin-groovy",
+    "arguments" : 1
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/localdate-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/localdate-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/localdate-v3d0.json
new file mode 100644
index 0000000..36fb81d
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/localdate-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:LocalDate",
+  "@value" : "2016-01-01"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/localdatetime-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/localdatetime-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/localdatetime-v3d0.json
new file mode 100644
index 0000000..2d83668
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/localdatetime-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:LocalDateTime",
+  "@value" : "2016-01-01T12:30"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/localtime-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/localtime-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/localtime-v3d0.json
new file mode 100644
index 0000000..eff65a7
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/localtime-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:LocalTime",
+  "@value" : "12:30:45"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/long-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/long-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/long-v3d0.json
new file mode 100644
index 0000000..84b9a23
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/long-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:Int64",
+  "@value" : 100
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/metrics-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/metrics-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/metrics-v3d0.json
new file mode 100644
index 0000000..7b1e964
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/metrics-v3d0.json
@@ -0,0 +1,54 @@
+{
+  "@type" : "g:Metrics",
+  "@value" : {
+    "dur" : {
+      "@type" : "g:Double",
+      "@value" : 100.0
+    },
+    "counts" : {
+      "traverserCount" : {
+        "@type" : "g:Int64",
+        "@value" : 4
+      },
+      "elementCount" : {
+        "@type" : "g:Int64",
+        "@value" : 4
+      }
+    },
+    "name" : "TinkerGraphStep(vertex,[~label.eq(person)])",
+    "annotations" : {
+      "percentDur" : {
+        "@type" : "g:Double",
+        "@value" : 25.0
+      }
+    },
+    "id" : "7.0.0()",
+    "metrics" : [ {
+      "@type" : "g:Metrics",
+      "@value" : {
+        "dur" : {
+          "@type" : "g:Double",
+          "@value" : 100.0
+        },
+        "counts" : {
+          "traverserCount" : {
+            "@type" : "g:Int64",
+            "@value" : 7
+          },
+          "elementCount" : {
+            "@type" : "g:Int64",
+            "@value" : 7
+          }
+        },
+        "name" : "VertexStep(OUT,vertex)",
+        "annotations" : {
+          "percentDur" : {
+            "@type" : "g:Double",
+            "@value" : 25.0
+          }
+        },
+        "id" : "3.0.0()"
+      }
+    } ]
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/monthday-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/monthday-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/monthday-v3d0.json
new file mode 100644
index 0000000..5da5914
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/monthday-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:MonthDay",
+  "@value" : "--01-01"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/offsetdatetime-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/offsetdatetime-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/offsetdatetime-v3d0.json
new file mode 100644
index 0000000..03f45cd
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/offsetdatetime-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:OffsetDateTime",
+  "@value" : "2007-12-03T10:15:30+01:00"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/offsettime-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/offsettime-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/offsettime-v3d0.json
new file mode 100644
index 0000000..b124953
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/offsettime-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:OffsetTime",
+  "@value" : "10:15:30+01:00"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/operator-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/operator-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/operator-v3d0.json
new file mode 100644
index 0000000..14c1400
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/operator-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:Operator",
+  "@value" : "sum"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/order-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/order-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/order-v3d0.json
new file mode 100644
index 0000000..6ad66d8
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/order-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:Order",
+  "@value" : "incr"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/p-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/p-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/p-v3d0.json
new file mode 100644
index 0000000..5bdfb3b
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/p-v3d0.json
@@ -0,0 +1,10 @@
+{
+  "@type" : "g:P",
+  "@value" : {
+    "predicate" : "gt",
+    "value" : {
+      "@type" : "g:Int32",
+      "@value" : 0
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/pand-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/pand-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/pand-v3d0.json
new file mode 100644
index 0000000..c271958
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/pand-v3d0.json
@@ -0,0 +1,25 @@
+{
+  "@type" : "g:P",
+  "@value" : {
+    "predicate" : "and",
+    "value" : [ {
+      "@type" : "g:P",
+      "@value" : {
+        "predicate" : "gt",
+        "value" : {
+          "@type" : "g:Int32",
+          "@value" : 0
+        }
+      }
+    }, {
+      "@type" : "g:P",
+      "@value" : {
+        "predicate" : "lt",
+        "value" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        }
+      }
+    } ]
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v2d0-no-types.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v2d0-no-types.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v2d0-no-types.json
index f271288..06a9522 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v2d0-no-types.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v2d0-no-types.json
@@ -7,13 +7,11 @@
       "name" : [ {
         "id" : 0,
         "value" : "marko",
-        "vertex" : 1,
         "label" : "name"
       } ],
       "location" : [ {
         "id" : 6,
         "value" : "san diego",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 1997,
@@ -22,7 +20,6 @@
       }, {
         "id" : 7,
         "value" : "santa cruz",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 2001,
@@ -31,7 +28,6 @@
       }, {
         "id" : 8,
         "value" : "brussels",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 2004,
@@ -40,7 +36,6 @@
       }, {
         "id" : 9,
         "value" : "santa fe",
-        "vertex" : 1,
         "label" : "location",
         "properties" : {
           "startTime" : 2005
@@ -54,7 +49,6 @@
       "name" : [ {
         "id" : 4,
         "value" : "gremlin",
-        "vertex" : 10,
         "label" : "name"
       } ]
     }
@@ -65,7 +59,6 @@
       "name" : [ {
         "id" : 5,
         "value" : "tinkergraph",
-        "vertex" : 11,
         "label" : "name"
       } ]
     }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v2d0-partial.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v2d0-partial.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v2d0-partial.json
index 1acc3ea..54ff76d 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v2d0-partial.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v2d0-partial.json
@@ -19,10 +19,6 @@
                 "@value" : 0
               },
               "value" : "marko",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "name"
             }
           } ],
@@ -34,10 +30,6 @@
                 "@value" : 6
               },
               "value" : "san diego",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -58,10 +50,6 @@
                 "@value" : 7
               },
               "value" : "santa cruz",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -82,10 +70,6 @@
                 "@value" : 8
               },
               "value" : "brussels",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -106,10 +90,6 @@
                 "@value" : 9
               },
               "value" : "santa fe",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 1
-              },
               "label" : "location",
               "properties" : {
                 "startTime" : {
@@ -138,10 +118,6 @@
                 "@value" : 4
               },
               "value" : "gremlin",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 10
-              },
               "label" : "name"
             }
           } ]
@@ -164,10 +140,6 @@
                 "@value" : 5
               },
               "value" : "tinkergraph",
-              "vertex" : {
-                "@type" : "g:Int32",
-                "@value" : 11
-              },
               "label" : "name"
             }
           } ]

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v3d0.json
new file mode 100644
index 0000000..8a41507
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v3d0.json
@@ -0,0 +1,122 @@
+{
+  "@type" : "g:Path",
+  "@value" : {
+    "labels" : [ [ ], [ ], [ ] ],
+    "objects" : [ {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "label" : "person",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 0
+            },
+            "value" : "marko"
+          } ],
+          "location" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 6
+            },
+            "value" : "san diego",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 1997
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 7
+            },
+            "value" : "santa cruz",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 8
+            },
+            "value" : "brussels",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 9
+            },
+            "value" : "santa fe",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          } ]
+        }
+      }
+    }, {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "label" : "software",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 4
+            },
+            "value" : "gremlin"
+          } ]
+        }
+      }
+    }, {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "label" : "software",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 5
+            },
+            "value" : "tinkergraph"
+          } ]
+        }
+      }
+    } ]
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/period-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/period-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/period-v3d0.json
new file mode 100644
index 0000000..20438a1
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/period-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "gx:Period",
+  "@value" : "P1Y6M15D"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/pick-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/pick-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/pick-v3d0.json
new file mode 100644
index 0000000..3ca2f2e
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/pick-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:Pick",
+  "@value" : "any"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/pop-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/pop-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/pop-v3d0.json
new file mode 100644
index 0000000..271515f
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/pop-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:Pop",
+  "@value" : "all"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/por-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/por-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/por-v3d0.json
new file mode 100644
index 0000000..71fcb7d
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/por-v3d0.json
@@ -0,0 +1,31 @@
+{
+  "@type" : "g:P",
+  "@value" : {
+    "predicate" : "or",
+    "value" : [ {
+      "@type" : "g:P",
+      "@value" : {
+        "predicate" : "gt",
+        "value" : {
+          "@type" : "g:Int32",
+          "@value" : 0
+        }
+      }
+    }, {
+      "@type" : "g:P",
+      "@value" : {
+        "predicate" : "within",
+        "value" : [ {
+          "@type" : "g:Int32",
+          "@value" : -1
+        }, {
+          "@type" : "g:Int32",
+          "@value" : -10
+        }, {
+          "@type" : "g:Int32",
+          "@value" : -100
+        } ]
+      }
+    } ]
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/property-v2d0-no-types.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/property-v2d0-no-types.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/property-v2d0-no-types.json
index 187d0ab..c051c0a 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/property-v2d0-no-types.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/property-v2d0-no-types.json
@@ -1,13 +1,4 @@
 {
   "key" : "since",
-  "value" : 2009,
-  "element" : {
-    "@type" : "g:Edge",
-    "@value" : {
-      "id" : 13,
-      "label" : "develops",
-      "outV" : 1,
-      "inV" : 10
-    }
-  }
+  "value" : 2009
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/property-v2d0-partial.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/property-v2d0-partial.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/property-v2d0-partial.json
index 1c6afa0..296fe32 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/property-v2d0-partial.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/property-v2d0-partial.json
@@ -5,24 +5,6 @@
     "value" : {
       "@type" : "g:Int32",
       "@value" : 2009
-    },
-    "element" : {
-      "@type" : "g:Edge",
-      "@value" : {
-        "id" : {
-          "@type" : "g:Int32",
-          "@value" : 13
-        },
-        "label" : "develops",
-        "outV" : {
-          "@type" : "g:Int32",
-          "@value" : 1
-        },
-        "inV" : {
-          "@type" : "g:Int32",
-          "@value" : 10
-        }
-      }
     }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/property-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/property-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/property-v3d0.json
new file mode 100644
index 0000000..db2e8db
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/property-v3d0.json
@@ -0,0 +1,25 @@
+{
+  "@type" : "g:Property",
+  "@value" : {
+    "key" : "since",
+    "value" : {
+      "@type" : "g:Int32",
+      "@value" : 2009
+    },
+    "edge" : {
+      "id" : {
+        "@type" : "g:Int32",
+        "@value" : 13
+      },
+      "label" : "develops",
+      "inV" : {
+        "@type" : "g:Int32",
+        "@value" : 10
+      },
+      "outV" : {
+        "@type" : "g:Int32",
+        "@value" : 1
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/scope-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/scope-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/scope-v3d0.json
new file mode 100644
index 0000000..4a74af0
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/scope-v3d0.json
@@ -0,0 +1,4 @@
+{
+  "@type" : "g:Scope",
+  "@value" : "local"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionclose-v2d0-partial.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionclose-v2d0-partial.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionclose-v2d0-partial.json
index ee860ea..cc4386b 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionclose-v2d0-partial.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionclose-v2d0-partial.json
@@ -1,8 +1,5 @@
 {
-  "requestId" : {
-    "@type" : "g:UUID",
-    "@value" : "cb682578-9d92-4499-9ebc-5c6aa73c5397"
-  },
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
   "op" : "close",
   "processor" : "session",
   "args" : {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionclose-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionclose-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionclose-v3d0.json
new file mode 100644
index 0000000..cc4386b
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionclose-v3d0.json
@@ -0,0 +1,11 @@
+{
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
+  "op" : "close",
+  "processor" : "session",
+  "args" : {
+    "session" : {
+      "@type" : "g:UUID",
+      "@value" : "41d2e28a-20a4-4ab0-b379-d810dede3786"
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessioneval-v2d0-partial.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessioneval-v2d0-partial.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessioneval-v2d0-partial.json
index 8d1d535..900e1ab 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessioneval-v2d0-partial.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessioneval-v2d0-partial.json
@@ -1,8 +1,5 @@
 {
-  "requestId" : {
-    "@type" : "g:UUID",
-    "@value" : "cb682578-9d92-4499-9ebc-5c6aa73c5397"
-  },
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
   "op" : "eval",
   "processor" : "session",
   "args" : {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessioneval-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessioneval-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessioneval-v3d0.json
new file mode 100644
index 0000000..900e1ab
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessioneval-v3d0.json
@@ -0,0 +1,19 @@
+{
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
+  "op" : "eval",
+  "processor" : "session",
+  "args" : {
+    "gremlin" : "g.V(x)",
+    "language" : "gremlin-groovy",
+    "session" : {
+      "@type" : "g:UUID",
+      "@value" : "41d2e28a-20a4-4ab0-b379-d810dede3786"
+    },
+    "bindings" : {
+      "x" : {
+        "@type" : "g:Int32",
+        "@value" : 1
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionevalaliased-v2d0-partial.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionevalaliased-v2d0-partial.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionevalaliased-v2d0-partial.json
index f1f2dc2..924bf77 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionevalaliased-v2d0-partial.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionevalaliased-v2d0-partial.json
@@ -1,8 +1,5 @@
 {
-  "requestId" : {
-    "@type" : "g:UUID",
-    "@value" : "cb682578-9d92-4499-9ebc-5c6aa73c5397"
-  },
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
   "op" : "eval",
   "processor" : "session",
   "args" : {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionevalaliased-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionevalaliased-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionevalaliased-v3d0.json
new file mode 100644
index 0000000..924bf77
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionevalaliased-v3d0.json
@@ -0,0 +1,22 @@
+{
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
+  "op" : "eval",
+  "processor" : "session",
+  "args" : {
+    "gremlin" : "social.V(x)",
+    "language" : "gremlin-groovy",
+    "aliases" : {
+      "g" : "social"
+    },
+    "session" : {
+      "@type" : "g:UUID",
+      "@value" : "41d2e28a-20a4-4ab0-b379-d810dede3786"
+    },
+    "bindings" : {
+      "x" : {
+        "@type" : "g:Int32",
+        "@value" : 1
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlesseval-v2d0-partial.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlesseval-v2d0-partial.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlesseval-v2d0-partial.json
index b01e98f..81e2f6c 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlesseval-v2d0-partial.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlesseval-v2d0-partial.json
@@ -1,8 +1,5 @@
 {
-  "requestId" : {
-    "@type" : "g:UUID",
-    "@value" : "cb682578-9d92-4499-9ebc-5c6aa73c5397"
-  },
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
   "op" : "eval",
   "processor" : "",
   "args" : {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlesseval-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlesseval-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlesseval-v3d0.json
new file mode 100644
index 0000000..81e2f6c
--- /dev/null
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/sessionlesseval-v3d0.json
@@ -0,0 +1,15 @@
+{
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
+  "op" : "eval",
+  "processor" : "",
+  "args" : {
+    "gremlin" : "g.V(x)",
+    "language" : "gremlin-groovy",
+    "bindings" : {
+      "x" : {
+        "@type" : "g:Int32",
+        "@value" : 1
+      }
+    }
+  }
+}
\ No newline at end of file


[06/31] tinkerpop git commit: Not going to touch GroupStep in this ticket. Too complicated ... will do for the next release. Minor tweaks and cleanups.

Posted by sp...@apache.org.
Not going to touch GroupStep in this ticket. Too complicated ... will do for the next release. Minor tweaks and cleanups.


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

Branch: refs/heads/TINKERPOP-1565
Commit: 973484d19a81ada87e95f9d3c2b0b66988f362d9
Parents: ee1ab08
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Jan 18 12:25:41 2017 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Wed Jan 18 12:25:41 2017 -0700

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |  2 +-
 .../traversal/step/filter/SampleGlobalStep.java |  8 +++-----
 .../traversal/step/map/OrderGlobalStep.java     | 21 ++++++++------------
 3 files changed, 12 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/973484d1/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index 052257f..25ff3e9 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -27,7 +27,7 @@ TinkerPop 3.2.4 (Release Date: NOT OFFICIALLY RELEASED YET)
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
 * Added `ProjectedTraverser` which wraps a traverser with a `List<Object>` of projected data.
-* `OrderGlobalStep` and `SampleGlobalStep` now emit traversers with their `by()`-projections and thus, can move beyond the local star graph.
+* `OrderGlobalStep` and `SampleGlobalStep` use `ProjectedTraverser` and now can work up to the local star graph in OLAP.
 * SASL negotiation supports both a byte array and Base64 encoded bytes as a string for authentication to Gremlin Server.
 * Deprecated `TinkerIoRegistry` replacing it with the more consistently named `TinkerIoRegistryV1d0`.
 * Made error messaging more consistent during result iteration timeouts in Gremlin Server.

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/973484d1/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/SampleGlobalStep.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/SampleGlobalStep.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/SampleGlobalStep.java
index 2b2cf20..28d2fb4 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/SampleGlobalStep.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/filter/SampleGlobalStep.java
@@ -66,10 +66,8 @@ public final class SampleGlobalStep<S> extends CollectingBarrierStep<S> implemen
 
     @Override
     public void processAllStarts() {
-        if (this.starts.hasNext()) {
-            while (this.starts.hasNext()) {
-                this.traverserSet.add(this.createProjectedTraverser(this.starts.next()));
-            }
+        while (this.starts.hasNext()) {
+            this.traverserSet.add(this.createProjectedTraverser(this.starts.next()));
         }
     }
 
@@ -97,7 +95,7 @@ public final class SampleGlobalStep<S> extends CollectingBarrierStep<S> implemen
                         runningWeight = runningWeight + currentWeight;
                         if (RANDOM.nextDouble() <= ((runningWeight / totalWeight))) {
                             final Traverser.Admin<S> split = s.split();
-                            split.setBulk(1l);
+                            split.setBulk(1L);
                             sampledSet.add(split);
                             runningAmountToSample++;
                             totalWeight = totalWeight - currentWeight;

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/973484d1/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
index 55d8650..e5c5834 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
@@ -53,7 +53,6 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
     private List<Pair<Traversal.Admin<S, C>, Comparator<C>>> comparators = new ArrayList<>();
     private MultiComparator<C> multiComparator = null;
     private long limit = Long.MAX_VALUE;
-    private boolean isShuffle = false;
 
     public OrderGlobalStep(final Traversal.Admin traversal) {
         super(traversal);
@@ -61,7 +60,9 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
 
     @Override
     public void barrierConsumer(final TraverserSet<S> traverserSet) {
-        if (this.isShuffle)
+        if (null == this.multiComparator) this.multiComparator = this.createMultiComparator();
+        //
+        if (this.multiComparator.isShuffle())
             traverserSet.shuffle();
         else
             traverserSet.sort((Comparator) this.multiComparator);
@@ -69,12 +70,8 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
 
     @Override
     public void processAllStarts() {
-        if (null == this.multiComparator)
-            this.multiComparator = this.createMultiComparator();
-        if (this.starts.hasNext()) {
-            while (this.starts.hasNext()) {
-                this.traverserSet.add(this.createProjectedTraverser(this.starts.next()));
-            }
+        while (this.starts.hasNext()) {
+            this.traverserSet.add(this.createProjectedTraverser(this.starts.next()));
         }
     }
 
@@ -88,7 +85,6 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
 
     @Override
     public void addComparator(final Traversal.Admin<S, C> traversal, final Comparator<C> comparator) {
-        this.isShuffle = Order.shuffle == (Comparator) comparator;
         this.comparators.add(new Pair<>(this.integrateChild(traversal), comparator));
     }
 
@@ -149,13 +145,12 @@ public final class OrderGlobalStep<S, C extends Comparable> extends CollectingBa
 
     @Override
     public MemoryComputeKey<TraverserSet<S>> getMemoryComputeKey() {
-        if (null == this.multiComparator)
-            this.multiComparator = this.createMultiComparator();
+        if (null == this.multiComparator) this.multiComparator = this.createMultiComparator();
         return MemoryComputeKey.of(this.getId(), new OrderBiOperator<>(this.limit, this.multiComparator), false, true);
     }
 
-    private final ProjectedTraverser<S,Object> createProjectedTraverser(final Traverser.Admin<S> traverser) {
-        final List<Object> projections = new ArrayList<>(this.comparators.size());
+    private final ProjectedTraverser<S, C> createProjectedTraverser(final Traverser.Admin<S> traverser) {
+        final List<C> projections = new ArrayList<>(this.comparators.size());
         for (final Pair<Traversal.Admin<S, C>, Comparator<C>> pair : this.comparators) {
             projections.add(TraversalUtil.apply(traverser, pair.getValue0()));
         }


[08/31] tinkerpop git commit: Merge branch 'TINKERPOP-1248' into tp32

Posted by sp...@apache.org.
Merge branch 'TINKERPOP-1248' into tp32


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

Branch: refs/heads/TINKERPOP-1565
Commit: 379a6e5e9e19b1de72fba3c7c401e4424028a88a
Parents: 3496402 973484d
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Thu Jan 19 05:33:22 2017 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Thu Jan 19 05:33:22 2017 -0700

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |   3 +
 .../traversal/step/filter/SampleGlobalStep.java |  19 +-
 .../traversal/step/map/OrderGlobalStep.java     |  68 ++++---
 .../step/util/CollectingBarrierStep.java        |  24 ++-
 .../ComputerVerificationStrategy.java           |   7 -
 .../traversal/traverser/ProjectedTraverser.java | 198 +++++++++++++++++++
 .../gremlin/structure/io/gryo/GryoVersion.java  |   8 +-
 .../gremlin/util/function/MultiComparator.java  |  72 +++++++
 8 files changed, 349 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/379a6e5e/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --cc CHANGELOG.asciidoc
index 74751fa,25ff3e9..88cbf32
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@@ -26,7 -26,8 +26,10 @@@ image::https://raw.githubusercontent.co
  TinkerPop 3.2.4 (Release Date: NOT OFFICIALLY RELEASED YET)
  ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
  
 +* `GroupBiOperator` no longer maintains state and thus, no more side-effect related OLAP inconsistencies.
+ * Added `ProjectedTraverser` which wraps a traverser with a `List<Object>` of projected data.
++* Fixed an optimization bug in `CollectionBarrierSteps` where the barrier was being consumed on each `addBarrier()`.
+ * `OrderGlobalStep` and `SampleGlobalStep` use `ProjectedTraverser` and now can work up to the local star graph in OLAP.
  * SASL negotiation supports both a byte array and Base64 encoded bytes as a string for authentication to Gremlin Server.
  * Deprecated `TinkerIoRegistry` replacing it with the more consistently named `TinkerIoRegistryV1d0`.
  * Made error messaging more consistent during result iteration timeouts in Gremlin Server.


[11/31] tinkerpop git commit: updated CHANGELOG.

Posted by sp...@apache.org.
updated CHANGELOG.


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

Branch: refs/heads/TINKERPOP-1565
Commit: 6002299925e7ef68275b9576331aac1194642bef
Parents: d54b490
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Thu Jan 19 10:16:43 2017 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Thu Jan 19 10:16:43 2017 -0700

----------------------------------------------------------------------
 CHANGELOG.asciidoc | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/60022999/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index 5f28790..fb0f8da 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -26,6 +26,7 @@ image::https://raw.githubusercontent.com/apache/tinkerpop/master/docs/static/ima
 TinkerPop 3.2.4 (Release Date: NOT OFFICIALLY RELEASED YET)
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
+* Fixed a bug associated with user-provided maps and `GroupSideEffectStep`.
 * `GroupBiOperator` no longer maintains a detached traversal and thus, no more side-effect related OLAP inconsistencies.
 * Added `ProjectedTraverser` which wraps a traverser with a `List<Object>` of projected data.
 * Fixed an optimization bug in `CollectionBarrierSteps` where the barrier was being consumed on each `addBarrier()`.


[20/31] tinkerpop git commit: TINKERPOP-1565 GraphSON 3.0 defaulted in drivers

Posted by sp...@apache.org.
TINKERPOP-1565 GraphSON 3.0 defaulted in drivers

Set this up for both python and java. Altered the sasl auth to use Base64 encoded strings in the drivers because GraphSON didn't have a byte array established as a type.


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

Branch: refs/heads/TINKERPOP-1565
Commit: a3515cfdf2681d645512bfff1e0090a102334833
Parents: 729af57
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Fri Jan 13 13:50:01 2017 -0500
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Jan 19 15:15:33 2017 -0500

----------------------------------------------------------------------
 .../ser/GraphSONMessageSerializerV3d0.java      |  4 ++--
 .../tinkerpop/gremlin/driver/ser/SerTokens.java |  1 +
 .../gremlin/driver/ser/Serializers.java         |  8 ++++++-
 .../driver/driver_remote_connection.py          |  4 ++--
 .../jsr223/PythonGraphSONJavaTranslator.java    |  4 ++--
 .../handler/SaslAuthenticationHandler.java      |  1 +
 .../server/GremlinServerAuthIntegrateTest.java  | 25 ++++++++++----------
 .../GremlinServerAuthOldIntegrateTest.java      | 17 ++++++-------
 .../server/GremlinServerHttpIntegrateTest.java  |  2 +-
 .../remote/gremlin-server-integration.yaml      |  1 +
 .../server/gremlin-server-integration.yaml      |  1 +
 11 files changed, 40 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a3515cfd/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0.java
----------------------------------------------------------------------
diff --git a/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0.java b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0.java
index f6f2fc8..d1c9e46 100644
--- a/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0.java
+++ b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0.java
@@ -38,7 +38,7 @@ import java.util.UUID;
  */
 public final class GraphSONMessageSerializerV3d0 extends AbstractGraphSONMessageSerializerV2d0 implements MessageTextSerializer {
     private static final Logger logger = LoggerFactory.getLogger(GraphSONMessageSerializerV3d0.class);
-    private static final String MIME_TYPE = SerTokens.MIME_JSON;
+    private static final String MIME_TYPE = SerTokens.MIME_GRAPHSON_V3D0;
 
     private static byte[] header;
 
@@ -65,7 +65,7 @@ public final class GraphSONMessageSerializerV3d0 extends AbstractGraphSONMessage
     @Override
     GraphSONMapper.Builder configureBuilder(final GraphSONMapper.Builder builder) {
         // override the 2.0 in AbstractGraphSONMessageSerializerV2d0
-        return builder.typeInfo(TypeInfo.PARTIAL_TYPES).version(GraphSONVersion.V3_0);
+        return builder.version(GraphSONVersion.V3_0);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a3515cfd/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 ecda014..f342a57 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,6 +39,7 @@ 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_GRAPHSON_V2D0 = "application/vnd.gremlin-v2.0+json";
+    public static final String MIME_GRAPHSON_V3D0 = "application/vnd.gremlin-v3.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/tinkerpop/blob/a3515cfd/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 bb0d377..69e8287 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
@@ -26,9 +26,13 @@ import org.apache.tinkerpop.gremlin.driver.MessageSerializer;
  * @author Stephen Mallette (http://stephen.genoprime.com)
  */
 public enum Serializers {
+    /**
+     * GraphSON 3.0.
+     */
     GRAPHSON(SerTokens.MIME_JSON),
     GRAPHSON_V1D0(SerTokens.MIME_GRAPHSON_V1D0),
     GRAPHSON_V2D0(SerTokens.MIME_GRAPHSON_V2D0),
+    GRAPHSON_V3D0(SerTokens.MIME_GRAPHSON_V3D0),
     GRYO_V1D0(SerTokens.MIME_GRYO_V1D0),
     GRYO_LITE_V1D0(SerTokens.MIME_GRYO_LITE_V1D0);
 
@@ -59,11 +63,13 @@ public enum Serializers {
     public MessageSerializer simpleInstance() {
         switch (value) {
             case SerTokens.MIME_JSON:
-                return new GraphSONMessageSerializerV1d0();
+                return new GraphSONMessageSerializerV3d0();
             case SerTokens.MIME_GRAPHSON_V1D0:
                 return new GraphSONMessageSerializerGremlinV1d0();
             case SerTokens.MIME_GRAPHSON_V2D0:
                 return new GraphSONMessageSerializerGremlinV2d0();
+            case SerTokens.MIME_GRAPHSON_V3D0:
+                return new GraphSONMessageSerializerV3d0();
             case SerTokens.MIME_GRYO_V1D0:
                 return new GryoMessageSerializerV1d0();
             case SerTokens.MIME_GRYO_LITE_V1D0:

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a3515cfd/gremlin-python/src/main/jython/gremlin_python/driver/driver_remote_connection.py
----------------------------------------------------------------------
diff --git a/gremlin-python/src/main/jython/gremlin_python/driver/driver_remote_connection.py b/gremlin-python/src/main/jython/gremlin_python/driver/driver_remote_connection.py
index 2cbe0e7..7bc792f 100644
--- a/gremlin-python/src/main/jython/gremlin_python/driver/driver_remote_connection.py
+++ b/gremlin-python/src/main/jython/gremlin_python/driver/driver_remote_connection.py
@@ -153,7 +153,7 @@ class DriverRemoteConnection(RemoteConnection):
     @gen.coroutine
     def _execute_message(self, send_message):
         send_message = b"".join([b"\x21",
-                                 b"application/vnd.gremlin-v2.0+json",
+                                 b"application/vnd.gremlin-v3.0+json",
                                  json.dumps(send_message, separators=(',', ':')).encode("utf-8")])
         if self._websocket.protocol is None:
             self._websocket = yield websocket.websocket_connect(self.url)
@@ -219,7 +219,7 @@ class Response:
         if status_code == 407:
             self._websocket.write_message(
                 b"".join([b"\x21",
-                          b"application/vnd.gremlin-v2.0+json",
+                          b"application/vnd.gremlin-v3.0+json",
                           json.dumps({
                               "requestId": {
                                   "@type": "g:UUID",

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a3515cfd/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/jsr223/PythonGraphSONJavaTranslator.java
----------------------------------------------------------------------
diff --git a/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/jsr223/PythonGraphSONJavaTranslator.java b/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/jsr223/PythonGraphSONJavaTranslator.java
index 09a457c..df3bb9d 100644
--- a/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/jsr223/PythonGraphSONJavaTranslator.java
+++ b/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/jsr223/PythonGraphSONJavaTranslator.java
@@ -53,10 +53,10 @@ final class PythonGraphSONJavaTranslator<S extends TraversalSource, T extends Tr
     private final JavaTranslator<S, T> javaTranslator;
     private final GraphSONReader reader = GraphSONReader.build().mapper(
             GraphSONMapper.build().addCustomModule(GraphSONXModuleV2d0.build().create(false))
-                    .version(GraphSONVersion.V2_0).create()).create();
+                    .version(GraphSONVersion.V3_0).create()).create();
     private final GraphSONWriter writer = GraphSONWriter.build().mapper(
             GraphSONMapper.build().addCustomModule(GraphSONXModuleV2d0.build().create(false))
-                    .version(GraphSONVersion.V2_0).create()).create();
+                    .version(GraphSONVersion.V3_0).create()).create();
 
     public PythonGraphSONJavaTranslator(final PythonTranslator pythonTranslator, final JavaTranslator<S, T> javaTranslator) {
         this.pythonTranslator = pythonTranslator;

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a3515cfd/gremlin-server/src/main/java/org/apache/tinkerpop/gremlin/server/handler/SaslAuthenticationHandler.java
----------------------------------------------------------------------
diff --git a/gremlin-server/src/main/java/org/apache/tinkerpop/gremlin/server/handler/SaslAuthenticationHandler.java b/gremlin-server/src/main/java/org/apache/tinkerpop/gremlin/server/handler/SaslAuthenticationHandler.java
index 6cb0ddb..31171b5 100644
--- a/gremlin-server/src/main/java/org/apache/tinkerpop/gremlin/server/handler/SaslAuthenticationHandler.java
+++ b/gremlin-server/src/main/java/org/apache/tinkerpop/gremlin/server/handler/SaslAuthenticationHandler.java
@@ -22,6 +22,7 @@ import io.netty.channel.Channel;
 import io.netty.channel.ChannelHandler;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.handler.codec.base64.Base64Decoder;
 import io.netty.util.Attribute;
 
 import java.net.InetAddress;

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a3515cfd/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthIntegrateTest.java
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthIntegrateTest.java b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthIntegrateTest.java
index 5d0f82e..8488f28 100644
--- a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthIntegrateTest.java
+++ b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthIntegrateTest.java
@@ -23,6 +23,8 @@ import org.apache.tinkerpop.gremlin.driver.Client;
 import org.apache.tinkerpop.gremlin.driver.Cluster;
 import org.apache.tinkerpop.gremlin.driver.exception.ResponseException;
 import org.apache.tinkerpop.gremlin.server.auth.SimpleAuthenticator;
+import org.apache.tinkerpop.gremlin.structure.Property;
+import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.junit.Test;
 
 import java.util.HashMap;
@@ -167,22 +169,22 @@ public class GremlinServerAuthIntegrateTest extends AbstractGremlinServerIntegra
     }
 
     @Test
-    public void shouldAuthenticateWithPlainTextOverJSONSerialization() throws Exception {
+    public void shouldAuthenticateWithPlainTextOverDefaultJSONSerialization() throws Exception {
         final Cluster cluster = TestClientFactory.build().serializer(Serializers.GRAPHSON)
                 .credentials("stephen", "password").create();
         final Client client = cluster.connect();
 
         try {
-            assertEquals(3, client.submit("1+2").all().get().get(0).get(Map.class).get("@value"));
-            assertEquals(2, client.submit("1+1").all().get().get(0).get(Map.class).get("@value"));
-            assertEquals(4, client.submit("1+3").all().get().get(0).get(Map.class).get("@value"));
+            assertEquals(3, client.submit("1+2").all().get().get(0).getInt());
+            assertEquals(2, client.submit("1+1").all().get().get(0).getInt());
+            assertEquals(4, client.submit("1+3").all().get().get(0).getInt());
         } finally {
             cluster.close();
         }
     }
 
     @Test
-    public void shouldAuthenticateWithPlainTextOverGraphSONSerialization() throws Exception {
+    public void shouldAuthenticateWithPlainTextOverGraphSONV1Serialization() throws Exception {
         final Cluster cluster = TestClientFactory.build().serializer(Serializers.GRAPHSON_V1D0)
                 .credentials("stephen", "password").create();
         final Client client = cluster.connect();
@@ -197,25 +199,24 @@ public class GremlinServerAuthIntegrateTest extends AbstractGremlinServerIntegra
     }
 
     @Test
-    public void shouldAuthenticateAndWorkWithVariablesOverJsonSerialization() throws Exception {
+    public void shouldAuthenticateAndWorkWithVariablesOverDefaultJsonSerialization() throws Exception {
         final Cluster cluster = TestClientFactory.build().serializer(Serializers.GRAPHSON)
                 .credentials("stephen", "password").create();
         final Client client = cluster.connect(name.getMethodName());
 
         try {
-            final Map vertex = (Map) client.submit("v=graph.addVertex(\"name\", \"stephen\")").all().get().get(0).getObject();
-            final Map<String, List<Map>> properties = (Map) ((Map) vertex.get("@value")).get("properties");
-            assertEquals("stephen", properties.get("name").get(0).get("value"));
+            final Vertex vertex = (Vertex) client.submit("v=graph.addVertex(\"name\", \"stephen\")").all().get().get(0).getObject();
+            assertEquals("stephen", vertex.value("name"));
 
-            final Map vpName = (Map)client.submit("v.property('name')").all().get().get(0).getObject();
-            assertEquals("stephen", ((Map) vpName.get("@value")).get("value"));
+            final Property vpName = (Property)client.submit("v.property('name')").all().get().get(0).getObject();
+            assertEquals("stephen", vpName.value());
         } finally {
             cluster.close();
         }
     }
 
     @Test
-    public void shouldAuthenticateAndWorkWithVariablesOverGraphSONSerialization() throws Exception {
+    public void shouldAuthenticateAndWorkWithVariablesOverGraphSONV1Serialization() throws Exception {
         final Cluster cluster = TestClientFactory.build().serializer(Serializers.GRAPHSON_V1D0)
                 .credentials("stephen", "password").create();
         final Client client = cluster.connect(name.getMethodName());

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a3515cfd/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthOldIntegrateTest.java
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthOldIntegrateTest.java b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthOldIntegrateTest.java
index c8312ae..7fd3c61 100644
--- a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthOldIntegrateTest.java
+++ b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerAuthOldIntegrateTest.java
@@ -24,6 +24,8 @@ import org.apache.tinkerpop.gremlin.driver.Cluster;
 import org.apache.tinkerpop.gremlin.driver.exception.ResponseException;
 import org.apache.tinkerpop.gremlin.driver.ser.Serializers;
 import org.apache.tinkerpop.gremlin.server.auth.SimpleAuthenticator;
+import org.apache.tinkerpop.gremlin.structure.Property;
+import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.junit.Test;
 
 import java.util.HashMap;
@@ -179,9 +181,9 @@ public class GremlinServerAuthOldIntegrateTest extends AbstractGremlinServerInte
         final Client client = cluster.connect();
 
         try {
-            assertEquals(3, client.submit("1+2").all().get().get(0).get(Map.class).get("@value"));
-            assertEquals(2, client.submit("1+1").all().get().get(0).get(Map.class).get("@value"));
-            assertEquals(4, client.submit("1+3").all().get().get(0).get(Map.class).get("@value"));
+            assertEquals(3, client.submit("1+2").all().get().get(0).getInt());
+            assertEquals(2, client.submit("1+1").all().get().get(0).getInt());
+            assertEquals(4, client.submit("1+3").all().get().get(0).getInt());
         } finally {
             cluster.close();
         }
@@ -209,12 +211,11 @@ public class GremlinServerAuthOldIntegrateTest extends AbstractGremlinServerInte
         final Client client = cluster.connect(name.getMethodName());
 
         try {
-            final Map vertex = (Map) client.submit("v=graph.addVertex(\"name\", \"stephen\")").all().get().get(0).getObject();
-            final Map<String, List<Map>> properties = (Map) ((Map) vertex.get("@value")).get("properties");
-            assertEquals("stephen", properties.get("name").get(0).get("value"));
+            final Vertex vertex = (Vertex) client.submit("v=graph.addVertex(\"name\", \"stephen\")").all().get().get(0).getObject();
+            assertEquals("stephen", vertex.value("name"));
 
-            final Map vpName = (Map)client.submit("v.property('name')").all().get().get(0).getObject();
-            assertEquals("stephen", ((Map) vpName.get("@value")).get("value"));
+            final Property vpName = (Property)client.submit("v.property('name')").all().get().get(0).getObject();
+            assertEquals("stephen", vpName.value());
         } finally {
             cluster.close();
         }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a3515cfd/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java
index b2a1568..e46871b 100644
--- a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java
+++ b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java
@@ -366,7 +366,7 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
             assertEquals("application/json", response.getEntity().getContentType().getValue());
             final String json = EntityUtils.toString(response.getEntity());
             final JsonNode node = mapper.readTree(json);
-            assertEquals("stephen", node.get("result").get("data").get(0).get("@value").get("properties").get("name").get(0).get(GraphSONTokens.VALUE).asText());
+            assertEquals("stephen", node.get("result").get("data").get(0).get(GraphSONTokens.VALUEPROP).get("properties").get("name").get(0).get(GraphSONTokens.VALUEPROP).get(GraphSONTokens.VALUE).asText());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a3515cfd/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/driver/remote/gremlin-server-integration.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/driver/remote/gremlin-server-integration.yaml b/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/driver/remote/gremlin-server-integration.yaml
index 7d23fb9..98cdef0 100644
--- a/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/driver/remote/gremlin-server-integration.yaml
+++ b/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/driver/remote/gremlin-server-integration.yaml
@@ -38,6 +38,7 @@ serializers:
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }}
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }}
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}
 processors:
   - { className: org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor, config: { sessionTimeout: 28800000 }}
 metrics: {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a3515cfd/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/server/gremlin-server-integration.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/server/gremlin-server-integration.yaml b/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/server/gremlin-server-integration.yaml
index a467111..1c1ef8d 100644
--- a/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/server/gremlin-server-integration.yaml
+++ b/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/server/gremlin-server-integration.yaml
@@ -34,6 +34,7 @@ serializers:
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }}
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }}
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}
 processors:
   - { className: org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor, config: { sessionTimeout: 28800000 }}
 metrics: {


[15/31] tinkerpop git commit: Merge branch 'tp32'

Posted by sp...@apache.org.
Merge branch 'tp32'


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

Branch: refs/heads/TINKERPOP-1565
Commit: bfe432eb8dd0bc06ab335add0919944d21739e54
Parents: 1bc66ab 8ad2911
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Thu Jan 19 13:01:18 2017 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Thu Jan 19 13:01:18 2017 -0700

----------------------------------------------------------------------
 .../process/traversal/step/util/CollectingBarrierStep.java      | 4 ++--
 .../gremlin/process/traversal/traverser/ProjectedTraverser.java | 5 +++++
 2 files changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[05/31] tinkerpop git commit: added a private no-arg constructor to MultiComparator so it can be serialized.

Posted by sp...@apache.org.
added a private no-arg constructor to MultiComparator so it can be serialized.


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

Branch: refs/heads/TINKERPOP-1565
Commit: ee1ab08b0b8c7c6dca85d9ff046dcff739549387
Parents: 91e1f50
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Jan 18 11:20:07 2017 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Wed Jan 18 11:20:07 2017 -0700

----------------------------------------------------------------------
 .../tinkerpop/gremlin/util/function/MultiComparator.java     | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/ee1ab08b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java
index b7176ab..5d24ddf 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java
@@ -31,8 +31,12 @@ import java.util.List;
  */
 public final class MultiComparator<C> implements Comparator<C>, Serializable {
 
-    private final List<Comparator> comparators;
-    private final boolean isShuffle;
+    private List<Comparator> comparators;
+    private boolean isShuffle;
+
+    private MultiComparator() {
+        // for serialization purposes
+    }
 
     public MultiComparator(final List<Comparator<C>> comparators) {
         this.comparators = (List) comparators;


[24/31] tinkerpop git commit: TINKERPOP-1565 Stub out GraphSON 3.0

Posted by sp...@apache.org.
TINKERPOP-1565 Stub out GraphSON 3.0

Set up the new testing model and generated test files. Reverted changes on property serialization that went in as breaking changes on GraphSON 2.0 and moved them forward to GraphSON 3.0.


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

Branch: refs/heads/TINKERPOP-1565
Commit: 7db10c40cc143473eff78866d989a90df8da7206
Parents: a004162
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Thu Jan 12 14:58:50 2017 -0500
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Jan 19 15:15:33 2017 -0500

----------------------------------------------------------------------
 .../structure/io/graphson/GraphSONMapper.java   |  71 +-
 .../structure/io/graphson/GraphSONModule.java   | 206 ++++++
 .../io/graphson/GraphSONSerializersV2d0.java    | 162 ++---
 .../io/graphson/GraphSONSerializersV3d0.java    | 632 +++++++++++++++++
 .../structure/io/graphson/GraphSONTokens.java   |   1 +
 .../structure/io/graphson/GraphSONUtil.java     |   7 +
 .../structure/io/graphson/GraphSONVersion.java  |   3 +-
 .../ser/GraphSONMessageSerializerV3d0.java      | 124 ++++
 .../ser/GraphSONMessageSerializerV3d0Test.java  | 342 ++++++++++
 .../driver/gremlin-server-modern-secure-py.yaml |   2 +-
 gremlin-server/conf/gremlin-server-classic.yaml |   2 +-
 .../conf/gremlin-server-modern-py.yaml          |   2 +-
 gremlin-server/conf/gremlin-server-modern.yaml  |   2 +-
 gremlin-server/conf/gremlin-server.yaml         |   2 +-
 gremlin-tools/gremlin-io-test/pom.xml           |  20 +-
 .../io/graphson/GraphSONCompatibility.java      |   3 +-
 .../GraphSONTypedCompatibilityTest.java         |   9 +-
 .../_3_3_0/authenticationchallenge-v3d0.json    |  12 +
 .../authenticationresponse-v2d0-partial.json    |   5 +-
 .../_3_3_0/authenticationresponse-v3d0.json     |   9 +
 .../io/graphson/_3_3_0/barrier-v3d0.json        |   4 +
 .../io/graphson/_3_3_0/bigdecimal-v3d0.json     |   4 +
 .../io/graphson/_3_3_0/biginteger-v3d0.json     |   4 +
 .../io/graphson/_3_3_0/binding-v3d0.json        |  10 +
 .../structure/io/graphson/_3_3_0/byte-v3d0.json |   4 +
 .../io/graphson/_3_3_0/bytebuffer-v3d0.json     |   4 +
 .../io/graphson/_3_3_0/bytecode-v3d0.json       |   6 +
 .../io/graphson/_3_3_0/cardinality-v3d0.json    |   4 +
 .../structure/io/graphson/_3_3_0/char-v3d0.json |   4 +
 .../io/graphson/_3_3_0/class-v3d0.json          |   4 +
 .../io/graphson/_3_3_0/column-v3d0.json         |   4 +
 .../structure/io/graphson/_3_3_0/date-v3d0.json |   4 +
 .../io/graphson/_3_3_0/direction-v3d0.json      |   4 +
 .../io/graphson/_3_3_0/double-v3d0.json         |   4 +
 .../io/graphson/_3_3_0/duration-v3d0.json       |   4 +
 .../io/graphson/_3_3_0/edge-v2d0-no-types.json  |   5 +-
 .../io/graphson/_3_3_0/edge-v2d0-partial.json   |  10 +-
 .../structure/io/graphson/_3_3_0/edge-v3d0.json |  26 +
 .../io/graphson/_3_3_0/float-v3d0.json          |   4 +
 .../io/graphson/_3_3_0/inetaddress-v3d0.json    |   4 +
 .../io/graphson/_3_3_0/instant-v3d0.json        |   4 +
 .../io/graphson/_3_3_0/integer-v3d0.json        |   4 +
 .../io/graphson/_3_3_0/lambda-v3d0.json         |   8 +
 .../io/graphson/_3_3_0/localdate-v3d0.json      |   4 +
 .../io/graphson/_3_3_0/localdatetime-v3d0.json  |   4 +
 .../io/graphson/_3_3_0/localtime-v3d0.json      |   4 +
 .../structure/io/graphson/_3_3_0/long-v3d0.json |   4 +
 .../io/graphson/_3_3_0/metrics-v3d0.json        |  54 ++
 .../io/graphson/_3_3_0/monthday-v3d0.json       |   4 +
 .../io/graphson/_3_3_0/offsetdatetime-v3d0.json |   4 +
 .../io/graphson/_3_3_0/offsettime-v3d0.json     |   4 +
 .../io/graphson/_3_3_0/operator-v3d0.json       |   4 +
 .../io/graphson/_3_3_0/order-v3d0.json          |   4 +
 .../structure/io/graphson/_3_3_0/p-v3d0.json    |  10 +
 .../structure/io/graphson/_3_3_0/pand-v3d0.json |  25 +
 .../io/graphson/_3_3_0/path-v2d0-no-types.json  |   7 -
 .../io/graphson/_3_3_0/path-v2d0-partial.json   |  28 -
 .../structure/io/graphson/_3_3_0/path-v3d0.json | 122 ++++
 .../io/graphson/_3_3_0/period-v3d0.json         |   4 +
 .../structure/io/graphson/_3_3_0/pick-v3d0.json |   4 +
 .../structure/io/graphson/_3_3_0/pop-v3d0.json  |   4 +
 .../structure/io/graphson/_3_3_0/por-v3d0.json  |  31 +
 .../graphson/_3_3_0/property-v2d0-no-types.json |  11 +-
 .../graphson/_3_3_0/property-v2d0-partial.json  |  18 -
 .../io/graphson/_3_3_0/property-v3d0.json       |  25 +
 .../io/graphson/_3_3_0/scope-v3d0.json          |   4 +
 .../_3_3_0/sessionclose-v2d0-partial.json       |   5 +-
 .../io/graphson/_3_3_0/sessionclose-v3d0.json   |  11 +
 .../_3_3_0/sessioneval-v2d0-partial.json        |   5 +-
 .../io/graphson/_3_3_0/sessioneval-v3d0.json    |  19 +
 .../_3_3_0/sessionevalaliased-v2d0-partial.json |   5 +-
 .../_3_3_0/sessionevalaliased-v3d0.json         |  22 +
 .../_3_3_0/sessionlesseval-v2d0-partial.json    |   5 +-
 .../graphson/_3_3_0/sessionlesseval-v3d0.json   |  15 +
 .../sessionlessevalaliased-v2d0-partial.json    |   5 +-
 .../_3_3_0/sessionlessevalaliased-v3d0.json     |  18 +
 .../io/graphson/_3_3_0/short-v3d0.json          |   4 +
 .../_3_3_0/standardresult-v2d0-no-types.json    |   5 -
 .../_3_3_0/standardresult-v2d0-partial.json     |  20 -
 .../io/graphson/_3_3_0/standardresult-v3d0.json |  91 +++
 .../_3_3_0/stargraph-v2d0-no-types.json         |   5 -
 .../graphson/_3_3_0/stargraph-v2d0-partial.json |  20 -
 .../io/graphson/_3_3_0/stargraph-v3d0.json      |  82 +++
 .../structure/io/graphson/_3_3_0/t-v3d0.json    |   4 +
 .../io/graphson/_3_3_0/timestamp-v3d0.json      |   4 +
 .../_3_3_0/tinkergraph-v2d0-no-types.json       |  85 ++-
 .../_3_3_0/tinkergraph-v2d0-partial.json        | 210 +++---
 .../io/graphson/_3_3_0/tinkergraph-v3d0.json    | 671 +++++++++++++++++++
 .../graphson/_3_3_0/traversalmetrics-v3d0.json  | 114 ++++
 .../_3_3_0/traverser-v2d0-no-types.json         |   5 -
 .../graphson/_3_3_0/traverser-v2d0-partial.json |  20 -
 .../io/graphson/_3_3_0/traverser-v3d0.json      |  89 +++
 .../io/graphson/_3_3_0/tree-v2d0-no-types.json  |   7 -
 .../io/graphson/_3_3_0/tree-v2d0-partial.json   |  28 -
 .../structure/io/graphson/_3_3_0/tree-v3d0.json | 137 ++++
 .../structure/io/graphson/_3_3_0/uuid-v3d0.json |   4 +
 .../graphson/_3_3_0/vertex-v2d0-no-types.json   |   5 -
 .../io/graphson/_3_3_0/vertex-v2d0-partial.json |  20 -
 .../io/graphson/_3_3_0/vertex-v3d0.json         |  80 +++
 .../_3_3_0/vertexproperty-v2d0-no-types.json    |   1 -
 .../_3_3_0/vertexproperty-v2d0-partial.json     |   4 -
 .../io/graphson/_3_3_0/vertexproperty-v3d0.json |  15 +
 .../structure/io/graphson/_3_3_0/year-v3d0.json |   4 +
 .../io/graphson/_3_3_0/yearmonth-v3d0.json      |   4 +
 .../io/graphson/_3_3_0/zoneddatetime-v3d0.json  |   4 +
 .../io/graphson/_3_3_0/zoneoffset-v3d0.json     |   4 +
 106 files changed, 3465 insertions(+), 523 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONMapper.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONMapper.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONMapper.java
index f82ebb7..d442dbc 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONMapper.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONMapper.java
@@ -87,13 +87,44 @@ public class GraphSONMapper implements Mapper<ObjectMapper> {
         om.registerModule(graphSONModule);
         customModules.forEach(om::registerModule);
 
-
         // plugin external serialization modules
         if (loadCustomSerializers)
             om.findAndRegisterModules();
 
-
-        if (version == GraphSONVersion.V1_0) {
+        // graphson 3.0 only allows type - there is no option to remove embedded types
+        if (version == GraphSONVersion.V3_0 || (version == GraphSONVersion.V2_0 && typeInfo != TypeInfo.NO_TYPES)) {
+            final GraphSONTypeIdResolver graphSONTypeIdResolver = new GraphSONTypeIdResolver();
+            final TypeResolverBuilder typer = new GraphSONTypeResolverBuilder()
+                    .typesEmbedding(getTypeInfo())
+                    .valuePropertyName(GraphSONTokens.VALUEPROP)
+                    .init(JsonTypeInfo.Id.CUSTOM, graphSONTypeIdResolver)
+                    .typeProperty(GraphSONTokens.VALUETYPE);
+
+            // Registers native Java types that are supported by Jackson
+            registerJavaBaseTypes(graphSONTypeIdResolver);
+
+            // Registers the GraphSON Module's types
+            graphSONModule.getTypeDefinitions().forEach(
+                    (targetClass, typeId) -> graphSONTypeIdResolver.addCustomType(
+                            String.format("%s:%s", graphSONModule.getTypeNamespace(), typeId), targetClass));
+
+            // Register types to typeResolver for the Custom modules
+            customModules.forEach(e -> {
+                if (e instanceof TinkerPopJacksonModule) {
+                    final TinkerPopJacksonModule mod = (TinkerPopJacksonModule) e;
+                    final Map<Class, String> moduleTypeDefinitions = mod.getTypeDefinitions();
+                    if (moduleTypeDefinitions != null) {
+                        if (mod.getTypeNamespace() == null || mod.getTypeNamespace().isEmpty())
+                            throw new IllegalStateException("Cannot specify a module for GraphSON 2.0 with type definitions but without a type Domain. " +
+                                    "If no specific type domain is required, use Gremlin's default domain, \"gremlin\" but there may be collisions.");
+
+                        moduleTypeDefinitions.forEach((targetClass, typeId) -> graphSONTypeIdResolver.addCustomType(
+                                        String.format("%s:%s", mod.getTypeNamespace(), typeId), targetClass));
+                    }
+                }
+            });
+            om.setDefaultTyping(typer);
+        } else if (version == GraphSONVersion.V1_0 || version == GraphSONVersion.V2_0) {
             if (embedTypes) {
                 final TypeResolverBuilder<?> typer = new StdTypeResolverBuilder()
                         .init(JsonTypeInfo.Id.CLASS, null)
@@ -101,40 +132,6 @@ public class GraphSONMapper implements Mapper<ObjectMapper> {
                         .typeProperty(GraphSONTokens.CLASS);
                 om.setDefaultTyping(typer);
             }
-        } else if (version == GraphSONVersion.V2_0) {
-            if (typeInfo != TypeInfo.NO_TYPES) {
-                final GraphSONTypeIdResolver graphSONTypeIdResolver = new GraphSONTypeIdResolver();
-                final TypeResolverBuilder typer = new GraphSONTypeResolverBuilder()
-                        .typesEmbedding(getTypeInfo())
-                        .valuePropertyName(GraphSONTokens.VALUEPROP)
-                        .init(JsonTypeInfo.Id.CUSTOM, graphSONTypeIdResolver)
-                        .typeProperty(GraphSONTokens.VALUETYPE);
-
-                // Registers native Java types that are supported by Jackson
-                registerJavaBaseTypes(graphSONTypeIdResolver);
-
-                // Registers the GraphSON Module's types
-                graphSONModule.getTypeDefinitions().forEach(
-                        (targetClass, typeId) -> graphSONTypeIdResolver.addCustomType(
-                                String.format("%s:%s", graphSONModule.getTypeNamespace(), typeId), targetClass));
-
-                // Register types to typeResolver for the Custom modules
-                customModules.forEach(e -> {
-                    if (e instanceof TinkerPopJacksonModule) {
-                        final TinkerPopJacksonModule mod = (TinkerPopJacksonModule) e;
-                        final Map<Class, String> moduleTypeDefinitions = mod.getTypeDefinitions();
-                        if (moduleTypeDefinitions != null) {
-                            if (mod.getTypeNamespace() == null || mod.getTypeNamespace().isEmpty())
-                                throw new IllegalStateException("Cannot specify a module for GraphSON 2.0 with type definitions but without a type Domain. " +
-                                        "If no specific type domain is required, use Gremlin's default domain, \"gremlin\" but there may be collisions.");
-
-                            moduleTypeDefinitions.forEach((targetClass, typeId) -> graphSONTypeIdResolver.addCustomType(
-                                            String.format("%s:%s", mod.getTypeNamespace(), typeId), targetClass));
-                        }
-                    }
-                });
-                om.setDefaultTyping(typer);
-            }
         } else {
             throw new IllegalStateException("Unknown GraphSONVersion : " + version);
         }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java
index a061d1c..dceae6b 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java
@@ -109,6 +109,212 @@ abstract class GraphSONModule extends TinkerPopJacksonModule {
     }
 
     /**
+     * Version 3.0 of GraphSON.
+     */
+    static final class GraphSONModuleV3d0 extends GraphSONModule {
+
+        private static final Map<Class, String> TYPE_DEFINITIONS = Collections.unmodifiableMap(
+                new LinkedHashMap<Class, String>() {{
+                    // Those don't have deserializers because handled by Jackson,
+                    // but we still want to rename them in GraphSON
+                    put(Integer.class, "Int32");
+                    put(Long.class, "Int64");
+                    put(Double.class, "Double");
+                    put(Float.class, "Float");
+
+                    // Tinkerpop Graph objects
+                    put(Lambda.class, "Lambda");
+                    put(Vertex.class, "Vertex");
+                    put(Edge.class, "Edge");
+                    put(Property.class, "Property");
+                    put(Path.class, "Path");
+                    put(VertexProperty.class, "VertexProperty");
+                    put(Metrics.class, "Metrics");
+                    put(TraversalMetrics.class, "TraversalMetrics");
+                    put(Traverser.class, "Traverser");
+                    put(Tree.class, "Tree");
+                    put(Bytecode.class, "Bytecode");
+                    put(Bytecode.Binding.class, "Binding");
+                    put(AndP.class, "P");
+                    put(OrP.class, "P");
+                    put(P.class, "P");
+                    Stream.of(
+                            VertexProperty.Cardinality.class,
+                            Column.class,
+                            Direction.class,
+                            Operator.class,
+                            Order.class,
+                            Pop.class,
+                            SackFunctions.Barrier.class,
+                            TraversalOptionParent.Pick.class,
+                            Scope.class,
+                            T.class).forEach(e -> put(e, e.getSimpleName()));
+                    Arrays.asList(
+                            ConnectiveStrategy.class,
+                            ElementIdStrategy.class,
+                            EventStrategy.class,
+                            HaltedTraverserStrategy.class,
+                            PartitionStrategy.class,
+                            SubgraphStrategy.class,
+                            LazyBarrierStrategy.class,
+                            MatchAlgorithmStrategy.class,
+                            AdjacentToIncidentStrategy.class,
+                            FilterRankingStrategy.class,
+                            IdentityRemovalStrategy.class,
+                            IncidentToAdjacentStrategy.class,
+                            InlineFilterStrategy.class,
+                            MatchPredicateStrategy.class,
+                            OrderLimitStrategy.class,
+                            PathProcessorStrategy.class,
+                            PathRetractionStrategy.class,
+                            RangeByIsCountStrategy.class,
+                            RepeatUnrollStrategy.class,
+                            ComputerVerificationStrategy.class,
+                            LambdaRestrictionStrategy.class,
+                            ReadOnlyStrategy.class,
+                            StandardVerificationStrategy.class,
+                            //
+                            GraphFilterStrategy.class,
+                            VertexProgramStrategy.class
+                    ).forEach(strategy -> put(strategy, strategy.getSimpleName()));
+                }});
+
+        /**
+         * Constructs a new object.
+         */
+        protected GraphSONModuleV3d0(final boolean normalize) {
+            super("graphson-3.0");
+
+            /////////////////////// SERIALIZERS ////////////////////////////
+
+            // graph
+            addSerializer(Edge.class, new GraphSONSerializersV3d0.EdgeJacksonSerializer(normalize));
+            addSerializer(Vertex.class, new GraphSONSerializersV3d0.VertexJacksonSerializer(normalize));
+            addSerializer(VertexProperty.class, new GraphSONSerializersV3d0.VertexPropertyJacksonSerializer(normalize, true));
+            addSerializer(Property.class, new GraphSONSerializersV3d0.PropertyJacksonSerializer());
+            addSerializer(Metrics.class, new GraphSONSerializersV3d0.MetricsJacksonSerializer());
+            addSerializer(TraversalMetrics.class, new GraphSONSerializersV3d0.TraversalMetricsJacksonSerializer());
+            addSerializer(TraversalExplanation.class, new GraphSONSerializersV3d0.TraversalExplanationJacksonSerializer());
+            addSerializer(Path.class, new GraphSONSerializersV3d0.PathJacksonSerializer());
+            addSerializer(DirectionalStarGraph.class, new StarGraphGraphSONSerializerV2d0(normalize));
+            addSerializer(Tree.class, new GraphSONSerializersV3d0.TreeJacksonSerializer());
+
+            // java.util
+            addSerializer(Map.Entry.class, new JavaUtilSerializersV2d0.MapEntryJacksonSerializer());
+
+            // need to explicitly add serializers for those types because Jackson doesn't do it at all.
+            addSerializer(Integer.class, new GraphSONSerializersV3d0.IntegerGraphSONSerializer());
+            addSerializer(Double.class, new GraphSONSerializersV3d0.DoubleGraphSONSerializer());
+
+            // traversal
+            addSerializer(Traversal.class, new GraphSONTraversalSerializersV2d0.TraversalJacksonSerializer());
+            addSerializer(Bytecode.class, new GraphSONTraversalSerializersV2d0.BytecodeJacksonSerializer());
+            Stream.of(VertexProperty.Cardinality.class,
+                    Column.class,
+                    Direction.class,
+                    Operator.class,
+                    Order.class,
+                    Pop.class,
+                    SackFunctions.Barrier.class,
+                    Scope.class,
+                    TraversalOptionParent.Pick.class,
+                    T.class).forEach(e -> addSerializer(e, new GraphSONTraversalSerializersV2d0.EnumJacksonSerializer()));
+            addSerializer(P.class, new GraphSONTraversalSerializersV2d0.PJacksonSerializer());
+            addSerializer(Lambda.class, new GraphSONTraversalSerializersV2d0.LambdaJacksonSerializer());
+            addSerializer(Bytecode.Binding.class, new GraphSONTraversalSerializersV2d0.BindingJacksonSerializer());
+            addSerializer(Traverser.class, new GraphSONTraversalSerializersV2d0.TraverserJacksonSerializer());
+            addSerializer(TraversalStrategy.class, new GraphSONTraversalSerializersV2d0.TraversalStrategyJacksonSerializer());
+
+            /////////////////////// DESERIALIZERS ////////////////////////////
+
+            // Tinkerpop Graph
+            addDeserializer(Vertex.class, new GraphSONSerializersV3d0.VertexJacksonDeserializer());
+            addDeserializer(Edge.class, new GraphSONSerializersV3d0.EdgeJacksonDeserializer());
+            addDeserializer(Property.class, new GraphSONSerializersV3d0.PropertyJacksonDeserializer());
+            addDeserializer(Path.class, new GraphSONSerializersV3d0.PathJacksonDeserializer());
+            addDeserializer(VertexProperty.class, new GraphSONSerializersV3d0.VertexPropertyJacksonDeserializer());
+            addDeserializer(Metrics.class, new GraphSONSerializersV3d0.MetricsJacksonDeserializer());
+            addDeserializer(TraversalMetrics.class, new GraphSONSerializersV3d0.TraversalMetricsJacksonDeserializer());
+            addDeserializer(Tree.class, new GraphSONSerializersV3d0.TreeJacksonDeserializer());
+
+            // numbers
+            addDeserializer(Integer.class, new GraphSONSerializersV2d0.IntegerJackonsDeserializer());
+            addDeserializer(Double.class, new GraphSONSerializersV2d0.DoubleJackonsDeserializer());
+
+            // traversal
+            addDeserializer(Bytecode.class, new GraphSONTraversalSerializersV2d0.BytecodeJacksonDeserializer());
+            addDeserializer(Bytecode.Binding.class, new GraphSONTraversalSerializersV2d0.BindingJacksonDeserializer());
+            Stream.of(VertexProperty.Cardinality.values(),
+                    Column.values(),
+                    Direction.values(),
+                    Operator.values(),
+                    Order.values(),
+                    Pop.values(),
+                    SackFunctions.Barrier.values(),
+                    Scope.values(),
+                    TraversalOptionParent.Pick.values(),
+                    T.values()).flatMap(Stream::of).forEach(e -> addDeserializer(e.getClass(), new GraphSONTraversalSerializersV2d0.EnumJacksonDeserializer(e.getDeclaringClass())));
+            addDeserializer(P.class, new GraphSONTraversalSerializersV2d0.PJacksonDeserializer());
+            addDeserializer(Lambda.class, new GraphSONTraversalSerializersV2d0.LambdaJacksonDeserializer());
+            addDeserializer(Traverser.class, new GraphSONTraversalSerializersV2d0.TraverserJacksonDeserializer());
+            Arrays.asList(
+                    ConnectiveStrategy.class,
+                    ElementIdStrategy.class,
+                    EventStrategy.class,
+                    HaltedTraverserStrategy.class,
+                    PartitionStrategy.class,
+                    SubgraphStrategy.class,
+                    LazyBarrierStrategy.class,
+                    MatchAlgorithmStrategy.class,
+                    AdjacentToIncidentStrategy.class,
+                    FilterRankingStrategy.class,
+                    IdentityRemovalStrategy.class,
+                    IncidentToAdjacentStrategy.class,
+                    InlineFilterStrategy.class,
+                    MatchPredicateStrategy.class,
+                    OrderLimitStrategy.class,
+                    PathProcessorStrategy.class,
+                    PathRetractionStrategy.class,
+                    RangeByIsCountStrategy.class,
+                    RepeatUnrollStrategy.class,
+                    ComputerVerificationStrategy.class,
+                    LambdaRestrictionStrategy.class,
+                    ReadOnlyStrategy.class,
+                    StandardVerificationStrategy.class,
+                    //
+                    GraphFilterStrategy.class,
+                    VertexProgramStrategy.class
+            ).forEach(strategy -> addDeserializer(strategy, new GraphSONTraversalSerializersV2d0.TraversalStrategyProxyJacksonDeserializer(strategy)));
+        }
+
+        public static Builder build() {
+            return new Builder();
+        }
+
+        @Override
+        public Map<Class, String> getTypeDefinitions() {
+            return TYPE_DEFINITIONS;
+        }
+
+        @Override
+        public String getTypeNamespace() {
+            return GraphSONTokens.GREMLIN_TYPE_NAMESPACE;
+        }
+
+        static final class Builder implements GraphSONModuleBuilder {
+
+            private Builder() {
+            }
+
+            @Override
+            public GraphSONModule create(final boolean normalize) {
+                return new GraphSONModuleV3d0(normalize);
+            }
+
+        }
+    }
+
+    /**
      * Version 2.0 of GraphSON.
      */
     static final class GraphSONModuleV2d0 extends GraphSONModule {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV2d0.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV2d0.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV2d0.java
index 824fc7f..f8f4cd1 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV2d0.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV2d0.java
@@ -54,7 +54,6 @@ import org.javatuples.Pair;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -64,6 +63,8 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONUtil.safeWriteObjectField;
+
 /**
  * GraphSON serializers for graph-based objects such as vertices, edges, properties, and paths. These serializers
  * present a generalized way to serialize the implementations of core interfaces.
@@ -100,34 +101,22 @@ class GraphSONSerializersV2d0 {
         }
 
         private void writeProperties(final Vertex vertex, final JsonGenerator jsonGenerator) throws IOException {
-            if (vertex.keys().isEmpty())
+            if (vertex.keys().size() == 0)
                 return;
             jsonGenerator.writeFieldName(GraphSONTokens.PROPERTIES);
             jsonGenerator.writeStartObject();
 
             final List<String> keys = normalize ?
                     IteratorUtils.list(vertex.keys().iterator(), Comparator.naturalOrder()) : new ArrayList<>(vertex.keys());
-            for (final String key : keys) {
+            for (String key : keys) {
                 final Iterator<VertexProperty<Object>> vertexProperties = normalize ?
                         IteratorUtils.list(vertex.properties(key), Comparators.PROPERTY_COMPARATOR).iterator() : vertex.properties(key);
                 if (vertexProperties.hasNext()) {
                     jsonGenerator.writeFieldName(key);
+
                     jsonGenerator.writeStartArray();
                     while (vertexProperties.hasNext()) {
-                        final VertexProperty<?> vertexProperty = vertexProperties.next();
-                        jsonGenerator.writeStartObject();
-                        jsonGenerator.writeObjectField(GraphSONTokens.ID, vertexProperty.id());
-                        jsonGenerator.writeObjectField(GraphSONTokens.VALUE, vertexProperty.value());
-                        if (!vertexProperty.keys().isEmpty()) {
-                            jsonGenerator.writeObjectFieldStart(GraphSONTokens.PROPERTIES);
-                            final Iterator<Property<?>> properties = (Iterator) vertexProperty.properties();
-                            while (properties.hasNext()) {
-                                final Property<?> property = properties.next();
-                                jsonGenerator.writeObjectField(property.key(), property.value());
-                            }
-                            jsonGenerator.writeEndObject();
-                        }
-                        jsonGenerator.writeEndObject();
+                        jsonGenerator.writeObject(vertexProperties.next());
                     }
                     jsonGenerator.writeEndArray();
                 }
@@ -164,15 +153,13 @@ class GraphSONSerializersV2d0 {
         }
 
         private void writeProperties(final Edge edge, final JsonGenerator jsonGenerator) throws IOException {
-            final Iterator<Property<Object>> edgeProperties = normalize ?
+            final Iterator<Property<Object>> elementProperties = normalize ?
                     IteratorUtils.list(edge.properties(), Comparators.PROPERTY_COMPARATOR).iterator() : edge.properties();
-            if (edgeProperties.hasNext()) {
+            if (elementProperties.hasNext()) {
                 jsonGenerator.writeFieldName(GraphSONTokens.PROPERTIES);
+
                 jsonGenerator.writeStartObject();
-                while (edgeProperties.hasNext()) {
-                    final Property<?> property = edgeProperties.next();
-                    jsonGenerator.writeObjectField(property.key(), property.value());
-                }
+                elementProperties.forEachRemaining(prop -> safeWriteObjectField(jsonGenerator, prop.key(), prop));
                 jsonGenerator.writeEndObject();
             }
         }
@@ -188,25 +175,8 @@ class GraphSONSerializersV2d0 {
         public void serialize(final Property property, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
                 throws IOException {
             jsonGenerator.writeStartObject();
-            jsonGenerator.writeStringField(GraphSONTokens.KEY, property.key());
+            jsonGenerator.writeObjectField(GraphSONTokens.KEY, property.key());
             jsonGenerator.writeObjectField(GraphSONTokens.VALUE, property.value());
-            if (property.element() instanceof VertexProperty) {
-                VertexProperty vertexProperty = (VertexProperty) property.element();
-                jsonGenerator.writeObjectFieldStart(GraphSONTokens.VERTEX_PROPERTY);
-                jsonGenerator.writeObjectField(GraphSONTokens.ID, vertexProperty.id());
-                jsonGenerator.writeStringField(GraphSONTokens.LABEL, vertexProperty.label());
-                jsonGenerator.writeObjectField(GraphSONTokens.VALUE, vertexProperty.value());
-                jsonGenerator.writeObjectField(GraphSONTokens.VERTEX, vertexProperty.element().id());
-                jsonGenerator.writeEndObject();
-            } else if (property.element() instanceof Edge) {
-                Edge edge = (Edge) property.element();
-                jsonGenerator.writeObjectFieldStart(GraphSONTokens.EDGE);
-                jsonGenerator.writeObjectField(GraphSONTokens.ID, edge.id());
-                jsonGenerator.writeStringField(GraphSONTokens.LABEL, edge.label());
-                jsonGenerator.writeObjectField(GraphSONTokens.IN, edge.inVertex().id());
-                jsonGenerator.writeObjectField(GraphSONTokens.OUT, edge.outVertex().id());
-                jsonGenerator.writeEndObject();
-            }
             jsonGenerator.writeEndObject();
         }
     }
@@ -223,46 +193,44 @@ class GraphSONSerializersV2d0 {
         }
 
         @Override
-        public void serialize(final VertexProperty vertexProperty, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+        public void serialize(final VertexProperty property, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
                 throws IOException {
             jsonGenerator.writeStartObject();
 
-            jsonGenerator.writeObjectField(GraphSONTokens.ID, vertexProperty.id());
-            jsonGenerator.writeObjectField(GraphSONTokens.VALUE, vertexProperty.value());
-            if (null != vertexProperty.element())
-                jsonGenerator.writeObjectField(GraphSONTokens.VERTEX, vertexProperty.element().id());
-            if (this.includeLabel)
-                jsonGenerator.writeStringField(GraphSONTokens.LABEL, vertexProperty.label());
-            tryWriteMetaProperties(vertexProperty, jsonGenerator, normalize);
+            jsonGenerator.writeObjectField(GraphSONTokens.ID, property.id());
+            jsonGenerator.writeObjectField(GraphSONTokens.VALUE, property.value());
+            if (includeLabel)
+                jsonGenerator.writeStringField(GraphSONTokens.LABEL, property.label());
+            tryWriteMetaProperties(property, jsonGenerator, normalize);
 
             jsonGenerator.writeEndObject();
         }
 
-        private static void tryWriteMetaProperties(final VertexProperty vertexProperty, final JsonGenerator jsonGenerator,
+        private static void tryWriteMetaProperties(final VertexProperty property, final JsonGenerator jsonGenerator,
                                                    final boolean normalize) throws IOException {
             // when "detached" you can't check features of the graph it detached from so it has to be
             // treated differently from a regular VertexProperty implementation.
-            if (vertexProperty instanceof DetachedVertexProperty) {
+            if (property instanceof DetachedVertexProperty) {
                 // only write meta properties key if they exist
-                if (vertexProperty.properties().hasNext()) {
-                    writeMetaProperties(vertexProperty, jsonGenerator, normalize);
+                if (property.properties().hasNext()) {
+                    writeMetaProperties(property, jsonGenerator, normalize);
                 }
             } else {
                 // still attached - so we can check the features to see if it's worth even trying to write the
                 // meta properties key
-                if (vertexProperty.graph().features().vertex().supportsMetaProperties() && vertexProperty.properties().hasNext()) {
-                    writeMetaProperties(vertexProperty, jsonGenerator, normalize);
+                if (property.graph().features().vertex().supportsMetaProperties() && property.properties().hasNext()) {
+                    writeMetaProperties(property, jsonGenerator, normalize);
                 }
             }
         }
 
-        private static void writeMetaProperties(final VertexProperty vertexProperty, final JsonGenerator jsonGenerator,
+        private static void writeMetaProperties(final VertexProperty property, final JsonGenerator jsonGenerator,
                                                 final boolean normalize) throws IOException {
             jsonGenerator.writeFieldName(GraphSONTokens.PROPERTIES);
             jsonGenerator.writeStartObject();
 
             final Iterator<Property<Object>> metaProperties = normalize ?
-                    IteratorUtils.list((Iterator<Property<Object>>) vertexProperty.properties(), Comparators.PROPERTY_COMPARATOR).iterator() : vertexProperty.properties();
+                    IteratorUtils.list((Iterator<Property<Object>>) property.properties(), Comparators.PROPERTY_COMPARATOR).iterator() : property.properties();
             while (metaProperties.hasNext()) {
                 final Property<Object> metaProperty = metaProperties.next();
                 jsonGenerator.writeObjectField(metaProperty.key(), metaProperty.value());
@@ -358,7 +326,7 @@ class GraphSONSerializersV2d0 {
         @Override
         public void serialize(final Integer integer, final JsonGenerator jsonGenerator,
                               final SerializerProvider serializerProvider) throws IOException {
-            jsonGenerator.writeNumber(integer.intValue());
+            jsonGenerator.writeNumber(((Integer) integer).intValue());
         }
     }
 
@@ -459,7 +427,7 @@ class GraphSONSerializersV2d0 {
         public Vertex createObject(final Map<String, Object> vertexData) {
             return new DetachedVertex(
                     vertexData.get(GraphSONTokens.ID),
-                    (String) vertexData.getOrDefault(GraphSONTokens.LABEL, Vertex.DEFAULT_LABEL),
+                    vertexData.get(GraphSONTokens.LABEL).toString(),
                     (Map<String, Object>) vertexData.get(GraphSONTokens.PROPERTIES)
             );
         }
@@ -475,10 +443,10 @@ class GraphSONSerializersV2d0 {
         public Edge createObject(final Map<String, Object> edgeData) {
             return new DetachedEdge(
                     edgeData.get(GraphSONTokens.ID),
-                    (String) edgeData.getOrDefault(GraphSONTokens.LABEL, Edge.DEFAULT_LABEL),
-                    (Map<String, Object>) edgeData.get(GraphSONTokens.PROPERTIES),
-                    Pair.with(edgeData.get(GraphSONTokens.OUT), (String) edgeData.getOrDefault(GraphSONTokens.OUT_LABEL, Vertex.DEFAULT_LABEL)),
-                    Pair.with(edgeData.get(GraphSONTokens.IN), (String) edgeData.getOrDefault(GraphSONTokens.IN_LABEL, Vertex.DEFAULT_LABEL))
+                    edgeData.get(GraphSONTokens.LABEL).toString(),
+                    (Map) edgeData.get(GraphSONTokens.PROPERTIES),
+                    Pair.with(edgeData.get(GraphSONTokens.OUT), edgeData.get(GraphSONTokens.OUT_LABEL).toString()),
+                    Pair.with(edgeData.get(GraphSONTokens.IN), edgeData.get(GraphSONTokens.IN_LABEL).toString())
             );
         }
     }
@@ -491,41 +459,9 @@ class GraphSONSerializersV2d0 {
 
         @Override
         public Property createObject(final Map<String, Object> propData) {
-            Element element = null;
-            if (propData.containsKey(GraphSONTokens.VERTEX_PROPERTY)) {
-                final Map<String, Object> elementData = (Map<String, Object>) propData.get(GraphSONTokens.VERTEX_PROPERTY);
-                element = new VertexPropertyJacksonDeserializer().createObject(elementData);
-            } else if (propData.containsKey(GraphSONTokens.EDGE)) {
-                final Map<String, Object> elementData = (Map<String, Object>) propData.get(GraphSONTokens.EDGE);
-                element = new EdgeJacksonDeserializer().createObject(elementData);
-            }
-            return null != element ? // graphson-non-embedded is treated differently, but since this is a hard coded embedding...
-                    new DetachedProperty<>((String) propData.get(GraphSONTokens.KEY), propData.get(GraphSONTokens.VALUE), element) :
-                    new DetachedProperty<>((String) propData.get(GraphSONTokens.KEY), propData.get(GraphSONTokens.VALUE));
-        }
-    }
-
-    static class VertexPropertyJacksonDeserializer extends AbstractObjectDeserializer<VertexProperty> {
-
-        protected VertexPropertyJacksonDeserializer() {
-            super(VertexProperty.class);
-        }
-
-        @Override
-        public VertexProperty createObject(final Map<String, Object> propData) {
-            return propData.containsKey(GraphSONTokens.VERTEX) ?
-                    new DetachedVertexProperty<>(
-                            propData.get(GraphSONTokens.ID),
-                            (String) propData.get(GraphSONTokens.LABEL),
-                            propData.get(GraphSONTokens.VALUE),
-                            (Map<String, Object>) propData.get(GraphSONTokens.PROPERTIES),
-                            new DetachedVertex(propData.get(GraphSONTokens.VERTEX), Vertex.DEFAULT_LABEL, null)) :
-                    new DetachedVertexProperty<>(
-                            propData.get(GraphSONTokens.ID),
-                            (String) propData.get(GraphSONTokens.LABEL),
-                            propData.get(GraphSONTokens.VALUE),
-                            (Map<String, Object>) propData.get(GraphSONTokens.PROPERTIES));
-
+            return new DetachedProperty(
+                    (String) propData.get(GraphSONTokens.KEY),
+                    propData.get(GraphSONTokens.VALUE));
         }
     }
 
@@ -549,6 +485,23 @@ class GraphSONSerializersV2d0 {
         }
     }
 
+    static class VertexPropertyJacksonDeserializer extends AbstractObjectDeserializer<VertexProperty> {
+
+        protected VertexPropertyJacksonDeserializer() {
+            super(VertexProperty.class);
+        }
+
+        @Override
+        public VertexProperty createObject(final Map<String, Object> propData) {
+            return new DetachedVertexProperty(
+                    propData.get(GraphSONTokens.ID),
+                    (String) propData.get(GraphSONTokens.LABEL),
+                    propData.get(GraphSONTokens.VALUE),
+                    (Map) propData.get(GraphSONTokens.PROPERTIES)
+            );
+        }
+    }
+
     static class MetricsJacksonDeserializer extends AbstractObjectDeserializer<Metrics> {
         public MetricsJacksonDeserializer() {
             super(Metrics.class);
@@ -556,16 +509,16 @@ class GraphSONSerializersV2d0 {
 
         @Override
         public Metrics createObject(final Map<String, Object> metricsData) {
-            final MutableMetrics m = new MutableMetrics((String) metricsData.get(GraphSONTokens.ID), (String) metricsData.get(GraphSONTokens.NAME));
+            final MutableMetrics m = new MutableMetrics((String)metricsData.get(GraphSONTokens.ID), (String)metricsData.get(GraphSONTokens.NAME));
 
             m.setDuration(Math.round((Double) metricsData.get(GraphSONTokens.DURATION) * 1000000), TimeUnit.NANOSECONDS);
-            for (Map.Entry<String, Long> count : ((Map<String, Long>) metricsData.getOrDefault(GraphSONTokens.COUNTS, Collections.emptyMap())).entrySet()) {
+            for (Map.Entry<String, Long> count : ((Map<String, Long>)metricsData.getOrDefault(GraphSONTokens.COUNTS, new HashMap<>(0))).entrySet()) {
                 m.setCount(count.getKey(), count.getValue());
             }
-            for (Map.Entry<String, Long> count : ((Map<String, Long>) metricsData.getOrDefault(GraphSONTokens.ANNOTATIONS, Collections.emptyMap())).entrySet()) {
+            for (Map.Entry<String, Long> count : ((Map<String, Long>) metricsData.getOrDefault(GraphSONTokens.ANNOTATIONS, new HashMap<>(0))).entrySet()) {
                 m.setAnnotation(count.getKey(), count.getValue());
             }
-            for (MutableMetrics nested : (List<MutableMetrics>) metricsData.getOrDefault(GraphSONTokens.METRICS, Collections.emptyList())) {
+            for (MutableMetrics nested : (List<MutableMetrics>)metricsData.getOrDefault(GraphSONTokens.METRICS, new ArrayList<>(0))) {
                 m.addNested(nested);
             }
             return m;
@@ -627,7 +580,4 @@ class GraphSONSerializersV2d0 {
             return jsonParser.getDoubleValue();
         }
     }
-}
-
-
-
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV3d0.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV3d0.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV3d0.java
new file mode 100644
index 0000000..165e7e1
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV3d0.java
@@ -0,0 +1,632 @@
+/*
+ * 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.graphson;
+
+import org.apache.tinkerpop.gremlin.process.traversal.Path;
+import org.apache.tinkerpop.gremlin.process.traversal.Traversal;
+import org.apache.tinkerpop.gremlin.process.traversal.TraversalStrategy;
+import org.apache.tinkerpop.gremlin.process.traversal.step.util.MutablePath;
+import org.apache.tinkerpop.gremlin.process.traversal.step.util.Tree;
+import org.apache.tinkerpop.gremlin.process.traversal.util.DefaultTraversalMetrics;
+import org.apache.tinkerpop.gremlin.process.traversal.util.Metrics;
+import org.apache.tinkerpop.gremlin.process.traversal.util.MutableMetrics;
+import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalExplanation;
+import org.apache.tinkerpop.gremlin.process.traversal.util.TraversalMetrics;
+import org.apache.tinkerpop.gremlin.structure.Edge;
+import org.apache.tinkerpop.gremlin.structure.Element;
+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.util.Comparators;
+import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedEdge;
+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.util.iterator.IteratorUtils;
+import org.apache.tinkerpop.shaded.jackson.core.JsonGenerationException;
+import org.apache.tinkerpop.shaded.jackson.core.JsonGenerator;
+import org.apache.tinkerpop.shaded.jackson.core.JsonParser;
+import org.apache.tinkerpop.shaded.jackson.core.JsonProcessingException;
+import org.apache.tinkerpop.shaded.jackson.databind.DeserializationContext;
+import org.apache.tinkerpop.shaded.jackson.databind.SerializerProvider;
+import org.apache.tinkerpop.shaded.jackson.databind.deser.std.StdDeserializer;
+import org.apache.tinkerpop.shaded.jackson.databind.jsontype.TypeSerializer;
+import org.apache.tinkerpop.shaded.jackson.databind.ser.std.StdKeySerializer;
+import org.apache.tinkerpop.shaded.jackson.databind.ser.std.StdScalarSerializer;
+import org.apache.tinkerpop.shaded.jackson.databind.ser.std.StdSerializer;
+import org.javatuples.Pair;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * GraphSON serializers for graph-based objects such as vertices, edges, properties, and paths. These serializers
+ * present a generalized way to serialize the implementations of core interfaces.
+ *
+ * @author Stephen Mallette (http://stephen.genoprime.com)
+ */
+class GraphSONSerializersV3d0 {
+
+    private GraphSONSerializersV3d0() {
+    }
+
+    ////////////////////////////// SERIALIZERS /////////////////////////////////
+
+    final static class VertexJacksonSerializer extends StdScalarSerializer<Vertex> {
+
+        private final boolean normalize;
+
+        public VertexJacksonSerializer(final boolean normalize) {
+            super(Vertex.class);
+            this.normalize = normalize;
+        }
+
+        @Override
+        public void serialize(final Vertex vertex, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+                throws IOException {
+            jsonGenerator.writeStartObject();
+
+            jsonGenerator.writeObjectField(GraphSONTokens.ID, vertex.id());
+            jsonGenerator.writeStringField(GraphSONTokens.LABEL, vertex.label());
+            writeProperties(vertex, jsonGenerator);
+
+            jsonGenerator.writeEndObject();
+
+        }
+
+        private void writeProperties(final Vertex vertex, final JsonGenerator jsonGenerator) throws IOException {
+            if (vertex.keys().isEmpty())
+                return;
+            jsonGenerator.writeFieldName(GraphSONTokens.PROPERTIES);
+            jsonGenerator.writeStartObject();
+
+            final List<String> keys = normalize ?
+                    IteratorUtils.list(vertex.keys().iterator(), Comparator.naturalOrder()) : new ArrayList<>(vertex.keys());
+            for (final String key : keys) {
+                final Iterator<VertexProperty<Object>> vertexProperties = normalize ?
+                        IteratorUtils.list(vertex.properties(key), Comparators.PROPERTY_COMPARATOR).iterator() : vertex.properties(key);
+                if (vertexProperties.hasNext()) {
+                    jsonGenerator.writeFieldName(key);
+                    jsonGenerator.writeStartArray();
+                    while (vertexProperties.hasNext()) {
+                        final VertexProperty<?> vertexProperty = vertexProperties.next();
+                        jsonGenerator.writeStartObject();
+                        jsonGenerator.writeObjectField(GraphSONTokens.ID, vertexProperty.id());
+                        jsonGenerator.writeObjectField(GraphSONTokens.VALUE, vertexProperty.value());
+                        if (!vertexProperty.keys().isEmpty()) {
+                            jsonGenerator.writeObjectFieldStart(GraphSONTokens.PROPERTIES);
+                            final Iterator<Property<?>> properties = (Iterator) vertexProperty.properties();
+                            while (properties.hasNext()) {
+                                final Property<?> property = properties.next();
+                                jsonGenerator.writeObjectField(property.key(), property.value());
+                            }
+                            jsonGenerator.writeEndObject();
+                        }
+                        jsonGenerator.writeEndObject();
+                    }
+                    jsonGenerator.writeEndArray();
+                }
+            }
+
+            jsonGenerator.writeEndObject();
+        }
+    }
+
+    final static class EdgeJacksonSerializer extends StdScalarSerializer<Edge> {
+
+        private final boolean normalize;
+
+        public EdgeJacksonSerializer(final boolean normalize) {
+            super(Edge.class);
+            this.normalize = normalize;
+        }
+
+
+        @Override
+        public void serialize(final Edge edge, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+                throws IOException {
+            jsonGenerator.writeStartObject();
+
+            jsonGenerator.writeObjectField(GraphSONTokens.ID, edge.id());
+            jsonGenerator.writeStringField(GraphSONTokens.LABEL, edge.label());
+            jsonGenerator.writeStringField(GraphSONTokens.IN_LABEL, edge.inVertex().label());
+            jsonGenerator.writeStringField(GraphSONTokens.OUT_LABEL, edge.outVertex().label());
+            jsonGenerator.writeObjectField(GraphSONTokens.IN, edge.inVertex().id());
+            jsonGenerator.writeObjectField(GraphSONTokens.OUT, edge.outVertex().id());
+            writeProperties(edge, jsonGenerator);
+
+            jsonGenerator.writeEndObject();
+        }
+
+        private void writeProperties(final Edge edge, final JsonGenerator jsonGenerator) throws IOException {
+            final Iterator<Property<Object>> edgeProperties = normalize ?
+                    IteratorUtils.list(edge.properties(), Comparators.PROPERTY_COMPARATOR).iterator() : edge.properties();
+            if (edgeProperties.hasNext()) {
+                jsonGenerator.writeFieldName(GraphSONTokens.PROPERTIES);
+                jsonGenerator.writeStartObject();
+                while (edgeProperties.hasNext()) {
+                    final Property<?> property = edgeProperties.next();
+                    jsonGenerator.writeObjectField(property.key(), property.value());
+                }
+                jsonGenerator.writeEndObject();
+            }
+        }
+    }
+
+    final static class PropertyJacksonSerializer extends StdScalarSerializer<Property> {
+
+        public PropertyJacksonSerializer() {
+            super(Property.class);
+        }
+
+        @Override
+        public void serialize(final Property property, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+                throws IOException {
+            jsonGenerator.writeStartObject();
+            jsonGenerator.writeStringField(GraphSONTokens.KEY, property.key());
+            jsonGenerator.writeObjectField(GraphSONTokens.VALUE, property.value());
+            if (property.element() instanceof VertexProperty) {
+                VertexProperty vertexProperty = (VertexProperty) property.element();
+                jsonGenerator.writeObjectFieldStart(GraphSONTokens.VERTEX_PROPERTY);
+                jsonGenerator.writeObjectField(GraphSONTokens.ID, vertexProperty.id());
+                jsonGenerator.writeStringField(GraphSONTokens.LABEL, vertexProperty.label());
+                jsonGenerator.writeObjectField(GraphSONTokens.VALUE, vertexProperty.value());
+                jsonGenerator.writeObjectField(GraphSONTokens.VERTEX, vertexProperty.element().id());
+                jsonGenerator.writeEndObject();
+            } else if (property.element() instanceof Edge) {
+                Edge edge = (Edge) property.element();
+                jsonGenerator.writeObjectFieldStart(GraphSONTokens.EDGE);
+                jsonGenerator.writeObjectField(GraphSONTokens.ID, edge.id());
+                jsonGenerator.writeStringField(GraphSONTokens.LABEL, edge.label());
+                jsonGenerator.writeObjectField(GraphSONTokens.IN, edge.inVertex().id());
+                jsonGenerator.writeObjectField(GraphSONTokens.OUT, edge.outVertex().id());
+                jsonGenerator.writeEndObject();
+            }
+            jsonGenerator.writeEndObject();
+        }
+    }
+
+    final static class VertexPropertyJacksonSerializer extends StdScalarSerializer<VertexProperty> {
+
+        private final boolean normalize;
+        private final boolean includeLabel;
+
+        public VertexPropertyJacksonSerializer(final boolean normalize, final boolean includeLabel) {
+            super(VertexProperty.class);
+            this.normalize = normalize;
+            this.includeLabel = includeLabel;
+        }
+
+        @Override
+        public void serialize(final VertexProperty vertexProperty, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+                throws IOException {
+            jsonGenerator.writeStartObject();
+
+            jsonGenerator.writeObjectField(GraphSONTokens.ID, vertexProperty.id());
+            jsonGenerator.writeObjectField(GraphSONTokens.VALUE, vertexProperty.value());
+            if (null != vertexProperty.element())
+                jsonGenerator.writeObjectField(GraphSONTokens.VERTEX, vertexProperty.element().id());
+            if (this.includeLabel)
+                jsonGenerator.writeStringField(GraphSONTokens.LABEL, vertexProperty.label());
+            tryWriteMetaProperties(vertexProperty, jsonGenerator, normalize);
+
+            jsonGenerator.writeEndObject();
+        }
+
+        private static void tryWriteMetaProperties(final VertexProperty vertexProperty, final JsonGenerator jsonGenerator,
+                                                   final boolean normalize) throws IOException {
+            // when "detached" you can't check features of the graph it detached from so it has to be
+            // treated differently from a regular VertexProperty implementation.
+            if (vertexProperty instanceof DetachedVertexProperty) {
+                // only write meta properties key if they exist
+                if (vertexProperty.properties().hasNext()) {
+                    writeMetaProperties(vertexProperty, jsonGenerator, normalize);
+                }
+            } else {
+                // still attached - so we can check the features to see if it's worth even trying to write the
+                // meta properties key
+                if (vertexProperty.graph().features().vertex().supportsMetaProperties() && vertexProperty.properties().hasNext()) {
+                    writeMetaProperties(vertexProperty, jsonGenerator, normalize);
+                }
+            }
+        }
+
+        private static void writeMetaProperties(final VertexProperty vertexProperty, final JsonGenerator jsonGenerator,
+                                                final boolean normalize) throws IOException {
+            jsonGenerator.writeFieldName(GraphSONTokens.PROPERTIES);
+            jsonGenerator.writeStartObject();
+
+            final Iterator<Property<Object>> metaProperties = normalize ?
+                    IteratorUtils.list((Iterator<Property<Object>>) vertexProperty.properties(), Comparators.PROPERTY_COMPARATOR).iterator() : vertexProperty.properties();
+            while (metaProperties.hasNext()) {
+                final Property<Object> metaProperty = metaProperties.next();
+                jsonGenerator.writeObjectField(metaProperty.key(), metaProperty.value());
+            }
+
+            jsonGenerator.writeEndObject();
+        }
+
+
+    }
+
+    final static class PathJacksonSerializer extends StdScalarSerializer<Path> {
+
+        public PathJacksonSerializer() {
+            super(Path.class);
+        }
+
+        @Override
+        public void serialize(final Path path, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+                throws IOException, JsonGenerationException {
+            jsonGenerator.writeStartObject();
+
+            jsonGenerator.writeObjectField(GraphSONTokens.LABELS, path.labels());
+            jsonGenerator.writeObjectField(GraphSONTokens.OBJECTS, path.objects());
+
+            jsonGenerator.writeEndObject();
+        }
+    }
+
+    final static class TreeJacksonSerializer extends StdScalarSerializer<Tree> {
+
+        public TreeJacksonSerializer() {
+            super(Tree.class);
+        }
+
+        @Override
+        public void serialize(final Tree tree, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider) throws IOException, JsonGenerationException {
+            jsonGenerator.writeStartArray();
+            final Set<Map.Entry<Element, Tree>> set = tree.entrySet();
+            for (Map.Entry<Element, Tree> entry : set) {
+                jsonGenerator.writeStartObject();
+                jsonGenerator.writeObjectField(GraphSONTokens.KEY, entry.getKey());
+                jsonGenerator.writeObjectField(GraphSONTokens.VALUE, entry.getValue());
+                jsonGenerator.writeEndObject();
+            }
+            jsonGenerator.writeEndArray();
+        }
+    }
+
+    final static class TraversalExplanationJacksonSerializer extends StdSerializer<TraversalExplanation> {
+        public TraversalExplanationJacksonSerializer() {
+            super(TraversalExplanation.class);
+        }
+
+        @Override
+        public void serialize(final TraversalExplanation traversalExplanation, final JsonGenerator jsonGenerator,
+                              final SerializerProvider serializerProvider) throws IOException {
+            final Map<String, Object> m = new HashMap<>();
+            m.put(GraphSONTokens.ORIGINAL, getStepsAsList(traversalExplanation.getOriginalTraversal()));
+
+            final List<Pair<TraversalStrategy, Traversal.Admin<?, ?>>> strategyTraversals = traversalExplanation.getStrategyTraversals();
+
+            final List<Map<String, Object>> intermediates = new ArrayList<>();
+            for (final Pair<TraversalStrategy, Traversal.Admin<?, ?>> pair : strategyTraversals) {
+                final Map<String, Object> intermediate = new HashMap<>();
+                intermediate.put(GraphSONTokens.STRATEGY, pair.getValue0().toString());
+                intermediate.put(GraphSONTokens.CATEGORY, pair.getValue0().getTraversalCategory().getSimpleName());
+                intermediate.put(GraphSONTokens.TRAVERSAL, getStepsAsList(pair.getValue1()));
+                intermediates.add(intermediate);
+            }
+            m.put(GraphSONTokens.INTERMEDIATE, intermediates);
+
+            if (strategyTraversals.isEmpty())
+                m.put(GraphSONTokens.FINAL, getStepsAsList(traversalExplanation.getOriginalTraversal()));
+            else
+                m.put(GraphSONTokens.FINAL, getStepsAsList(strategyTraversals.get(strategyTraversals.size() - 1).getValue1()));
+
+            jsonGenerator.writeObject(m);
+        }
+
+        private List<String> getStepsAsList(final Traversal.Admin<?, ?> t) {
+            final List<String> steps = new ArrayList<>();
+            t.getSteps().iterator().forEachRemaining(s -> steps.add(s.toString()));
+            return steps;
+        }
+    }
+
+    final static class IntegerGraphSONSerializer extends StdScalarSerializer<Integer> {
+        public IntegerGraphSONSerializer() {
+            super(Integer.class);
+        }
+
+        @Override
+        public void serialize(final Integer integer, final JsonGenerator jsonGenerator,
+                              final SerializerProvider serializerProvider) throws IOException {
+            jsonGenerator.writeNumber(integer.intValue());
+        }
+    }
+
+    final static class DoubleGraphSONSerializer extends StdScalarSerializer<Double> {
+        public DoubleGraphSONSerializer() {
+            super(Double.class);
+        }
+
+        @Override
+        public void serialize(final Double doubleValue, final JsonGenerator jsonGenerator,
+                              final SerializerProvider serializerProvider) throws IOException {
+            jsonGenerator.writeNumber(doubleValue);
+        }
+    }
+
+    final static class TraversalMetricsJacksonSerializer extends StdScalarSerializer<TraversalMetrics> {
+        public TraversalMetricsJacksonSerializer() {
+            super(TraversalMetrics.class);
+        }
+
+        @Override
+        public void serialize(final TraversalMetrics traversalMetrics, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+                throws IOException {
+            // creation of the map enables all the fields to be properly written with their type if required
+            final Map<String, Object> m = new HashMap<>();
+            m.put(GraphSONTokens.DURATION, traversalMetrics.getDuration(TimeUnit.NANOSECONDS) / 1000000d);
+            final List<Metrics> metrics = new ArrayList<>();
+            metrics.addAll(traversalMetrics.getMetrics());
+            m.put(GraphSONTokens.METRICS, metrics);
+
+            jsonGenerator.writeObject(m);
+        }
+    }
+
+    final static class MetricsJacksonSerializer extends StdScalarSerializer<Metrics> {
+        public MetricsJacksonSerializer() {
+            super(Metrics.class);
+        }
+
+        @Override
+        public void serialize(final Metrics metrics, final JsonGenerator jsonGenerator,
+                              final SerializerProvider serializerProvider) throws IOException {
+            final Map<String, Object> m = new HashMap<>();
+            m.put(GraphSONTokens.ID, metrics.getId());
+            m.put(GraphSONTokens.NAME, metrics.getName());
+            m.put(GraphSONTokens.COUNTS, metrics.getCounts());
+            m.put(GraphSONTokens.DURATION, metrics.getDuration(TimeUnit.NANOSECONDS) / 1000000d);
+
+            if (!metrics.getAnnotations().isEmpty()) {
+                m.put(GraphSONTokens.ANNOTATIONS, metrics.getAnnotations());
+            }
+            if (!metrics.getNested().isEmpty()) {
+                final List<Metrics> nested = new ArrayList<>();
+                metrics.getNested().forEach(it -> nested.add(it));
+                m.put(GraphSONTokens.METRICS, nested);
+            }
+            jsonGenerator.writeObject(m);
+        }
+    }
+
+
+    /**
+     * Maps in the JVM can have {@link Object} as a key, but in JSON they must be a {@link String}.
+     */
+    final static class GraphSONKeySerializer extends StdKeySerializer {
+
+        @Override
+        public void serialize(final Object o, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider) throws IOException {
+            ser(o, jsonGenerator, serializerProvider);
+        }
+
+        @Override
+        public void serializeWithType(final Object o, final JsonGenerator jsonGenerator,
+                                      final SerializerProvider serializerProvider, final TypeSerializer typeSerializer) throws IOException {
+            ser(o, jsonGenerator, serializerProvider);
+        }
+
+        private void ser(final Object o, final JsonGenerator jsonGenerator,
+                         final SerializerProvider serializerProvider) throws IOException {
+            if (Element.class.isAssignableFrom(o.getClass()))
+                jsonGenerator.writeFieldName((((Element) o).id()).toString());
+            else
+                super.serialize(o, jsonGenerator, serializerProvider);
+        }
+    }
+
+
+    //////////////////////////// DESERIALIZERS ///////////////////////////
+
+
+    static class VertexJacksonDeserializer extends AbstractObjectDeserializer<Vertex> {
+
+        public VertexJacksonDeserializer() {
+            super(Vertex.class);
+        }
+
+        @Override
+        public Vertex createObject(final Map<String, Object> vertexData) {
+            return new DetachedVertex(
+                    vertexData.get(GraphSONTokens.ID),
+                    (String) vertexData.getOrDefault(GraphSONTokens.LABEL, Vertex.DEFAULT_LABEL),
+                    (Map<String, Object>) vertexData.get(GraphSONTokens.PROPERTIES)
+            );
+        }
+    }
+
+    static class EdgeJacksonDeserializer extends AbstractObjectDeserializer<Edge> {
+
+        public EdgeJacksonDeserializer() {
+            super(Edge.class);
+        }
+
+        @Override
+        public Edge createObject(final Map<String, Object> edgeData) {
+            return new DetachedEdge(
+                    edgeData.get(GraphSONTokens.ID),
+                    (String) edgeData.getOrDefault(GraphSONTokens.LABEL, Edge.DEFAULT_LABEL),
+                    (Map<String, Object>) edgeData.get(GraphSONTokens.PROPERTIES),
+                    Pair.with(edgeData.get(GraphSONTokens.OUT), (String) edgeData.getOrDefault(GraphSONTokens.OUT_LABEL, Vertex.DEFAULT_LABEL)),
+                    Pair.with(edgeData.get(GraphSONTokens.IN), (String) edgeData.getOrDefault(GraphSONTokens.IN_LABEL, Vertex.DEFAULT_LABEL))
+            );
+        }
+    }
+
+    static class PropertyJacksonDeserializer extends AbstractObjectDeserializer<Property> {
+
+        public PropertyJacksonDeserializer() {
+            super(Property.class);
+        }
+
+        @Override
+        public Property createObject(final Map<String, Object> propData) {
+            Element element = null;
+            if (propData.containsKey(GraphSONTokens.VERTEX_PROPERTY)) {
+                final Map<String, Object> elementData = (Map<String, Object>) propData.get(GraphSONTokens.VERTEX_PROPERTY);
+                element = new VertexPropertyJacksonDeserializer().createObject(elementData);
+            } else if (propData.containsKey(GraphSONTokens.EDGE)) {
+                final Map<String, Object> elementData = (Map<String, Object>) propData.get(GraphSONTokens.EDGE);
+                element = new EdgeJacksonDeserializer().createObject(elementData);
+            }
+            return null != element ? // graphson-non-embedded is treated differently, but since this is a hard coded embedding...
+                    new DetachedProperty<>((String) propData.get(GraphSONTokens.KEY), propData.get(GraphSONTokens.VALUE), element) :
+                    new DetachedProperty<>((String) propData.get(GraphSONTokens.KEY), propData.get(GraphSONTokens.VALUE));
+        }
+    }
+
+    static class VertexPropertyJacksonDeserializer extends AbstractObjectDeserializer<VertexProperty> {
+
+        protected VertexPropertyJacksonDeserializer() {
+            super(VertexProperty.class);
+        }
+
+        @Override
+        public VertexProperty createObject(final Map<String, Object> propData) {
+            return propData.containsKey(GraphSONTokens.VERTEX) ?
+                    new DetachedVertexProperty<>(
+                            propData.get(GraphSONTokens.ID),
+                            (String) propData.get(GraphSONTokens.LABEL),
+                            propData.get(GraphSONTokens.VALUE),
+                            (Map<String, Object>) propData.get(GraphSONTokens.PROPERTIES),
+                            new DetachedVertex(propData.get(GraphSONTokens.VERTEX), Vertex.DEFAULT_LABEL, null)) :
+                    new DetachedVertexProperty<>(
+                            propData.get(GraphSONTokens.ID),
+                            (String) propData.get(GraphSONTokens.LABEL),
+                            propData.get(GraphSONTokens.VALUE),
+                            (Map<String, Object>) propData.get(GraphSONTokens.PROPERTIES));
+
+        }
+    }
+
+    static class PathJacksonDeserializer extends AbstractObjectDeserializer<Path> {
+
+        public PathJacksonDeserializer() {
+            super(Path.class);
+        }
+
+        @Override
+        public Path createObject(final Map<String, Object> pathData) {
+            final Path p = MutablePath.make();
+
+            final List labels = (List) pathData.get(GraphSONTokens.LABELS);
+            final List objects = (List) pathData.get(GraphSONTokens.OBJECTS);
+
+            for (int i = 0; i < objects.size(); i++) {
+                p.extend(objects.get(i), new HashSet((List) labels.get(i)));
+            }
+            return p;
+        }
+    }
+
+    static class MetricsJacksonDeserializer extends AbstractObjectDeserializer<Metrics> {
+        public MetricsJacksonDeserializer() {
+            super(Metrics.class);
+        }
+
+        @Override
+        public Metrics createObject(final Map<String, Object> metricsData) {
+            final MutableMetrics m = new MutableMetrics((String) metricsData.get(GraphSONTokens.ID), (String) metricsData.get(GraphSONTokens.NAME));
+
+            m.setDuration(Math.round((Double) metricsData.get(GraphSONTokens.DURATION) * 1000000), TimeUnit.NANOSECONDS);
+            for (Map.Entry<String, Long> count : ((Map<String, Long>) metricsData.getOrDefault(GraphSONTokens.COUNTS, Collections.emptyMap())).entrySet()) {
+                m.setCount(count.getKey(), count.getValue());
+            }
+            for (Map.Entry<String, Long> count : ((Map<String, Long>) metricsData.getOrDefault(GraphSONTokens.ANNOTATIONS, Collections.emptyMap())).entrySet()) {
+                m.setAnnotation(count.getKey(), count.getValue());
+            }
+            for (MutableMetrics nested : (List<MutableMetrics>) metricsData.getOrDefault(GraphSONTokens.METRICS, Collections.emptyList())) {
+                m.addNested(nested);
+            }
+            return m;
+        }
+    }
+
+    static class TraversalMetricsJacksonDeserializer extends AbstractObjectDeserializer<TraversalMetrics> {
+
+        public TraversalMetricsJacksonDeserializer() {
+            super(TraversalMetrics.class);
+        }
+
+        @Override
+        public TraversalMetrics createObject(final Map<String, Object> traversalMetricsData) {
+            return new DefaultTraversalMetrics(
+                    Math.round((Double) traversalMetricsData.get(GraphSONTokens.DURATION) * 1000000),
+                    (List<MutableMetrics>) traversalMetricsData.get(GraphSONTokens.METRICS)
+            );
+        }
+    }
+
+    static class TreeJacksonDeserializer extends StdDeserializer<Tree> {
+
+        public TreeJacksonDeserializer() {
+            super(Tree.class);
+        }
+
+        @Override
+        public Tree deserialize(final JsonParser jsonParser, final DeserializationContext deserializationContext) throws IOException, JsonProcessingException {
+            final List<Map> data = deserializationContext.readValue(jsonParser, List.class);
+            final Tree t = new Tree();
+            for (Map<String, Object> entry : data) {
+                t.put(entry.get(GraphSONTokens.KEY), entry.get(GraphSONTokens.VALUE));
+            }
+            return t;
+        }
+    }
+
+    static class IntegerJackonsDeserializer extends StdDeserializer<Integer> {
+
+        protected IntegerJackonsDeserializer() {
+            super(Integer.class);
+        }
+
+        @Override
+        public Integer deserialize(JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException, JsonProcessingException {
+            return jsonParser.getIntValue();
+        }
+    }
+
+    static class DoubleJackonsDeserializer extends StdDeserializer<Double> {
+
+        protected DoubleJackonsDeserializer() {
+            super(Double.class);
+        }
+
+        @Override
+        public Double deserialize(JsonParser jsonParser, DeserializationContext deserializationContext) throws IOException, JsonProcessingException {
+            return jsonParser.getDoubleValue();
+        }
+    }
+}
+
+

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTokens.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTokens.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTokens.java
index d804f0b..10e36e0 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTokens.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTokens.java
@@ -36,6 +36,7 @@ public final class GraphSONTokens {
     public static final String KEY = "key";
     public static final String EDGE = "edge";
     public static final String EDGES = "edges";
+    public static final String ELEMENT = "ELEMENT";
     public static final String VERTEX = "vertex";
     public static final String VERTEX_PROPERTY = "vertexProperty";
     public static final String VERTICES = "vertices";

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONUtil.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONUtil.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONUtil.java
index 97292af..710c2bc 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONUtil.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONUtil.java
@@ -86,4 +86,11 @@ public final class GraphSONUtil {
             jsonGenerator.writeEndArray();
     }
 
+    static void safeWriteObjectField(final JsonGenerator jsonGenerator, final String key, final Object value) {
+        try {
+            jsonGenerator.writeObjectField(key, value);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONVersion.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONVersion.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONVersion.java
index 7740b06..cb163e7 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONVersion.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONVersion.java
@@ -25,7 +25,8 @@ package org.apache.tinkerpop.gremlin.structure.io.graphson;
  */
 public enum GraphSONVersion {
     V1_0(GraphSONModule.GraphSONModuleV1d0.build(), "1.0"),
-    V2_0(GraphSONModule.GraphSONModuleV2d0.build(), "2.0");
+    V2_0(GraphSONModule.GraphSONModuleV2d0.build(), "2.0"),
+    V3_0(GraphSONModule.GraphSONModuleV3d0.build(), "3.0");
 
     private final GraphSONModule.GraphSONModuleBuilder builder;
     private final String versionNumber;

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7db10c40/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0.java
----------------------------------------------------------------------
diff --git a/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0.java b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0.java
new file mode 100644
index 0000000..f6f2fc8
--- /dev/null
+++ b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0.java
@@ -0,0 +1,124 @@
+/*
+ * 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.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.io.graphson.GraphSONMapper;
+import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONVersion;
+import org.apache.tinkerpop.gremlin.structure.io.graphson.TypeInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ * Serialize results to JSON with version 3.0.x schema and the extended module.
+ *
+ * @author Stephen Mallette (http://stephen.genoprime.com)
+ */
+public final class GraphSONMessageSerializerV3d0 extends AbstractGraphSONMessageSerializerV2d0 implements MessageTextSerializer {
+    private static final Logger logger = LoggerFactory.getLogger(GraphSONMessageSerializerV3d0.class);
+    private static final String MIME_TYPE = SerTokens.MIME_JSON;
+
+    private static byte[] header;
+
+    static {
+        final ByteBuffer buffer = ByteBuffer.allocate(MIME_TYPE.length() + 1);
+        buffer.put((byte) MIME_TYPE.length());
+        buffer.put(MIME_TYPE.getBytes());
+        header = buffer.array();
+    }
+
+    public GraphSONMessageSerializerV3d0() {
+        super();
+    }
+
+    public GraphSONMessageSerializerV3d0(final GraphSONMapper mapper) {
+        super(mapper);
+    }
+
+    @Override
+    public String[] mimeTypesSupported() {
+        return new String[]{MIME_TYPE, "application/json"};
+    }
+
+    @Override
+    GraphSONMapper.Builder configureBuilder(final GraphSONMapper.Builder builder) {
+        // override the 2.0 in AbstractGraphSONMessageSerializerV2d0
+        return builder.typeInfo(TypeInfo.PARTIAL_TYPES).version(GraphSONVersion.V3_0);
+    }
+
+    @Override
+    byte[] obtainHeader() {
+        return header;
+    }
+
+    @Override
+    public ResponseMessage deserializeResponse(final String msg) throws SerializationException {
+        try {
+            final Map<String, Object> responseData = mapper.readValue(msg, mapTypeReference);
+            final Map<String, Object> status = (Map<String, Object>) responseData.get(SerTokens.TOKEN_STATUS);
+            final Map<String, Object> result = (Map<String, Object>) responseData.get(SerTokens.TOKEN_RESULT);
+            return ResponseMessage.build(UUID.fromString(responseData.get(SerTokens.TOKEN_REQUEST).toString()))
+                    .code(ResponseStatusCode.getFromValue((Integer) status.get(SerTokens.TOKEN_CODE)))
+                    .statusMessage(status.get(SerTokens.TOKEN_MESSAGE).toString())
+                    .statusAttributes((Map<String, Object>) status.get(SerTokens.TOKEN_ATTRIBUTES))
+                    .result(result.get(SerTokens.TOKEN_DATA))
+                    .responseMetaData((Map<String, Object>) result.get(SerTokens.TOKEN_META))
+                    .create();
+        } catch (Exception ex) {
+            logger.warn("Response [{}] could not be deserialized by {}.", msg, AbstractGraphSONMessageSerializerV2d0.class.getName());
+            throw new SerializationException(ex);
+        }
+    }
+
+    @Override
+    public String serializeResponseAsString(final ResponseMessage responseMessage) throws SerializationException {
+        try {
+            return mapper.writeValueAsString(responseMessage);
+        } catch (Exception ex) {
+            logger.warn("Response [{}] could not be serialized by {}.", responseMessage.toString(), AbstractGraphSONMessageSerializerV2d0.class.getName());
+            throw new SerializationException(ex);
+        }
+    }
+
+    @Override
+    public RequestMessage deserializeRequest(final String msg) throws SerializationException {
+        try {
+            return mapper.readValue(msg, RequestMessage.class);
+        } catch (Exception ex) {
+            logger.warn("Request [{}] could not be deserialized by {}.", msg, AbstractGraphSONMessageSerializerV2d0.class.getName());
+            throw new SerializationException(ex);
+        }
+    }
+
+    @Override
+    public String serializeRequestAsString(final RequestMessage requestMessage) throws SerializationException {
+        try {
+            return mapper.writeValueAsString(requestMessage);
+        } catch (Exception ex) {
+            logger.warn("Request [{}] could not be serialized by {}.", requestMessage.toString(), AbstractGraphSONMessageSerializerV2d0.class.getName());
+            throw new SerializationException(ex);
+        }
+    }
+}


[09/31] tinkerpop git commit: @dkuppitz found an optimization trick for MultiComparator. The startIndex for comparing should start after the last Order.shuffle.

Posted by sp...@apache.org.
@dkuppitz found an optimization trick for MultiComparator. The startIndex for comparing should start after the last Order.shuffle.


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

Branch: refs/heads/TINKERPOP-1565
Commit: 341ebf9811c15d65231ea31c3617723c2fd0ab09
Parents: 379a6e5
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Thu Jan 19 08:23:42 2017 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Thu Jan 19 08:23:42 2017 -0700

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |  2 +-
 .../gremlin/structure/io/gryo/GryoVersion.java  |  4 +-
 .../gremlin/util/function/MultiComparator.java  | 15 +++--
 .../util/function/MultiComparatorTest.java      | 69 ++++++++++++++++++++
 4 files changed, 81 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/341ebf98/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index 88cbf32..5f28790 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -26,7 +26,7 @@ image::https://raw.githubusercontent.com/apache/tinkerpop/master/docs/static/ima
 TinkerPop 3.2.4 (Release Date: NOT OFFICIALLY RELEASED YET)
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
-* `GroupBiOperator` no longer maintains state and thus, no more side-effect related OLAP inconsistencies.
+* `GroupBiOperator` no longer maintains a detached traversal and thus, no more side-effect related OLAP inconsistencies.
 * Added `ProjectedTraverser` which wraps a traverser with a `List<Object>` of projected data.
 * Fixed an optimization bug in `CollectionBarrierSteps` where the barrier was being consumed on each `addBarrier()`.
 * `OrderGlobalStep` and `SampleGlobalStep` use `ProjectedTraverser` and now can work up to the local star graph in OLAP.

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/341ebf98/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
index f4e31fd..7818f6b 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
@@ -292,13 +292,13 @@ public enum GryoVersion {
             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(GroupStep.GroupBiOperator.class, 117, new JavaSerializer()));
             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(OrderGlobalStep.OrderBiOperator.class, 118, new JavaSerializer()));
             add(GryoTypeReg.of(ProfileStep.ProfileBiOperator.class, 119));
         }};
     }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/341ebf98/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java
index 5d24ddf..d97d147 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/util/function/MultiComparator.java
@@ -33,6 +33,7 @@ public final class MultiComparator<C> implements Comparator<C>, Serializable {
 
     private List<Comparator> comparators;
     private boolean isShuffle;
+    int startIndex = 0;
 
     private MultiComparator() {
         // for serialization purposes
@@ -41,6 +42,10 @@ public final class MultiComparator<C> implements Comparator<C>, Serializable {
     public MultiComparator(final List<Comparator<C>> comparators) {
         this.comparators = (List) comparators;
         this.isShuffle = !this.comparators.isEmpty() && Order.shuffle == this.comparators.get(this.comparators.size() - 1);
+        for (int i = 0; i < this.comparators.size(); i++) {
+            if (this.comparators.get(i) == Order.shuffle)
+                this.startIndex = i + 1;
+        }
     }
 
     @Override
@@ -48,12 +53,10 @@ public final class MultiComparator<C> implements Comparator<C>, Serializable {
         if (this.comparators.isEmpty()) {
             return Order.incr.compare(objectA, objectB);
         } else {
-            for (int i = 0; i < this.comparators.size(); i++) {
-                if (Order.shuffle != this.comparators.get(i)) {
-                    final int comparison = this.comparators.get(i).compare(this.getObject(objectA, i), this.getObject(objectB, i));
-                    if (comparison != 0)
-                        return comparison;
-                }
+            for (int i = this.startIndex; i < this.comparators.size(); i++) {
+                final int comparison = this.comparators.get(i).compare(this.getObject(objectA, i), this.getObject(objectB, i));
+                if (comparison != 0)
+                    return comparison;
             }
             return 0;
         }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/341ebf98/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/util/function/MultiComparatorTest.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/util/function/MultiComparatorTest.java b/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/util/function/MultiComparatorTest.java
new file mode 100644
index 0000000..de2a741
--- /dev/null
+++ b/gremlin-core/src/test/java/org/apache/tinkerpop/gremlin/util/function/MultiComparatorTest.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.util.function;
+
+import org.apache.tinkerpop.gremlin.process.traversal.Order;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * @author Marko A. Rodriguez (http://markorodriguez.com)
+ */
+public class MultiComparatorTest {
+
+    private static final Random RANDOM = new Random();
+
+    @Test
+    public void shouldHandleShuffleCorrectly() {
+        MultiComparator<Object> comparator = new MultiComparator<>(Arrays.asList(Order.incr, Order.decr, Order.shuffle));
+        assertTrue(comparator.isShuffle()); // because its a shuffle, the comparator simply returns 0
+        for (int i = 0; i < 100; i++) {
+            assertEquals(0, comparator.compare(RANDOM.nextInt(), RANDOM.nextInt()));
+        }
+        //
+        comparator = new MultiComparator<>(Arrays.asList(Order.incr, Order.shuffle, Order.decr));
+        assertEquals(1, comparator.compare(1, 2));
+        assertEquals(-1, comparator.compare(2, 1));
+        assertEquals(0, comparator.compare(2, 2));
+        assertEquals(2, comparator.startIndex);
+        assertFalse(comparator.isShuffle());
+        //
+        comparator = new MultiComparator<>(Arrays.asList(Order.incr, Order.shuffle, Order.decr, Order.shuffle, Order.incr));
+        assertEquals(-1, comparator.compare(1, 2));
+        assertEquals(1, comparator.compare(2, 1));
+        assertEquals(0, comparator.compare(2, 2));
+        assertEquals(4, comparator.startIndex);
+        assertFalse(comparator.isShuffle());
+        //
+        comparator = new MultiComparator<>(Collections.emptyList());
+        assertEquals(-1, comparator.compare(1, 2));
+        assertEquals(1, comparator.compare(2, 1));
+        assertEquals(0, comparator.compare(2, 2));
+        assertEquals(0, comparator.startIndex);
+        assertFalse(comparator.isShuffle());
+    }
+}


[27/31] tinkerpop git commit: TINKERPOP-1565 Reverted GraphSON 3.0 changes

Posted by sp...@apache.org.
TINKERPOP-1565 Reverted GraphSON 3.0 changes

These changes were part of the original work on this branch to alter the structure of GraphSON 2.0. I'd migrated those changes to 3.0 to avoid breaking change. Since that time there has been a lot of work on and off this branch related to this work and it seems that including GraphSON 3.0 changes to format is just complicating this ticket. As such the ticket itself has been modified to just be about setting up for GraphSON 3.0 and at this point 3.0 is equivalent to 2.0.


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

Branch: refs/heads/TINKERPOP-1565
Commit: 99a2703806705d896a7eb0eb88b940a82aac9d73
Parents: aec7fd1
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Thu Jan 19 15:10:20 2017 -0500
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Jan 19 15:15:33 2017 -0500

----------------------------------------------------------------------
 .../io/graphson/GraphSONSerializersV3d0.java    | 160 ++---
 .../ser/GraphSONMessageSerializerV3d0.java      |  11 +-
 .../gremlin_python/structure/io/graphson.py     |  54 +-
 .../jython/tests/structure/io/test_graphson.py  |  14 +-
 .../gremlin/python/jsr223/PythonProvider.java   |   1 +
 .../server/GremlinServerHttpIntegrateTest.java  |   2 +-
 .../gremlin/structure/io/IoPropertyTest.java    |  12 -
 .../structure/io/graphson/_3_3_0/edge-v3d0.json |  10 +-
 .../structure/io/graphson/_3_3_0/path-v3d0.json | 148 +++--
 .../io/graphson/_3_3_0/tinkergraph-v3d0.json    | 582 ++++++++++++-------
 .../io/graphson/_3_3_0/traverser-v3d0.json      | 120 ++--
 .../structure/io/graphson/_3_3_0/tree-v3d0.json | 148 +++--
 .../io/graphson/_3_3_0/vertex-v3d0.json         | 120 ++--
 .../io/graphson/_3_3_0/vertexproperty-v3d0.json |   4 -
 14 files changed, 777 insertions(+), 609 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/99a27038/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV3d0.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV3d0.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV3d0.java
index 165e7e1..b62094e 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV3d0.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV3d0.java
@@ -64,6 +64,8 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONUtil.safeWriteObjectField;
+
 /**
  * GraphSON serializers for graph-based objects such as vertices, edges, properties, and paths. These serializers
  * present a generalized way to serialize the implementations of core interfaces.
@@ -100,34 +102,22 @@ class GraphSONSerializersV3d0 {
         }
 
         private void writeProperties(final Vertex vertex, final JsonGenerator jsonGenerator) throws IOException {
-            if (vertex.keys().isEmpty())
+            if (vertex.keys().size() == 0)
                 return;
             jsonGenerator.writeFieldName(GraphSONTokens.PROPERTIES);
             jsonGenerator.writeStartObject();
 
             final List<String> keys = normalize ?
                     IteratorUtils.list(vertex.keys().iterator(), Comparator.naturalOrder()) : new ArrayList<>(vertex.keys());
-            for (final String key : keys) {
+            for (String key : keys) {
                 final Iterator<VertexProperty<Object>> vertexProperties = normalize ?
                         IteratorUtils.list(vertex.properties(key), Comparators.PROPERTY_COMPARATOR).iterator() : vertex.properties(key);
                 if (vertexProperties.hasNext()) {
                     jsonGenerator.writeFieldName(key);
+
                     jsonGenerator.writeStartArray();
                     while (vertexProperties.hasNext()) {
-                        final VertexProperty<?> vertexProperty = vertexProperties.next();
-                        jsonGenerator.writeStartObject();
-                        jsonGenerator.writeObjectField(GraphSONTokens.ID, vertexProperty.id());
-                        jsonGenerator.writeObjectField(GraphSONTokens.VALUE, vertexProperty.value());
-                        if (!vertexProperty.keys().isEmpty()) {
-                            jsonGenerator.writeObjectFieldStart(GraphSONTokens.PROPERTIES);
-                            final Iterator<Property<?>> properties = (Iterator) vertexProperty.properties();
-                            while (properties.hasNext()) {
-                                final Property<?> property = properties.next();
-                                jsonGenerator.writeObjectField(property.key(), property.value());
-                            }
-                            jsonGenerator.writeEndObject();
-                        }
-                        jsonGenerator.writeEndObject();
+                        jsonGenerator.writeObject(vertexProperties.next());
                     }
                     jsonGenerator.writeEndArray();
                 }
@@ -164,15 +154,13 @@ class GraphSONSerializersV3d0 {
         }
 
         private void writeProperties(final Edge edge, final JsonGenerator jsonGenerator) throws IOException {
-            final Iterator<Property<Object>> edgeProperties = normalize ?
+            final Iterator<Property<Object>> elementProperties = normalize ?
                     IteratorUtils.list(edge.properties(), Comparators.PROPERTY_COMPARATOR).iterator() : edge.properties();
-            if (edgeProperties.hasNext()) {
+            if (elementProperties.hasNext()) {
                 jsonGenerator.writeFieldName(GraphSONTokens.PROPERTIES);
+
                 jsonGenerator.writeStartObject();
-                while (edgeProperties.hasNext()) {
-                    final Property<?> property = edgeProperties.next();
-                    jsonGenerator.writeObjectField(property.key(), property.value());
-                }
+                elementProperties.forEachRemaining(prop -> safeWriteObjectField(jsonGenerator, prop.key(), prop));
                 jsonGenerator.writeEndObject();
             }
         }
@@ -188,25 +176,8 @@ class GraphSONSerializersV3d0 {
         public void serialize(final Property property, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
                 throws IOException {
             jsonGenerator.writeStartObject();
-            jsonGenerator.writeStringField(GraphSONTokens.KEY, property.key());
+            jsonGenerator.writeObjectField(GraphSONTokens.KEY, property.key());
             jsonGenerator.writeObjectField(GraphSONTokens.VALUE, property.value());
-            if (property.element() instanceof VertexProperty) {
-                VertexProperty vertexProperty = (VertexProperty) property.element();
-                jsonGenerator.writeObjectFieldStart(GraphSONTokens.VERTEX_PROPERTY);
-                jsonGenerator.writeObjectField(GraphSONTokens.ID, vertexProperty.id());
-                jsonGenerator.writeStringField(GraphSONTokens.LABEL, vertexProperty.label());
-                jsonGenerator.writeObjectField(GraphSONTokens.VALUE, vertexProperty.value());
-                jsonGenerator.writeObjectField(GraphSONTokens.VERTEX, vertexProperty.element().id());
-                jsonGenerator.writeEndObject();
-            } else if (property.element() instanceof Edge) {
-                Edge edge = (Edge) property.element();
-                jsonGenerator.writeObjectFieldStart(GraphSONTokens.EDGE);
-                jsonGenerator.writeObjectField(GraphSONTokens.ID, edge.id());
-                jsonGenerator.writeStringField(GraphSONTokens.LABEL, edge.label());
-                jsonGenerator.writeObjectField(GraphSONTokens.IN, edge.inVertex().id());
-                jsonGenerator.writeObjectField(GraphSONTokens.OUT, edge.outVertex().id());
-                jsonGenerator.writeEndObject();
-            }
             jsonGenerator.writeEndObject();
         }
     }
@@ -223,46 +194,44 @@ class GraphSONSerializersV3d0 {
         }
 
         @Override
-        public void serialize(final VertexProperty vertexProperty, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+        public void serialize(final VertexProperty property, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
                 throws IOException {
             jsonGenerator.writeStartObject();
 
-            jsonGenerator.writeObjectField(GraphSONTokens.ID, vertexProperty.id());
-            jsonGenerator.writeObjectField(GraphSONTokens.VALUE, vertexProperty.value());
-            if (null != vertexProperty.element())
-                jsonGenerator.writeObjectField(GraphSONTokens.VERTEX, vertexProperty.element().id());
-            if (this.includeLabel)
-                jsonGenerator.writeStringField(GraphSONTokens.LABEL, vertexProperty.label());
-            tryWriteMetaProperties(vertexProperty, jsonGenerator, normalize);
+            jsonGenerator.writeObjectField(GraphSONTokens.ID, property.id());
+            jsonGenerator.writeObjectField(GraphSONTokens.VALUE, property.value());
+            if (includeLabel)
+                jsonGenerator.writeStringField(GraphSONTokens.LABEL, property.label());
+            tryWriteMetaProperties(property, jsonGenerator, normalize);
 
             jsonGenerator.writeEndObject();
         }
 
-        private static void tryWriteMetaProperties(final VertexProperty vertexProperty, final JsonGenerator jsonGenerator,
+        private static void tryWriteMetaProperties(final VertexProperty property, final JsonGenerator jsonGenerator,
                                                    final boolean normalize) throws IOException {
             // when "detached" you can't check features of the graph it detached from so it has to be
             // treated differently from a regular VertexProperty implementation.
-            if (vertexProperty instanceof DetachedVertexProperty) {
+            if (property instanceof DetachedVertexProperty) {
                 // only write meta properties key if they exist
-                if (vertexProperty.properties().hasNext()) {
-                    writeMetaProperties(vertexProperty, jsonGenerator, normalize);
+                if (property.properties().hasNext()) {
+                    writeMetaProperties(property, jsonGenerator, normalize);
                 }
             } else {
                 // still attached - so we can check the features to see if it's worth even trying to write the
                 // meta properties key
-                if (vertexProperty.graph().features().vertex().supportsMetaProperties() && vertexProperty.properties().hasNext()) {
-                    writeMetaProperties(vertexProperty, jsonGenerator, normalize);
+                if (property.graph().features().vertex().supportsMetaProperties() && property.properties().hasNext()) {
+                    writeMetaProperties(property, jsonGenerator, normalize);
                 }
             }
         }
 
-        private static void writeMetaProperties(final VertexProperty vertexProperty, final JsonGenerator jsonGenerator,
+        private static void writeMetaProperties(final VertexProperty property, final JsonGenerator jsonGenerator,
                                                 final boolean normalize) throws IOException {
             jsonGenerator.writeFieldName(GraphSONTokens.PROPERTIES);
             jsonGenerator.writeStartObject();
 
             final Iterator<Property<Object>> metaProperties = normalize ?
-                    IteratorUtils.list((Iterator<Property<Object>>) vertexProperty.properties(), Comparators.PROPERTY_COMPARATOR).iterator() : vertexProperty.properties();
+                    IteratorUtils.list((Iterator<Property<Object>>) property.properties(), Comparators.PROPERTY_COMPARATOR).iterator() : property.properties();
             while (metaProperties.hasNext()) {
                 final Property<Object> metaProperty = metaProperties.next();
                 jsonGenerator.writeObjectField(metaProperty.key(), metaProperty.value());
@@ -358,7 +327,7 @@ class GraphSONSerializersV3d0 {
         @Override
         public void serialize(final Integer integer, final JsonGenerator jsonGenerator,
                               final SerializerProvider serializerProvider) throws IOException {
-            jsonGenerator.writeNumber(integer.intValue());
+            jsonGenerator.writeNumber(((Integer) integer).intValue());
         }
     }
 
@@ -459,7 +428,7 @@ class GraphSONSerializersV3d0 {
         public Vertex createObject(final Map<String, Object> vertexData) {
             return new DetachedVertex(
                     vertexData.get(GraphSONTokens.ID),
-                    (String) vertexData.getOrDefault(GraphSONTokens.LABEL, Vertex.DEFAULT_LABEL),
+                    vertexData.get(GraphSONTokens.LABEL).toString(),
                     (Map<String, Object>) vertexData.get(GraphSONTokens.PROPERTIES)
             );
         }
@@ -475,10 +444,10 @@ class GraphSONSerializersV3d0 {
         public Edge createObject(final Map<String, Object> edgeData) {
             return new DetachedEdge(
                     edgeData.get(GraphSONTokens.ID),
-                    (String) edgeData.getOrDefault(GraphSONTokens.LABEL, Edge.DEFAULT_LABEL),
-                    (Map<String, Object>) edgeData.get(GraphSONTokens.PROPERTIES),
-                    Pair.with(edgeData.get(GraphSONTokens.OUT), (String) edgeData.getOrDefault(GraphSONTokens.OUT_LABEL, Vertex.DEFAULT_LABEL)),
-                    Pair.with(edgeData.get(GraphSONTokens.IN), (String) edgeData.getOrDefault(GraphSONTokens.IN_LABEL, Vertex.DEFAULT_LABEL))
+                    edgeData.get(GraphSONTokens.LABEL).toString(),
+                    (Map) edgeData.get(GraphSONTokens.PROPERTIES),
+                    Pair.with(edgeData.get(GraphSONTokens.OUT), edgeData.get(GraphSONTokens.OUT_LABEL).toString()),
+                    Pair.with(edgeData.get(GraphSONTokens.IN), edgeData.get(GraphSONTokens.IN_LABEL).toString())
             );
         }
     }
@@ -491,41 +460,9 @@ class GraphSONSerializersV3d0 {
 
         @Override
         public Property createObject(final Map<String, Object> propData) {
-            Element element = null;
-            if (propData.containsKey(GraphSONTokens.VERTEX_PROPERTY)) {
-                final Map<String, Object> elementData = (Map<String, Object>) propData.get(GraphSONTokens.VERTEX_PROPERTY);
-                element = new VertexPropertyJacksonDeserializer().createObject(elementData);
-            } else if (propData.containsKey(GraphSONTokens.EDGE)) {
-                final Map<String, Object> elementData = (Map<String, Object>) propData.get(GraphSONTokens.EDGE);
-                element = new EdgeJacksonDeserializer().createObject(elementData);
-            }
-            return null != element ? // graphson-non-embedded is treated differently, but since this is a hard coded embedding...
-                    new DetachedProperty<>((String) propData.get(GraphSONTokens.KEY), propData.get(GraphSONTokens.VALUE), element) :
-                    new DetachedProperty<>((String) propData.get(GraphSONTokens.KEY), propData.get(GraphSONTokens.VALUE));
-        }
-    }
-
-    static class VertexPropertyJacksonDeserializer extends AbstractObjectDeserializer<VertexProperty> {
-
-        protected VertexPropertyJacksonDeserializer() {
-            super(VertexProperty.class);
-        }
-
-        @Override
-        public VertexProperty createObject(final Map<String, Object> propData) {
-            return propData.containsKey(GraphSONTokens.VERTEX) ?
-                    new DetachedVertexProperty<>(
-                            propData.get(GraphSONTokens.ID),
-                            (String) propData.get(GraphSONTokens.LABEL),
-                            propData.get(GraphSONTokens.VALUE),
-                            (Map<String, Object>) propData.get(GraphSONTokens.PROPERTIES),
-                            new DetachedVertex(propData.get(GraphSONTokens.VERTEX), Vertex.DEFAULT_LABEL, null)) :
-                    new DetachedVertexProperty<>(
-                            propData.get(GraphSONTokens.ID),
-                            (String) propData.get(GraphSONTokens.LABEL),
-                            propData.get(GraphSONTokens.VALUE),
-                            (Map<String, Object>) propData.get(GraphSONTokens.PROPERTIES));
-
+            return new DetachedProperty(
+                    (String) propData.get(GraphSONTokens.KEY),
+                    propData.get(GraphSONTokens.VALUE));
         }
     }
 
@@ -549,6 +486,23 @@ class GraphSONSerializersV3d0 {
         }
     }
 
+    static class VertexPropertyJacksonDeserializer extends AbstractObjectDeserializer<VertexProperty> {
+
+        protected VertexPropertyJacksonDeserializer() {
+            super(VertexProperty.class);
+        }
+
+        @Override
+        public VertexProperty createObject(final Map<String, Object> propData) {
+            return new DetachedVertexProperty(
+                    propData.get(GraphSONTokens.ID),
+                    (String) propData.get(GraphSONTokens.LABEL),
+                    propData.get(GraphSONTokens.VALUE),
+                    (Map) propData.get(GraphSONTokens.PROPERTIES)
+            );
+        }
+    }
+
     static class MetricsJacksonDeserializer extends AbstractObjectDeserializer<Metrics> {
         public MetricsJacksonDeserializer() {
             super(Metrics.class);
@@ -556,16 +510,16 @@ class GraphSONSerializersV3d0 {
 
         @Override
         public Metrics createObject(final Map<String, Object> metricsData) {
-            final MutableMetrics m = new MutableMetrics((String) metricsData.get(GraphSONTokens.ID), (String) metricsData.get(GraphSONTokens.NAME));
+            final MutableMetrics m = new MutableMetrics((String)metricsData.get(GraphSONTokens.ID), (String)metricsData.get(GraphSONTokens.NAME));
 
             m.setDuration(Math.round((Double) metricsData.get(GraphSONTokens.DURATION) * 1000000), TimeUnit.NANOSECONDS);
-            for (Map.Entry<String, Long> count : ((Map<String, Long>) metricsData.getOrDefault(GraphSONTokens.COUNTS, Collections.emptyMap())).entrySet()) {
+            for (Map.Entry<String, Long> count : ((Map<String, Long>)metricsData.getOrDefault(GraphSONTokens.COUNTS, new HashMap<>(0))).entrySet()) {
                 m.setCount(count.getKey(), count.getValue());
             }
-            for (Map.Entry<String, Long> count : ((Map<String, Long>) metricsData.getOrDefault(GraphSONTokens.ANNOTATIONS, Collections.emptyMap())).entrySet()) {
+            for (Map.Entry<String, Long> count : ((Map<String, Long>) metricsData.getOrDefault(GraphSONTokens.ANNOTATIONS, new HashMap<>(0))).entrySet()) {
                 m.setAnnotation(count.getKey(), count.getValue());
             }
-            for (MutableMetrics nested : (List<MutableMetrics>) metricsData.getOrDefault(GraphSONTokens.METRICS, Collections.emptyList())) {
+            for (MutableMetrics nested : (List<MutableMetrics>)metricsData.getOrDefault(GraphSONTokens.METRICS, new ArrayList<>(0))) {
                 m.addNested(nested);
             }
             return m;
@@ -627,6 +581,4 @@ class GraphSONSerializersV3d0 {
             return jsonParser.getDoubleValue();
         }
     }
-}
-
-
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/99a27038/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0.java
----------------------------------------------------------------------
diff --git a/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0.java b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0.java
index d1c9e46..46e5c3a 100644
--- a/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0.java
+++ b/gremlin-driver/src/main/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV3d0.java
@@ -76,16 +76,7 @@ public final class GraphSONMessageSerializerV3d0 extends AbstractGraphSONMessage
     @Override
     public ResponseMessage deserializeResponse(final String msg) throws SerializationException {
         try {
-            final Map<String, Object> responseData = mapper.readValue(msg, mapTypeReference);
-            final Map<String, Object> status = (Map<String, Object>) responseData.get(SerTokens.TOKEN_STATUS);
-            final Map<String, Object> result = (Map<String, Object>) responseData.get(SerTokens.TOKEN_RESULT);
-            return ResponseMessage.build(UUID.fromString(responseData.get(SerTokens.TOKEN_REQUEST).toString()))
-                    .code(ResponseStatusCode.getFromValue((Integer) status.get(SerTokens.TOKEN_CODE)))
-                    .statusMessage(status.get(SerTokens.TOKEN_MESSAGE).toString())
-                    .statusAttributes((Map<String, Object>) status.get(SerTokens.TOKEN_ATTRIBUTES))
-                    .result(result.get(SerTokens.TOKEN_DATA))
-                    .responseMetaData((Map<String, Object>) result.get(SerTokens.TOKEN_META))
-                    .create();
+            return mapper.readValue(msg, ResponseMessage.class);
         } catch (Exception ex) {
             logger.warn("Response [{}] could not be deserialized by {}.", msg, AbstractGraphSONMessageSerializerV2d0.class.getName());
             throw new SerializationException(ex);

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/99a27038/gremlin-python/src/main/jython/gremlin_python/structure/io/graphson.py
----------------------------------------------------------------------
diff --git a/gremlin-python/src/main/jython/gremlin_python/structure/io/graphson.py b/gremlin-python/src/main/jython/gremlin_python/structure/io/graphson.py
index 8795ff1..54faf56 100644
--- a/gremlin-python/src/main/jython/gremlin_python/structure/io/graphson.py
+++ b/gremlin-python/src/main/jython/gremlin_python/structure/io/graphson.py
@@ -208,26 +208,20 @@ class PropertySerializer(_GraphSONTypeIO):
 
     @classmethod
     def dictify(cls, property, writer):
-        element = property.element
-        elementDict = {}
-        if element is not None:
-            elementDict["id"] = element.id
-            elementDict["label"] = element.label
-            if isinstance(element, VertexProperty):
-                elementDict["value"] = element.value
-                elementDict["vertex"] = element.vertex.id
-                return GraphSONUtil.typedValue("Property", {"key": writer.toDict(property.key),
-                                                            "value": writer.toDict(property.value),
-                                                            "vertexProperty": writer.toDict(elementDict)})
-            elif isinstance(element, Edge):
-                elementDict["outV"] = element.outV.id
-                elementDict["inV"] = element.inV.id
-                return GraphSONUtil.typedValue("Property", {"key": writer.toDict(property.key),
-                                                            "value": writer.toDict(property.value),
-                                                            "edge": writer.toDict(elementDict)})
-        else:
-            return GraphSONUtil.typedValue("Property", {"key": writer.toDict(property.key),
-                                                        "value": writer.toDict(property.value)})
+        elementDict = writer.toDict(property.element)
+        if elementDict is not None:
+            valueDict = elementDict["@value"]
+            if "outVLabel" in valueDict:
+                del valueDict["outVLabel"]
+            if "inVLabel" in valueDict:
+                del valueDict["inVLabel"]
+            if "properties" in valueDict:
+                del valueDict["properties"]
+            if "value" in valueDict:
+                del valueDict["value"]
+        return GraphSONUtil.typedValue("Property", {"key": writer.toDict(property.key),
+                                                    "value": writer.toDict(property.value),
+                                                    "element": writer.toDict(elementDict)})
 
 
 class TraversalStrategySerializer(_GraphSONTypeIO):
@@ -392,22 +386,8 @@ class PropertyDeserializer(_GraphSONTypeIO):
 
     @classmethod
     def objectify(cls, d, reader):
-        if "edge" in d:
-            edge = reader.toObject(d["edge"])
-            return Property(d["key"], reader.toObject(d["value"]),
-                            Edge(edge["id"],
-                                 Vertex(edge["outV"]),
-                                 edge["label"],
-                                 Vertex(edge["inV"])))
-        elif "vertexProperty" in d:
-            vertex_property = reader.toObject(d["vertexProperty"])
-            return Property(d["key"], reader.toObject(d["value"]),
-                            VertexProperty(vertex_property["id"],
-                                           vertex_property["label"],
-                                           vertex_property["value"],
-                                           Vertex(vertex_property["vertex"])))
-        else:
-            return Property(d["key"], reader.toObject(d["value"]), None)
+        element = reader.toObject(d["element"]) if "element" in d else None
+        return Property(d["key"], reader.toObject(d["value"]), element)
 
 
 class PathDeserializer(_GraphSONTypeIO):
@@ -417,4 +397,4 @@ class PathDeserializer(_GraphSONTypeIO):
     def objectify(cls, d, reader):
         labels = [set(label) for label in d["labels"]]
         objects = [reader.toObject(o) for o in d["objects"]]
-        return Path(labels, objects)
+        return Path(labels, objects)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/99a27038/gremlin-python/src/main/jython/tests/structure/io/test_graphson.py
----------------------------------------------------------------------
diff --git a/gremlin-python/src/main/jython/tests/structure/io/test_graphson.py b/gremlin-python/src/main/jython/tests/structure/io/test_graphson.py
index 7e034c9..e8aa572 100644
--- a/gremlin-python/src/main/jython/tests/structure/io/test_graphson.py
+++ b/gremlin-python/src/main/jython/tests/structure/io/test_graphson.py
@@ -111,7 +111,7 @@ class TestGraphSONReader(TestCase):
         assert edge.outV == Vertex("y", "vertex")
         ##
         property = self.graphson_reader.readObject("""
-        {"@type":"g:Property", "@value":{"key":"aKey","value":{"@type":"g:Int64","@value":17},"edge":{"id":{"@type":"g:Int64","@value":122},"label":"knows","inV":"x","outV":"y"}}}""")
+        {"@type":"g:Property", "@value":{"key":"aKey","value":{"@type":"g:Int64","@value":17},"element":{"@type":"g:Edge","@value":{"id":{"@type":"g:Int64","@value":122},"label":"knows","inV":"x","outV":"y","inVLabel":"xLab"}}}}""")
         # print property
         assert isinstance(property, Property)
         assert "aKey" == property.key
@@ -208,11 +208,11 @@ class TestGraphSONWriter(TestCase):
             self.graphson_writer.writeObject(VertexProperty("blah", "keyA", True, Vertex("stephen"))))
 
         assert {"@type": "g:Property",
-                "@value": {"key": "name", "value": "marko", "vertexProperty": {
-                    "vertex": "vertexId",
-                    "id": "anId",
-                    "label": "aKey",
-                    "value": {"@type": "g:Int32", "@value": 21345}}}} == json.loads(
+                "@value": {"key": "name", "value": "marko", "element": {"@type": "g:VertexProperty",
+                                                                        "@value": {
+                                                                            "vertex": "vertexId",
+                                                                            "id": "anId",
+                                                                            "label": "aKey"}}}} == json.loads(
             self.graphson_writer.writeObject(
                 Property("name", "marko", VertexProperty("anId", "aKey", 21345, Vertex("vertexId")))))
 
@@ -255,4 +255,4 @@ class TestGraphSONWriter(TestCase):
 
 
 if __name__ == '__main__':
-    unittest.main()
+    unittest.main()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/99a27038/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/jsr223/PythonProvider.java
----------------------------------------------------------------------
diff --git a/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/jsr223/PythonProvider.java b/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/jsr223/PythonProvider.java
index 605539d..44d65fa 100644
--- a/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/jsr223/PythonProvider.java
+++ b/gremlin-python/src/test/java/org/apache/tinkerpop/gremlin/python/jsr223/PythonProvider.java
@@ -66,6 +66,7 @@ public class PythonProvider extends AbstractGraphProvider {
             "g_VX1X_out_injectXv2X_name",
             "shouldHidePartitionKeyForValues",
             "g_withSackXBigInteger_TEN_powX1000X_assignX_V_localXoutXknowsX_barrierXnormSackXX_inXknowsX_barrier_sack",
+            "g_injectXg_VX1X_propertiesXnameX_nextX_value",
             //
             ProgramTest.Traversals.class.getCanonicalName(),
             TraversalInterruptionTest.class.getCanonicalName(),

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/99a27038/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java
index ba9aba8..4192a6c 100644
--- a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java
+++ b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java
@@ -379,7 +379,7 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
             assertEquals("application/json", response.getEntity().getContentType().getValue());
             final String json = EntityUtils.toString(response.getEntity());
             final JsonNode node = mapper.readTree(json);
-            assertEquals("stephen", node.get("result").get("data").get(0).get(GraphSONTokens.VALUEPROP).get("properties").get("name").get(0).get(GraphSONTokens.VALUE).asText());
+            assertEquals("stephen", node.get("result").get("data").get(0).get(GraphSONTokens.VALUEPROP).get("properties").get("name").get(0).get(GraphSONTokens.VALUEPROP).get(GraphSONTokens.VALUE).asText());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/99a27038/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoPropertyTest.java
----------------------------------------------------------------------
diff --git a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoPropertyTest.java b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoPropertyTest.java
index ecde126..d16339f 100644
--- a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoPropertyTest.java
+++ b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoPropertyTest.java
@@ -106,10 +106,6 @@ public class IoPropertyTest extends AbstractGremlinTest {
                     assertEquals(IteratorUtils.count(p.properties()), IteratorUtils.count(propertyAttachable.get().properties()));
                     assertEquals(p.property("startTime").value(), ((Property) propertyAttachable.get().properties("startTime").next()).value());
                     assertEquals(p.property("endTime").value(), ((Property) propertyAttachable.get().properties("endTime").next()).value());
-                    if (ioType.equals("graphson-v3")) { // TODO: make this work with Gryo
-                        assertEquals(p, propertyAttachable.get());
-                        assertEquals(p.element(), propertyAttachable.get().element());
-                    }
                     called.set(true);
                     return propertyAttachable.get();
                 });
@@ -134,10 +130,6 @@ public class IoPropertyTest extends AbstractGremlinTest {
                     assertEquals(p.value(), propertyAttachable.get().value());
                     assertEquals(p.key(), propertyAttachable.get().key());
                     assertEquals(0, IteratorUtils.count(propertyAttachable.get().properties()));
-                    if (ioType.equals("graphson-v3")) { // TODO: make this work with Gryo
-                        assertEquals(p, propertyAttachable.get());
-                        assertEquals(p.element(), propertyAttachable.get().element());
-                    }
                     called.set(true);
                     return propertyAttachable.get();
                 });
@@ -161,10 +153,6 @@ public class IoPropertyTest extends AbstractGremlinTest {
                 reader.readProperty(bais, propertyAttachable -> {
                     assertEquals(p.value(), propertyAttachable.get().value());
                     assertEquals(p.key(), propertyAttachable.get().key());
-                    if (ioType.equals("graphson-v3")) { // TODO: make this work with Gryo
-                        assertEquals(p, propertyAttachable.get());
-                        assertEquals(p.element(), propertyAttachable.get().element());
-                    }
                     called.set(true);
                     return propertyAttachable.get();
                 });

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/99a27038/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v3d0.json
index 4c396f8..ba1c52e 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v3d0.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/edge-v3d0.json
@@ -18,8 +18,14 @@
     },
     "properties" : {
       "since" : {
-        "@type" : "g:Int32",
-        "@value" : 2009
+        "@type" : "g:Property",
+        "@value" : {
+          "key" : "since",
+          "value" : {
+            "@type" : "g:Int32",
+            "@value" : 2009
+          }
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/99a27038/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v3d0.json
index 8a41507..54ff76d 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v3d0.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/path-v3d0.json
@@ -12,70 +12,90 @@
         "label" : "person",
         "properties" : {
           "name" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 0
-            },
-            "value" : "marko"
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 0
+              },
+              "value" : "marko",
+              "label" : "name"
+            }
           } ],
           "location" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 6
-            },
-            "value" : "san diego",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 1997
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 6
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2001
+              "value" : "san diego",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 1997
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2001
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 7
-            },
-            "value" : "santa cruz",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2001
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 7
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2004
+              "value" : "santa cruz",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2001
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2004
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 8
-            },
-            "value" : "brussels",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2004
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 8
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2005
+              "value" : "brussels",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2004
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2005
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 9
-            },
-            "value" : "santa fe",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2005
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 9
+              },
+              "value" : "santa fe",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2005
+                }
               }
             }
           } ]
@@ -91,11 +111,15 @@
         "label" : "software",
         "properties" : {
           "name" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 4
-            },
-            "value" : "gremlin"
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 4
+              },
+              "value" : "gremlin",
+              "label" : "name"
+            }
           } ]
         }
       }
@@ -109,11 +133,15 @@
         "label" : "software",
         "properties" : {
           "name" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 5
-            },
-            "value" : "tinkergraph"
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 5
+              },
+              "value" : "tinkergraph",
+              "label" : "name"
+            }
           } ]
         }
       }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/99a27038/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v3d0.json
index a97c8c1..24e95ed 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v3d0.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tinkergraph-v3d0.json
@@ -11,70 +11,90 @@
         "label" : "person",
         "properties" : {
           "name" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 0
-            },
-            "value" : "marko"
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 0
+              },
+              "value" : "marko",
+              "label" : "name"
+            }
           } ],
           "location" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 6
-            },
-            "value" : "san diego",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 1997
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 6
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2001
+              "value" : "san diego",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 1997
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2001
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 7
-            },
-            "value" : "santa cruz",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2001
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 7
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2004
+              "value" : "santa cruz",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2001
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2004
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 8
-            },
-            "value" : "brussels",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2004
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 8
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2005
+              "value" : "brussels",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2004
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2005
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 9
-            },
-            "value" : "santa fe",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2005
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 9
+              },
+              "value" : "santa fe",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2005
+                }
               }
             }
           } ]
@@ -90,54 +110,70 @@
         "label" : "person",
         "properties" : {
           "name" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 1
-            },
-            "value" : "stephen"
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 1
+              },
+              "value" : "stephen",
+              "label" : "name"
+            }
           } ],
           "location" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 10
-            },
-            "value" : "centreville",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 1990
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 10
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2000
+              "value" : "centreville",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 1990
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2000
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 11
-            },
-            "value" : "dulles",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2000
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 11
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2006
+              "value" : "dulles",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2000
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2006
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 12
-            },
-            "value" : "purcellville",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2006
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 12
+              },
+              "value" : "purcellville",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2006
+                }
               }
             }
           } ]
@@ -153,70 +189,90 @@
         "label" : "person",
         "properties" : {
           "name" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 2
-            },
-            "value" : "matthias"
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 2
+              },
+              "value" : "matthias",
+              "label" : "name"
+            }
           } ],
           "location" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 13
-            },
-            "value" : "bremen",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2004
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 13
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2007
+              "value" : "bremen",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2004
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2007
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 14
-            },
-            "value" : "baltimore",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2007
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 14
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2011
+              "value" : "baltimore",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2007
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2011
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 15
-            },
-            "value" : "oakland",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2011
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 15
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2014
+              "value" : "oakland",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2011
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2014
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 16
-            },
-            "value" : "seattle",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2014
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 16
+              },
+              "value" : "seattle",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2014
+                }
               }
             }
           } ]
@@ -232,54 +288,70 @@
         "label" : "person",
         "properties" : {
           "name" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 3
-            },
-            "value" : "daniel"
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 3
+              },
+              "value" : "daniel",
+              "label" : "name"
+            }
           } ],
           "location" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 17
-            },
-            "value" : "spremberg",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 1982
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 17
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2005
+              "value" : "spremberg",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 1982
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2005
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 18
-            },
-            "value" : "kaiserslautern",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2005
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 18
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2009
+              "value" : "kaiserslautern",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2005
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2009
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 19
-            },
-            "value" : "aachen",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2009
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 19
+              },
+              "value" : "aachen",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2009
+                }
               }
             }
           } ]
@@ -295,11 +367,15 @@
         "label" : "software",
         "properties" : {
           "name" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 4
-            },
-            "value" : "gremlin"
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 4
+              },
+              "value" : "gremlin",
+              "label" : "name"
+            }
           } ]
         }
       }
@@ -313,11 +389,15 @@
         "label" : "software",
         "properties" : {
           "name" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 5
-            },
-            "value" : "tinkergraph"
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 5
+              },
+              "value" : "tinkergraph",
+              "label" : "name"
+            }
           } ]
         }
       }
@@ -342,8 +422,14 @@
         },
         "properties" : {
           "since" : {
-            "@type" : "g:Int32",
-            "@value" : 2009
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "since",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 2009
+              }
+            }
           }
         }
       }
@@ -367,8 +453,14 @@
         },
         "properties" : {
           "since" : {
-            "@type" : "g:Int32",
-            "@value" : 2010
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "since",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 2010
+              }
+            }
           }
         }
       }
@@ -392,8 +484,14 @@
         },
         "properties" : {
           "skill" : {
-            "@type" : "g:Int32",
-            "@value" : 4
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "skill",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 4
+              }
+            }
           }
         }
       }
@@ -417,8 +515,14 @@
         },
         "properties" : {
           "skill" : {
-            "@type" : "g:Int32",
-            "@value" : 5
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "skill",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 5
+              }
+            }
           }
         }
       }
@@ -442,8 +546,14 @@
         },
         "properties" : {
           "since" : {
-            "@type" : "g:Int32",
-            "@value" : 2010
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "since",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 2010
+              }
+            }
           }
         }
       }
@@ -467,8 +577,14 @@
         },
         "properties" : {
           "since" : {
-            "@type" : "g:Int32",
-            "@value" : 2011
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "since",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 2011
+              }
+            }
           }
         }
       }
@@ -492,8 +608,14 @@
         },
         "properties" : {
           "skill" : {
-            "@type" : "g:Int32",
-            "@value" : 5
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "skill",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 5
+              }
+            }
           }
         }
       }
@@ -517,8 +639,14 @@
         },
         "properties" : {
           "skill" : {
-            "@type" : "g:Int32",
-            "@value" : 4
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "skill",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 4
+              }
+            }
           }
         }
       }
@@ -542,8 +670,14 @@
         },
         "properties" : {
           "since" : {
-            "@type" : "g:Int32",
-            "@value" : 2012
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "since",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 2012
+              }
+            }
           }
         }
       }
@@ -567,8 +701,14 @@
         },
         "properties" : {
           "skill" : {
-            "@type" : "g:Int32",
-            "@value" : 3
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "skill",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 3
+              }
+            }
           }
         }
       }
@@ -592,8 +732,14 @@
         },
         "properties" : {
           "skill" : {
-            "@type" : "g:Int32",
-            "@value" : 3
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "skill",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 3
+              }
+            }
           }
         }
       }
@@ -617,8 +763,14 @@
         },
         "properties" : {
           "skill" : {
-            "@type" : "g:Int32",
-            "@value" : 5
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "skill",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 5
+              }
+            }
           }
         }
       }
@@ -642,8 +794,14 @@
         },
         "properties" : {
           "skill" : {
-            "@type" : "g:Int32",
-            "@value" : 3
+            "@type" : "g:Property",
+            "@value" : {
+              "key" : "skill",
+              "value" : {
+                "@type" : "g:Int32",
+                "@value" : 3
+              }
+            }
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/99a27038/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v3d0.json
index 1e6a2ce..a59a29e 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v3d0.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/traverser-v3d0.json
@@ -15,70 +15,90 @@
         "label" : "person",
         "properties" : {
           "name" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 0
-            },
-            "value" : "marko"
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 0
+              },
+              "value" : "marko",
+              "label" : "name"
+            }
           } ],
           "location" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 6
-            },
-            "value" : "san diego",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 1997
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 6
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2001
+              "value" : "san diego",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 1997
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2001
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 7
-            },
-            "value" : "santa cruz",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2001
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 7
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2004
+              "value" : "santa cruz",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2001
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2004
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 8
-            },
-            "value" : "brussels",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2004
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 8
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2005
+              "value" : "brussels",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2004
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2005
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 9
-            },
-            "value" : "santa fe",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2005
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 9
+              },
+              "value" : "santa fe",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2005
+                }
               }
             }
           } ]

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/99a27038/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v3d0.json
index 8868929..74dcffc 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v3d0.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/tree-v3d0.json
@@ -11,70 +11,90 @@
         "label" : "person",
         "properties" : {
           "name" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 0
-            },
-            "value" : "marko"
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 0
+              },
+              "value" : "marko",
+              "label" : "name"
+            }
           } ],
           "location" : [ {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 6
-            },
-            "value" : "san diego",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 1997
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 6
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2001
+              "value" : "san diego",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 1997
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2001
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 7
-            },
-            "value" : "santa cruz",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2001
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 7
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2004
+              "value" : "santa cruz",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2001
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2004
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 8
-            },
-            "value" : "brussels",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2004
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 8
               },
-              "endTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2005
+              "value" : "brussels",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2004
+                },
+                "endTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2005
+                }
               }
             }
           }, {
-            "id" : {
-              "@type" : "g:Int64",
-              "@value" : 9
-            },
-            "value" : "santa fe",
-            "properties" : {
-              "startTime" : {
-                "@type" : "g:Int32",
-                "@value" : 2005
+            "@type" : "g:VertexProperty",
+            "@value" : {
+              "id" : {
+                "@type" : "g:Int64",
+                "@value" : 9
+              },
+              "value" : "santa fe",
+              "label" : "location",
+              "properties" : {
+                "startTime" : {
+                  "@type" : "g:Int32",
+                  "@value" : 2005
+                }
               }
             }
           } ]
@@ -94,11 +114,15 @@
             "label" : "software",
             "properties" : {
               "name" : [ {
-                "id" : {
-                  "@type" : "g:Int64",
-                  "@value" : 4
-                },
-                "value" : "gremlin"
+                "@type" : "g:VertexProperty",
+                "@value" : {
+                  "id" : {
+                    "@type" : "g:Int64",
+                    "@value" : 4
+                  },
+                  "value" : "gremlin",
+                  "label" : "name"
+                }
               } ]
             }
           }
@@ -116,11 +140,15 @@
                 "label" : "software",
                 "properties" : {
                   "name" : [ {
-                    "id" : {
-                      "@type" : "g:Int64",
-                      "@value" : 5
-                    },
-                    "value" : "tinkergraph"
+                    "@type" : "g:VertexProperty",
+                    "@value" : {
+                      "id" : {
+                        "@type" : "g:Int64",
+                        "@value" : 5
+                      },
+                      "value" : "tinkergraph",
+                      "label" : "name"
+                    }
                   } ]
                 }
               }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/99a27038/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v3d0.json
index 86f305d..f102230 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v3d0.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertex-v3d0.json
@@ -8,70 +8,90 @@
     "label" : "person",
     "properties" : {
       "name" : [ {
-        "id" : {
-          "@type" : "g:Int64",
-          "@value" : 0
-        },
-        "value" : "marko"
+        "@type" : "g:VertexProperty",
+        "@value" : {
+          "id" : {
+            "@type" : "g:Int64",
+            "@value" : 0
+          },
+          "value" : "marko",
+          "label" : "name"
+        }
       } ],
       "location" : [ {
-        "id" : {
-          "@type" : "g:Int64",
-          "@value" : 6
-        },
-        "value" : "san diego",
-        "properties" : {
-          "startTime" : {
-            "@type" : "g:Int32",
-            "@value" : 1997
+        "@type" : "g:VertexProperty",
+        "@value" : {
+          "id" : {
+            "@type" : "g:Int64",
+            "@value" : 6
           },
-          "endTime" : {
-            "@type" : "g:Int32",
-            "@value" : 2001
+          "value" : "san diego",
+          "label" : "location",
+          "properties" : {
+            "startTime" : {
+              "@type" : "g:Int32",
+              "@value" : 1997
+            },
+            "endTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2001
+            }
           }
         }
       }, {
-        "id" : {
-          "@type" : "g:Int64",
-          "@value" : 7
-        },
-        "value" : "santa cruz",
-        "properties" : {
-          "startTime" : {
-            "@type" : "g:Int32",
-            "@value" : 2001
+        "@type" : "g:VertexProperty",
+        "@value" : {
+          "id" : {
+            "@type" : "g:Int64",
+            "@value" : 7
           },
-          "endTime" : {
-            "@type" : "g:Int32",
-            "@value" : 2004
+          "value" : "santa cruz",
+          "label" : "location",
+          "properties" : {
+            "startTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2001
+            },
+            "endTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2004
+            }
           }
         }
       }, {
-        "id" : {
-          "@type" : "g:Int64",
-          "@value" : 8
-        },
-        "value" : "brussels",
-        "properties" : {
-          "startTime" : {
-            "@type" : "g:Int32",
-            "@value" : 2004
+        "@type" : "g:VertexProperty",
+        "@value" : {
+          "id" : {
+            "@type" : "g:Int64",
+            "@value" : 8
           },
-          "endTime" : {
-            "@type" : "g:Int32",
-            "@value" : 2005
+          "value" : "brussels",
+          "label" : "location",
+          "properties" : {
+            "startTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2004
+            },
+            "endTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2005
+            }
           }
         }
       }, {
-        "id" : {
-          "@type" : "g:Int64",
-          "@value" : 9
-        },
-        "value" : "santa fe",
-        "properties" : {
-          "startTime" : {
-            "@type" : "g:Int32",
-            "@value" : 2005
+        "@type" : "g:VertexProperty",
+        "@value" : {
+          "id" : {
+            "@type" : "g:Int64",
+            "@value" : 9
+          },
+          "value" : "santa fe",
+          "label" : "location",
+          "properties" : {
+            "startTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2005
+            }
           }
         }
       } ]

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/99a27038/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v3d0.json
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v3d0.json b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v3d0.json
index 0319bd1..af184b1 100644
--- a/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v3d0.json
+++ b/gremlin-tools/gremlin-io-test/src/test/resources/org/apache/tinkerpop/gremlin/structure/io/graphson/_3_3_0/vertexproperty-v3d0.json
@@ -6,10 +6,6 @@
       "@value" : 0
     },
     "value" : "marko",
-    "vertex" : {
-      "@type" : "g:Int32",
-      "@value" : 1
-    },
     "label" : "name"
   }
 }
\ No newline at end of file


[29/31] tinkerpop git commit: TINKERPOP-1414 Changed Gremlin Server and TinkerGraph to default GraphSON 2.0

Posted by sp...@apache.org.
http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoTest.java
----------------------------------------------------------------------
diff --git a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoTest.java b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoTest.java
index 17ffed2..e16bbcc 100644
--- a/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoTest.java
+++ b/gremlin-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/IoTest.java
@@ -39,8 +39,10 @@ import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONIo;
 import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONMapper;
 import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONResourceAccess;
 import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONTokens;
+import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONVersion;
 import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONWriter;
 import org.apache.tinkerpop.gremlin.structure.io.graphson.LegacyGraphSONReader;
+import org.apache.tinkerpop.gremlin.structure.io.graphson.TypeInfo;
 import org.apache.tinkerpop.gremlin.structure.io.util.CustomId;
 import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 import org.apache.tinkerpop.shaded.jackson.databind.JsonNode;
@@ -355,7 +357,7 @@ public class IoTest {
         @LoadGraphWith(LoadGraphWith.GraphData.CLASSIC)
         public void shouldWriteNormalizedGraphSON() throws Exception {
             try (ByteArrayOutputStream bos = new ByteArrayOutputStream()) {
-                final GraphSONMapper mapper = graph.io(graphson).mapper().normalize(true).create();
+                final GraphSONMapper mapper = graph.io(graphson).mapper().version(GraphSONVersion.V1_0).normalize(true).create();
                 final GraphSONWriter w = graph.io(graphson).writer().mapper(mapper).create();
                 w.writeGraph(bos, graph);
 
@@ -369,14 +371,15 @@ public class IoTest {
         @FeatureRequirement(featureClass = Graph.Features.EdgeFeatures.class, feature = Graph.Features.EdgeFeatures.FEATURE_ADD_EDGES)
         @FeatureRequirement(featureClass = Graph.Features.VertexFeatures.class, feature = Graph.Features.VertexFeatures.FEATURE_ADD_VERTICES)
         public void shouldReadWriteModernWrappedInJsonObject() throws Exception {
+            final GraphSONMapper mapper = graph.io(graphson).mapper().version(GraphSONVersion.V1_0).create();
             try (final ByteArrayOutputStream os = new ByteArrayOutputStream()) {
-                final GraphWriter writer = graph.io(graphson()).writer().wrapAdjacencyList(true).create();
+                final GraphWriter writer = graph.io(graphson()).writer().wrapAdjacencyList(true).mapper(mapper).create();
                 writer.writeGraph(os, graph);
 
                 final Configuration configuration = graphProvider.newGraphConfiguration("readGraph", this.getClass(), name.getMethodName(), LoadGraphWith.GraphData.MODERN);
                 graphProvider.clear(configuration);
                 final Graph g1 = graphProvider.openTestGraph(configuration);
-                final GraphReader reader = graph.io(graphson()).reader().unwrapAdjacencyList(true).create();
+                final GraphReader reader = graph.io(graphson()).reader().mapper(mapper).unwrapAdjacencyList(true).create();
                 try (final ByteArrayInputStream bais = new ByteArrayInputStream(os.toByteArray())) {
                     reader.readGraph(bais, g1);
                 }
@@ -402,7 +405,7 @@ public class IoTest {
             final SimpleModule module = new SimpleModule();
             module.addSerializer(CustomId.class, new CustomId.CustomIdJacksonSerializerV1d0());
             final GraphWriter writer = graph.io(graphson).writer().mapper(
-                    graph.io(graphson).mapper().addCustomModule(module).embedTypes(true).create()).create();
+                    graph.io(graphson).mapper().version(GraphSONVersion.V1_0).addCustomModule(module).embedTypes(true).create()).create();
 
             try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
                 writer.writeGraph(baos, graph);
@@ -421,7 +424,7 @@ public class IoTest {
 
                 try (final InputStream is = new ByteArrayInputStream(baos.toByteArray())) {
                     final GraphReader reader = graph.io(graphson).reader()
-                            .mapper(graph.io(graphson).mapper().embedTypes(true).addCustomModule(module).create()).create();
+                            .mapper(graph.io(graphson).mapper().version(GraphSONVersion.V1_0).embedTypes(true).addCustomModule(module).create()).create();
                     reader.readGraph(is, g2);
                 }
 
@@ -437,6 +440,35 @@ public class IoTest {
 
         @Test
         @FeatureRequirement(featureClass = Graph.Features.EdgeFeatures.class, feature = Graph.Features.EdgeFeatures.FEATURE_ADD_EDGES)
+        @FeatureRequirement(featureClass = EdgePropertyFeatures.class, feature = FEATURE_STRING_VALUES)
+        @FeatureRequirement(featureClass = Graph.Features.VertexFeatures.class, feature = Graph.Features.VertexFeatures.FEATURE_ADD_VERTICES)
+        public void shouldReadWriteSelfLoopingEdges() throws Exception {
+            final GraphSONMapper mapper = graph.io(graphson).mapper().version(GraphSONVersion.V1_0).create();
+            final Graph source = graph;
+            final Vertex v1 = source.addVertex();
+            final Vertex v2 = source.addVertex();
+            v1.addEdge("CONTROL", v2);
+            v1.addEdge("SELFLOOP", v1);
+
+            final Configuration targetConf = graphProvider.newGraphConfiguration("target", this.getClass(), name.getMethodName(), null);
+            final Graph target = graphProvider.openTestGraph(targetConf);
+            try (ByteArrayOutputStream os = new ByteArrayOutputStream()) {
+                source.io(IoCore.graphson()).writer().mapper(mapper).create().writeGraph(os, source);
+                try (ByteArrayInputStream is = new ByteArrayInputStream(os.toByteArray())) {
+                    target.io(IoCore.graphson()).reader().mapper(mapper).create().readGraph(is, target);
+                }
+            } catch (IOException ioe) {
+                throw new RuntimeException(ioe);
+            }
+
+            assertEquals(IteratorUtils.count(source.vertices()), IteratorUtils.count(target.vertices()));
+            assertEquals(IteratorUtils.count(source.edges()), IteratorUtils.count(target.edges()));
+        }
+    }
+
+    public static final class GraphSONLegacyTest extends AbstractGremlinTest {
+        @Test
+        @FeatureRequirement(featureClass = Graph.Features.EdgeFeatures.class, feature = Graph.Features.EdgeFeatures.FEATURE_ADD_EDGES)
         @FeatureRequirement(featureClass = Graph.Features.VertexFeatures.class, feature = Graph.Features.VertexFeatures.FEATURE_ADD_VERTICES)
         @FeatureRequirement(featureClass = VertexPropertyFeatures.class, feature = FEATURE_STRING_VALUES)
         @FeatureRequirement(featureClass = VertexPropertyFeatures.class, feature = FEATURE_INTEGER_VALUES)
@@ -450,12 +482,109 @@ public class IoTest {
             // the id is lossy in migration because TP2 treated ID as String
             assertClassicGraph(graph, false, true);
         }
+    }
+
+    public static final class GraphSONV2D0Test extends AbstractGremlinTest {
+        private Io.Builder<GraphSONIo> graphson;
+
+        @Before
+        public void setupBeforeEachTest() {
+            graphson = graphson();
+        }
+
+        /**
+         * Only need to execute this test with TinkerGraph or other graphs that support user supplied identifiers.
+         */
+        @Test
+        @FeatureRequirement(featureClass = VertexPropertyFeatures.class, feature = FEATURE_STRING_VALUES)
+        @FeatureRequirement(featureClass = VertexPropertyFeatures.class, feature = FEATURE_INTEGER_VALUES)
+        @FeatureRequirement(featureClass = EdgePropertyFeatures.class, feature = EdgePropertyFeatures.FEATURE_FLOAT_VALUES)
+        @FeatureRequirement(featureClass = Graph.Features.VertexFeatures.class, feature = Graph.Features.VertexFeatures.FEATURE_USER_SUPPLIED_IDS)
+        @FeatureRequirement(featureClass = Graph.Features.VertexFeatures.class, feature = Graph.Features.VertexFeatures.FEATURE_NUMERIC_IDS)
+        @FeatureRequirement(featureClass = Graph.Features.VertexPropertyFeatures.class, feature = Graph.Features.VertexPropertyFeatures.FEATURE_USER_SUPPLIED_IDS)
+        @FeatureRequirement(featureClass = Graph.Features.VariableFeatures.class, feature = FEATURE_VARIABLES)
+        @LoadGraphWith(LoadGraphWith.GraphData.CLASSIC)
+        public void shouldWriteNormalizedGraphSON() throws Exception {
+            try (ByteArrayOutputStream bos = new ByteArrayOutputStream()) {
+                final GraphSONMapper mapper = graph.io(graphson).mapper().version(GraphSONVersion.V2_0).typeInfo(TypeInfo.NO_TYPES).normalize(true).create();
+                final GraphSONWriter w = graph.io(graphson).writer().mapper(mapper).create();
+                w.writeGraph(bos, graph);
+
+                final String expected = streamToString(IoTest.class.getResourceAsStream(TestHelper.convertPackageToResourcePath(GraphSONResourceAccess.class) + "tinkerpop-classic-normalized-v2d0.json"));
+                assertEquals(expected.replace("\n", "").replace("\r", ""), bos.toString().replace("\n", "").replace("\r", ""));
+            }
+        }
+
+        @Test
+        @LoadGraphWith(LoadGraphWith.GraphData.MODERN)
+        @FeatureRequirement(featureClass = Graph.Features.EdgeFeatures.class, feature = Graph.Features.EdgeFeatures.FEATURE_ADD_EDGES)
+        @FeatureRequirement(featureClass = Graph.Features.VertexFeatures.class, feature = Graph.Features.VertexFeatures.FEATURE_ADD_VERTICES)
+        public void shouldReadWriteModernWrappedInJsonObject() throws Exception {
+            final GraphSONMapper mapper = graph.io(graphson).mapper().version(GraphSONVersion.V2_0).create();
+            try (final ByteArrayOutputStream os = new ByteArrayOutputStream()) {
+                final GraphWriter writer = graph.io(graphson()).writer().wrapAdjacencyList(true).mapper(mapper).create();
+                writer.writeGraph(os, graph);
+
+                final Configuration configuration = graphProvider.newGraphConfiguration("readGraph", this.getClass(), name.getMethodName(), LoadGraphWith.GraphData.MODERN);
+                graphProvider.clear(configuration);
+                final Graph g1 = graphProvider.openTestGraph(configuration);
+                final GraphReader reader = graph.io(graphson()).reader().mapper(mapper).unwrapAdjacencyList(true).create();
+                try (final ByteArrayInputStream bais = new ByteArrayInputStream(os.toByteArray())) {
+                    reader.readGraph(bais, g1);
+                }
+
+                // modern uses double natively so always assert as such
+                IoTest.assertModernGraph(g1, true, true);
+
+                graphProvider.clear(g1, configuration);
+            }
+        }
+
+        /**
+         * This is just a serialization check for JSON.
+         */
+        @Test
+        @FeatureRequirement(featureClass = Graph.Features.VertexFeatures.class, feature = Graph.Features.VertexFeatures.FEATURE_ADD_VERTICES)
+        @FeatureRequirement(featureClass = Graph.Features.VertexFeatures.class, feature = FEATURE_USER_SUPPLIED_IDS)
+        @FeatureRequirement(featureClass = Graph.Features.VertexFeatures.class, feature = FEATURE_ANY_IDS)
+        public void shouldProperlySerializeCustomIdWithGraphSON() throws Exception {
+            final UUID id = UUID.fromString("AF4B5965-B176-4552-B3C1-FBBE2F52C305");
+            graph.addVertex(T.id, new CustomId("vertex", id));
+
+            final SimpleModule module = new CustomId.CustomIdTinkerPopJacksonModule();
+            final GraphWriter writer = graph.io(graphson).writer().mapper(
+                    graph.io(graphson).mapper().version(GraphSONVersion.V2_0).addCustomModule(module).create()).create();
+
+            try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
+                writer.writeGraph(baos, graph);
+
+                // reusing the same config used for creation of "g".
+                final Configuration configuration = graphProvider.newGraphConfiguration("g2", this.getClass(), name.getMethodName(), null);
+                graphProvider.clear(configuration);
+                final Graph g2 = graphProvider.openTestGraph(configuration);
+
+                try (final InputStream is = new ByteArrayInputStream(baos.toByteArray())) {
+                    final GraphReader reader = graph.io(graphson).reader()
+                            .mapper(graph.io(graphson).mapper().version(GraphSONVersion.V2_0).addCustomModule(module).create()).create();
+                    reader.readGraph(is, g2);
+                }
+
+                final Vertex v2 = g2.vertices().next();
+                final CustomId customId = (CustomId) v2.id();
+                assertEquals(id, customId.getElementId());
+                assertEquals("vertex", customId.getCluster());
+
+                // need to manually close the "g2" instance
+                graphProvider.clear(g2, configuration);
+            }
+        }
 
         @Test
         @FeatureRequirement(featureClass = Graph.Features.EdgeFeatures.class, feature = Graph.Features.EdgeFeatures.FEATURE_ADD_EDGES)
         @FeatureRequirement(featureClass = EdgePropertyFeatures.class, feature = FEATURE_STRING_VALUES)
         @FeatureRequirement(featureClass = Graph.Features.VertexFeatures.class, feature = Graph.Features.VertexFeatures.FEATURE_ADD_VERTICES)
         public void shouldReadWriteSelfLoopingEdges() throws Exception {
+            final GraphSONMapper mapper = graph.io(graphson).mapper().version(GraphSONVersion.V2_0).create();
             final Graph source = graph;
             final Vertex v1 = source.addVertex();
             final Vertex v2 = source.addVertex();
@@ -465,9 +594,9 @@ public class IoTest {
             final Configuration targetConf = graphProvider.newGraphConfiguration("target", this.getClass(), name.getMethodName(), null);
             final Graph target = graphProvider.openTestGraph(targetConf);
             try (ByteArrayOutputStream os = new ByteArrayOutputStream()) {
-                source.io(IoCore.graphson()).writer().create().writeGraph(os, source);
+                source.io(IoCore.graphson()).writer().mapper(mapper).create().writeGraph(os, source);
                 try (ByteArrayInputStream is = new ByteArrayInputStream(os.toByteArray())) {
-                    target.io(IoCore.graphson()).reader().create().readGraph(is, target);
+                    target.io(IoCore.graphson()).reader().mapper(mapper).create().readGraph(is, target);
                 }
             } catch (IOException ioe) {
                 throw new RuntimeException(ioe);

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/a0041629/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraph.java
----------------------------------------------------------------------
diff --git a/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraph.java b/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraph.java
index b40515e..e47229b 100644
--- a/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraph.java
+++ b/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraph.java
@@ -230,7 +230,7 @@ public final class TinkerGraph implements Graph {
 
     @Override
     public <I extends Io> I io(final Io.Builder<I> builder) {
-        return (I) builder.graph(this).onMapper(mapper -> mapper.addRegistry(TinkerIoRegistryV1d0.instance())).create();
+        return (I) builder.graph(this).onMapper(mapper -> mapper.addRegistry(TinkerIoRegistryV2d0.instance())).create();
     }
 
     @Override


[19/31] tinkerpop git commit: TINKERPOP-1565 Updated changelog

Posted by sp...@apache.org.
TINKERPOP-1565 Updated changelog


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

Branch: refs/heads/TINKERPOP-1565
Commit: 8ffd0b8fb50faf2047f98ac9a31ef3d80eafa048
Parents: 99a2703
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Thu Jan 19 15:14:28 2017 -0500
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Jan 19 15:15:33 2017 -0500

----------------------------------------------------------------------
 CHANGELOG.asciidoc | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/8ffd0b8f/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index 934022f..ab20ae8 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -29,7 +29,8 @@ TinkerPop 3.3.0 (Release Date: NOT OFFICIALLY RELEASED YET)
 * Added more specific typing to various `__` traversal steps. E.g. `<A,Vertex>out()` is `<Vertex,Vertex>out()`.
 * Updated Docker build scripts to include Python dependencies (NOTE: users should remove any previously generated TinkerPop Docker images).
 * Added "attachment requisite" `VertexProperty.element()` and `Property.element()` data in GraphSON serialization.
-* GraphSON 2.0 is now the default serialization format in TinkerGraph and Gremlin Server.
+* GraphSON 3.0 is now the default serialization format in TinkerGraph and Gremlin Server.
+* Established the GraphSON 3.0 format.
 * Added `Vertex`, `Edge`, `VertexProperty`, and `Property` serializers to Gremlin-Python and exposed tests that use graph object arguments.
 * `Bytecode.getSourceInstructions()` and `Bytecode.getStepInstructions()` now returns `List<Instruction>` instead of `Iterable<Instruction>`.
 * Added various `TraversalStrategy` registrations with `GryoMapper`.


[21/31] tinkerpop git commit: TINKERPOP-1565 Updated the IO dev docs with GraphSON 3.0.

Posted by sp...@apache.org.
TINKERPOP-1565 Updated the IO dev docs with GraphSON 3.0.


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

Branch: refs/heads/TINKERPOP-1565
Commit: 2dc9b51dc73ab7bb9d1ce042b7401e9a7b6e6cc0
Parents: 5e69a51
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Thu Jan 12 15:12:45 2017 -0500
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Jan 19 15:15:33 2017 -0500

----------------------------------------------------------------------
 docs/src/dev/io/graphson.asciidoc | 2367 ++++++++++++++++++++++++++++++++
 1 file changed, 2367 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/2dc9b51d/docs/src/dev/io/graphson.asciidoc
----------------------------------------------------------------------
diff --git a/docs/src/dev/io/graphson.asciidoc b/docs/src/dev/io/graphson.asciidoc
index fafb4f9..9ae9ef7 100644
--- a/docs/src/dev/io/graphson.asciidoc
+++ b/docs/src/dev/io/graphson.asciidoc
@@ -3848,4 +3848,2371 @@ The following example is a `ZoneOffset` of three hours, six minutes, and nine se
   "@type" : "gx:ZoneOffset",
   "@value" : "+03:06:09"
 }
+----
+
+[[graphson-3d0]]
+Version 3.0
+-----------
+
+Version 3.0 of GraphSON was first introduced on TinkerPop 3.3.0. It is quite similar to GraphSON 2.0 in many ways in
+terms of features, but it does not have an "untyped" version as previous version of GraphSON had.
+
+Core
+~~~~
+
+Class
+^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Class",
+  "@value" : "java.io.File"
+}
+----
+
+Date
+^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Date",
+  "@value" : 1481750076295
+}
+----
+
+Double
+^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Double",
+  "@value" : 100.0
+}
+----
+
+Float
+^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Float",
+  "@value" : 100.0
+}
+----
+
+Integer
+^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Int32",
+  "@value" : 100
+}
+----
+
+Long
+^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Int64",
+  "@value" : 100
+}
+----
+
+Timestamp
+^^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Timestamp",
+  "@value" : 1481750076295
+}
+----
+
+UUID
+^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:UUID",
+  "@value" : "41d2e28a-20a4-4ab0-b379-d810dede3786"
+}
+----
+
+
+Graph Structure
+~~~~~~~~~~~~~~~
+
+Edge
+^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Edge",
+  "@value" : {
+    "id" : {
+      "@type" : "g:Int32",
+      "@value" : 13
+    },
+    "label" : "develops",
+    "inVLabel" : "software",
+    "outVLabel" : "person",
+    "inV" : {
+      "@type" : "g:Int32",
+      "@value" : 10
+    },
+    "outV" : {
+      "@type" : "g:Int32",
+      "@value" : 1
+    },
+    "properties" : {
+      "since" : {
+        "@type" : "g:Int32",
+        "@value" : 2009
+      }
+    }
+  }
+}
+----
+
+Path
+^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Path",
+  "@value" : {
+    "labels" : [ [ ], [ ], [ ] ],
+    "objects" : [ {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "label" : "person",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 0
+            },
+            "value" : "marko"
+          } ],
+          "location" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 6
+            },
+            "value" : "san diego",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 1997
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 7
+            },
+            "value" : "santa cruz",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 8
+            },
+            "value" : "brussels",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 9
+            },
+            "value" : "santa fe",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          } ]
+        }
+      }
+    }, {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "label" : "software",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 4
+            },
+            "value" : "gremlin"
+          } ]
+        }
+      }
+    }, {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "label" : "software",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 5
+            },
+            "value" : "tinkergraph"
+          } ]
+        }
+      }
+    } ]
+  }
+}
+----
+
+Property
+^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Property",
+  "@value" : {
+    "key" : "since",
+    "value" : {
+      "@type" : "g:Int32",
+      "@value" : 2009
+    },
+    "edge" : {
+      "id" : {
+        "@type" : "g:Int32",
+        "@value" : 13
+      },
+      "label" : "develops",
+      "inV" : {
+        "@type" : "g:Int32",
+        "@value" : 10
+      },
+      "outV" : {
+        "@type" : "g:Int32",
+        "@value" : 1
+      }
+    }
+  }
+}
+----
+
+StarGraph
+^^^^^^^^^
+
+[source,json]
+----
+{
+  "starVertex" : {
+    "@type" : "g:Vertex",
+    "@value" : {
+      "id" : {
+        "@type" : "g:Int32",
+        "@value" : 1
+      },
+      "label" : "person",
+      "properties" : {
+        "name" : [ {
+          "id" : {
+            "@type" : "g:Int64",
+            "@value" : 0
+          },
+          "value" : "marko"
+        } ],
+        "location" : [ {
+          "id" : {
+            "@type" : "g:Int64",
+            "@value" : 6
+          },
+          "value" : "san diego",
+          "properties" : {
+            "startTime" : {
+              "@type" : "g:Int32",
+              "@value" : 1997
+            },
+            "endTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2001
+            }
+          }
+        }, {
+          "id" : {
+            "@type" : "g:Int64",
+            "@value" : 7
+          },
+          "value" : "santa cruz",
+          "properties" : {
+            "startTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2001
+            },
+            "endTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2004
+            }
+          }
+        }, {
+          "id" : {
+            "@type" : "g:Int64",
+            "@value" : 8
+          },
+          "value" : "brussels",
+          "properties" : {
+            "startTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2004
+            },
+            "endTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2005
+            }
+          }
+        }, {
+          "id" : {
+            "@type" : "g:Int64",
+            "@value" : 9
+          },
+          "value" : "santa fe",
+          "properties" : {
+            "startTime" : {
+              "@type" : "g:Int32",
+              "@value" : 2005
+            }
+          }
+        } ]
+      }
+    }
+  }
+}
+----
+
+TinkerGraph
+^^^^^^^^^^^
+
+`TinkerGraph` has a custom serializer that is registered as part of the `TinkerIoRegistry`.
+
+[source,json]
+----
+{
+  "@type" : "tinker:graph",
+  "@value" : {
+    "vertices" : [ {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "label" : "person",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 0
+            },
+            "value" : "marko"
+          } ],
+          "location" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 6
+            },
+            "value" : "san diego",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 1997
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 7
+            },
+            "value" : "santa cruz",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 8
+            },
+            "value" : "brussels",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 9
+            },
+            "value" : "santa fe",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          } ]
+        }
+      }
+    }, {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 7
+        },
+        "label" : "person",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 1
+            },
+            "value" : "stephen"
+          } ],
+          "location" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 10
+            },
+            "value" : "centreville",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 1990
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2000
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 11
+            },
+            "value" : "dulles",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2000
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2006
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 12
+            },
+            "value" : "purcellville",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2006
+              }
+            }
+          } ]
+        }
+      }
+    }, {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 8
+        },
+        "label" : "person",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 2
+            },
+            "value" : "matthias"
+          } ],
+          "location" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 13
+            },
+            "value" : "bremen",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2007
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 14
+            },
+            "value" : "baltimore",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2007
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2011
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 15
+            },
+            "value" : "oakland",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2011
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2014
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 16
+            },
+            "value" : "seattle",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2014
+              }
+            }
+          } ]
+        }
+      }
+    }, {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 9
+        },
+        "label" : "person",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 3
+            },
+            "value" : "daniel"
+          } ],
+          "location" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 17
+            },
+            "value" : "spremberg",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 1982
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 18
+            },
+            "value" : "kaiserslautern",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2009
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 19
+            },
+            "value" : "aachen",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2009
+              }
+            }
+          } ]
+        }
+      }
+    }, {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "label" : "software",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 4
+            },
+            "value" : "gremlin"
+          } ]
+        }
+      }
+    }, {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "label" : "software",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 5
+            },
+            "value" : "tinkergraph"
+          } ]
+        }
+      }
+    } ],
+    "edges" : [ {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 13
+        },
+        "label" : "develops",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "properties" : {
+          "since" : {
+            "@type" : "g:Int32",
+            "@value" : 2009
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 14
+        },
+        "label" : "develops",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "properties" : {
+          "since" : {
+            "@type" : "g:Int32",
+            "@value" : 2010
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 15
+        },
+        "label" : "uses",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "properties" : {
+          "skill" : {
+            "@type" : "g:Int32",
+            "@value" : 4
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 16
+        },
+        "label" : "uses",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "properties" : {
+          "skill" : {
+            "@type" : "g:Int32",
+            "@value" : 5
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 17
+        },
+        "label" : "develops",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 7
+        },
+        "properties" : {
+          "since" : {
+            "@type" : "g:Int32",
+            "@value" : 2010
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 18
+        },
+        "label" : "develops",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 7
+        },
+        "properties" : {
+          "since" : {
+            "@type" : "g:Int32",
+            "@value" : 2011
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 19
+        },
+        "label" : "uses",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 7
+        },
+        "properties" : {
+          "skill" : {
+            "@type" : "g:Int32",
+            "@value" : 5
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 20
+        },
+        "label" : "uses",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 7
+        },
+        "properties" : {
+          "skill" : {
+            "@type" : "g:Int32",
+            "@value" : 4
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 21
+        },
+        "label" : "develops",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 8
+        },
+        "properties" : {
+          "since" : {
+            "@type" : "g:Int32",
+            "@value" : 2012
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 22
+        },
+        "label" : "uses",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 8
+        },
+        "properties" : {
+          "skill" : {
+            "@type" : "g:Int32",
+            "@value" : 3
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 23
+        },
+        "label" : "uses",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 8
+        },
+        "properties" : {
+          "skill" : {
+            "@type" : "g:Int32",
+            "@value" : 3
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 24
+        },
+        "label" : "uses",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 9
+        },
+        "properties" : {
+          "skill" : {
+            "@type" : "g:Int32",
+            "@value" : 5
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 25
+        },
+        "label" : "uses",
+        "inVLabel" : "software",
+        "outVLabel" : "person",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 9
+        },
+        "properties" : {
+          "skill" : {
+            "@type" : "g:Int32",
+            "@value" : 3
+          }
+        }
+      }
+    }, {
+      "@type" : "g:Edge",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 26
+        },
+        "label" : "traverses",
+        "inVLabel" : "software",
+        "outVLabel" : "software",
+        "inV" : {
+          "@type" : "g:Int32",
+          "@value" : 11
+        },
+        "outV" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        }
+      }
+    } ]
+  }
+}
+----
+
+Tree
+^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Tree",
+  "@value" : [ {
+    "key" : {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "label" : "person",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 0
+            },
+            "value" : "marko"
+          } ],
+          "location" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 6
+            },
+            "value" : "san diego",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 1997
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 7
+            },
+            "value" : "santa cruz",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 8
+            },
+            "value" : "brussels",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 9
+            },
+            "value" : "santa fe",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          } ]
+        }
+      }
+    },
+    "value" : {
+      "@type" : "g:Tree",
+      "@value" : [ {
+        "key" : {
+          "@type" : "g:Vertex",
+          "@value" : {
+            "id" : {
+              "@type" : "g:Int32",
+              "@value" : 10
+            },
+            "label" : "software",
+            "properties" : {
+              "name" : [ {
+                "id" : {
+                  "@type" : "g:Int64",
+                  "@value" : 4
+                },
+                "value" : "gremlin"
+              } ]
+            }
+          }
+        },
+        "value" : {
+          "@type" : "g:Tree",
+          "@value" : [ {
+            "key" : {
+              "@type" : "g:Vertex",
+              "@value" : {
+                "id" : {
+                  "@type" : "g:Int32",
+                  "@value" : 11
+                },
+                "label" : "software",
+                "properties" : {
+                  "name" : [ {
+                    "id" : {
+                      "@type" : "g:Int64",
+                      "@value" : 5
+                    },
+                    "value" : "tinkergraph"
+                  } ]
+                }
+              }
+            },
+            "value" : {
+              "@type" : "g:Tree",
+              "@value" : [ ]
+            }
+          } ]
+        }
+      } ]
+    }
+  } ]
+}
+----
+
+Vertex
+^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Vertex",
+  "@value" : {
+    "id" : {
+      "@type" : "g:Int32",
+      "@value" : 1
+    },
+    "label" : "person",
+    "properties" : {
+      "name" : [ {
+        "id" : {
+          "@type" : "g:Int64",
+          "@value" : 0
+        },
+        "value" : "marko"
+      } ],
+      "location" : [ {
+        "id" : {
+          "@type" : "g:Int64",
+          "@value" : 6
+        },
+        "value" : "san diego",
+        "properties" : {
+          "startTime" : {
+            "@type" : "g:Int32",
+            "@value" : 1997
+          },
+          "endTime" : {
+            "@type" : "g:Int32",
+            "@value" : 2001
+          }
+        }
+      }, {
+        "id" : {
+          "@type" : "g:Int64",
+          "@value" : 7
+        },
+        "value" : "santa cruz",
+        "properties" : {
+          "startTime" : {
+            "@type" : "g:Int32",
+            "@value" : 2001
+          },
+          "endTime" : {
+            "@type" : "g:Int32",
+            "@value" : 2004
+          }
+        }
+      }, {
+        "id" : {
+          "@type" : "g:Int64",
+          "@value" : 8
+        },
+        "value" : "brussels",
+        "properties" : {
+          "startTime" : {
+            "@type" : "g:Int32",
+            "@value" : 2004
+          },
+          "endTime" : {
+            "@type" : "g:Int32",
+            "@value" : 2005
+          }
+        }
+      }, {
+        "id" : {
+          "@type" : "g:Int64",
+          "@value" : 9
+        },
+        "value" : "santa fe",
+        "properties" : {
+          "startTime" : {
+            "@type" : "g:Int32",
+            "@value" : 2005
+          }
+        }
+      } ]
+    }
+  }
+}
+----
+
+VertexProperty
+^^^^^^^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:VertexProperty",
+  "@value" : {
+    "id" : {
+      "@type" : "g:Int64",
+      "@value" : 0
+    },
+    "value" : "marko",
+    "vertex" : {
+      "@type" : "g:Int32",
+      "@value" : 1
+    },
+    "label" : "name"
+  }
+}
+----
+
+
+Graph Process
+~~~~~~~~~~~~~
+
+Barrier
+^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Barrier",
+  "@value" : "normSack"
+}
+----
+
+Binding
+^^^^^^^
+
+A "Binding" refers to a `Bytecode.Binding`.
+
+[source,json]
+----
+{
+  "@type" : "g:Binding",
+  "@value" : {
+    "key" : "x",
+    "value" : {
+      "@type" : "g:Int32",
+      "@value" : 1
+    }
+  }
+}
+----
+
+Bytecode
+^^^^^^^^
+
+The following `Bytecode` example represents the traversal of `g.V().hasLabel('person').out().in().tree()`. Obviously the serialized `Bytecode` woudl be quite different for the endless variations of commands that could be used together in the Gremlin language.
+
+[source,json]
+----
+{
+  "@type" : "g:Bytecode",
+  "@value" : {
+    "step" : [ [ "V" ], [ "hasLabel", "person" ], [ "out" ], [ "in" ], [ "tree" ] ]
+  }
+}
+----
+
+Cardinality
+^^^^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Cardinality",
+  "@value" : "list"
+}
+----
+
+Column
+^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Column",
+  "@value" : "keys"
+}
+----
+
+Direction
+^^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Direction",
+  "@value" : "OUT"
+}
+----
+
+Operator
+^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Operator",
+  "@value" : "sum"
+}
+----
+
+Order
+^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Order",
+  "@value" : "incr"
+}
+----
+
+Pick
+^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Pick",
+  "@value" : "any"
+}
+----
+
+Pop
+^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Pop",
+  "@value" : "all"
+}
+----
+
+Lambda
+^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Lambda",
+  "@value" : {
+    "script" : "{ it.get() }",
+    "language" : "gremlin-groovy",
+    "arguments" : 1
+  }
+}
+----
+
+Metrics
+^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Metrics",
+  "@value" : {
+    "dur" : {
+      "@type" : "g:Double",
+      "@value" : 100.0
+    },
+    "counts" : {
+      "traverserCount" : {
+        "@type" : "g:Int64",
+        "@value" : 4
+      },
+      "elementCount" : {
+        "@type" : "g:Int64",
+        "@value" : 4
+      }
+    },
+    "name" : "TinkerGraphStep(vertex,[~label.eq(person)])",
+    "annotations" : {
+      "percentDur" : {
+        "@type" : "g:Double",
+        "@value" : 25.0
+      }
+    },
+    "id" : "7.0.0()",
+    "metrics" : [ {
+      "@type" : "g:Metrics",
+      "@value" : {
+        "dur" : {
+          "@type" : "g:Double",
+          "@value" : 100.0
+        },
+        "counts" : {
+          "traverserCount" : {
+            "@type" : "g:Int64",
+            "@value" : 7
+          },
+          "elementCount" : {
+            "@type" : "g:Int64",
+            "@value" : 7
+          }
+        },
+        "name" : "VertexStep(OUT,vertex)",
+        "annotations" : {
+          "percentDur" : {
+            "@type" : "g:Double",
+            "@value" : 25.0
+          }
+        },
+        "id" : "3.0.0()"
+      }
+    } ]
+  }
+}
+----
+
+P
+^
+
+[source,json]
+----
+{
+  "@type" : "g:P",
+  "@value" : {
+    "predicate" : "gt",
+    "value" : {
+      "@type" : "g:Int32",
+      "@value" : 0
+    }
+  }
+}
+----
+
+P and
+^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:P",
+  "@value" : {
+    "predicate" : "and",
+    "value" : [ {
+      "@type" : "g:P",
+      "@value" : {
+        "predicate" : "gt",
+        "value" : {
+          "@type" : "g:Int32",
+          "@value" : 0
+        }
+      }
+    }, {
+      "@type" : "g:P",
+      "@value" : {
+        "predicate" : "lt",
+        "value" : {
+          "@type" : "g:Int32",
+          "@value" : 10
+        }
+      }
+    } ]
+  }
+}
+----
+
+P or
+^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:P",
+  "@value" : {
+    "predicate" : "or",
+    "value" : [ {
+      "@type" : "g:P",
+      "@value" : {
+        "predicate" : "gt",
+        "value" : {
+          "@type" : "g:Int32",
+          "@value" : 0
+        }
+      }
+    }, {
+      "@type" : "g:P",
+      "@value" : {
+        "predicate" : "within",
+        "value" : [ {
+          "@type" : "g:Int32",
+          "@value" : -1
+        }, {
+          "@type" : "g:Int32",
+          "@value" : -10
+        }, {
+          "@type" : "g:Int32",
+          "@value" : -100
+        } ]
+      }
+    } ]
+  }
+}
+----
+
+Scope
+^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Scope",
+  "@value" : "local"
+}
+----
+
+T
+^
+
+[source,json]
+----
+{
+  "@type" : "g:T",
+  "@value" : "label"
+}
+----
+
+TraversalMetrics
+^^^^^^^^^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:TraversalMetrics",
+  "@value" : {
+    "dur" : {
+      "@type" : "g:Double",
+      "@value" : 0.004
+    },
+    "metrics" : [ {
+      "@type" : "g:Metrics",
+      "@value" : {
+        "dur" : {
+          "@type" : "g:Double",
+          "@value" : 100.0
+        },
+        "counts" : {
+          "traverserCount" : {
+            "@type" : "g:Int64",
+            "@value" : 4
+          },
+          "elementCount" : {
+            "@type" : "g:Int64",
+            "@value" : 4
+          }
+        },
+        "name" : "TinkerGraphStep(vertex,[~label.eq(person)])",
+        "annotations" : {
+          "percentDur" : {
+            "@type" : "g:Double",
+            "@value" : 25.0
+          }
+        },
+        "id" : "7.0.0()"
+      }
+    }, {
+      "@type" : "g:Metrics",
+      "@value" : {
+        "dur" : {
+          "@type" : "g:Double",
+          "@value" : 100.0
+        },
+        "counts" : {
+          "traverserCount" : {
+            "@type" : "g:Int64",
+            "@value" : 13
+          },
+          "elementCount" : {
+            "@type" : "g:Int64",
+            "@value" : 13
+          }
+        },
+        "name" : "VertexStep(OUT,vertex)",
+        "annotations" : {
+          "percentDur" : {
+            "@type" : "g:Double",
+            "@value" : 25.0
+          }
+        },
+        "id" : "2.0.0()"
+      }
+    }, {
+      "@type" : "g:Metrics",
+      "@value" : {
+        "dur" : {
+          "@type" : "g:Double",
+          "@value" : 100.0
+        },
+        "counts" : {
+          "traverserCount" : {
+            "@type" : "g:Int64",
+            "@value" : 7
+          },
+          "elementCount" : {
+            "@type" : "g:Int64",
+            "@value" : 7
+          }
+        },
+        "name" : "VertexStep(OUT,vertex)",
+        "annotations" : {
+          "percentDur" : {
+            "@type" : "g:Double",
+            "@value" : 25.0
+          }
+        },
+        "id" : "3.0.0()"
+      }
+    }, {
+      "@type" : "g:Metrics",
+      "@value" : {
+        "dur" : {
+          "@type" : "g:Double",
+          "@value" : 100.0
+        },
+        "counts" : {
+          "traverserCount" : {
+            "@type" : "g:Int64",
+            "@value" : 1
+          },
+          "elementCount" : {
+            "@type" : "g:Int64",
+            "@value" : 1
+          }
+        },
+        "name" : "TreeStep",
+        "annotations" : {
+          "percentDur" : {
+            "@type" : "g:Double",
+            "@value" : 25.0
+          }
+        },
+        "id" : "4.0.0()"
+      }
+    } ]
+  }
+}
+----
+
+Traverser
+^^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "g:Traverser",
+  "@value" : {
+    "bulk" : {
+      "@type" : "g:Int64",
+      "@value" : 1
+    },
+    "value" : {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "label" : "person",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 0
+            },
+            "value" : "marko"
+          } ],
+          "location" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 6
+            },
+            "value" : "san diego",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 1997
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 7
+            },
+            "value" : "santa cruz",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 8
+            },
+            "value" : "brussels",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 9
+            },
+            "value" : "santa fe",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          } ]
+        }
+      }
+    }
+  }
+}
+----
+
+
+RequestMessage
+~~~~~~~~~~~~~~
+
+Authentication Response
+^^^^^^^^^^^^^^^^^^^^^^^
+
+The following `RequestMessage` is an example of the response that should be made to a SASL-based authentication challenge.
+
+[source,json]
+----
+{
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
+  "op" : "authentication",
+  "processor" : "",
+  "args" : {
+    "saslMechanism" : "PLAIN",
+    "sasl" : "AHN0ZXBocGhlbgBwYXNzd29yZA=="
+  }
+}
+----
+
+Session Eval
+^^^^^^^^^^^^
+
+The following `RequestMessage` is an example of a simple session request for a script evaluation with parameters.
+
+[source,json]
+----
+{
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
+  "op" : "eval",
+  "processor" : "session",
+  "args" : {
+    "gremlin" : "g.V(x)",
+    "language" : "gremlin-groovy",
+    "session" : {
+      "@type" : "g:UUID",
+      "@value" : "41d2e28a-20a4-4ab0-b379-d810dede3786"
+    },
+    "bindings" : {
+      "x" : {
+        "@type" : "g:Int32",
+        "@value" : 1
+      }
+    }
+  }
+}
+----
+
+Session Eval Aliased
+^^^^^^^^^^^^^^^^^^^^
+
+The following `RequestMessage` is an example of a session request for a script evaluation with an alias that binds the `TraversalSource` of "g" to "social".
+
+[source,json]
+----
+{
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
+  "op" : "eval",
+  "processor" : "session",
+  "args" : {
+    "gremlin" : "social.V(x)",
+    "language" : "gremlin-groovy",
+    "aliases" : {
+      "g" : "social"
+    },
+    "session" : {
+      "@type" : "g:UUID",
+      "@value" : "41d2e28a-20a4-4ab0-b379-d810dede3786"
+    },
+    "bindings" : {
+      "x" : {
+        "@type" : "g:Int32",
+        "@value" : 1
+      }
+    }
+  }
+}
+----
+
+Session Close
+^^^^^^^^^^^^^
+
+The following `RequestMessage` is an example of a request to close a session.
+
+[source,json]
+----
+{
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
+  "op" : "close",
+  "processor" : "session",
+  "args" : {
+    "session" : {
+      "@type" : "g:UUID",
+      "@value" : "41d2e28a-20a4-4ab0-b379-d810dede3786"
+    }
+  }
+}
+----
+
+Sessionless Eval
+^^^^^^^^^^^^^^^^
+
+The following `RequestMessage` is an example of a simple sessionless request for a script evaluation with parameters.
+
+[source,json]
+----
+{
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
+  "op" : "eval",
+  "processor" : "",
+  "args" : {
+    "gremlin" : "g.V(x)",
+    "language" : "gremlin-groovy",
+    "bindings" : {
+      "x" : {
+        "@type" : "g:Int32",
+        "@value" : 1
+      }
+    }
+  }
+}
+----
+
+Sessionless Eval Aliased
+^^^^^^^^^^^^^^^^^^^^^^^^
+
+The following `RequestMessage` is an example of a sessionless request for a script evaluation with an alias that binds the `TraversalSource` of "g" to "social".
+
+[source,json]
+----
+{
+  "requestId" : "cb682578-9d92-4499-9ebc-5c6aa73c5397",
+  "op" : "eval",
+  "processor" : "",
+  "args" : {
+    "gremlin" : "social.V(x)",
+    "language" : "gremlin-groovy",
+    "aliases" : {
+      "g" : "social"
+    },
+    "bindings" : {
+      "x" : {
+        "@type" : "g:Int32",
+        "@value" : 1
+      }
+    }
+  }
+}
+----
+
+
+ResponseMessage
+~~~~~~~~~~~~~~~
+
+Authentication Challenge
+^^^^^^^^^^^^^^^^^^^^^^^^
+
+When authentication is enabled, an initial request to the server will result in an authentication challenge. The typical response message will appear as follows, but handling it could be different depending on the SASL implementation (e.g. multiple challenges maybe requested in some cases, but no in the default provided by Gremlin Server).
+
+[source,json]
+----
+{
+  "requestId" : "41d2e28a-20a4-4ab0-b379-d810dede3786",
+  "status" : {
+    "message" : "",
+    "code" : 407,
+    "attributes" : { }
+  },
+  "result" : {
+    "data" : null,
+    "meta" : { }
+  }
+}
+----
+
+Standard Result
+^^^^^^^^^^^^^^^
+
+The following `ResponseMessage` is a typical example of the typical successful response Gremlin Server will return when returning results from a script.
+
+[source,json]
+----
+{
+  "requestId" : "41d2e28a-20a4-4ab0-b379-d810dede3786",
+  "status" : {
+    "message" : "",
+    "code" : 200,
+    "attributes" : { }
+  },
+  "result" : {
+    "data" : [ {
+      "@type" : "g:Vertex",
+      "@value" : {
+        "id" : {
+          "@type" : "g:Int32",
+          "@value" : 1
+        },
+        "label" : "person",
+        "properties" : {
+          "name" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 0
+            },
+            "value" : "marko"
+          } ],
+          "location" : [ {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 6
+            },
+            "value" : "san diego",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 1997
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 7
+            },
+            "value" : "santa cruz",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2001
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 8
+            },
+            "value" : "brussels",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2004
+              },
+              "endTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          }, {
+            "id" : {
+              "@type" : "g:Int64",
+              "@value" : 9
+            },
+            "value" : "santa fe",
+            "properties" : {
+              "startTime" : {
+                "@type" : "g:Int32",
+                "@value" : 2005
+              }
+            }
+          } ]
+        }
+      }
+    } ],
+    "meta" : { }
+  }
+}
+----
+
+
+Extended
+~~~~~~~~
+
+Note that the "extended" types require the addition of the separate `GraphSONXModuleV2d0` module as follows:
+
+[source,java]
+----
+mapper = GraphSONMapper.build().
+                        typeInfo(TypeInfo.PARTIAL_TYPES).
+                        addCustomModule(GraphSONXModuleV2d0.build().create(false)).
+                        version(GraphSONVersion.V2_0).create().createMapper()
+----
+
+BigDecimal
+^^^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "gx:BigDecimal",
+  "@value" : 123456789987654321123456789987654321
+}
+----
+
+BigInteger
+^^^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "gx:BigInteger",
+  "@value" : 123456789987654321123456789987654321
+}
+----
+
+Byte
+^^^^
+
+[source,json]
+----
+{
+  "@type" : "gx:Byte",
+  "@value" : 1
+}
+----
+
+ByteBuffer
+^^^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "gx:ByteBuffer",
+  "@value" : "c29tZSBieXRlcyBmb3IgeW91"
+}
+----
+
+Char
+^^^^
+
+[source,json]
+----
+{
+  "@type" : "gx:Char",
+  "@value" : "x"
+}
+----
+
+Duration
+^^^^^^^^
+
+The following example is a `Duration` of five days.
+
+[source,json]
+----
+{
+  "@type" : "gx:Duration",
+  "@value" : "PT120H"
+}
+----
+
+InetAddress
+^^^^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "gx:InetAddress",
+  "@value" : "localhost"
+}
+----
+
+Instant
+^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "gx:Instant",
+  "@value" : "2016-12-14T16:39:19.349Z"
+}
+----
+
+LocalDate
+^^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "gx:LocalDate",
+  "@value" : "2016-01-01"
+}
+----
+
+LocalDateTime
+^^^^^^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "gx:LocalDateTime",
+  "@value" : "2016-01-01T12:30"
+}
+----
+
+LocalTime
+^^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "gx:LocalTime",
+  "@value" : "12:30:45"
+}
+----
+
+MonthDay
+^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "gx:MonthDay",
+  "@value" : "--01-01"
+}
+----
+
+OffsetDateTime
+^^^^^^^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "gx:OffsetDateTime",
+  "@value" : "2007-12-03T10:15:30+01:00"
+}
+----
+
+OffsetTime
+^^^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "gx:OffsetTime",
+  "@value" : "10:15:30+01:00"
+}
+----
+
+Period
+^^^^^^
+
+The following example is a `Period` of one year, six months and fifteen days.
+
+[source,json]
+----
+{
+  "@type" : "gx:Period",
+  "@value" : "P1Y6M15D"
+}
+----
+
+Short
+^^^^^
+
+[source,json]
+----
+{
+  "@type" : "gx:Int16",
+  "@value" : 100
+}
+----
+
+Year
+^^^^
+
+The following example is of the `Year` "2016".
+
+[source,json]
+----
+{
+  "@type" : "gx:Year",
+  "@value" : "2016"
+}
+----
+
+YearMonth
+^^^^^^^^^
+
+The following example is a `YearMonth` of "June 2016"
+
+[source,json]
+----
+{
+  "@type" : "gx:YearMonth",
+  "@value" : "2016-06"
+}
+----
+
+ZonedDateTime
+^^^^^^^^^^^^^
+
+[source,json]
+----
+{
+  "@type" : "gx:ZonedDateTime",
+  "@value" : "2016-12-23T12:12:24.000000036+02:00[GMT+02:00]"
+}
+----
+
+ZoneOffset
+^^^^^^^^^^
+
+The following example is a `ZoneOffset` of three hours, six minutes, and nine seconds.
+
+[source,json]
+----
+{
+  "@type" : "gx:ZoneOffset",
+  "@value" : "+03:06:09"
+}
 ----
\ No newline at end of file


[14/31] tinkerpop git commit: introduced a minor bug into CollectingBarrierStep that would only be noticed by asynchrnous traversal execution engines. I noticed it in the GraphActors branch. Also, added a toString() to ProjectedTraverser. CTR.

Posted by sp...@apache.org.
introduced a minor bug into CollectingBarrierStep that would only be noticed by asynchrnous traversal execution engines. I noticed it in the GraphActors branch. Also, added a toString() to ProjectedTraverser. CTR.


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

Branch: refs/heads/TINKERPOP-1565
Commit: 8ad291134d1b9febdae437e855812185b41c73db
Parents: 6002299
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Thu Jan 19 13:01:07 2017 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Thu Jan 19 13:01:07 2017 -0700

----------------------------------------------------------------------
 .../process/traversal/step/util/CollectingBarrierStep.java      | 4 ++--
 .../gremlin/process/traversal/traverser/ProjectedTraverser.java | 5 +++++
 2 files changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/8ad29113/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/CollectingBarrierStep.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/CollectingBarrierStep.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/CollectingBarrierStep.java
index f99201d..8409c9f 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/CollectingBarrierStep.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/util/CollectingBarrierStep.java
@@ -97,8 +97,8 @@ public abstract class CollectingBarrierStep<S> extends AbstractStep<S, S> implem
 
     @Override
     public void addBarrier(final TraverserSet<S> barrier) {
-        this.traverserSet = barrier;
-        this.traverserSet.forEach(traverser -> traverser.setSideEffects(this.getTraversal().getSideEffects()));
+        barrier.forEach(traverser -> traverser.setSideEffects(this.getTraversal().getSideEffects()));
+        this.traverserSet.addAll(barrier);
         this.barrierConsumed = false;
     }
 

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/8ad29113/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java
index 128e377..602f88f 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java
@@ -182,6 +182,11 @@ public final class ProjectedTraverser<T, P> implements Traverser.Admin<T> {
     }
 
     @Override
+    public String toString() {
+        return this.baseTraverser.toString();
+    }
+
+    @Override
     public ProjectedTraverser<T, P> clone() {
         try {
             final ProjectedTraverser<T, P> clone = (ProjectedTraverser<T, P>) super.clone();


[16/31] tinkerpop git commit: first draft of new updates to GraphSON 2.0. The object models are concise, all the elements/properties are attachable. Really wasn't that much of a headache to change -- a few test cases in Gremlin-Python needed updating and

Posted by sp...@apache.org.
first draft of new updates to GraphSON 2.0. The object models are concise, all the elements/properties are attachable. Really wasn't that much of a headache to change -- a few test cases in Gremlin-Python needed updating and in GraphSONMessageSerializerV2d0Test. The corresponding ticket in JIRA has a comment with the pretty print JSON of the various elements/properties.


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

Branch: refs/heads/TINKERPOP-1565
Commit: 580970064c5dad70aa6635960f288862e07ac5d0
Parents: bfe432e
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Tue Nov 29 11:32:21 2016 -0700
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Jan 19 15:15:32 2017 -0500

----------------------------------------------------------------------
 .../io/graphson/GraphSONSerializersV2d0.java    | 50 ++++++++++++-------
 .../structure/io/graphson/GraphSONTokens.java   |  2 +-
 .../ser/GraphSONMessageSerializerV2d0Test.java  | 18 +++----
 .../gremlin_python/structure/io/graphson.py     | 52 ++++++++++++++------
 .../jython/tests/structure/io/test_graphson.py  | 12 ++---
 5 files changed, 84 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/58097006/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV2d0.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV2d0.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV2d0.java
index e4cc755..824fc7f 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV2d0.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONSerializersV2d0.java
@@ -100,22 +100,34 @@ class GraphSONSerializersV2d0 {
         }
 
         private void writeProperties(final Vertex vertex, final JsonGenerator jsonGenerator) throws IOException {
-            if (vertex.keys().size() == 0)
+            if (vertex.keys().isEmpty())
                 return;
             jsonGenerator.writeFieldName(GraphSONTokens.PROPERTIES);
             jsonGenerator.writeStartObject();
 
             final List<String> keys = normalize ?
                     IteratorUtils.list(vertex.keys().iterator(), Comparator.naturalOrder()) : new ArrayList<>(vertex.keys());
-            for (String key : keys) {
+            for (final String key : keys) {
                 final Iterator<VertexProperty<Object>> vertexProperties = normalize ?
                         IteratorUtils.list(vertex.properties(key), Comparators.PROPERTY_COMPARATOR).iterator() : vertex.properties(key);
                 if (vertexProperties.hasNext()) {
                     jsonGenerator.writeFieldName(key);
-
                     jsonGenerator.writeStartArray();
                     while (vertexProperties.hasNext()) {
-                        jsonGenerator.writeObject(vertexProperties.next());
+                        final VertexProperty<?> vertexProperty = vertexProperties.next();
+                        jsonGenerator.writeStartObject();
+                        jsonGenerator.writeObjectField(GraphSONTokens.ID, vertexProperty.id());
+                        jsonGenerator.writeObjectField(GraphSONTokens.VALUE, vertexProperty.value());
+                        if (!vertexProperty.keys().isEmpty()) {
+                            jsonGenerator.writeObjectFieldStart(GraphSONTokens.PROPERTIES);
+                            final Iterator<Property<?>> properties = (Iterator) vertexProperty.properties();
+                            while (properties.hasNext()) {
+                                final Property<?> property = properties.next();
+                                jsonGenerator.writeObjectField(property.key(), property.value());
+                            }
+                            jsonGenerator.writeEndObject();
+                        }
+                        jsonGenerator.writeEndObject();
                     }
                     jsonGenerator.writeEndArray();
                 }
@@ -156,7 +168,6 @@ class GraphSONSerializersV2d0 {
                     IteratorUtils.list(edge.properties(), Comparators.PROPERTY_COMPARATOR).iterator() : edge.properties();
             if (edgeProperties.hasNext()) {
                 jsonGenerator.writeFieldName(GraphSONTokens.PROPERTIES);
-
                 jsonGenerator.writeStartObject();
                 while (edgeProperties.hasNext()) {
                     final Property<?> property = edgeProperties.next();
@@ -181,24 +192,19 @@ class GraphSONSerializersV2d0 {
             jsonGenerator.writeObjectField(GraphSONTokens.VALUE, property.value());
             if (property.element() instanceof VertexProperty) {
                 VertexProperty vertexProperty = (VertexProperty) property.element();
-                jsonGenerator.writeObjectFieldStart(GraphSONTokens.ELEMENT);
-                jsonGenerator.writeStringField(GraphSONTokens.VALUETYPE, "g:VertexProperty");
-                jsonGenerator.writeObjectFieldStart(GraphSONTokens.VALUEPROP);
+                jsonGenerator.writeObjectFieldStart(GraphSONTokens.VERTEX_PROPERTY);
                 jsonGenerator.writeObjectField(GraphSONTokens.ID, vertexProperty.id());
                 jsonGenerator.writeStringField(GraphSONTokens.LABEL, vertexProperty.label());
+                jsonGenerator.writeObjectField(GraphSONTokens.VALUE, vertexProperty.value());
                 jsonGenerator.writeObjectField(GraphSONTokens.VERTEX, vertexProperty.element().id());
                 jsonGenerator.writeEndObject();
-                jsonGenerator.writeEndObject();
             } else if (property.element() instanceof Edge) {
                 Edge edge = (Edge) property.element();
-                jsonGenerator.writeObjectFieldStart(GraphSONTokens.ELEMENT);
-                jsonGenerator.writeStringField(GraphSONTokens.VALUETYPE, "g:Edge");
-                jsonGenerator.writeObjectFieldStart(GraphSONTokens.VALUEPROP);
+                jsonGenerator.writeObjectFieldStart(GraphSONTokens.EDGE);
                 jsonGenerator.writeObjectField(GraphSONTokens.ID, edge.id());
                 jsonGenerator.writeStringField(GraphSONTokens.LABEL, edge.label());
-                jsonGenerator.writeObjectField(GraphSONTokens.OUT, edge.outVertex().id());
                 jsonGenerator.writeObjectField(GraphSONTokens.IN, edge.inVertex().id());
-                jsonGenerator.writeEndObject();
+                jsonGenerator.writeObjectField(GraphSONTokens.OUT, edge.outVertex().id());
                 jsonGenerator.writeEndObject();
             }
             jsonGenerator.writeEndObject();
@@ -485,9 +491,16 @@ class GraphSONSerializersV2d0 {
 
         @Override
         public Property createObject(final Map<String, Object> propData) {
-            final Object element = propData.get(GraphSONTokens.ELEMENT);
-            return element instanceof Element ? // graphson-non-embedded is treated differently, but since this is a hard coded embedding...
-                    new DetachedProperty<>((String) propData.get(GraphSONTokens.KEY), propData.get(GraphSONTokens.VALUE), (Element) element) :
+            Element element = null;
+            if (propData.containsKey(GraphSONTokens.VERTEX_PROPERTY)) {
+                final Map<String, Object> elementData = (Map<String, Object>) propData.get(GraphSONTokens.VERTEX_PROPERTY);
+                element = new VertexPropertyJacksonDeserializer().createObject(elementData);
+            } else if (propData.containsKey(GraphSONTokens.EDGE)) {
+                final Map<String, Object> elementData = (Map<String, Object>) propData.get(GraphSONTokens.EDGE);
+                element = new EdgeJacksonDeserializer().createObject(elementData);
+            }
+            return null != element ? // graphson-non-embedded is treated differently, but since this is a hard coded embedding...
+                    new DetachedProperty<>((String) propData.get(GraphSONTokens.KEY), propData.get(GraphSONTokens.VALUE), element) :
                     new DetachedProperty<>((String) propData.get(GraphSONTokens.KEY), propData.get(GraphSONTokens.VALUE));
         }
     }
@@ -504,7 +517,8 @@ class GraphSONSerializersV2d0 {
                     new DetachedVertexProperty<>(
                             propData.get(GraphSONTokens.ID),
                             (String) propData.get(GraphSONTokens.LABEL),
-                            propData.get(GraphSONTokens.VALUE), (Map<String, Object>) propData.get(GraphSONTokens.PROPERTIES),
+                            propData.get(GraphSONTokens.VALUE),
+                            (Map<String, Object>) propData.get(GraphSONTokens.PROPERTIES),
                             new DetachedVertex(propData.get(GraphSONTokens.VERTEX), Vertex.DEFAULT_LABEL, null)) :
                     new DetachedVertexProperty<>(
                             propData.get(GraphSONTokens.ID),

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/58097006/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTokens.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTokens.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTokens.java
index 4f804ad..d804f0b 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTokens.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONTokens.java
@@ -37,8 +37,8 @@ public final class GraphSONTokens {
     public static final String EDGE = "edge";
     public static final String EDGES = "edges";
     public static final String VERTEX = "vertex";
+    public static final String VERTEX_PROPERTY = "vertexProperty";
     public static final String VERTICES = "vertices";
-    public static final String ELEMENT = "element";
     public static final String IN = "inV";
     public static final String OUT = "outV";
     public static final String IN_E = "inE";

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/58097006/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV2d0Test.java
----------------------------------------------------------------------
diff --git a/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV2d0Test.java b/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV2d0Test.java
index 13e2e69..4125946 100644
--- a/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV2d0Test.java
+++ b/gremlin-driver/src/test/java/org/apache/tinkerpop/gremlin/driver/ser/GraphSONMessageSerializerV2d0Test.java
@@ -315,17 +315,17 @@ public class GraphSONMessageSerializerV2d0Test {
 
         final JsonNode friendProperties = properties.get("friends");
         assertEquals(1, friendProperties.size());
-        final JsonNode friendsProperty = friendProperties.get(0).get(GraphSONTokens.VALUEPROP);
+        final JsonNode friendsProperty = friendProperties.get(0).get(GraphSONTokens.VALUE);
         assertNotNull(friendsProperty);
-        assertEquals(4, friendsProperty.size());
+        assertEquals(3, friendsProperty.size());
 
-        final String object1 = friendsProperty.get(GraphSONTokens.VALUE).get(0).asText();
+        final String object1 = friendsProperty.get(0).asText();
         assertEquals("x", object1);
 
-        final int object2 = friendsProperty.get(GraphSONTokens.VALUE).get(1).get(GraphSONTokens.VALUEPROP).asInt();
+        final int object2 = friendsProperty.get(1).get(GraphSONTokens.VALUEPROP).asInt();
         assertEquals(5, object2);
 
-        final JsonNode object3 = friendsProperty.get(GraphSONTokens.VALUE).get(2);
+        final JsonNode object3 = friendsProperty.get(2);
         assertEquals(500, object3.get("x").get(GraphSONTokens.VALUEPROP).asInt());
         assertEquals("some", object3.get("y").asText());
     }
@@ -439,7 +439,7 @@ public class GraphSONMessageSerializerV2d0Test {
                 .get(GraphSONTokens.KEY).get(GraphSONTokens.VALUEPROP)
                 .get(GraphSONTokens.PROPERTIES)
                 .get("name")
-                .get(0).get(GraphSONTokens.VALUEPROP)
+                .get(0)
                 .get(GraphSONTokens.VALUE).asText());
 
         //check the leafs
@@ -450,7 +450,7 @@ public class GraphSONMessageSerializerV2d0Test {
                 .get(GraphSONTokens.KEY).get(GraphSONTokens.VALUEPROP)
                 .get(GraphSONTokens.PROPERTIES)
                 .get("name")
-                .get(0).get(GraphSONTokens.VALUEPROP)
+                .get(0)
                 .get(GraphSONTokens.VALUE).asText());
 
         assertEquals("lop", converted.get(GraphSONTokens.VALUEPROP)
@@ -460,7 +460,7 @@ public class GraphSONMessageSerializerV2d0Test {
                 .get(GraphSONTokens.KEY).get(GraphSONTokens.VALUEPROP)
                 .get(GraphSONTokens.PROPERTIES)
                 .get("name")
-                .get(0).get(GraphSONTokens.VALUEPROP)
+                .get(0)
                 .get(GraphSONTokens.VALUE).asText());
 
         assertEquals("josh", converted.get(GraphSONTokens.VALUEPROP)
@@ -470,7 +470,7 @@ public class GraphSONMessageSerializerV2d0Test {
                 .get(GraphSONTokens.KEY).get(GraphSONTokens.VALUEPROP)
                 .get(GraphSONTokens.PROPERTIES)
                 .get("name")
-                .get(0).get(GraphSONTokens.VALUEPROP)
+                .get(0)
                 .get(GraphSONTokens.VALUE).asText());
     }
 

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/58097006/gremlin-python/src/main/jython/gremlin_python/structure/io/graphson.py
----------------------------------------------------------------------
diff --git a/gremlin-python/src/main/jython/gremlin_python/structure/io/graphson.py b/gremlin-python/src/main/jython/gremlin_python/structure/io/graphson.py
index a3276e2..8795ff1 100644
--- a/gremlin-python/src/main/jython/gremlin_python/structure/io/graphson.py
+++ b/gremlin-python/src/main/jython/gremlin_python/structure/io/graphson.py
@@ -208,20 +208,26 @@ class PropertySerializer(_GraphSONTypeIO):
 
     @classmethod
     def dictify(cls, property, writer):
-        elementDict = writer.toDict(property.element)
-        if elementDict is not None:
-            valueDict = elementDict["@value"]
-            if "outVLabel" in valueDict:
-                del valueDict["outVLabel"]
-            if "inVLabel" in valueDict:
-                del valueDict["inVLabel"]
-            if "properties" in valueDict:
-                del valueDict["properties"]
-            if "value" in valueDict:
-                del valueDict["value"]
-        return GraphSONUtil.typedValue("Property", {"key": writer.toDict(property.key),
-                                                    "value": writer.toDict(property.value),
-                                                    "element": writer.toDict(elementDict)})
+        element = property.element
+        elementDict = {}
+        if element is not None:
+            elementDict["id"] = element.id
+            elementDict["label"] = element.label
+            if isinstance(element, VertexProperty):
+                elementDict["value"] = element.value
+                elementDict["vertex"] = element.vertex.id
+                return GraphSONUtil.typedValue("Property", {"key": writer.toDict(property.key),
+                                                            "value": writer.toDict(property.value),
+                                                            "vertexProperty": writer.toDict(elementDict)})
+            elif isinstance(element, Edge):
+                elementDict["outV"] = element.outV.id
+                elementDict["inV"] = element.inV.id
+                return GraphSONUtil.typedValue("Property", {"key": writer.toDict(property.key),
+                                                            "value": writer.toDict(property.value),
+                                                            "edge": writer.toDict(elementDict)})
+        else:
+            return GraphSONUtil.typedValue("Property", {"key": writer.toDict(property.key),
+                                                        "value": writer.toDict(property.value)})
 
 
 class TraversalStrategySerializer(_GraphSONTypeIO):
@@ -386,8 +392,22 @@ class PropertyDeserializer(_GraphSONTypeIO):
 
     @classmethod
     def objectify(cls, d, reader):
-        element = reader.toObject(d["element"]) if "element" in d else None
-        return Property(d["key"], reader.toObject(d["value"]), element)
+        if "edge" in d:
+            edge = reader.toObject(d["edge"])
+            return Property(d["key"], reader.toObject(d["value"]),
+                            Edge(edge["id"],
+                                 Vertex(edge["outV"]),
+                                 edge["label"],
+                                 Vertex(edge["inV"])))
+        elif "vertexProperty" in d:
+            vertex_property = reader.toObject(d["vertexProperty"])
+            return Property(d["key"], reader.toObject(d["value"]),
+                            VertexProperty(vertex_property["id"],
+                                           vertex_property["label"],
+                                           vertex_property["value"],
+                                           Vertex(vertex_property["vertex"])))
+        else:
+            return Property(d["key"], reader.toObject(d["value"]), None)
 
 
 class PathDeserializer(_GraphSONTypeIO):

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/58097006/gremlin-python/src/main/jython/tests/structure/io/test_graphson.py
----------------------------------------------------------------------
diff --git a/gremlin-python/src/main/jython/tests/structure/io/test_graphson.py b/gremlin-python/src/main/jython/tests/structure/io/test_graphson.py
index f01100d..7e034c9 100644
--- a/gremlin-python/src/main/jython/tests/structure/io/test_graphson.py
+++ b/gremlin-python/src/main/jython/tests/structure/io/test_graphson.py
@@ -111,7 +111,7 @@ class TestGraphSONReader(TestCase):
         assert edge.outV == Vertex("y", "vertex")
         ##
         property = self.graphson_reader.readObject("""
-        {"@type":"g:Property", "@value":{"key":"aKey","value":{"@type":"g:Int64","@value":17},"element":{"@type":"g:Edge","@value":{"id":{"@type":"g:Int64","@value":122},"label":"knows","inV":"x","outV":"y","inVLabel":"xLab"}}}}""")
+        {"@type":"g:Property", "@value":{"key":"aKey","value":{"@type":"g:Int64","@value":17},"edge":{"id":{"@type":"g:Int64","@value":122},"label":"knows","inV":"x","outV":"y"}}}""")
         # print property
         assert isinstance(property, Property)
         assert "aKey" == property.key
@@ -208,11 +208,11 @@ class TestGraphSONWriter(TestCase):
             self.graphson_writer.writeObject(VertexProperty("blah", "keyA", True, Vertex("stephen"))))
 
         assert {"@type": "g:Property",
-                "@value": {"key": "name", "value": "marko", "element": {"@type": "g:VertexProperty",
-                                                                        "@value": {
-                                                                            "vertex": "vertexId",
-                                                                            "id": "anId",
-                                                                            "label": "aKey"}}}} == json.loads(
+                "@value": {"key": "name", "value": "marko", "vertexProperty": {
+                    "vertex": "vertexId",
+                    "id": "anId",
+                    "label": "aKey",
+                    "value": {"@type": "g:Int32", "@value": 21345}}}} == json.loads(
             self.graphson_writer.writeObject(
                 Property("name", "marko", VertexProperty("anId", "aKey", 21345, Vertex("vertexId")))))
 


[26/31] tinkerpop git commit: TINKERPOP-1414 Made GraphSON 3.0 the default serialization format.

Posted by sp...@apache.org.
TINKERPOP-1414 Made GraphSON 3.0 the default serialization format.

This was previously GraphSON 2.0 but recent consensus has this now at 3.0 since 2.0 will not be altered to support breaking change.


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

Branch: refs/heads/TINKERPOP-1565
Commit: aec7fd1ac1cc56375d52d358c14f57816b466b67
Parents: 9dc30cd
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Thu Jan 19 10:28:38 2017 -0500
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Jan 19 15:15:33 2017 -0500

----------------------------------------------------------------------
 gremlin-server/conf/gremlin-server-classic.yaml |   4 +-
 .../conf/gremlin-server-modern-py.yaml          |   4 +-
 .../conf/gremlin-server-modern-readonly.yaml    |   6 +-
 gremlin-server/conf/gremlin-server-modern.yaml  |   4 +-
 gremlin-server/conf/gremlin-server-neo4j.yaml   |   6 +-
 .../conf/gremlin-server-rest-modern.yaml        |   6 +-
 .../conf/gremlin-server-rest-secure.yaml        |   6 +-
 gremlin-server/conf/gremlin-server-secure.yaml  |   6 +-
 gremlin-server/conf/gremlin-server-spark.yaml   |   6 +-
 gremlin-server/conf/gremlin-server.yaml         |   4 +-
 .../server/GremlinServerHttpIntegrateTest.java  |  51 ++++-
 .../remote/gremlin-server-integration.yaml      |   7 +-
 .../server/gremlin-server-integration.yaml      |   7 +-
 .../tinkergraph/structure/TinkerGraph.java      |   2 +-
 .../structure/TinkerIoRegistryV3d0.java         | 217 +++++++++++++++++++
 15 files changed, 297 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/aec7fd1a/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 da9068f..ed256b5 100644
--- a/gremlin-server/conf/gremlin-server-classic.yaml
+++ b/gremlin-server/conf/gremlin-server-classic.yaml
@@ -31,8 +31,8 @@ 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV1d0] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0]  }} # application/vnd.gremlin-v2.0+json
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}         # application/json,application/vnd.gremlin-v3.0+json
 metrics: {
   slf4jReporter: {enabled: true, interval: 180000}}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/aec7fd1a/gremlin-server/conf/gremlin-server-modern-py.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/conf/gremlin-server-modern-py.yaml b/gremlin-server/conf/gremlin-server-modern-py.yaml
index d8583ee..d030832 100644
--- a/gremlin-server/conf/gremlin-server-modern-py.yaml
+++ b/gremlin-server/conf/gremlin-server-modern-py.yaml
@@ -46,8 +46,8 @@ 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV1d0] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0]  }} # application/vnd.gremlin-v2.0+json
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}         # application/json,application/vnd.gremlin-v3.0+json
 metrics: {
   slf4jReporter: {enabled: true, interval: 180000}}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/aec7fd1a/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 18f7ca4..d4ff8d4 100644
--- a/gremlin-server/conf/gremlin-server-modern-readonly.yaml
+++ b/gremlin-server/conf/gremlin-server-modern-readonly.yaml
@@ -31,9 +31,9 @@ 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV1d0] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0]  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}         # application/json,application/vnd.gremlin-v3.0+json
 metrics: {
   slf4jReporter: {enabled: true, interval: 180000}}
 strictTransactionManagement: false

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/aec7fd1a/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 c67f20f..08096f6 100644
--- a/gremlin-server/conf/gremlin-server-modern.yaml
+++ b/gremlin-server/conf/gremlin-server-modern.yaml
@@ -31,8 +31,8 @@ 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV1d0] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0]  }} # application/vnd.gremlin-v2.0+json
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}         # application/json,application/vnd.gremlin-v3.0+json
 metrics: {
   slf4jReporter: {enabled: true, interval: 180000}}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/aec7fd1a/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 6aac0f2..1d4003e 100644
--- a/gremlin-server/conf/gremlin-server-neo4j.yaml
+++ b/gremlin-server/conf/gremlin-server-neo4j.yaml
@@ -42,9 +42,9 @@ 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV1d0] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0]  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}         # application/json,application/vnd.gremlin-v3.0+json
 processors:
   - { className: org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor, config: { sessionTimeout: 28800000 }}
   - { className: org.apache.tinkerpop.gremlin.server.op.traversal.TraversalOpProcessor, config: { cacheExpirationTime: 600000, cacheMaxSize: 1000 }}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/aec7fd1a/gremlin-server/conf/gremlin-server-rest-modern.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/conf/gremlin-server-rest-modern.yaml b/gremlin-server/conf/gremlin-server-rest-modern.yaml
index 1fa23f9..e39e497 100644
--- a/gremlin-server/conf/gremlin-server-rest-modern.yaml
+++ b/gremlin-server/conf/gremlin-server-rest-modern.yaml
@@ -29,9 +29,9 @@ scriptEngines: {
     staticImports: [java.lang.Math.PI],
     scripts: [scripts/generate-modern.groovy]}}
 serializers:
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV1d0] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0]  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}         # application/json,application/vnd.gremlin-v3.0+json
 metrics: {
   slf4jReporter: {enabled: true, interval: 180000}}
 strictTransactionManagement: false

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/aec7fd1a/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 8ed8f41..e3d9330 100644
--- a/gremlin-server/conf/gremlin-server-rest-secure.yaml
+++ b/gremlin-server/conf/gremlin-server-rest-secure.yaml
@@ -42,9 +42,9 @@ 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.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV1d0] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0]  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}         # application/json,application/vnd.gremlin-v3.0+json
 processors:
   - { className: org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor, config: { sessionTimeout: 28800000 }}
   - { className: org.apache.tinkerpop.gremlin.server.op.traversal.TraversalOpProcessor, config: { cacheExpirationTime: 600000, cacheMaxSize: 1000 }}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/aec7fd1a/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 14f7034..e4b6049 100644
--- a/gremlin-server/conf/gremlin-server-secure.yaml
+++ b/gremlin-server/conf/gremlin-server-secure.yaml
@@ -45,9 +45,9 @@ 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV1d0] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0]  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}         # application/json,application/vnd.gremlin-v3.0+json
 processors:
   - { className: org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor, config: { sessionTimeout: 28800000 }}
   - { className: org.apache.tinkerpop.gremlin.server.op.traversal.TraversalOpProcessor, config: { cacheExpirationTime: 600000, cacheMaxSize: 1000 }}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/aec7fd1a/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 3c198f5..2335c31 100644
--- a/gremlin-server/conf/gremlin-server-spark.yaml
+++ b/gremlin-server/conf/gremlin-server-spark.yaml
@@ -55,9 +55,9 @@ 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}         # application/json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV1d0] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0]  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}         # application/json,application/vnd.gremlin-v3.0+json
 processors:
   - { className: org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor, config: { sessionTimeout: 28800000 }}
   - { className: org.apache.tinkerpop.gremlin.server.op.traversal.TraversalOpProcessor, config: { cacheExpirationTime: 600000, cacheMaxSize: 1000 }}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/aec7fd1a/gremlin-server/conf/gremlin-server.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/conf/gremlin-server.yaml b/gremlin-server/conf/gremlin-server.yaml
index 3f3fc00..a468096 100644
--- a/gremlin-server/conf/gremlin-server.yaml
+++ b/gremlin-server/conf/gremlin-server.yaml
@@ -32,8 +32,8 @@ 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: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }} # application/vnd.gremlin-v1.0+json
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV1d0] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0]  }} # application/vnd.gremlin-v2.0+json
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}         # application/json,application/vnd.gremlin-v3.0+json
 processors:
   - { className: org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor, config: { sessionTimeout: 28800000 }}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/aec7fd1a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java
index e46871b..ba9aba8 100644
--- a/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java
+++ b/gremlin-server/src/test/java/org/apache/tinkerpop/gremlin/server/GremlinServerHttpIntegrateTest.java
@@ -20,6 +20,7 @@ package org.apache.tinkerpop.gremlin.server;
 
 import org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV1d0;
 import org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0;
+import org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0;
 import org.apache.tinkerpop.gremlin.server.auth.SimpleAuthenticator;
 import org.apache.tinkerpop.gremlin.server.channel.HttpChannelizer;
 import org.apache.http.Consts;
@@ -80,9 +81,21 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
                 break;
             case "should200OnPOSTWithGraphSON1d0AcceptHeaderDefaultResultToJson":
                 settings.serializers.clear();
-                final Settings.SerializerSettings serializerSettings = new Settings.SerializerSettings();
-                serializerSettings.className = GraphSONMessageSerializerV1d0.class.getName();
-                settings.serializers.add(serializerSettings);
+                final Settings.SerializerSettings serializerSettingsV1 = new Settings.SerializerSettings();
+                serializerSettingsV1.className = GraphSONMessageSerializerV1d0.class.getName();
+                settings.serializers.add(serializerSettingsV1);
+                break;
+            case "should200OnPOSTWithGraphSON2d0AcceptHeaderDefaultResultToJson":
+                settings.serializers.clear();
+                final Settings.SerializerSettings serializerSettingsV2 = new Settings.SerializerSettings();
+                serializerSettingsV2.className = GraphSONMessageSerializerV2d0.class.getName();
+                settings.serializers.add(serializerSettingsV2);
+                break;
+            case "should200OnPOSTWithGraphSON3d0AcceptHeaderDefaultResultToJson":
+                settings.serializers.clear();
+                final Settings.SerializerSettings serializerSettingsV3 = new Settings.SerializerSettings();
+                serializerSettingsV3.className = GraphSONMessageSerializerV3d0.class.getName();
+                settings.serializers.add(serializerSettingsV3);
                 break;
             case "should401OnGETWithNoAuthorizationHeader":
             case "should401OnPOSTWithNoAuthorizationHeader":
@@ -366,7 +379,7 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
             assertEquals("application/json", response.getEntity().getContentType().getValue());
             final String json = EntityUtils.toString(response.getEntity());
             final JsonNode node = mapper.readTree(json);
-            assertEquals("stephen", node.get("result").get("data").get(0).get(GraphSONTokens.VALUEPROP).get("properties").get("name").get(0).get(GraphSONTokens.VALUEPROP).get(GraphSONTokens.VALUE).asText());
+            assertEquals("stephen", node.get("result").get("data").get(0).get(GraphSONTokens.VALUEPROP).get("properties").get("name").get(0).get(GraphSONTokens.VALUE).asText());
         }
     }
 
@@ -804,6 +817,21 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
     }
 
     @Test
+    public void should200OnPOSTWithGraphSON1d0AcceptHeaderDefaultResultToJson() throws Exception {
+        final CloseableHttpClient httpclient = HttpClients.createDefault();
+        final HttpPost httppost = new HttpPost(TestClientFactory.createURLString());
+        httppost.setEntity(new StringEntity("{\"gremlin\":\"1-1\"}", Consts.UTF_8));
+
+        try (final CloseableHttpResponse response = httpclient.execute(httppost)) {
+            assertEquals(200, response.getStatusLine().getStatusCode());
+            assertEquals("application/json", response.getEntity().getContentType().getValue());
+            final String json = EntityUtils.toString(response.getEntity());
+            final JsonNode node = mapper.readTree(json);
+            assertEquals(0, node.get("result").get("data").get(0).asInt());
+        }
+    }
+
+    @Test
     public void should200OnPOSTWithGraphSON2d0AcceptHeaderDefaultResultToJson() throws Exception {
         final CloseableHttpClient httpclient = HttpClients.createDefault();
         final HttpPost httppost = new HttpPost(TestClientFactory.createURLString());
@@ -817,4 +845,19 @@ public class GremlinServerHttpIntegrateTest extends AbstractGremlinServerIntegra
             assertEquals(0, node.get("result").get("data").get(0).asInt());
         }
     }
+
+    @Test
+    public void should200OnPOSTWithGraphSON3d0AcceptHeaderDefaultResultToJson() throws Exception {
+        final CloseableHttpClient httpclient = HttpClients.createDefault();
+        final HttpPost httppost = new HttpPost(TestClientFactory.createURLString());
+        httppost.setEntity(new StringEntity("{\"gremlin\":\"1-1\"}", Consts.UTF_8));
+
+        try (final CloseableHttpResponse response = httpclient.execute(httppost)) {
+            assertEquals(200, response.getStatusLine().getStatusCode());
+            assertEquals("application/json", response.getEntity().getContentType().getValue());
+            final String json = EntityUtils.toString(response.getEntity());
+            final JsonNode node = mapper.readTree(json);
+            assertEquals(0, node.get("result").get("data").get(0).asInt());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/aec7fd1a/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/driver/remote/gremlin-server-integration.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/driver/remote/gremlin-server-integration.yaml b/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/driver/remote/gremlin-server-integration.yaml
index 98cdef0..ca8a0df 100644
--- a/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/driver/remote/gremlin-server-integration.yaml
+++ b/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/driver/remote/gremlin-server-integration.yaml
@@ -35,10 +35,9 @@ serializers:
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { useMapperFromGraph: graph, custom: [groovy.json.JsonBuilder;org.apache.tinkerpop.gremlin.driver.ser.JsonBuilderGryoSerializer]}}
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GryoLiteMessageSerializerV1d0, config: { useMapperFromGraph: graph, custom: [groovy.json.JsonBuilder;org.apache.tinkerpop.gremlin.driver.ser.JsonBuilderGryoSerializer]}}
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { serializeResultToString: true}}
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }}
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }}
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV1d0] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0]  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}         # application/json,application/vnd.gremlin-v3.0+json
 processors:
   - { className: org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor, config: { sessionTimeout: 28800000 }}
 metrics: {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/aec7fd1a/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/server/gremlin-server-integration.yaml
----------------------------------------------------------------------
diff --git a/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/server/gremlin-server-integration.yaml b/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/server/gremlin-server-integration.yaml
index 1c1ef8d..623c35e 100644
--- a/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/server/gremlin-server-integration.yaml
+++ b/gremlin-server/src/test/resources/org/apache/tinkerpop/gremlin/server/gremlin-server-integration.yaml
@@ -31,10 +31,9 @@ serializers:
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { useMapperFromGraph: graph, custom: [groovy.json.JsonBuilder;org.apache.tinkerpop.gremlin.driver.ser.JsonBuilderGryoSerializer]}}
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GryoLiteMessageSerializerV1d0, config: { useMapperFromGraph: graph, custom: [groovy.json.JsonBuilder;org.apache.tinkerpop.gremlin.driver.ser.JsonBuilderGryoSerializer]}}
   - { className: org.apache.tinkerpop.gremlin.driver.ser.GryoMessageSerializerV1d0, config: { serializeResultToString: true}}
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistry] }}
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { useMapperFromGraph: graph  }}
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV2d0, config: { useMapperFromGraph: graph }}
-  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV1d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV1d0] }} # application/vnd.gremlin-v1.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerGremlinV2d0, config: { ioRegistries: [org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerIoRegistryV2d0]  }} # application/vnd.gremlin-v2.0+json
+  - { className: org.apache.tinkerpop.gremlin.driver.ser.GraphSONMessageSerializerV3d0, config: { useMapperFromGraph: graph }}         # application/json,application/vnd.gremlin-v3.0+json
 processors:
   - { className: org.apache.tinkerpop.gremlin.server.op.session.SessionOpProcessor, config: { sessionTimeout: 28800000 }}
 metrics: {

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/aec7fd1a/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraph.java
----------------------------------------------------------------------
diff --git a/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraph.java b/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraph.java
index e47229b..9d2df0a 100644
--- a/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraph.java
+++ b/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerGraph.java
@@ -230,7 +230,7 @@ public final class TinkerGraph implements Graph {
 
     @Override
     public <I extends Io> I io(final Io.Builder<I> builder) {
-        return (I) builder.graph(this).onMapper(mapper -> mapper.addRegistry(TinkerIoRegistryV2d0.instance())).create();
+        return (I) builder.graph(this).onMapper(mapper -> mapper.addRegistry(TinkerIoRegistryV3d0.instance())).create();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/aec7fd1a/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerIoRegistryV3d0.java
----------------------------------------------------------------------
diff --git a/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerIoRegistryV3d0.java b/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerIoRegistryV3d0.java
new file mode 100644
index 0000000..685a8cc
--- /dev/null
+++ b/tinkergraph-gremlin/src/main/java/org/apache/tinkerpop/gremlin/tinkergraph/structure/TinkerIoRegistryV3d0.java
@@ -0,0 +1,217 @@
+/*
+ * 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.tinkergraph.structure;
+
+import org.apache.commons.configuration.BaseConfiguration;
+import org.apache.commons.configuration.Configuration;
+import org.apache.tinkerpop.gremlin.structure.Edge;
+import org.apache.tinkerpop.gremlin.structure.Vertex;
+import org.apache.tinkerpop.gremlin.structure.io.AbstractIoRegistry;
+import org.apache.tinkerpop.gremlin.structure.io.IoRegistry;
+import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONIo;
+import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONTokens;
+import org.apache.tinkerpop.gremlin.structure.io.graphson.TinkerPopJacksonModule;
+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.GryoWriter;
+import org.apache.tinkerpop.gremlin.structure.util.Attachable;
+import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedEdge;
+import org.apache.tinkerpop.gremlin.structure.util.detached.DetachedVertex;
+import org.apache.tinkerpop.shaded.jackson.core.JsonGenerator;
+import org.apache.tinkerpop.shaded.jackson.core.JsonParser;
+import org.apache.tinkerpop.shaded.jackson.core.JsonProcessingException;
+import org.apache.tinkerpop.shaded.jackson.databind.DeserializationContext;
+import org.apache.tinkerpop.shaded.jackson.databind.SerializerProvider;
+import org.apache.tinkerpop.shaded.jackson.databind.deser.std.StdDeserializer;
+import org.apache.tinkerpop.shaded.jackson.databind.ser.std.StdScalarSerializer;
+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.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * An implementation of the {@link IoRegistry} interface that provides serializers with custom configurations for
+ * implementation specific classes that might need to be serialized.  This registry allows a {@link TinkerGraph} to
+ * be serialized directly which is useful for moving small graphs around on the network.
+ * <p/>
+ * Most providers need not implement this kind of custom serializer as they will deal with much larger graphs that
+ * wouldn't be practical to serialize in this fashion.  This is a bit of a special case for TinkerGraph given its
+ * in-memory status.  Typical implementations would create serializers for a complex vertex identifier or a
+ * custom data class like a "geographic point".
+ *
+ * @author Stephen Mallette (http://stephen.genoprime.com)
+ */
+public final class TinkerIoRegistryV3d0 extends AbstractIoRegistry {
+
+    private static final TinkerIoRegistryV3d0 INSTANCE = new TinkerIoRegistryV3d0();
+
+    private TinkerIoRegistryV3d0() {
+        register(GryoIo.class, TinkerGraph.class, new TinkerGraphGryoSerializer());
+        register(GraphSONIo.class, null, new TinkerModuleV2d0());
+    }
+
+    public static TinkerIoRegistryV3d0 instance() {
+        return INSTANCE;
+    }
+
+    /**
+     * Provides a method to serialize an entire {@link TinkerGraph} into itself for Gryo.  This is useful when
+     * shipping small graphs around through Gremlin Server. Reuses the existing Kryo instance for serialization.
+     */
+    final static class TinkerGraphGryoSerializer extends Serializer<TinkerGraph> {
+        @Override
+        public void write(final Kryo kryo, final Output output, final TinkerGraph graph) {
+            try (final ByteArrayOutputStream stream = new ByteArrayOutputStream()) {
+                GryoWriter.build().mapper(() -> kryo).create().writeGraph(stream, graph);
+                final byte[] bytes = stream.toByteArray();
+                output.writeInt(bytes.length);
+                output.write(bytes);
+            } catch (Exception io) {
+                throw new RuntimeException(io);
+            }
+        }
+
+        @Override
+        public TinkerGraph read(final Kryo kryo, final Input input, final Class<TinkerGraph> tinkerGraphClass) {
+            final Configuration conf = new BaseConfiguration();
+            conf.setProperty("gremlin.tinkergraph.defaultVertexPropertyCardinality", "list");
+            final TinkerGraph graph = TinkerGraph.open(conf);
+            final int len = input.readInt();
+            final byte[] bytes = input.readBytes(len);
+            try (final ByteArrayInputStream stream = new ByteArrayInputStream(bytes)) {
+                GryoReader.build().mapper(() -> kryo).create().readGraph(stream, graph);
+            } catch (Exception io) {
+                throw new RuntimeException(io);
+            }
+
+            return graph;
+        }
+    }
+
+    /**
+     * Provides a method to serialize an entire {@link TinkerGraph} into itself for GraphSON. This is useful when
+     * shipping small graphs around through Gremlin Server.
+     */
+    final static class TinkerModuleV2d0 extends TinkerPopJacksonModule {
+        public TinkerModuleV2d0() {
+            super("tinkergraph-2.0");
+            addSerializer(TinkerGraph.class, new TinkerGraphJacksonSerializer());
+            addDeserializer(TinkerGraph.class, new TinkerGraphJacksonDeserializer());
+        }
+
+        @Override
+        public Map<Class, String> getTypeDefinitions() {
+            return new HashMap<Class, String>(){{
+                put(TinkerGraph.class, "graph");
+            }};
+        }
+
+        @Override
+        public String getTypeNamespace() {
+            return "tinker";
+        }
+    }
+
+    /**
+     * Serializes the graph into an edge list format.  Edge list is a better choices than adjacency list (which is
+     * typically standard from the {@link org.apache.tinkerpop.gremlin.structure.io.GraphReader} and {@link org.apache.tinkerpop.gremlin.structure.io.GraphWriter} perspective) in this case because
+     * the use case for this isn't around massive graphs.  The use case is for "small" subgraphs that are being
+     * shipped over the wire from Gremlin Server. Edge list format is a bit easier for non-JVM languages to work
+     * with as a format and doesn't require a cache for loading (as vertex labels are not serialized in adjacency
+     * list).
+     */
+    final static class TinkerGraphJacksonSerializer extends StdScalarSerializer<TinkerGraph> {
+
+        public TinkerGraphJacksonSerializer() {
+            super(TinkerGraph.class);
+        }
+
+        @Override
+        public void serialize(final TinkerGraph graph, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+                throws IOException {
+            jsonGenerator.writeStartObject();
+            jsonGenerator.writeFieldName(GraphSONTokens.VERTICES);
+            jsonGenerator.writeStartArray();
+
+            final Iterator<Vertex> vertices = graph.vertices();
+            while (vertices.hasNext()) {
+                serializerProvider.defaultSerializeValue(vertices.next(), jsonGenerator);
+            }
+
+            jsonGenerator.writeEndArray();
+            jsonGenerator.writeFieldName(GraphSONTokens.EDGES);
+            jsonGenerator.writeStartArray();
+
+            final Iterator<Edge> edges = graph.edges();
+            while (edges.hasNext()) {
+                serializerProvider.defaultSerializeValue(edges.next(), jsonGenerator);
+            }
+
+            jsonGenerator.writeEndArray();
+            jsonGenerator.writeEndObject();
+        }
+    }
+
+    /**
+     * Deserializes the edge list format.
+     */
+    static class TinkerGraphJacksonDeserializer extends StdDeserializer<TinkerGraph> {
+        public TinkerGraphJacksonDeserializer() {
+            super(TinkerGraph.class);
+        }
+
+        @Override
+        public TinkerGraph deserialize(final JsonParser jsonParser, final DeserializationContext deserializationContext) throws IOException, JsonProcessingException {
+            final Configuration conf = new BaseConfiguration();
+            conf.setProperty("gremlin.tinkergraph.defaultVertexPropertyCardinality", "list");
+            final TinkerGraph graph = TinkerGraph.open(conf);
+
+            final List<? extends Edge> edges;
+            final List<? extends Vertex> vertices;
+
+            jsonParser.nextToken();
+            final Map<String, Object> graphData = deserializationContext.readValue(jsonParser, Map.class);
+            vertices = (List<DetachedVertex>) graphData.get(GraphSONTokens.VERTICES);
+            edges = (List<DetachedEdge>) graphData.get(GraphSONTokens.EDGES);
+
+
+            vertices.forEach(e -> {
+                if (e instanceof DetachedVertex) {
+                    ((DetachedVertex)e).attach(Attachable.Method.getOrCreate(graph));
+                }
+            });
+
+            edges.forEach(e -> {
+                if (e instanceof DetachedEdge) {
+                    ((DetachedEdge) e).attach(Attachable.Method.getOrCreate(graph));
+                }
+            });
+
+            return graph;
+        }
+    }
+}


[28/31] tinkerpop git commit: TINKERPOP-1565 Added a few GraphSON 3.0 classes

Posted by sp...@apache.org.
TINKERPOP-1565 Added a few GraphSON 3.0 classes


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

Branch: refs/heads/TINKERPOP-1565
Commit: 7ca3a29826bbafca1733fd34f138b8b8d4252210
Parents: a3515cf
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Wed Jan 18 12:53:22 2017 -0500
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Jan 19 15:15:33 2017 -0500

----------------------------------------------------------------------
 .../structure/io/graphson/GraphSONModule.java   |   6 +-
 .../io/graphson/GraphSONXModuleV3d0.java        | 142 ++++++++
 .../io/graphson/JavaTimeSerializersV3d0.java    | 323 +++++++++++++++++++
 .../io/graphson/JavaUtilSerializersV3d0.java    |  85 +++++
 4 files changed, 553 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7ca3a298/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java
index dceae6b..f4acf12 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONModule.java
@@ -200,7 +200,7 @@ abstract class GraphSONModule extends TinkerPopJacksonModule {
             addSerializer(Tree.class, new GraphSONSerializersV3d0.TreeJacksonSerializer());
 
             // java.util
-            addSerializer(Map.Entry.class, new JavaUtilSerializersV2d0.MapEntryJacksonSerializer());
+            addSerializer(Map.Entry.class, new JavaUtilSerializersV3d0.MapEntryJacksonSerializer());
 
             // need to explicitly add serializers for those types because Jackson doesn't do it at all.
             addSerializer(Integer.class, new GraphSONSerializersV3d0.IntegerGraphSONSerializer());
@@ -238,8 +238,8 @@ abstract class GraphSONModule extends TinkerPopJacksonModule {
             addDeserializer(Tree.class, new GraphSONSerializersV3d0.TreeJacksonDeserializer());
 
             // numbers
-            addDeserializer(Integer.class, new GraphSONSerializersV2d0.IntegerJackonsDeserializer());
-            addDeserializer(Double.class, new GraphSONSerializersV2d0.DoubleJackonsDeserializer());
+            addDeserializer(Integer.class, new GraphSONSerializersV3d0.IntegerJackonsDeserializer());
+            addDeserializer(Double.class, new GraphSONSerializersV3d0.DoubleJackonsDeserializer());
 
             // traversal
             addDeserializer(Bytecode.class, new GraphSONTraversalSerializersV2d0.BytecodeJacksonDeserializer());

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7ca3a298/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONXModuleV3d0.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONXModuleV3d0.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONXModuleV3d0.java
new file mode 100644
index 0000000..777c6c4
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/GraphSONXModuleV3d0.java
@@ -0,0 +1,142 @@
+/*
+ * 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.graphson;
+
+/**
+ * @author Stephen Mallette (http://stephen.genoprime.com)
+ */
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.MonthDay;
+import java.time.OffsetDateTime;
+import java.time.OffsetTime;
+import java.time.Period;
+import java.time.Year;
+import java.time.YearMonth;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Version 3.0 of GraphSON extensions.
+ */
+public final class GraphSONXModuleV3d0 extends GraphSONModule {
+
+    private static final Map<Class, String> TYPE_DEFINITIONS = Collections.unmodifiableMap(
+            new LinkedHashMap<Class, String>() {{
+                put(ByteBuffer.class, "ByteBuffer");
+                put(Short.class, "Int16");
+                put(BigInteger.class, "BigInteger");
+                put(BigDecimal.class, "BigDecimal");
+                put(Byte.class, "Byte");
+                put(Character.class, "Char");
+                put(InetAddress.class, "InetAddress");
+
+                // Time serializers/deserializers
+                put(Duration.class, "Duration");
+                put(Instant.class, "Instant");
+                put(LocalDate.class, "LocalDate");
+                put(LocalDateTime.class, "LocalDateTime");
+                put(LocalTime.class, "LocalTime");
+                put(MonthDay.class, "MonthDay");
+                put(OffsetDateTime.class, "OffsetDateTime");
+                put(OffsetTime.class, "OffsetTime");
+                put(Period.class, "Period");
+                put(Year.class, "Year");
+                put(YearMonth.class, "YearMonth");
+                put(ZonedDateTime.class, "ZonedDateTime");
+                put(ZoneOffset.class, "ZoneOffset");
+            }});
+
+    /**
+     * Constructs a new object.
+     */
+    protected GraphSONXModuleV3d0(final boolean normalize) {
+        super("graphsonx-3.0");
+
+        /////////////////////// SERIALIZERS ////////////////////////////
+
+        // java.time
+        addSerializer(Duration.class, new JavaTimeSerializersV3d0.DurationJacksonSerializer());
+        addSerializer(Instant.class, new JavaTimeSerializersV3d0.InstantJacksonSerializer());
+        addSerializer(LocalDate.class, new JavaTimeSerializersV3d0.LocalDateJacksonSerializer());
+        addSerializer(LocalDateTime.class, new JavaTimeSerializersV3d0.LocalDateTimeJacksonSerializer());
+        addSerializer(LocalTime.class, new JavaTimeSerializersV3d0.LocalTimeJacksonSerializer());
+        addSerializer(MonthDay.class, new JavaTimeSerializersV3d0.MonthDayJacksonSerializer());
+        addSerializer(OffsetDateTime.class, new JavaTimeSerializersV3d0.OffsetDateTimeJacksonSerializer());
+        addSerializer(OffsetTime.class, new JavaTimeSerializersV3d0.OffsetTimeJacksonSerializer());
+        addSerializer(Period.class, new JavaTimeSerializersV3d0.PeriodJacksonSerializer());
+        addSerializer(Year.class, new JavaTimeSerializersV3d0.YearJacksonSerializer());
+        addSerializer(YearMonth.class, new JavaTimeSerializersV3d0.YearMonthJacksonSerializer());
+        addSerializer(ZonedDateTime.class, new JavaTimeSerializersV3d0.ZonedDateTimeJacksonSerializer());
+        addSerializer(ZoneOffset.class, new JavaTimeSerializersV3d0.ZoneOffsetJacksonSerializer());
+
+        /////////////////////// DESERIALIZERS ////////////////////////////
+
+        // java.time
+        addDeserializer(Duration.class, new JavaTimeSerializersV3d0.DurationJacksonDeserializer());
+        addDeserializer(Instant.class, new JavaTimeSerializersV3d0.InstantJacksonDeserializer());
+        addDeserializer(LocalDate.class, new JavaTimeSerializersV3d0.LocalDateJacksonDeserializer());
+        addDeserializer(LocalDateTime.class, new JavaTimeSerializersV3d0.LocalDateTimeJacksonDeserializer());
+        addDeserializer(LocalTime.class, new JavaTimeSerializersV3d0.LocalTimeJacksonDeserializer());
+        addDeserializer(MonthDay.class, new JavaTimeSerializersV3d0.MonthDayJacksonDeserializer());
+        addDeserializer(OffsetDateTime.class, new JavaTimeSerializersV3d0.OffsetDateTimeJacksonDeserializer());
+        addDeserializer(OffsetTime.class, new JavaTimeSerializersV3d0.OffsetTimeJacksonDeserializer());
+        addDeserializer(Period.class, new JavaTimeSerializersV3d0.PeriodJacksonDeserializer());
+        addDeserializer(Year.class, new JavaTimeSerializersV3d0.YearJacksonDeserializer());
+        addDeserializer(YearMonth.class, new JavaTimeSerializersV3d0.YearMonthJacksonDeserializer());
+        addDeserializer(ZonedDateTime.class, new JavaTimeSerializersV3d0.ZonedDateTimeJacksonDeserializer());
+        addDeserializer(ZoneOffset.class, new JavaTimeSerializersV3d0.ZoneOffsetJacksonDeserializer());
+    }
+
+    public static Builder build() {
+        return new Builder();
+    }
+
+    @Override
+    public Map<Class, String> getTypeDefinitions() {
+        return TYPE_DEFINITIONS;
+    }
+
+    @Override
+    public String getTypeNamespace() {
+        return GraphSONTokens.GREMLINX_TYPE_NAMESPACE;
+    }
+
+    public static final class Builder implements GraphSONModuleBuilder {
+
+        private Builder() {
+        }
+
+        @Override
+        public GraphSONModule create(final boolean normalize) {
+            return new GraphSONXModuleV3d0(normalize);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7ca3a298/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/JavaTimeSerializersV3d0.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/JavaTimeSerializersV3d0.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/JavaTimeSerializersV3d0.java
new file mode 100644
index 0000000..1cb75e0
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/JavaTimeSerializersV3d0.java
@@ -0,0 +1,323 @@
+/*
+ * 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.graphson;
+
+import org.apache.tinkerpop.shaded.jackson.core.JsonGenerator;
+import org.apache.tinkerpop.shaded.jackson.core.JsonParser;
+import org.apache.tinkerpop.shaded.jackson.databind.DeserializationContext;
+import org.apache.tinkerpop.shaded.jackson.databind.SerializerProvider;
+import org.apache.tinkerpop.shaded.jackson.databind.deser.std.StdDeserializer;
+import org.apache.tinkerpop.shaded.jackson.databind.jsontype.TypeSerializer;
+import org.apache.tinkerpop.shaded.jackson.databind.ser.std.StdSerializer;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.MonthDay;
+import java.time.OffsetDateTime;
+import java.time.OffsetTime;
+import java.time.Period;
+import java.time.Year;
+import java.time.YearMonth;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+
+/**
+ * GraphSON serializers for classes in {@code java.time.*} for the version 3.0 of GraphSON.
+ */
+final class JavaTimeSerializersV3d0 {
+
+    private JavaTimeSerializersV3d0() {}
+
+    /**
+     * Base class for serializing the {@code java.time.*} to ISO-8061 formats.
+     */
+    static abstract class AbstractJavaTimeSerializer<T> extends StdSerializer<T> {
+
+        public AbstractJavaTimeSerializer(final Class<T> clazz) {
+            super(clazz);
+        }
+
+        @Override
+        public void serialize(final T value, final JsonGenerator gen,
+                              final SerializerProvider serializerProvider) throws IOException {
+            gen.writeString(value.toString());
+        }
+
+        @Override
+        public void serializeWithType(final T value, final JsonGenerator gen,
+                                      final SerializerProvider serializers, final TypeSerializer typeSer) throws IOException {
+            typeSer.writeTypePrefixForScalar(value, gen);
+            gen.writeString(value.toString());
+            typeSer.writeTypeSuffixForScalar(value, gen);
+        }
+    }
+    /**
+     * Base class for serializing the {@code java.time.*} from ISO-8061 formats.
+     */
+    abstract static class AbstractJavaTimeJacksonDeserializer<T> extends StdDeserializer<T> {
+        public AbstractJavaTimeJacksonDeserializer(final Class<T> clazz) {
+            super(clazz);
+        }
+
+        public abstract T parse(final String val);
+
+        @Override
+        public T deserialize(final JsonParser jsonParser, final DeserializationContext deserializationContext) throws IOException {
+            return parse(jsonParser.getText());
+        }
+    }
+
+    final static class DurationJacksonSerializer extends AbstractJavaTimeSerializer<Duration> {
+
+        public DurationJacksonSerializer() {
+            super(Duration.class);
+        }
+    }
+
+    final static class DurationJacksonDeserializer extends AbstractJavaTimeJacksonDeserializer<Duration> {
+        public DurationJacksonDeserializer() {
+            super(Duration.class);
+        }
+
+        @Override
+        public Duration parse(final String val) {
+            return Duration.parse(val);
+        }
+    }
+
+    final static class InstantJacksonSerializer extends AbstractJavaTimeSerializer<Instant> {
+
+        public InstantJacksonSerializer() {
+            super(Instant.class);
+        }
+    }
+
+    final static class InstantJacksonDeserializer extends AbstractJavaTimeJacksonDeserializer<Instant> {
+        public InstantJacksonDeserializer() {
+            super(Instant.class);
+        }
+
+        @Override
+        public Instant parse(final String val) {
+            return Instant.parse(val);
+        }
+    }
+
+    final static class LocalDateJacksonSerializer extends AbstractJavaTimeSerializer<LocalDate> {
+
+        public LocalDateJacksonSerializer() {
+            super(LocalDate.class);
+        }
+    }
+
+    final static class LocalDateJacksonDeserializer extends AbstractJavaTimeJacksonDeserializer<LocalDate> {
+        public LocalDateJacksonDeserializer() {
+            super(LocalDate.class);
+        }
+
+        @Override
+        public LocalDate parse(final String val) {
+            return LocalDate.parse(val);
+        }
+    }
+
+    final static class LocalDateTimeJacksonSerializer extends AbstractJavaTimeSerializer<LocalDateTime> {
+
+        public LocalDateTimeJacksonSerializer() {
+            super(LocalDateTime.class);
+        }
+    }
+
+    final static class LocalDateTimeJacksonDeserializer extends AbstractJavaTimeJacksonDeserializer<LocalDateTime> {
+        public LocalDateTimeJacksonDeserializer() {
+            super(LocalDateTime.class);
+        }
+
+        @Override
+        public LocalDateTime parse(final String val) {
+            return LocalDateTime.parse(val);
+        }
+    }
+
+    final static class LocalTimeJacksonSerializer extends AbstractJavaTimeSerializer<LocalTime> {
+
+        public LocalTimeJacksonSerializer() {
+            super(LocalTime.class);
+        }
+    }
+
+    final static class LocalTimeJacksonDeserializer extends AbstractJavaTimeJacksonDeserializer<LocalTime> {
+        public LocalTimeJacksonDeserializer() {
+            super(LocalTime.class);
+        }
+
+        @Override
+        public LocalTime parse(final String val) {
+            return LocalTime.parse(val);
+        }
+    }
+
+    final static class MonthDayJacksonSerializer extends AbstractJavaTimeSerializer<MonthDay> {
+
+        public MonthDayJacksonSerializer() {
+            super(MonthDay.class);
+        }
+    }
+
+    final static class MonthDayJacksonDeserializer extends AbstractJavaTimeJacksonDeserializer<MonthDay> {
+        public MonthDayJacksonDeserializer() {
+            super(MonthDay.class);
+        }
+
+        @Override
+        public MonthDay parse(final String val) {
+            return MonthDay.parse(val);
+        }
+    }
+
+    final static class OffsetDateTimeJacksonSerializer extends AbstractJavaTimeSerializer<OffsetDateTime> {
+
+        public OffsetDateTimeJacksonSerializer() {
+            super(OffsetDateTime.class);
+        }
+    }
+
+    final static class OffsetDateTimeJacksonDeserializer extends AbstractJavaTimeJacksonDeserializer<OffsetDateTime> {
+        public OffsetDateTimeJacksonDeserializer() {
+            super(OffsetDateTime.class);
+        }
+
+        @Override
+        public OffsetDateTime parse(final String val) {
+            return OffsetDateTime.parse(val);
+        }
+    }
+
+    final static class OffsetTimeJacksonSerializer extends AbstractJavaTimeSerializer<OffsetTime> {
+
+        public OffsetTimeJacksonSerializer() {
+            super(OffsetTime.class);
+        }
+    }
+
+    final static class OffsetTimeJacksonDeserializer extends AbstractJavaTimeJacksonDeserializer<OffsetTime> {
+        public OffsetTimeJacksonDeserializer() {
+            super(OffsetTime.class);
+        }
+
+        @Override
+        public OffsetTime parse(final String val) {
+            return OffsetTime.parse(val);
+        }
+    }
+
+    final static class PeriodJacksonSerializer extends AbstractJavaTimeSerializer<Period> {
+
+        public PeriodJacksonSerializer() {
+            super(Period.class);
+        }
+    }
+
+    final static class PeriodJacksonDeserializer extends AbstractJavaTimeJacksonDeserializer<Period> {
+        public PeriodJacksonDeserializer() {
+            super(Period.class);
+        }
+
+        @Override
+        public Period parse(final String val) {
+            return Period.parse(val);
+        }
+    }
+
+    final static class YearJacksonSerializer extends AbstractJavaTimeSerializer<Year> {
+
+        public YearJacksonSerializer() {
+            super(Year.class);
+        }
+    }
+
+    final static class YearJacksonDeserializer extends AbstractJavaTimeJacksonDeserializer<Year> {
+        public YearJacksonDeserializer() {
+            super(Year.class);
+        }
+
+        @Override
+        public Year parse(final String val) {
+            return Year.parse(val);
+        }
+    }
+
+    final static class YearMonthJacksonSerializer extends AbstractJavaTimeSerializer<YearMonth> {
+
+        public YearMonthJacksonSerializer() {
+            super(YearMonth.class);
+        }
+    }
+
+    final static class YearMonthJacksonDeserializer extends AbstractJavaTimeJacksonDeserializer<YearMonth> {
+        public YearMonthJacksonDeserializer() {
+            super(YearMonth.class);
+        }
+
+        @Override
+        public YearMonth parse(final String val) {
+            return YearMonth.parse(val);
+        }
+    }
+
+    final static class ZonedDateTimeJacksonSerializer extends AbstractJavaTimeSerializer<ZonedDateTime> {
+
+        public ZonedDateTimeJacksonSerializer() {
+            super(ZonedDateTime.class);
+        }
+    }
+
+    final static class ZonedDateTimeJacksonDeserializer extends AbstractJavaTimeJacksonDeserializer<ZonedDateTime> {
+        public ZonedDateTimeJacksonDeserializer() {
+            super(ZonedDateTime.class);
+        }
+
+        @Override
+        public ZonedDateTime parse(final String val) {
+            return ZonedDateTime.parse(val);
+        }
+    }
+
+    final static class ZoneOffsetJacksonSerializer extends AbstractJavaTimeSerializer<ZoneOffset> {
+
+        public ZoneOffsetJacksonSerializer() {
+            super(ZoneOffset.class);
+        }
+    }
+
+    final static class ZoneOffsetJacksonDeserializer extends AbstractJavaTimeJacksonDeserializer<ZoneOffset> {
+        public ZoneOffsetJacksonDeserializer() {
+            super(ZoneOffset.class);
+        }
+
+        @Override
+        public ZoneOffset parse(final String val) {
+            return ZoneOffset.of(val);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/7ca3a298/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/JavaUtilSerializersV3d0.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/JavaUtilSerializersV3d0.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/JavaUtilSerializersV3d0.java
new file mode 100644
index 0000000..816a3f7
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/graphson/JavaUtilSerializersV3d0.java
@@ -0,0 +1,85 @@
+/*
+ * 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.graphson;
+
+import org.apache.tinkerpop.gremlin.structure.Element;
+import org.apache.tinkerpop.shaded.jackson.core.JsonGenerator;
+import org.apache.tinkerpop.shaded.jackson.databind.SerializationFeature;
+import org.apache.tinkerpop.shaded.jackson.databind.SerializerProvider;
+import org.apache.tinkerpop.shaded.jackson.databind.jsontype.TypeSerializer;
+import org.apache.tinkerpop.shaded.jackson.databind.ser.std.StdSerializer;
+
+import java.io.IOException;
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * GraphSON serializers for classes in {@code java.util.*} for the version 3.0 of GraphSON.
+ */
+final class JavaUtilSerializersV3d0 {
+
+    private JavaUtilSerializersV3d0() {}
+
+    final static class MapEntryJacksonSerializer extends StdSerializer<Map.Entry> {
+
+        public MapEntryJacksonSerializer() {
+            super(Map.Entry.class);
+        }
+
+        @Override
+        public void serialize(final Map.Entry entry, final JsonGenerator jsonGenerator, final SerializerProvider serializerProvider)
+                throws IOException {
+            jsonGenerator.writeStartObject();
+            ser(entry, jsonGenerator, serializerProvider);
+            jsonGenerator.writeEndObject();
+        }
+
+        @Override
+        public void serializeWithType(final Map.Entry entry, final JsonGenerator jsonGenerator,
+                                      final SerializerProvider serializerProvider, final TypeSerializer typeSerializer) throws IOException {
+            typeSerializer.writeTypePrefixForObject(entry, jsonGenerator);
+            ser(entry, jsonGenerator, serializerProvider);
+            typeSerializer.writeTypeSuffixForObject(entry, jsonGenerator);
+        }
+
+        private static void ser(final Map.Entry entry, final JsonGenerator jsonGenerator,
+                                final SerializerProvider serializerProvider) throws IOException {
+            // this treatment of keys is consistent with the current GraphSONKeySerializer which extends the
+            // StdKeySerializer
+            final Object key = entry.getKey();
+            final Class cls = key.getClass();
+            String k;
+            if (cls == String.class)
+                k = (String) key;
+            else if (Element.class.isAssignableFrom(cls))
+                k = ((Element) key).id().toString();
+            else if(Date.class.isAssignableFrom(cls)) {
+                if (serializerProvider.isEnabled(SerializationFeature.WRITE_DATE_KEYS_AS_TIMESTAMPS))
+                    k = String.valueOf(((Date) key).getTime());
+                else
+                    k = serializerProvider.getConfig().getDateFormat().format((Date) key);
+            } else if(cls == Class.class)
+                k = ((Class) key).getName();
+            else
+                k = key.toString();
+
+            serializerProvider.defaultSerializeField(k, entry.getValue(), jsonGenerator);
+        }
+    }
+}


[04/31] tinkerpop git commit: updated CHANGELOG and tweaked ProjectTraverser variable naming.

Posted by sp...@apache.org.
updated CHANGELOG and tweaked ProjectTraverser variable naming.


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

Branch: refs/heads/TINKERPOP-1565
Commit: 91e1f50c8b95d295a86cba6f3c9db7a002664233
Parents: b2f0c57
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Wed Jan 18 11:17:05 2017 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Wed Jan 18 11:17:05 2017 -0700

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |  3 +-
 .../traversal/traverser/ProjectedTraverser.java | 69 +++++++++-----------
 2 files changed, 34 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/91e1f50c/CHANGELOG.asciidoc
----------------------------------------------------------------------
diff --git a/CHANGELOG.asciidoc b/CHANGELOG.asciidoc
index 86c6b4f..052257f 100644
--- a/CHANGELOG.asciidoc
+++ b/CHANGELOG.asciidoc
@@ -26,7 +26,8 @@ image::https://raw.githubusercontent.com/apache/tinkerpop/master/docs/static/ima
 TinkerPop 3.2.4 (Release Date: NOT OFFICIALLY RELEASED YET)
 ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
 
-* `OrderGlobalStep` now emits traversers with their `by()`-projections and thus, can move beyond the local star graph.
+* Added `ProjectedTraverser` which wraps a traverser with a `List<Object>` of projected data.
+* `OrderGlobalStep` and `SampleGlobalStep` now emit traversers with their `by()`-projections and thus, can move beyond the local star graph.
 * SASL negotiation supports both a byte array and Base64 encoded bytes as a string for authentication to Gremlin Server.
 * Deprecated `TinkerIoRegistry` replacing it with the more consistently named `TinkerIoRegistryV1d0`.
 * Made error messaging more consistent during result iteration timeouts in Gremlin Server.

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/91e1f50c/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java
index 5cecdc4..128e377 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/traverser/ProjectedTraverser.java
@@ -34,163 +34,158 @@ import java.util.function.Function;
  */
 public final class ProjectedTraverser<T, P> implements Traverser.Admin<T> {
 
-    private Traverser.Admin<T> internal;
+    private Traverser.Admin<T> baseTraverser;
     private List<P> projections;
 
     private ProjectedTraverser() {
         // for serialization
     }
 
-    public ProjectedTraverser(final Traverser.Admin<T> internal, final List<P> projections) {
-        this.internal = internal;
+    public ProjectedTraverser(final Traverser.Admin<T> baseTraverser, final List<P> projections) {
+        this.baseTraverser = baseTraverser;
         this.projections = projections;
     }
 
-
-    public Traverser.Admin<T> getInternal() {
-        return this.internal;
-    }
-
     public List<P> getProjections() {
         return this.projections;
     }
 
     @Override
     public void merge(final Admin<?> other) {
-        this.internal.merge(other);
+        this.baseTraverser.merge(other);
     }
 
     @Override
-    public <R> Admin<R> split(R r, Step<T, R> step) {
-        return new ProjectedTraverser<>(this.internal.split(r, step), this.projections);
+    public <R> Admin<R> split(final R r, final Step<T, R> step) {
+        return new ProjectedTraverser<>(this.baseTraverser.split(r, step), this.projections);
     }
 
     @Override
     public Admin<T> split() {
-        return new ProjectedTraverser<>(this.internal.split(), this.projections);
+        return new ProjectedTraverser<>(this.baseTraverser.split(), this.projections);
     }
 
     @Override
     public void addLabels(final Set<String> labels) {
-        this.internal.addLabels(labels);
+        this.baseTraverser.addLabels(labels);
     }
 
     @Override
     public void keepLabels(final Set<String> labels) {
-        this.internal.keepLabels(labels);
+        this.baseTraverser.keepLabels(labels);
     }
 
     @Override
     public void dropLabels(final Set<String> labels) {
-        this.internal.dropLabels(labels);
+        this.baseTraverser.dropLabels(labels);
     }
 
     @Override
     public void dropPath() {
-        this.internal.dropPath();
+        this.baseTraverser.dropPath();
     }
 
     @Override
     public void set(final T t) {
-        this.internal.set(t);
+        this.baseTraverser.set(t);
     }
 
     @Override
     public void incrLoops(final String stepLabel) {
-        this.internal.incrLoops(stepLabel);
+        this.baseTraverser.incrLoops(stepLabel);
     }
 
     @Override
     public void resetLoops() {
-        this.internal.resetLoops();
+        this.baseTraverser.resetLoops();
     }
 
     @Override
     public String getStepId() {
-        return this.internal.getStepId();
+        return this.baseTraverser.getStepId();
     }
 
     @Override
     public void setStepId(final String stepId) {
-        this.internal.setStepId(stepId);
+        this.baseTraverser.setStepId(stepId);
     }
 
     @Override
     public void setBulk(final long count) {
-        this.internal.setBulk(count);
+        this.baseTraverser.setBulk(count);
     }
 
     @Override
     public Admin<T> detach() {
-        this.internal = this.internal.detach();
+        this.baseTraverser = this.baseTraverser.detach();
         return this;
     }
 
     @Override
     public T attach(final Function<Attachable<T>, T> method) {
-        return this.internal.attach(method);
+        return this.baseTraverser.attach(method);
     }
 
     @Override
     public void setSideEffects(final TraversalSideEffects sideEffects) {
-        this.internal.setSideEffects(sideEffects);
+        this.baseTraverser.setSideEffects(sideEffects);
     }
 
     @Override
     public TraversalSideEffects getSideEffects() {
-        return this.internal.getSideEffects();
+        return this.baseTraverser.getSideEffects();
     }
 
     @Override
     public Set<String> getTags() {
-        return this.internal.getTags();
+        return this.baseTraverser.getTags();
     }
 
     @Override
     public T get() {
-        return this.internal.get();
+        return this.baseTraverser.get();
     }
 
     @Override
     public <S> S sack() {
-        return this.internal.sack();
+        return this.baseTraverser.sack();
     }
 
     @Override
     public <S> void sack(final S object) {
-        this.internal.sack(object);
+        this.baseTraverser.sack(object);
     }
 
     @Override
     public Path path() {
-        return this.internal.path();
+        return this.baseTraverser.path();
     }
 
     @Override
     public int loops() {
-        return this.internal.loops();
+        return this.baseTraverser.loops();
     }
 
     @Override
     public long bulk() {
-        return this.internal.bulk();
+        return this.baseTraverser.bulk();
     }
 
     @Override
     public int hashCode() {
-        return this.internal.hashCode();
+        return this.baseTraverser.hashCode();
     }
 
     @Override
     public boolean equals(final Object object) {
-        return object instanceof ProjectedTraverser && ((ProjectedTraverser) object).internal.equals(this.internal);
+        return object instanceof ProjectedTraverser && ((ProjectedTraverser) object).baseTraverser.equals(this.baseTraverser);
     }
 
     @Override
     public ProjectedTraverser<T, P> clone() {
         try {
             final ProjectedTraverser<T, P> clone = (ProjectedTraverser<T, P>) super.clone();
-            clone.internal = (Traverser.Admin<T>) this.internal.clone();
+            clone.baseTraverser = (Traverser.Admin<T>) this.baseTraverser.clone();
             return clone;
         } catch (final CloneNotSupportedException e) {
             throw new IllegalStateException(e.getMessage(), e);
@@ -198,6 +193,6 @@ public final class ProjectedTraverser<T, P> implements Traverser.Admin<T> {
     }
 
     public static <T> Traverser.Admin<T> tryUnwrap(final Traverser.Admin<T> traverser) {
-        return traverser instanceof ProjectedTraverser ? ((ProjectedTraverser) traverser).getInternal() : traverser;
+        return traverser instanceof ProjectedTraverser ? ((ProjectedTraverser) traverser).baseTraverser : traverser;
     }
 }
\ No newline at end of file


[12/31] tinkerpop git commit: Merge branch 'tp32'

Posted by sp...@apache.org.
Merge branch 'tp32'


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

Branch: refs/heads/TINKERPOP-1565
Commit: 66f1b235a5a72d1c019deef0e7d27ccb0cb077d5
Parents: d1f0889 6002299
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Thu Jan 19 10:17:59 2017 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Thu Jan 19 10:17:59 2017 -0700

----------------------------------------------------------------------
 CHANGELOG.asciidoc                              |   5 +
 .../traversal/step/filter/SampleGlobalStep.java |  19 +-
 .../process/traversal/step/map/GroupStep.java   | 263 +++----------------
 .../traversal/step/map/OrderGlobalStep.java     |  68 +++--
 .../step/sideEffect/GroupSideEffectStep.java    |  50 ++--
 .../step/util/CollectingBarrierStep.java        |  24 +-
 .../ComputerVerificationStrategy.java           |   7 -
 .../traversal/traverser/ProjectedTraverser.java | 198 ++++++++++++++
 .../gremlin/structure/io/gryo/GryoVersion.java  |  10 +-
 .../gremlin/util/function/MultiComparator.java  |  75 ++++++
 .../util/function/MultiComparatorTest.java      |  69 +++++
 .../step/sideEffect/GroovyGroupTest.groovy      |  10 +
 .../traversal/step/sideEffect/GroupTest.java    |  60 ++++-
 13 files changed, 549 insertions(+), 309 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/66f1b235/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
----------------------------------------------------------------------
diff --cc gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
index 806dbf0,e5c5834..fa705f4
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/process/traversal/step/map/OrderGlobalStep.java
@@@ -172,12 -184,12 +184,12 @@@ public final class OrderGlobalStep<S, 
          @Override
          public TraverserSet<S> apply(final TraverserSet<S> setA, final TraverserSet<S> setB) {
              setA.addAll(setB);
 -            if (Long.MAX_VALUE != this.limit && setA.bulkSize() > this.limit) {
 +            if (this.limit != -1 && setA.bulkSize() > this.limit) {
-                 if (this.chainedComparator.isShuffle())
+                 if (this.comparator.isShuffle())
                      setA.shuffle();
                  else
-                     setA.sort(this.chainedComparator);
-                 long counter = 0l;
+                     setA.sort(this.comparator);
+                 long counter = 0L;
                  final Iterator<Traverser.Admin<S>> traversers = setA.iterator();
                  while (traversers.hasNext()) {
                      final Traverser.Admin<S> traverser = traversers.next();

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/66f1b235/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
----------------------------------------------------------------------
diff --cc gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
index 837095b,7818f6b..0657bc0
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
@@@ -100,9 -77,9 +99,10 @@@ import org.apache.tinkerpop.gremlin.str
  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.StarGraphSerializer;
 +import org.apache.tinkerpop.gremlin.util.function.FunctionUtils;
  import org.apache.tinkerpop.gremlin.util.function.HashSetSupplier;
  import org.apache.tinkerpop.gremlin.util.function.Lambda;
+ import org.apache.tinkerpop.gremlin.util.function.MultiComparator;
  import org.apache.tinkerpop.shaded.kryo.KryoSerializable;
  import org.apache.tinkerpop.shaded.kryo.serializers.JavaSerializer;
  import org.javatuples.Pair;
@@@ -454,32 -260,8 +455,33 @@@ public enum GryoVersion 
              add(GryoTypeReg.of(SackFunctions.Barrier.class, 135));
              add(GryoTypeReg.of(TraversalOptionParent.Pick.class, 137));
              add(GryoTypeReg.of(HashSetSupplier.class, 136, new UtilSerializers.HashSetSupplierSerializer()));
+             add(GryoTypeReg.of(MultiComparator.class, 165));   // ***LAST ID***
  
 +            add(GryoTypeReg.of(ConnectiveStrategy.class, 138));
 +            add(GryoTypeReg.of(HaltedTraverserStrategy.class, 139));
 +            add(GryoTypeReg.of(PartitionStrategy.class, 140, new JavaSerializer()));
 +            add(GryoTypeReg.of(SubgraphStrategy.class, 141, new JavaSerializer()));
 +            add(GryoTypeReg.of(VertexProgramStrategy.class, 142, new JavaSerializer()));
 +            add(GryoTypeReg.of(MatchAlgorithmStrategy.class, 143));
 +            add(GryoTypeReg.of(MatchStep.GreedyMatchAlgorithm.class, 144));
 +            add(GryoTypeReg.of(AdjacentToIncidentStrategy.class, 145));
 +            add(GryoTypeReg.of(FilterRankingStrategy.class, 146));
 +            add(GryoTypeReg.of(IdentityRemovalStrategy.class, 147));
 +            add(GryoTypeReg.of(IncidentToAdjacentStrategy.class, 148));
 +            add(GryoTypeReg.of(InlineFilterStrategy.class, 149));
 +            add(GryoTypeReg.of(LazyBarrierStrategy.class, 150));
 +            add(GryoTypeReg.of(MatchPredicateStrategy.class, 151));
 +            add(GryoTypeReg.of(OrderLimitStrategy.class, 152));
 +            add(GryoTypeReg.of(PathProcessorStrategy.class, 153));
 +            add(GryoTypeReg.of(PathRetractionStrategy.class, 154));
 +            add(GryoTypeReg.of(RangeByIsCountStrategy.class, 155));
 +            add(GryoTypeReg.of(RepeatUnrollStrategy.class, 156));
 +            add(GryoTypeReg.of(GraphFilterStrategy.class, 157));
 +            add(GryoTypeReg.of(LambdaRestrictionStrategy.class, 158));
 +            add(GryoTypeReg.of(ReadOnlyStrategy.class, 159));
 +            add(GryoTypeReg.of(MatchStep.CountMatchAlgorithm.class, 160));
 +            add(GryoTypeReg.of(MatchStep.GreedyMatchAlgorithm.class, 164)); // ### LAST_ID
 +
              add(GryoTypeReg.of(TraverserSet.class, 58));
              add(GryoTypeReg.of(Tree.class, 61));
              add(GryoTypeReg.of(HashSet.class, 62));


[13/31] tinkerpop git commit: conflicts in GryoVersion -- repeated values. fixed.

Posted by sp...@apache.org.
conflicts in GryoVersion -- repeated values. fixed.


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

Branch: refs/heads/TINKERPOP-1565
Commit: 1bc66ab06e6eae4afa88eeaddfb6e367d4ba153b
Parents: 66f1b23
Author: Marko A. Rodriguez <ok...@gmail.com>
Authored: Thu Jan 19 10:23:17 2017 -0700
Committer: Marko A. Rodriguez <ok...@gmail.com>
Committed: Thu Jan 19 10:23:17 2017 -0700

----------------------------------------------------------------------
 .../gremlin/structure/io/gryo/GryoVersion.java        | 14 ++++++++------
 1 file changed, 8 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/1bc66ab0/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
index 0657bc0..49bf3f4 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/io/gryo/GryoVersion.java
@@ -270,6 +270,7 @@ public enum GryoVersion {
             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(ProjectedTraverser.class, 168)); // ***LAST ID***
             add(GryoTypeReg.of(DefaultRemoteTraverser.class, 123, new GryoSerializers.DefaultRemoteTraverserSerializer()));
 
             add(GryoTypeReg.of(Bytecode.class, 122, new GryoSerializers.BytecodeSerializer()));
@@ -284,6 +285,7 @@ public enum GryoVersion {
             add(GryoTypeReg.of(SackFunctions.Barrier.class, 135));
             add(GryoTypeReg.of(TraversalOptionParent.Pick.class, 137));
             add(GryoTypeReg.of(HashSetSupplier.class, 136, new UtilSerializers.HashSetSupplierSerializer()));
+            add(GryoTypeReg.of(MultiComparator.class, 167));
 
             add(GryoTypeReg.of(ConnectiveStrategy.class, 138));
             add(GryoTypeReg.of(HaltedTraverserStrategy.class, 139));
@@ -340,13 +342,13 @@ public enum GryoVersion {
             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(GroupStep.GroupBiOperator.class, 117));
             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(OrderGlobalStep.OrderBiOperator.class, 118));
             add(GryoTypeReg.of(ProfileStep.ProfileBiOperator.class, 119));
 
             // placeholder serializers for classes that don't live here in core. this will allow them to be used if
@@ -355,7 +357,7 @@ public enum GryoVersion {
             tryAddDynamicType(this, "org.apache.tinkerpop.gremlin.driver.message.RequestMessage",
                     "org.apache.tinkerpop.gremlin.driver.ser.RequestMessageGryoSerializer", 165);
             tryAddDynamicType(this, "org.apache.tinkerpop.gremlin.driver.message.ResponseMessage",
-                    "org.apache.tinkerpop.gremlin.driver.ser.ResponseMessageGryoSerializer", 166);  // ### LAST_ID
+                    "org.apache.tinkerpop.gremlin.driver.ser.ResponseMessageGryoSerializer", 166);
         }};
     }
 
@@ -440,7 +442,7 @@ public enum GryoVersion {
             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(ProjectedTraverser.class, 164));
+            add(GryoTypeReg.of(ProjectedTraverser.class, 165));
             add(GryoTypeReg.of(DefaultRemoteTraverser.class, 123, new GryoSerializers.DefaultRemoteTraverserSerializer()));
 
             add(GryoTypeReg.of(Bytecode.class, 122, new GryoSerializers.BytecodeSerializer()));
@@ -455,7 +457,7 @@ public enum GryoVersion {
             add(GryoTypeReg.of(SackFunctions.Barrier.class, 135));
             add(GryoTypeReg.of(TraversalOptionParent.Pick.class, 137));
             add(GryoTypeReg.of(HashSetSupplier.class, 136, new UtilSerializers.HashSetSupplierSerializer()));
-            add(GryoTypeReg.of(MultiComparator.class, 165));   // ***LAST ID***
+            add(GryoTypeReg.of(MultiComparator.class, 166));   // ***LAST ID***
 
             add(GryoTypeReg.of(ConnectiveStrategy.class, 138));
             add(GryoTypeReg.of(HaltedTraverserStrategy.class, 139));
@@ -480,7 +482,7 @@ public enum GryoVersion {
             add(GryoTypeReg.of(LambdaRestrictionStrategy.class, 158));
             add(GryoTypeReg.of(ReadOnlyStrategy.class, 159));
             add(GryoTypeReg.of(MatchStep.CountMatchAlgorithm.class, 160));
-            add(GryoTypeReg.of(MatchStep.GreedyMatchAlgorithm.class, 164)); // ### LAST_ID
+            add(GryoTypeReg.of(MatchStep.GreedyMatchAlgorithm.class, 164));
 
             add(GryoTypeReg.of(TraverserSet.class, 58));
             add(GryoTypeReg.of(Tree.class, 61));


[31/31] tinkerpop git commit: TINKERPOP-1565 Enabled more GraphSON IO tests

Posted by sp...@apache.org.
TINKERPOP-1565 Enabled more GraphSON IO tests

It was now possible to enable more tests as certain changes that were made to master on the GraphSON 2.0 format could be reverted and then migrated to GraphSON 3.0.


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

Branch: refs/heads/TINKERPOP-1565
Commit: 5e69a51154a79b33cd019000da939a75d29a4489
Parents: 7db10c4
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Thu Jan 12 15:11:19 2017 -0500
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu Jan 19 15:15:33 2017 -0500

----------------------------------------------------------------------
 .../apache/tinkerpop/gremlin/structure/io/Model.java | 15 +++++----------
 1 file changed, 5 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/5e69a511/gremlin-tools/gremlin-io-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Model.java
----------------------------------------------------------------------
diff --git a/gremlin-tools/gremlin-io-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Model.java b/gremlin-tools/gremlin-io-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Model.java
index 605414e..6ce19f5 100644
--- a/gremlin-tools/gremlin-io-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Model.java
+++ b/gremlin-tools/gremlin-io-test/src/main/java/org/apache/tinkerpop/gremlin/structure/io/Model.java
@@ -98,8 +98,6 @@ public class Model {
         final GraphTraversalSource g = graph.traversal();
 
         final Compatibility[] noTypeGraphSONPlusGryo3_2_3 = Compatibilities.with(GryoCompatibility.class).beforeRelease("3.2.4").join(Compatibilities.UNTYPED_GRAPHSON).matchToArray();
-        final Compatibility[] graphsonV2NoType = Compatibilities.with(GraphSONCompatibility.class)
-                .configuredAs(".*no-types").matchToArray();
 
         // IMPORTANT - the "title" or name of the Entry needs to be unique
 
@@ -114,19 +112,16 @@ public class Model {
         addCoreEntry(new java.sql.Timestamp(1481750076295L), "Timestamp", "", noTypeGraphSONPlusGryo3_2_3);
         addCoreEntry(UUID.fromString("41d2e28a-20a4-4ab0-b379-d810dede3786"), "UUID");
 
-        // TODO: remove incompatibilities in Element on GraphSON 2.0
-        // temporary incompatibility in v2 graphson starting at 3.3.0 with Element properties - need to revert some
-        // changes on master (which is what helped start this mess) once this work is merged
-        addGraphStructureEntry(graph.edges().next(), "Edge", "", graphsonV2NoType);
-        addGraphStructureEntry(g.V().out().out().path().next(), "Path", "", graphsonV2NoType);
-        addGraphStructureEntry(graph.edges().next().properties().next(), "Property", "", graphsonV2NoType);
+        addGraphStructureEntry(graph.edges().next(), "Edge", "");
+        addGraphStructureEntry(g.V().out().out().path().next(), "Path", "");
+        addGraphStructureEntry(graph.edges().next().properties().next(), "Property", "");
         // TODO: missing a stargraph deserializer in graphson v1/v2
         addEntry("Graph Structure", StarGraph.of(graph.vertices().next()), "StarGraph", "", Compatibilities.GRYO_ONLY.match());
         addGraphStructureEntry(graph, "TinkerGraph", "`TinkerGraph` has a custom serializer that is registered as part of the `TinkerIoRegistry`.");
         // TODO: tree has bugs for graphson
         addEntry("Graph Structure", g.V(1).out().out().tree().next(), "Tree", "", Compatibilities.GRYO_ONLY.match());
-        addGraphStructureEntry(graph.vertices().next(), "Vertex", "", graphsonV2NoType);
-        addGraphStructureEntry(graph.vertices().next().properties().next(), "VertexProperty", "", graphsonV2NoType);
+        addGraphStructureEntry(graph.vertices().next(), "Vertex", "");
+        addGraphStructureEntry(graph.vertices().next().properties().next(), "VertexProperty", "");
 
         addGraphProcessEntry(SackFunctions.Barrier.normSack, "Barrier", "", Compatibilities.UNTYPED_GRAPHSON.matchToArray());
         addGraphProcessEntry(new Bytecode.Binding("x", 1), "Binding", "A \"Binding\" refers to a `Bytecode.Binding`.", Compatibilities.UNTYPED_GRAPHSON.matchToArray());