You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2015/11/18 13:01:12 UTC

[1/6] ignite git commit: IGNITE-1917: Added fast ID resolution.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1917 df518bdbd -> cbfd19a4d


IGNITE-1917: Added fast ID resolution.


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

Branch: refs/heads/ignite-1917
Commit: 6e773a0d02d761243f83f832aeae9910c3e9e6ac
Parents: df518bd
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Nov 17 13:11:29 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Nov 17 13:11:29 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/PortableSchema.java       |  14 +-
 .../portable/PortableSchemaIntIntMap.java       | 217 +++++++++++++++++++
 2 files changed, 226 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6e773a0d/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java
index 86ca5f8..0c49451 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchema.java
@@ -47,6 +47,8 @@ public class PortableSchema implements Externalizable {
     /** Map with ID to order. */
     private HashMap<Integer, Integer> idToOrder;
 
+    private PortableSchemaIntIntMap fastIdToOrder;
+
     /** IDs depending on order. */
     private ArrayList<Integer> ids;
 
@@ -108,6 +110,7 @@ public class PortableSchema implements Externalizable {
             id7 = iter.hasNext() ? iter.next() : 0;
 
             idToOrder = null;
+            fastIdToOrder = null;
         }
         else {
             inline = false;
@@ -123,6 +126,8 @@ public class PortableSchema implements Externalizable {
                 ids.add(fieldId);
                 idToOrder.put(fieldId, i);
             }
+
+            fastIdToOrder = new PortableSchemaIntIntMap(idToOrder);
         }
     }
 
@@ -210,11 +215,8 @@ public class PortableSchema implements Externalizable {
 
             return ORDER_NOT_FOUND;
         }
-        else {
-            Integer order = idToOrder.get(id);
-
-            return order != null ? order : ORDER_NOT_FOUND;
-        }
+        else
+            return fastIdToOrder.get(id);
     }
 
     /** {@inheritDoc} */
@@ -283,6 +285,8 @@ public class PortableSchema implements Externalizable {
                 ids.add(fieldId);
                 idToOrder.put(fieldId, i);
             }
+
+            fastIdToOrder = new PortableSchemaIntIntMap(idToOrder);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6e773a0d/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchemaIntIntMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchemaIntIntMap.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchemaIntIntMap.java
new file mode 100644
index 0000000..f554b30
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableSchemaIntIntMap.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.ignite.internal.portable;
+
+import java.util.Map;
+
+/**
+ * Map for fast access to field order by ID.
+ */
+// TODO: IGNITE-1917: Inline into schema.
+public class PortableSchemaIntIntMap {
+    /** Minimum sensible size. */
+    private static final int MIN_SIZE = 32;
+
+    /** Empty cell. */
+    private static final int EMPTY = 0;
+
+    /** Data. */
+    private final int[] data;
+
+    /** Mask for index calculation. */
+    private final int mask;
+
+    /**
+     * Constructor.
+     *
+     * @param vals Values.
+     */
+    public PortableSchemaIntIntMap(Map<Integer, Integer> vals) {
+        int size = Math.max(nextPowerOfTwo(vals.size()) << 2, MIN_SIZE);
+
+        assert size > 0;
+
+        ParseResult finalRes;
+
+        ParseResult res1 = parse(vals, size);
+
+        if (res1.collisions == 0)
+            finalRes = res1;
+        else {
+            ParseResult res2 = parse(vals, size * 2);
+
+            // Failed to decrease aom
+            if (res2.collisions == 0)
+                finalRes = res2;
+            else
+                finalRes = parse(vals, size * 4);
+        }
+
+        data = finalRes.data;
+
+        mask = maskForPowerOfTwo(data.length / 2);
+    }
+
+    /**
+     * Get order.
+     *
+     * @param id ID.
+     * @return Order.
+     */
+    public int get(int id) {
+        int idx = (id & mask) << 1;
+
+        int curId = data[idx];
+
+        if (id == curId) // Hit!
+            return data[idx + 1];
+        else if (curId == EMPTY) // No such ID!
+            return PortableSchema.ORDER_NOT_FOUND;
+        else {
+            // Unlikely collision scenario.
+            for (int i = 2; i < data.length; i += 2) {
+                int newIdx = (idx + i) % data.length;
+
+                assert newIdx < data.length - 1;
+
+                curId = data[newIdx];
+
+                if (id == curId)
+                    return data[newIdx + 1];
+                else if (curId == EMPTY)
+                    return PortableSchema.ORDER_NOT_FOUND;
+            }
+
+            return PortableSchema.ORDER_NOT_FOUND;
+        }
+    }
+
+    /**
+     * Parse values.
+     *
+     * @param vals Values.
+     * @param size Proposed result size.
+     * @return Parse result.
+     */
+    private static ParseResult parse(Map<Integer, Integer> vals, int size) {
+        int mask = maskForPowerOfTwo(size);
+
+        int totalSize = size * 2;
+
+        int[] data = new int[totalSize];
+        int collisions = 0;
+
+        for (Map.Entry<Integer, Integer> val : vals.entrySet()) {
+            int id = val.getKey();
+            int order = val.getValue();
+
+            assert id != 0;
+
+            int idIdx = (id & mask) << 1;
+
+            if (data[idIdx] == 0) {
+                // Found empty slot.
+                data[idIdx] = id;
+                data[idIdx + 1] = order;
+            }
+            else {
+                // Collision!
+                collisions++;
+
+                boolean placeFound = false;
+
+                for (int i = 2; i < totalSize; i += 2) {
+                    int newIdIdx = (idIdx + i) % totalSize;
+
+                    if (data[newIdIdx] == 0) {
+                        data[newIdIdx] = id;
+                        data[newIdIdx + 1] = order;
+
+                        placeFound = true;
+
+                        break;
+                    }
+                }
+
+                assert placeFound : "Should always have a place for entry!";
+            }
+        }
+
+        return new ParseResult(data, collisions);
+    }
+
+    /**
+     * Get next power of two which greater or equal to the given number.
+     * This implementation is not meant to be very efficient, so it is expected to be used relatively rare.
+     *
+     * @param val Number
+     * @return Nearest pow2.
+     */
+    private static int nextPowerOfTwo(int val) {
+        int res = 1;
+
+        while (res < val)
+            res = res << 1;
+
+        if (res < 0)
+            throw new IllegalArgumentException("Value is too big to find positive pow2: " + val);
+
+        return res;
+    }
+
+    /**
+     * Calculate mask for the given value which is a power of two.
+     *
+     * @param val Value.
+     * @return Mask.
+     */
+    private static int maskForPowerOfTwo(int val) {
+        int mask = 0;
+        int comparand = 1;
+
+        while (comparand < val) {
+            mask |= comparand;
+
+            comparand <<= 1;
+        }
+
+        return mask;
+    }
+
+    /**
+     * Result of map parsing.
+     */
+    private static class ParseResult {
+        /** Data. */
+        public int[] data;
+
+        /** Collisions. */
+        public int collisions;
+
+        /**
+         * Constructor.
+         *
+         * @param data Data.
+         * @param collisions Collisions.
+         */
+        public ParseResult(int[] data, int collisions) {
+            this.data = data;
+            this.collisions = collisions;
+        }
+    }
+}


[5/6] ignite git commit: IGNITE-1917: Reader constructors simplification (2).

Posted by vo...@apache.org.
IGNITE-1917: Reader constructors simplification (2).


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

Branch: refs/heads/ignite-1917
Commit: d4071a2b14a14c58d78df4dfebaf95bb70e3f923
Parents: ee88850
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Nov 17 15:09:47 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Nov 17 15:09:47 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/BinaryObjectImpl.java      |  2 +-
 .../portable/BinaryObjectOffheapImpl.java        | 10 ++++------
 .../internal/portable/BinaryReaderExImpl.java    | 19 +++++--------------
 .../portable/GridPortableMarshaller.java         |  8 +++++---
 .../portable/builder/PortableBuilderReader.java  |  7 +++++--
 5 files changed, 20 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d4071a2b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
index 7455c70..fabbdca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
@@ -583,6 +583,6 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
      * @return Reader.
      */
     private BinaryReaderExImpl newReader() {
-        return new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), start, null);
+        return new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), start, null, new BinaryReaderHandles());
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d4071a2b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
index 66cbf20..91c68cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
@@ -324,11 +324,8 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Nullable @Override protected <F> F field(BinaryReaderHandles rCtx, String fieldName) {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx,
-            new PortableOffheapInputStream(ptr, size, false),
-            start,
-            null,
-            rCtx);
+        BinaryReaderExImpl reader =
+            new BinaryReaderExImpl(ctx, new PortableOffheapInputStream(ptr, size, false), start, null, rCtx);
 
         return (F)reader.unmarshalField(fieldName);
     }
@@ -430,6 +427,7 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
      * @return Reader.
      */
     private BinaryReaderExImpl newReader() {
-        return new BinaryReaderExImpl(ctx, new PortableOffheapInputStream(ptr, size, false), start, null);
+        return new BinaryReaderExImpl(ctx, new PortableOffheapInputStream(ptr, size, false), start, null,
+            new BinaryReaderHandles());
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d4071a2b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
index 7cffe28..60ec1ca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
@@ -126,14 +126,14 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
     private final ClassLoader ldr;
 
     /** */
-    private int start;
-
-    /** */
     private BinaryReaderHandles rCtx;
 
     /** */
     private PortableClassDescriptor desc;
 
+    /** */
+    private int start;
+
     /** Flag indicating that object header was parsed. */
     private boolean hdrParsed;
 
@@ -174,18 +174,9 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
      * @param ctx Context.
      * @param in Input stream.
      * @param start Start.
-     */
-    public BinaryReaderExImpl(PortableContext ctx, PortableInputStream in, int start, ClassLoader ldr) {
-        this(ctx, in, start, ldr, new BinaryReaderHandles());
-    }
-
-    /**
-     * @param ctx Context.
-     * @param in Input stream.
-     * @param start Start.
      * @param rCtx Context.
      */
-    BinaryReaderExImpl(PortableContext ctx, PortableInputStream in, int start, ClassLoader ldr,
+    public BinaryReaderExImpl(PortableContext ctx, PortableInputStream in, int start, ClassLoader ldr,
         BinaryReaderHandles rCtx) {
         this.ctx = ctx;
         this.in = in;
@@ -1780,7 +1771,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
 
     /**
      * @return Deserialized object.
-     * @throws org.apache.ignite.binary.BinaryObjectException If failed.
+     * @throws BinaryObjectException If failed.
      */
     @Nullable Object deserialize() throws BinaryObjectException {
         Object obj;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d4071a2b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
index 36c76c4..dbf4a8b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
@@ -255,7 +255,8 @@ public class GridPortableMarshaller {
     @Nullable public <T> T unmarshal(byte[] bytes, @Nullable ClassLoader clsLdr) throws BinaryObjectException {
         assert bytes != null;
 
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, new PortableHeapInputStream(bytes), 0, clsLdr);
+        BinaryReaderExImpl reader =
+            new BinaryReaderExImpl(ctx, new PortableHeapInputStream(bytes), 0, clsLdr, new BinaryReaderHandles());
 
         return (T)reader.unmarshal();
     }
@@ -284,7 +285,8 @@ public class GridPortableMarshaller {
         if (arr[0] == NULL)
             return null;
 
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), 0, ldr);
+        BinaryReaderExImpl reader =
+            new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), 0, ldr, new BinaryReaderHandles());
 
         return (T)reader.deserialize();
     }
@@ -307,7 +309,7 @@ public class GridPortableMarshaller {
      */
     public BinaryReaderExImpl reader(PortableInputStream in) {
         // TODO: IGNITE-1272 - Is class loader needed here?
-        return new BinaryReaderExImpl(ctx, in, in.position(), null);
+        return new BinaryReaderExImpl(ctx, in, in.position(), null, new BinaryReaderHandles());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/d4071a2b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
index fae9417..a9ec65a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
@@ -21,6 +21,8 @@ import java.sql.Timestamp;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.Map;
+
+import org.apache.ignite.internal.portable.BinaryReaderHandles;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.PortableContext;
 import org.apache.ignite.internal.portable.PortablePositionReadable;
@@ -65,7 +67,7 @@ public class PortableBuilderReader implements PortablePositionReadable {
         pos = objImpl.start();
 
         // TODO: IGNITE-1272 - Is class loader needed here?
-        reader = new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), pos, null);
+        reader = new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), pos, null, new BinaryReaderHandles());
     }
 
     /**
@@ -95,7 +97,8 @@ public class PortableBuilderReader implements PortablePositionReadable {
         if (start == pos)
             targetReader = reader;
         else
-            targetReader = new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), start, null);
+            targetReader =
+                new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), start, null, new BinaryReaderHandles());
 
         return targetReader.getOrCreateSchema();
     }


[2/6] ignite git commit: IGNITE-1917: Reworked reader context (handles).

Posted by vo...@apache.org.
IGNITE-1917: Reworked reader context (handles).


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

Branch: refs/heads/ignite-1917
Commit: 1a32d3991f33b2ea8bfd2c567253989b16ecf85b
Parents: 6e773a0
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Nov 17 13:30:20 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Nov 17 13:30:20 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/BinaryObjectEx.java       |   8 +-
 .../internal/portable/BinaryObjectImpl.java     |   2 +-
 .../portable/BinaryObjectOffheapImpl.java       |   2 +-
 .../internal/portable/BinaryReaderExImpl.java   |  20 ++--
 .../internal/portable/BinaryReaderHandles.java  | 109 +++++++++++++++++++
 .../portable/PortableReaderContext.java         | 105 ------------------
 6 files changed, 125 insertions(+), 121 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1a32d399/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
index 6902675..597fad5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectEx.java
@@ -79,7 +79,7 @@ public abstract class BinaryObjectEx implements BinaryObject {
      * @param fieldName Field name.
      * @return Field value.
      */
-    @Nullable protected abstract <F> F field(PortableReaderContext ctx, String fieldName);
+    @Nullable protected abstract <F> F field(BinaryReaderHandles ctx, String fieldName);
 
     /**
      * Get schema ID.
@@ -157,7 +157,7 @@ public abstract class BinaryObjectEx implements BinaryObject {
      * @param handles Handles for already traversed objects.
      * @return String representation.
      */
-    private String toString(PortableReaderContext ctx, IdentityHashMap<BinaryObject, Integer> handles) {
+    private String toString(BinaryReaderHandles ctx, IdentityHashMap<BinaryObject, Integer> handles) {
         int idHash = System.identityHashCode(this);
 
         BinaryType meta;
@@ -232,9 +232,9 @@ public abstract class BinaryObjectEx implements BinaryObject {
     /** {@inheritDoc} */
     @Override public String toString() {
         try {
-            PortableReaderContext ctx = new PortableReaderContext();
+            BinaryReaderHandles ctx = new BinaryReaderHandles();
 
-            ctx.setPortableHandler(start(), this);
+            ctx.put(start(), this);
 
             return toString(ctx, new IdentityHashMap<BinaryObject, Integer>());
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1a32d399/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
index d04dc7d..86dc247 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
@@ -412,7 +412,7 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Nullable @Override protected <F> F field(PortableReaderContext rCtx, String fieldName) {
+    @Nullable @Override protected <F> F field(BinaryReaderHandles rCtx, String fieldName) {
         BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx,
             new PortableHeapInputStream(arr),
             start,

http://git-wip-us.apache.org/repos/asf/ignite/blob/1a32d399/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
index 7d8c044..f97ab94 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
@@ -342,7 +342,7 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Nullable @Override protected <F> F field(PortableReaderContext rCtx, String fieldName) {
+    @Nullable @Override protected <F> F field(BinaryReaderHandles rCtx, String fieldName) {
         BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx,
             new PortableOffheapInputStream(ptr, size, false),
             start,

http://git-wip-us.apache.org/repos/asf/ignite/blob/1a32d399/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
index 8b256bc..7cdfced 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
@@ -127,7 +127,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
     private final int start;
 
     /** */
-    private final PortableReaderContext rCtx;
+    private final BinaryReaderHandles rCtx;
 
     /** */
     private final ClassLoader ldr;
@@ -178,7 +178,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
      * @param ldr Class loader.
      */
     public BinaryReaderExImpl(PortableContext ctx, byte[] arr, int start, ClassLoader ldr) {
-        this(ctx, new PortableHeapInputStream(arr), start, ldr, new PortableReaderContext());
+        this(ctx, new PortableHeapInputStream(arr), start, ldr, new BinaryReaderHandles());
     }
 
     /**
@@ -187,7 +187,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
      * @param start Start.
      */
     BinaryReaderExImpl(PortableContext ctx, PortableInputStream in, int start, ClassLoader ldr) {
-        this(ctx, in, start, ldr, new PortableReaderContext());
+        this(ctx, in, start, ldr, new BinaryReaderHandles());
     }
 
     /**
@@ -197,7 +197,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
      * @param rCtx Context.
      */
     BinaryReaderExImpl(PortableContext ctx, PortableInputStream in, int start, ClassLoader ldr,
-        PortableReaderContext rCtx) {
+        BinaryReaderHandles rCtx) {
         this.ctx = ctx;
         this.in = in;
         this.start = start;
@@ -1016,7 +1016,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
      * @param obj Object.
      */
     void setHandler(Object obj) {
-        rCtx.setObjectHandler(start, obj);
+        rCtx.put(start, obj);
     }
 
     /**
@@ -1024,7 +1024,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
      * @param pos Position.
      */
     void setHandler(Object obj, int pos) {
-        rCtx.setObjectHandler(pos, obj);
+        rCtx.put(pos, obj);
     }
 
     /**
@@ -1036,7 +1036,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
     private <T> T readHandleField() {
         int handle = (in.position() - 1) - in.readInt();
 
-        Object obj = rCtx.getObjectByHandle(handle);
+        Object obj = rCtx.get(handle);
 
         if (obj == null) {
             in.position(handle);
@@ -1556,7 +1556,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
             case HANDLE:
                 int handle = start - in.readInt();
 
-                BinaryObject handledPo = rCtx.getPortableByHandle(handle);
+                BinaryObject handledPo = rCtx.get(handle);
 
                 if (handledPo != null)
                     return handledPo;
@@ -1585,7 +1585,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
                         in.remaining() + in.position())
                         : new BinaryObjectImpl(ctx, in.array(), start);
 
-                rCtx.setPortableHandler(start, po);
+                rCtx.put(start, po);
 
                 in.position(start + po.length());
 
@@ -1807,7 +1807,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
             case HANDLE:
                 int handle = start - in.readInt();
 
-                obj = rCtx.getObjectByHandle(handle);
+                obj = rCtx.get(handle);
 
                 if (obj == null) {
                     int retPos = in.position();

http://git-wip-us.apache.org/repos/asf/ignite/blob/1a32d399/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderHandles.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderHandles.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderHandles.java
new file mode 100644
index 0000000..cb4ddaa
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderHandles.java
@@ -0,0 +1,109 @@
+/*
+ * 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.ignite.internal.portable;
+
+import org.jetbrains.annotations.Nullable;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Reader handles.
+ */
+// TODO: IGNITE-1917: This is "add-only" map => can be improved!
+public class BinaryReaderHandles {
+    /** Mode: empty. */
+    private static final int MODE_EMPTY = 0;
+
+    /** Mode: single object. */
+    private static final int MODE_SINGLE = 1;
+
+    /** Mode: multiple objects. */
+    private static final int MODE_MULTIPLE = 2;
+
+    /** Position.  */
+    private int singlePos;
+
+    /** Data. This is either an object or a map. */
+    private Object data;
+
+    /** Mode. */
+    private int mode = MODE_EMPTY;
+
+    /**
+     * Get object by position.
+     *
+     * @param pos Position.
+     * @return Object.
+     */
+    @SuppressWarnings("unchecked")
+    public @Nullable <T> T get(int pos) {
+        switch (mode) {
+            case MODE_EMPTY:
+                return null;
+
+            case MODE_SINGLE:
+                return (T)data;
+
+            default:
+                assert mode == MODE_MULTIPLE;
+
+                return (T)((Map<Integer, Object>)data).get(pos);
+        }
+    }
+
+    /**
+     * Put object to registry and return previous position (if any).
+     *
+     * @param pos Position.
+     * @param obj Object.
+     */
+    @SuppressWarnings("unchecked")
+    public void put(int pos, Object obj) {
+        assert pos >= 0;
+        assert obj != null;
+
+        switch (mode) {
+            case MODE_EMPTY:
+                this.singlePos = pos;
+                this.data = obj;
+                this.mode = MODE_SINGLE;
+
+                break;
+
+            case MODE_SINGLE:
+                Map<Integer, Object> newData = new HashMap(3, 1.0f);
+
+                newData.put(singlePos, data);
+                newData.put(pos, obj);
+
+                this.singlePos = -1;
+                this.data = newData;
+                this.mode = MODE_MULTIPLE;
+
+                break;
+
+            default:
+                assert mode == MODE_MULTIPLE;
+
+                Map<Integer, Object> data0 = (Map<Integer, Object>)data;
+
+                data0.put(pos, obj);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1a32d399/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderContext.java
deleted file mode 100644
index 869f81d..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderContext.java
+++ /dev/null
@@ -1,105 +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.ignite.internal.portable;
-
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.lang.IgniteBiTuple;
-import org.jetbrains.annotations.Nullable;
-
-/**
-* Reader context.
-*/
-class PortableReaderContext {
-    /** */
-    private Object oHandles;
-
-    /** */
-    private Map<Integer, BinaryObject> poHandles;
-
-    /**
-     * @param handle Handle.
-     * @param obj Object.
-     */
-    @SuppressWarnings("unchecked")
-    void setObjectHandler(int handle, Object obj) {
-        assert obj != null;
-
-        if (oHandles == null)
-            oHandles = new IgniteBiTuple(handle, obj);
-        else if (oHandles instanceof IgniteBiTuple) {
-            Map map = new HashMap(3, 1.0f);
-
-            IgniteBiTuple t = (IgniteBiTuple)oHandles;
-
-            map.put(t.getKey(), t.getValue());
-            map.put(handle, obj);
-
-            oHandles = map;
-        }
-        else
-            ((Map)oHandles).put(handle, obj);
-    }
-
-    /**
-     * @param handle Handle.
-     * @param po Portable object.
-     */
-    void setPortableHandler(int handle, BinaryObject po) {
-        assert po != null;
-
-        if (poHandles == null)
-            poHandles = new HashMap<>(3, 1.0f);
-
-        poHandles.put(handle, po);
-    }
-
-    /**
-     * @param handle Handle.
-     * @return Object.
-     */
-    @Nullable Object getObjectByHandle(int handle) {
-        if (oHandles != null) {
-            if (oHandles instanceof IgniteBiTuple) {
-                IgniteBiTuple t = (IgniteBiTuple)oHandles;
-
-                if ((int)t.get1() == handle)
-                    return t.get2();
-            }
-            else
-                return ((Map)oHandles).get(handle);
-        }
-
-        return null;
-    }
-
-    /**
-     * @param handle Handle.
-     * @return Object.
-     */
-    @Nullable BinaryObject getPortableByHandle(int handle) {
-        return poHandles != null ? poHandles.get(handle) : null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(PortableReaderContext.class, this);
-    }
-}


[3/6] ignite git commit: IGNITE-1917: Minor (preparing to big reader refactoring).

Posted by vo...@apache.org.
IGNITE-1917: Minor (preparing to big reader refactoring).


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

Branch: refs/heads/ignite-1917
Commit: f8f4d52524e5cee781ffb75cc603e9fc90702ac7
Parents: 1a32d39
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Nov 17 14:48:49 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Nov 17 14:48:49 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/portable/BinaryReaderExImpl.java    | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f8f4d525/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
index 7cdfced..d79df22 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
@@ -117,20 +117,20 @@ import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID_AR
  */
 @SuppressWarnings("unchecked")
 public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, ObjectInput {
-    /** */
+    /** Portable context. */
     private final PortableContext ctx;
 
-    /** */
+    /** Input stream. */
     private final PortableInputStream in;
 
-    /** */
-    private final int start;
+    /** Class loaded. */
+    private final ClassLoader ldr;
 
     /** */
-    private final BinaryReaderHandles rCtx;
+    private int start;
 
     /** */
-    private final ClassLoader ldr;
+    private BinaryReaderHandles rCtx;
 
     /** */
     private PortableClassDescriptor desc;


[4/6] ignite git commit: IGNITE-1917: Reader constructors simplification.

Posted by vo...@apache.org.
IGNITE-1917: Reader constructors simplification.


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

Branch: refs/heads/ignite-1917
Commit: ee88850d232610c905f1903375a43f6d6181abb2
Parents: f8f4d52
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Nov 17 14:59:30 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Nov 17 14:59:30 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/BinaryObjectImpl.java     | 40 +++++-----
 .../portable/BinaryObjectOffheapImpl.java       | 83 ++++++++------------
 .../internal/portable/BinaryReaderExImpl.java   | 13 +--
 .../portable/GridPortableMarshaller.java        |  5 +-
 .../portable/builder/PortableBuilderReader.java |  5 +-
 5 files changed, 56 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ee88850d/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
index 86dc247..7455c70 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
@@ -257,17 +257,13 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Nullable @Override public <F> F field(String fieldName) throws BinaryObjectException {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, arr, start, null);
-
-        return (F)reader.unmarshalField(fieldName);
+        return (F)newReader().unmarshalField(fieldName);
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Nullable @Override public <F> F field(int fieldId) throws BinaryObjectException {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, arr, start, null);
-
-        return (F)reader.unmarshalField(fieldId);
+        return (F)newReader().unmarshalField(fieldId);
     }
 
     /** {@inheritDoc} */
@@ -400,11 +396,10 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
 
                 break;
 
-            default: {
-                BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, arr, start, null);
+            default:
+                val = newReader().unmarshalFieldByAbsolutePosition(fieldPos);
 
-                val = reader.unmarshalFieldByAbsolutePosition(fieldPos);
-            }
+                break;
         }
 
         return (F)val;
@@ -413,20 +408,14 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Nullable @Override protected <F> F field(BinaryReaderHandles rCtx, String fieldName) {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx,
-            new PortableHeapInputStream(arr),
-            start,
-            null,
-            rCtx);
+        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), start, null, rCtx);
 
         return (F)reader.unmarshalField(fieldName);
     }
 
     /** {@inheritDoc} */
     @Override public boolean hasField(String fieldName) {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, arr, start, null);
-
-        return reader.hasField(fieldName);
+        return newReader().hasField(fieldName);
     }
 
     /** {@inheritDoc} */
@@ -458,9 +447,7 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
 
     /** {@inheritDoc} */
     @Override protected PortableSchema createSchema() {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, arr, start, null);
-
-        return reader.getOrCreateSchema();
+        return newReader().getOrCreateSchema();
     }
 
     /** {@inheritDoc} */
@@ -576,7 +563,7 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
      */
     private Object deserializeValue() {
         // TODO: IGNITE-1272 - Deserialize with proper class loader.
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, arr, start, null);
+        BinaryReaderExImpl reader = newReader();
 
         Object obj0 = reader.deserialize();
 
@@ -589,4 +576,13 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
 
         return obj0;
     }
+
+    /**
+     * Create new reader for this object.
+     *
+     * @return Reader.
+     */
+    private BinaryReaderExImpl newReader() {
+        return new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), start, null);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee88850d/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
index f97ab94..66cbf20 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
@@ -17,18 +17,11 @@
 
 package org.apache.ignite.internal.portable;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.sql.Timestamp;
-import java.util.Date;
-import java.util.UUID;
-
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryField;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.internal.portable.streams.PortableOffheapInputStream;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
@@ -37,13 +30,20 @@ import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryType;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.binary.BinaryField;
 import org.jetbrains.annotations.Nullable;
 import sun.misc.Unsafe;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.sql.Timestamp;
+import java.util.Date;
+import java.util.UUID;
+
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE;
@@ -131,12 +131,7 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
 
     /** {@inheritDoc} */
     @Override protected PortableSchema createSchema() {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx,
-            new PortableOffheapInputStream(ptr, size, false),
-            start,
-            null);
-
-        return reader.getOrCreateSchema();
+        return newReader().getOrCreateSchema();
     }
 
     /** {@inheritDoc} */
@@ -177,23 +172,13 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Nullable @Override public <F> F field(String fieldName) throws BinaryObjectException {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx,
-            new PortableOffheapInputStream(ptr, size, false),
-            start,
-            null);
-
-        return (F)reader.unmarshalField(fieldName);
+        return (F)newReader().unmarshalField(fieldName);
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Nullable @Override public <F> F field(int fieldId) throws BinaryObjectException {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx,
-            new PortableOffheapInputStream(ptr, size, false),
-            start,
-            null);
-
-        return (F)reader.unmarshalField(fieldId);
+        return (F)newReader().unmarshalField(fieldId);
     }
 
     /** {@inheritDoc} */
@@ -327,14 +312,10 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
 
                 break;
 
-            default: {
-                BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx,
-                    new PortableOffheapInputStream(ptr, size, false),
-                    start,
-                    null);
+            default:
+                val = newReader().unmarshalFieldByAbsolutePosition(fieldPos);
 
-                val = reader.unmarshalFieldByAbsolutePosition(fieldPos);
-            }
+                break;
         }
 
         return (F)val;
@@ -354,12 +335,7 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
 
     /** {@inheritDoc} */
     @Override public boolean hasField(String fieldName) {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx,
-            new PortableOffheapInputStream(ptr, size, false),
-            start,
-            null);
-
-        return reader.hasField(fieldName);
+        return newReader().hasField(fieldName);
     }
 
     /** {@inheritDoc} */
@@ -445,12 +421,15 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
      */
     private Object deserializeValue() {
         // TODO: IGNITE-1272 - Deserialize with proper class loader.
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(
-            ctx,
-            new PortableOffheapInputStream(ptr, size, false),
-            start,
-            null);
+        return newReader().deserialize();
+    }
 
-        return reader.deserialize();
+    /**
+     * Create new reader for this object.
+     *
+     * @return Reader.
+     */
+    private BinaryReaderExImpl newReader() {
+        return new BinaryReaderExImpl(ctx, new PortableOffheapInputStream(ptr, size, false), start, null);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee88850d/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
index d79df22..7cffe28 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
@@ -24,7 +24,6 @@ import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryRawReader;
 import org.apache.ignite.binary.BinaryReader;
-import org.apache.ignite.internal.portable.streams.PortableHeapInputStream;
 import org.apache.ignite.internal.portable.streams.PortableInputStream;
 import org.apache.ignite.internal.util.GridEnumCache;
 import org.apache.ignite.internal.util.lang.GridMapEntry;
@@ -173,20 +172,10 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
 
     /**
      * @param ctx Context.
-     * @param arr Array.
-     * @param start Start.
-     * @param ldr Class loader.
-     */
-    public BinaryReaderExImpl(PortableContext ctx, byte[] arr, int start, ClassLoader ldr) {
-        this(ctx, new PortableHeapInputStream(arr), start, ldr, new BinaryReaderHandles());
-    }
-
-    /**
-     * @param ctx Context.
      * @param in Input stream.
      * @param start Start.
      */
-    BinaryReaderExImpl(PortableContext ctx, PortableInputStream in, int start, ClassLoader ldr) {
+    public BinaryReaderExImpl(PortableContext ctx, PortableInputStream in, int start, ClassLoader ldr) {
         this(ctx, in, start, ldr, new BinaryReaderHandles());
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee88850d/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
index 989f16d..36c76c4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.portable;
 
+import org.apache.ignite.internal.portable.streams.PortableHeapInputStream;
 import org.apache.ignite.internal.portable.streams.PortableInputStream;
 import org.apache.ignite.internal.portable.streams.PortableOutputStream;
 import org.apache.ignite.binary.BinaryObjectException;
@@ -254,7 +255,7 @@ public class GridPortableMarshaller {
     @Nullable public <T> T unmarshal(byte[] bytes, @Nullable ClassLoader clsLdr) throws BinaryObjectException {
         assert bytes != null;
 
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, bytes, 0, clsLdr);
+        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, new PortableHeapInputStream(bytes), 0, clsLdr);
 
         return (T)reader.unmarshal();
     }
@@ -283,7 +284,7 @@ public class GridPortableMarshaller {
         if (arr[0] == NULL)
             return null;
 
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, arr, 0, ldr);
+        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), 0, ldr);
 
         return (T)reader.deserialize();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee88850d/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
index b6a6b54..fae9417 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.portable.PortableSchema;
 import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.portable.BinaryWriterExImpl;
 import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.portable.streams.PortableHeapInputStream;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;
@@ -64,7 +65,7 @@ public class PortableBuilderReader implements PortablePositionReadable {
         pos = objImpl.start();
 
         // TODO: IGNITE-1272 - Is class loader needed here?
-        reader = new BinaryReaderExImpl(ctx, arr, pos, null);
+        reader = new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), pos, null);
     }
 
     /**
@@ -94,7 +95,7 @@ public class PortableBuilderReader implements PortablePositionReadable {
         if (start == pos)
             targetReader = reader;
         else
-            targetReader = new BinaryReaderExImpl(ctx, arr, start, null);
+            targetReader = new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), start, null);
 
         return targetReader.getOrCreateSchema();
     }


[6/6] ignite git commit: IGNITE-1917: Reader constructors simplification (3).

Posted by vo...@apache.org.
IGNITE-1917: Reader constructors simplification (3).


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

Branch: refs/heads/ignite-1917
Commit: cbfd19a4d2b43b2a9b7bc4ac5ca3c7d7b74e35f5
Parents: d4071a2
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Nov 17 15:18:34 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Nov 17 15:18:34 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/portable/BinaryObjectImpl.java |  4 ++--
 .../internal/portable/BinaryObjectOffheapImpl.java | 14 ++++++++++----
 .../internal/portable/BinaryReaderExImpl.java      | 15 +++++++--------
 .../internal/portable/GridPortableMarshaller.java  |  6 +++---
 .../portable/builder/PortableBuilderReader.java    |  6 +++---
 .../portable/streams/PortableHeapInputStream.java  | 17 +++++++++++++++++
 6 files changed, 42 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cbfd19a4/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
index fabbdca..4b86e5a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
@@ -408,7 +408,7 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Nullable @Override protected <F> F field(BinaryReaderHandles rCtx, String fieldName) {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), start, null, rCtx);
+        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, PortableHeapInputStream.create(arr, start), null, rCtx);
 
         return (F)reader.unmarshalField(fieldName);
     }
@@ -583,6 +583,6 @@ public final class BinaryObjectImpl extends BinaryObjectEx implements Externaliz
      * @return Reader.
      */
     private BinaryReaderExImpl newReader() {
-        return new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), start, null, new BinaryReaderHandles());
+        return new BinaryReaderExImpl(ctx, PortableHeapInputStream.create(arr, start), null, new BinaryReaderHandles());
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbfd19a4/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
index 91c68cd..d859897 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectOffheapImpl.java
@@ -324,8 +324,11 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Nullable @Override protected <F> F field(BinaryReaderHandles rCtx, String fieldName) {
-        BinaryReaderExImpl reader =
-            new BinaryReaderExImpl(ctx, new PortableOffheapInputStream(ptr, size, false), start, null, rCtx);
+        PortableOffheapInputStream stream = new PortableOffheapInputStream(ptr, size, false);
+
+        stream.position(start);
+
+        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, stream, null, rCtx);
 
         return (F)reader.unmarshalField(fieldName);
     }
@@ -427,7 +430,10 @@ public class BinaryObjectOffheapImpl extends BinaryObjectEx implements Externali
      * @return Reader.
      */
     private BinaryReaderExImpl newReader() {
-        return new BinaryReaderExImpl(ctx, new PortableOffheapInputStream(ptr, size, false), start, null,
-            new BinaryReaderHandles());
+        PortableOffheapInputStream stream = new PortableOffheapInputStream(ptr, size, false);
+
+        stream.position(start);
+
+        return new BinaryReaderExImpl(ctx, stream, null, new BinaryReaderHandles());
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbfd19a4/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
index 60ec1ca..a4d126a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryReaderExImpl.java
@@ -125,14 +125,14 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
     /** Class loaded. */
     private final ClassLoader ldr;
 
-    /** */
-    private BinaryReaderHandles rCtx;
+    /** Reader context which is constantly passed between objects. */
+    private final BinaryReaderHandles rCtx;
 
     /** */
     private PortableClassDescriptor desc;
 
     /** */
-    private int start;
+    private final int start;
 
     /** Flag indicating that object header was parsed. */
     private boolean hdrParsed;
@@ -173,18 +173,17 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
     /**
      * @param ctx Context.
      * @param in Input stream.
-     * @param start Start.
+     * @param ldr Class loader.
      * @param rCtx Context.
      */
-    public BinaryReaderExImpl(PortableContext ctx, PortableInputStream in, int start, ClassLoader ldr,
+    public BinaryReaderExImpl(PortableContext ctx, PortableInputStream in, ClassLoader ldr,
         BinaryReaderHandles rCtx) {
         this.ctx = ctx;
         this.in = in;
-        this.start = start;
         this.ldr = ldr;
         this.rCtx = rCtx;
 
-        in.position(start);
+        start = in.position();
     }
 
     /**
@@ -1764,7 +1763,7 @@ public class BinaryReaderExImpl implements BinaryReader, BinaryRawReaderEx, Obje
      * @throws org.apache.ignite.binary.BinaryObjectException In case of error.
      */
     @Nullable private Object doReadObject() throws BinaryObjectException {
-        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, in, in.position(), ldr, rCtx);
+        BinaryReaderExImpl reader = new BinaryReaderExImpl(ctx, in, ldr, rCtx);
 
         return reader.deserialize();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbfd19a4/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
index dbf4a8b..9c61ef2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
@@ -256,7 +256,7 @@ public class GridPortableMarshaller {
         assert bytes != null;
 
         BinaryReaderExImpl reader =
-            new BinaryReaderExImpl(ctx, new PortableHeapInputStream(bytes), 0, clsLdr, new BinaryReaderHandles());
+            new BinaryReaderExImpl(ctx, PortableHeapInputStream.create(bytes, 0), clsLdr, new BinaryReaderHandles());
 
         return (T)reader.unmarshal();
     }
@@ -286,7 +286,7 @@ public class GridPortableMarshaller {
             return null;
 
         BinaryReaderExImpl reader =
-            new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), 0, ldr, new BinaryReaderHandles());
+            new BinaryReaderExImpl(ctx, PortableHeapInputStream.create(arr, 0), ldr, new BinaryReaderHandles());
 
         return (T)reader.deserialize();
     }
@@ -309,7 +309,7 @@ public class GridPortableMarshaller {
      */
     public BinaryReaderExImpl reader(PortableInputStream in) {
         // TODO: IGNITE-1272 - Is class loader needed here?
-        return new BinaryReaderExImpl(ctx, in, in.position(), null, new BinaryReaderHandles());
+        return new BinaryReaderExImpl(ctx, in, null, new BinaryReaderHandles());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbfd19a4/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
index a9ec65a..907c360 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
@@ -67,7 +67,7 @@ public class PortableBuilderReader implements PortablePositionReadable {
         pos = objImpl.start();
 
         // TODO: IGNITE-1272 - Is class loader needed here?
-        reader = new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), pos, null, new BinaryReaderHandles());
+        reader = new BinaryReaderExImpl(ctx, PortableHeapInputStream.create(arr, pos), null, new BinaryReaderHandles());
     }
 
     /**
@@ -97,8 +97,8 @@ public class PortableBuilderReader implements PortablePositionReadable {
         if (start == pos)
             targetReader = reader;
         else
-            targetReader =
-                new BinaryReaderExImpl(ctx, new PortableHeapInputStream(arr), start, null, new BinaryReaderHandles());
+            targetReader = new BinaryReaderExImpl(ctx, PortableHeapInputStream.create(arr, start), null,
+                new BinaryReaderHandles());
 
         return targetReader.getOrCreateSchema();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cbfd19a4/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableHeapInputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableHeapInputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableHeapInputStream.java
index e027d70..1b39950 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableHeapInputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableHeapInputStream.java
@@ -23,6 +23,23 @@ import java.util.Arrays;
  * Portable off-heap input stream.
  */
 public final class PortableHeapInputStream extends PortableAbstractInputStream {
+    /**
+     * Create stream with pointer set at the given position.
+     *
+     * @param data Data.
+     * @param pos Position.
+     * @return Stream.
+     */
+    public static PortableHeapInputStream create(byte[] data, int pos) {
+        assert pos < data.length;
+
+        PortableHeapInputStream stream = new PortableHeapInputStream(data);
+
+        stream.pos = pos;
+
+        return stream;
+    }
+
     /** Data. */
     private byte[] data;