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/05/31 20:06:13 UTC

[04/10] tinkerpop git commit: TINKERPOP-1676 Performance enhancement to graphson serialization

TINKERPOP-1676 Performance enhancement to graphson serialization

Focuses on speeding up serialization of graph elements. Prevent use of generic maps and stream graphson data directly into "detached" elements. Not using intermediate maps cut down on memory usage and a bunch of jackson reflection calls (still not sure what they were for and why they were not made to be more efficient). It did mean some ugly changes to "detached" stuff. Will need to maybe refactor some more, but the basic premise seems to be proven.


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

Branch: refs/heads/master
Commit: 02b007366b434918fe1181f99d80689c1c03684b
Parents: 251f5b7
Author: Stephen Mallette <sp...@genoprime.com>
Authored: Tue May 23 20:13:46 2017 -0400
Committer: Stephen Mallette <sp...@genoprime.com>
Committed: Thu May 25 14:52:30 2017 -0400

----------------------------------------------------------------------
 .../io/graphson/GraphSONSerializersV2d0.java    | 71 ++++++++++----------
 .../structure/util/detached/DetachedEdge.java   | 18 ++++-
 .../util/detached/DetachedElement.java          | 15 ++++-
 .../structure/util/detached/DetachedUtil.java   | 70 +++++++++++++++++++
 .../structure/util/detached/DetachedVertex.java | 19 ++++--
 .../util/detached/DetachedVertexProperty.java   | 14 +++-
 6 files changed, 161 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/02b00736/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 0008d3a..bdf3fe5 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
@@ -36,6 +36,7 @@ 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.DetachedUtil;
 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;
@@ -425,25 +426,26 @@ class GraphSONSerializersV2d0 {
         }
 
         public Vertex deserialize(final JsonParser jsonParser, final DeserializationContext deserializationContext) throws IOException, JsonProcessingException {
-            final JavaType propertiesType = deserializationContext.getConfig().getTypeFactory().constructMapType(HashMap.class, String.class, Object.class);
-
-            Object id = null;
-            String label = null;
-            Map<String, Object> properties = null;
+            final DetachedVertex v = DetachedUtil.newDetachedVertex();
             while (jsonParser.nextToken() != JsonToken.END_OBJECT) {
                 if (jsonParser.getCurrentName().equals(GraphSONTokens.ID)) {
                     jsonParser.nextToken();
-                    id = deserializationContext.readValue(jsonParser, Object.class);
+                    DetachedUtil.setId(v, deserializationContext.readValue(jsonParser, Object.class));
                 } else if (jsonParser.getCurrentName().equals(GraphSONTokens.LABEL)) {
                     jsonParser.nextToken();
-                    label = jsonParser.getText();
+                    DetachedUtil.setLabel(v, jsonParser.getText());
                 } else if (jsonParser.getCurrentName().equals(GraphSONTokens.PROPERTIES)) {
                     jsonParser.nextToken();
-                    properties = deserializationContext.readValue(jsonParser, propertiesType);
+                    while (jsonParser.nextToken() != JsonToken.END_OBJECT) {
+                        jsonParser.nextToken();
+                        while (jsonParser.nextToken() != JsonToken.END_ARRAY) {
+                            DetachedUtil.addProperty(v, (DetachedVertexProperty) deserializationContext.readValue(jsonParser, VertexProperty.class));
+                        }
+                    }
                 }
             }
 
-            return new DetachedVertex(id, label, properties);
+            return v;
         }
 
         @Override
@@ -460,41 +462,41 @@ class GraphSONSerializersV2d0 {
 
         @Override
         public Edge deserialize(final JsonParser jsonParser, final DeserializationContext deserializationContext) throws IOException, JsonProcessingException {
-            final JavaType propertiesType = deserializationContext.getConfig().getTypeFactory().constructMapType(HashMap.class, String.class, Object.class);
-
-            Object id = null;
-            String label = null;
-            Object outVId = null;
-            String outVLabel = null;
-            Object inVId = null;
-            String inVLabel = null;
-            Map<String, Object> properties = null;
+            final DetachedEdge e = DetachedUtil.newDetachedEdge();
+            final DetachedVertex inV = DetachedUtil.newDetachedVertex();
+            final DetachedVertex outV = DetachedUtil.newDetachedVertex();
             while (jsonParser.nextToken() != JsonToken.END_OBJECT) {
                 if (jsonParser.getCurrentName().equals(GraphSONTokens.ID)) {
                     jsonParser.nextToken();
-                    id = deserializationContext.readValue(jsonParser, Object.class);
+                    DetachedUtil.setId(e, deserializationContext.readValue(jsonParser, Object.class));
                 } else if (jsonParser.getCurrentName().equals(GraphSONTokens.LABEL)) {
                     jsonParser.nextToken();
-                    label = jsonParser.getText();
+                    DetachedUtil.setLabel(e, jsonParser.getText());
                 } else if (jsonParser.getCurrentName().equals(GraphSONTokens.OUT)) {
                     jsonParser.nextToken();
-                    outVId = deserializationContext.readValue(jsonParser, Object.class);
+                    DetachedUtil.setId(outV, deserializationContext.readValue(jsonParser, Object.class));
                 } else if (jsonParser.getCurrentName().equals(GraphSONTokens.OUT_LABEL)) {
                     jsonParser.nextToken();
-                    outVLabel = jsonParser.getText();
+                    DetachedUtil.setLabel(outV, jsonParser.getText());
                 } else if (jsonParser.getCurrentName().equals(GraphSONTokens.IN)) {
                     jsonParser.nextToken();
-                    inVId = deserializationContext.readValue(jsonParser, Object.class);
+                    DetachedUtil.setId(inV, deserializationContext.readValue(jsonParser, Object.class));
                 } else if (jsonParser.getCurrentName().equals(GraphSONTokens.IN_LABEL)) {
                     jsonParser.nextToken();
-                    inVLabel = jsonParser.getText();
+                    DetachedUtil.setLabel(inV,jsonParser.getText());
                 } else if (jsonParser.getCurrentName().equals(GraphSONTokens.PROPERTIES)) {
                     jsonParser.nextToken();
-                    properties = deserializationContext.readValue(jsonParser, propertiesType);
+                    while (jsonParser.nextToken() != JsonToken.END_OBJECT) {
+                        jsonParser.nextToken();
+                        DetachedUtil.addProperty(e, (DetachedProperty) deserializationContext.readValue(jsonParser, Property.class));
+                    }
                 }
             }
 
-            return new DetachedEdge(id, label, properties, outVId, outVLabel, inVId, inVLabel);
+            DetachedUtil.setInV(e, inV);
+            DetachedUtil.setOutV(e, outV);
+
+            return e;
         }
 
         @Override
@@ -560,29 +562,28 @@ class GraphSONSerializersV2d0 {
 
         @Override
         public VertexProperty deserialize(final JsonParser jsonParser, final DeserializationContext deserializationContext) throws IOException, JsonProcessingException {
+            final DetachedVertexProperty vp = DetachedUtil.newDetachedVertexProperty();
             final JavaType propertiesType = deserializationContext.getConfig().getTypeFactory().constructMapType(HashMap.class, String.class, Object.class);
 
-            Object id = null;
-            String label = null;
-            Object value = null;
-            Map<String, Object> properties = null;
+            Map<String, Object> properties;
             while (jsonParser.nextToken() != JsonToken.END_OBJECT) {
                 if (jsonParser.getCurrentName().equals(GraphSONTokens.ID)) {
                     jsonParser.nextToken();
-                    id = deserializationContext.readValue(jsonParser, Object.class);
+                    DetachedUtil.setId(vp, deserializationContext.readValue(jsonParser, Object.class));
                 } else if (jsonParser.getCurrentName().equals(GraphSONTokens.LABEL)) {
                     jsonParser.nextToken();
-                    label = jsonParser.getText();
+                    DetachedUtil.setLabel(vp, jsonParser.getText());
                 } else if (jsonParser.getCurrentName().equals(GraphSONTokens.VALUE)) {
                     jsonParser.nextToken();
-                    value = deserializationContext.readValue(jsonParser, Object.class);
-                }else if (jsonParser.getCurrentName().equals(GraphSONTokens.PROPERTIES)) {
+                    DetachedUtil.setValue(vp, deserializationContext.readValue(jsonParser, Object.class));
+                } else if (jsonParser.getCurrentName().equals(GraphSONTokens.PROPERTIES)) {
                     jsonParser.nextToken();
                     properties = deserializationContext.readValue(jsonParser, propertiesType);
+                    properties.entrySet().iterator().forEachRemaining(kv -> DetachedUtil.addProperty(vp, new DetachedProperty(kv.getKey(), kv.getValue())));
                 }
             }
 
-            return new DetachedVertexProperty<>(id, label, value, properties);
+            return vp;
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/02b00736/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedEdge.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedEdge.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedEdge.java
index 1284ca7..40b6e34 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedEdge.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedEdge.java
@@ -51,9 +51,7 @@ public class DetachedEdge extends DetachedElement<Edge> implements Edge {
     private DetachedVertex outVertex;
     private DetachedVertex inVertex;
 
-    private DetachedEdge() {
-
-    }
+    DetachedEdge() {}
 
     protected DetachedEdge(final Edge edge, final boolean withProperties) {
         super(edge);
@@ -150,4 +148,18 @@ public class DetachedEdge extends DetachedElement<Edge> implements Edge {
     public <V> Iterator<Property<V>> properties(final String... propertyKeys) {
         return (Iterator) super.properties(propertyKeys);
     }
+
+    @Override
+    void internalAddProperty(final Property p) {
+        if (null == properties) properties = new HashMap<>();
+        this.properties.put(p.key(), Collections.singletonList(p));
+    }
+
+    void internalSetOutV(final DetachedVertex v) {
+        outVertex = v;
+    }
+
+    void internalSetInV(final DetachedVertex v) {
+        inVertex = v;
+    }
 }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/02b00736/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedElement.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedElement.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedElement.java
index f9c8c73..4e40303 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedElement.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedElement.java
@@ -25,9 +25,12 @@ import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.structure.util.Attachable;
 import org.apache.tinkerpop.gremlin.structure.util.ElementHelper;
 import org.apache.tinkerpop.gremlin.structure.util.empty.EmptyGraph;
+import org.apache.tinkerpop.gremlin.util.function.HashMapSupplier;
 
 import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -40,7 +43,7 @@ public abstract class DetachedElement<E> implements Element, Serializable, Attac
 
     protected Object id;
     protected String label;
-    protected Map<String, List<? extends Property>> properties = null;
+    protected Map<String, List<Property>> properties = null;
 
     protected DetachedElement() {
 
@@ -106,4 +109,14 @@ public abstract class DetachedElement<E> implements Element, Serializable, Attac
     public E get() {
         return (E) this;
     }
+
+    abstract void internalAddProperty(final Property p);
+
+    void internalSetId(final Object id) {
+        this.id = id;
+    }
+
+    void inernalSetLabel(final String label) {
+        this.label = label;
+    }
 }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/02b00736/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedUtil.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedUtil.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedUtil.java
new file mode 100644
index 0000000..f2c658d
--- /dev/null
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedUtil.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.tinkerpop.gremlin.structure.util.detached;
+
+/**
+ * @author Stephen Mallette (http://stephen.genoprime.com)
+ */
+public final class DetachedUtil {
+
+    private DetachedUtil() {}
+
+    /**
+     * Provides a way to mutate something that is "detached". This method is really for internal usage as there
+     * typically is not need for application developers to mutate a "detached" element.
+     */
+    public static void addProperty(final DetachedElement e, final DetachedProperty p) {
+        e.internalAddProperty(p);
+    }
+    public static void addProperty(final DetachedVertex v, final DetachedVertexProperty vp) {
+        v.internalAddProperty(vp);
+    }
+
+    public static void setId(final DetachedElement e, final Object id) {
+        e.internalSetId(id);
+    }
+
+    public static void setLabel(final DetachedElement e, final String label) {
+        e.inernalSetLabel(label);
+    }
+
+    public static void setValue(final DetachedVertexProperty vp, final Object value) {
+        vp.internalSetValue(value);
+    }
+
+    public static DetachedVertex newDetachedVertex() {
+        return new DetachedVertex();
+    }
+
+    public static DetachedVertexProperty newDetachedVertexProperty() {
+        return new DetachedVertexProperty();
+    }
+
+    public static DetachedEdge newDetachedEdge() {
+        return new DetachedEdge();
+    }
+
+    public static void setInV(final DetachedEdge e, final DetachedVertex v) {
+        e.internalSetInV(v);
+    }
+
+    public static void setOutV(final DetachedEdge e, final DetachedVertex v) {
+        e.internalSetOutV(v);
+    }
+}

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/02b00736/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedVertex.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedVertex.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedVertex.java
index d3fbd8e..15e956b 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedVertex.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedVertex.java
@@ -22,8 +22,10 @@ import org.apache.tinkerpop.gremlin.structure.Direction;
 import org.apache.tinkerpop.gremlin.structure.Edge;
 import org.apache.tinkerpop.gremlin.structure.Element;
 import org.apache.tinkerpop.gremlin.structure.Graph;
+import org.apache.tinkerpop.gremlin.structure.Property;
 import org.apache.tinkerpop.gremlin.structure.Vertex;
 import org.apache.tinkerpop.gremlin.structure.VertexProperty;
+import org.apache.tinkerpop.gremlin.structure.util.ElementHelper;
 import org.apache.tinkerpop.gremlin.structure.util.StringFactory;
 import org.apache.tinkerpop.gremlin.util.iterator.IteratorUtils;
 
@@ -53,8 +55,7 @@ public class DetachedVertex extends DetachedElement<Vertex> implements Vertex {
     private static final String VALUE = "value";
     private static final String PROPERTIES = "properties";
 
-    private DetachedVertex() {
-    }
+    DetachedVertex() {}
 
     protected DetachedVertex(final Vertex vertex, final boolean withProperties) {
         super(vertex);
@@ -67,7 +68,7 @@ public class DetachedVertex extends DetachedElement<Vertex> implements Vertex {
             if (propertyIterator.hasNext()) {
                 this.properties = new HashMap<>();
                 propertyIterator.forEachRemaining(property -> {
-                    final List<VertexProperty<?>> list = (List<VertexProperty<?>>) this.properties.getOrDefault(property.key(), new ArrayList<>());
+                    final List<Property> list = this.properties.getOrDefault(property.key(), new ArrayList<>());
                     list.add(DetachedFactory.detach(property, true));
                     this.properties.put(property.key(), list);
                 });
@@ -80,7 +81,7 @@ public class DetachedVertex extends DetachedElement<Vertex> implements Vertex {
         if (properties != null && !properties.isEmpty()) {
             this.properties = new HashMap<>();
             properties.entrySet().iterator().forEachRemaining(entry ->
-                this.properties.put(entry.getKey(), IteratorUtils.<VertexProperty>list(IteratorUtils.map(((List<Object>) entry.getValue()).iterator(),
+                this.properties.put(entry.getKey(), IteratorUtils.<Property>list(IteratorUtils.map(((List<Object>) entry.getValue()).iterator(),
                         m -> VertexProperty.class.isAssignableFrom(m.getClass())
                                 ? (VertexProperty) m
                                 : new DetachedVertexProperty<>(((Map) m).get(ID), entry.getKey(), ((Map) m).get(VALUE), (Map<String, Object>) ((Map) m).getOrDefault(PROPERTIES, new HashMap<>()), this)))));
@@ -143,4 +144,14 @@ public class DetachedVertex extends DetachedElement<Vertex> implements Vertex {
     public void remove() {
         throw Vertex.Exceptions.vertexRemovalNotSupported();
     }
+
+    @Override
+    void internalAddProperty(final Property p) {
+        if (null == properties) properties = new HashMap<>();
+
+        if (!properties.containsKey(p.key()))
+            properties.put(p.key(), new ArrayList<>());
+
+        this.properties.get(p.key()).add(p);
+    }
 }

http://git-wip-us.apache.org/repos/asf/tinkerpop/blob/02b00736/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedVertexProperty.java
----------------------------------------------------------------------
diff --git a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedVertexProperty.java b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedVertexProperty.java
index 069c14c..f682e04 100644
--- a/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedVertexProperty.java
+++ b/gremlin-core/src/main/java/org/apache/tinkerpop/gremlin/structure/util/detached/DetachedVertexProperty.java
@@ -18,7 +18,6 @@
  */
 package org.apache.tinkerpop.gremlin.structure.util.detached;
 
-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;
@@ -40,8 +39,7 @@ public class DetachedVertexProperty<V> extends DetachedElement<VertexProperty<V>
     protected V value;
     protected transient DetachedVertex vertex;
 
-    private DetachedVertexProperty() {
-    }
+    DetachedVertexProperty() {}
 
     protected DetachedVertexProperty(final VertexProperty<V> vertexProperty, final boolean withProperties) {
         super(vertexProperty);
@@ -127,4 +125,14 @@ public class DetachedVertexProperty<V> extends DetachedElement<VertexProperty<V>
     public <U> Iterator<Property<U>> properties(final String... propertyKeys) {
         return (Iterator) super.properties(propertyKeys);
     }
+
+    @Override
+    void internalAddProperty(final Property p) {
+        if (null == properties) properties = new HashMap<>();
+        this.properties.put(p.key(), Collections.singletonList(p));
+    }
+
+    void internalSetValue(final V value) {
+        this.value = value;
+    }
 }