You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2016/02/04 11:09:39 UTC

[36/50] [abbrv] ignite git commit: Page memory WIP.

Page memory WIP.


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

Branch: refs/heads/sql-store
Commit: ddfe632e20c9b13fb0e06490088d007abe1bac1b
Parents: 500bd3a
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Jan 19 18:58:36 2016 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Feb 3 17:13:40 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/binary/BinaryContext.java   |   2 +-
 .../ignite/internal/binary/BinaryFieldEx.java   |  45 +++
 .../ignite/internal/binary/BinaryFieldImpl.java | 173 ++++++++++-
 .../internal/binary/BinaryObjectExImpl.java     |  10 +
 .../internal/binary/BinaryObjectImpl.java       | 116 ++++++++
 .../binary/BinaryObjectOffheapImpl.java         |   5 +
 .../streams/BinaryByteBufferInputStream.java    | 291 +++++++++++++++++++
 .../binary/BinaryFieldExtractionSelfTest.java   | 161 ++++++++++
 8 files changed, 800 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ddfe632e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
index cf19bdf..35723df 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
@@ -1010,7 +1010,7 @@ public class BinaryContext {
 
         int fieldId = mapper.fieldId(typeId, fieldName);
 
-        return new BinaryFieldImpl(typeId, schemaReg, fieldName, fieldId);
+        return new BinaryFieldImpl(this, typeId, schemaReg, fieldName, fieldId);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ddfe632e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldEx.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldEx.java
new file mode 100644
index 0000000..42aa282
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldEx.java
@@ -0,0 +1,45 @@
+/*
+ * 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.binary;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.binary.BinaryField;
+import org.apache.ignite.binary.BinaryObject;
+
+/**
+ *
+ */
+public interface BinaryFieldEx extends BinaryField {
+    /**
+     * Writes field value to the given byte buffer.
+     *
+     * @param obj Object from which the field should be extracted.
+     * @param buf Buffer to write the field value to.
+     * @return {@code True} if the value was successfully written, {@code false} if there is not enough space
+     *      for the field in the buffer.
+     */
+    public boolean writeField(BinaryObject obj, ByteBuffer buf);
+
+    /**
+     * Reads field value from the given byte buffer.
+     *
+     * @param buf Buffer to read value from.
+     * @return Field value.
+     */
+    public <F> F readField(ByteBuffer buf);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ddfe632e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldImpl.java
index 78ed17a..4b59904 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldImpl.java
@@ -17,16 +17,29 @@
 
 package org.apache.ignite.internal.binary;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.sql.Timestamp;
+import java.util.Date;
+import java.util.UUID;
 import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.streams.BinaryByteBufferInputStream;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryField;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+
 /**
  * Implementation of binary field descriptor.
  */
-public class BinaryFieldImpl implements BinaryField {
+public class BinaryFieldImpl implements BinaryFieldEx {
+    /** Binary context that created this field. */
+    private final BinaryContext ctx;
+
     /** Type ID. */
     private final int typeId;
 
@@ -47,12 +60,20 @@ public class BinaryFieldImpl implements BinaryField {
      * @param fieldName Field name.
      * @param fieldId Field ID.
      */
-    public BinaryFieldImpl(int typeId, BinarySchemaRegistry schemas, String fieldName, int fieldId) {
+    public BinaryFieldImpl(
+        BinaryContext ctx,
+        int typeId,
+        BinarySchemaRegistry schemas,
+        String fieldName,
+        int fieldId
+    ) {
+        assert ctx != null;
         assert typeId != 0;
         assert schemas != null;
         assert fieldName != null;
         assert fieldId != 0;
 
+        this.ctx = ctx;
         this.typeId = typeId;
         this.schemas = schemas;
         this.fieldName = fieldName;
@@ -81,6 +102,154 @@ public class BinaryFieldImpl implements BinaryField {
         return order != BinarySchema.ORDER_NOT_FOUND ? (T)obj0.fieldByOrder(order) : null;
     }
 
+    /** {@inheritDoc} */
+    @Override public boolean writeField(BinaryObject obj, ByteBuffer buf) {
+        BinaryObjectExImpl obj0 = (BinaryObjectExImpl)obj;
+
+        int order = fieldOrder(obj0);
+
+        return obj0.writeFieldByOrder(order, buf);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <F> F readField(ByteBuffer buf) {
+        ByteOrder oldOrder = buf.order();
+
+        try {
+            buf.order(ByteOrder.LITTLE_ENDIAN);
+
+            int pos = buf.position();
+
+            byte hdr = buf.get();
+
+            Object val;
+
+            switch (hdr) {
+                case GridBinaryMarshaller.INT:
+                    val = buf.getInt();
+
+                    break;
+
+                case GridBinaryMarshaller.LONG:
+                    val = buf.getLong();
+
+                    break;
+
+                case GridBinaryMarshaller.BOOLEAN:
+                    val = buf.get() != 0;
+
+                    break;
+
+                case GridBinaryMarshaller.SHORT:
+                    val = buf.getShort();
+
+                    break;
+
+                case GridBinaryMarshaller.BYTE:
+                    val = buf.get();
+
+                    break;
+
+                case GridBinaryMarshaller.CHAR:
+                    val = buf.getChar();
+
+                    break;
+
+                case GridBinaryMarshaller.FLOAT:
+                    val = buf.getFloat();
+
+                    break;
+
+                case GridBinaryMarshaller.DOUBLE:
+                    val = buf.getDouble();
+
+                    break;
+
+                case GridBinaryMarshaller.STRING: {
+                    int dataLen = buf.getInt();
+
+                    byte[] data = new byte[dataLen];
+
+                    buf.get(data);
+
+                    val = new String(data, 0, dataLen, UTF_8);
+
+                    break;
+                }
+
+                case GridBinaryMarshaller.DATE: {
+                    long time = buf.getLong();
+
+                    val = new Date(time);
+
+                    break;
+                }
+
+                case GridBinaryMarshaller.TIMESTAMP: {
+                    long time = buf.getLong();
+                    int nanos = buf.getInt();
+
+                    Timestamp ts = new Timestamp(time);
+
+                    ts.setNanos(ts.getNanos() + nanos);
+
+                    val = ts;
+
+                    break;
+                }
+
+                case GridBinaryMarshaller.UUID: {
+                    long most = buf.getLong();
+                    long least = buf.getLong();
+
+                    val = new UUID(most, least);
+
+                    break;
+                }
+
+                case GridBinaryMarshaller.DECIMAL: {
+                    int scale = buf.getInt();
+
+                    int dataLen = buf.getInt();
+
+                    byte[] data = new byte[dataLen];
+
+                    buf.get(data);
+
+                    BigInteger intVal = new BigInteger(data);
+
+                    if (scale < 0) {
+                        scale &= 0x7FFFFFFF;
+
+                        intVal = intVal.negate();
+                    }
+
+                    val = new BigDecimal(intVal, scale);
+
+                    break;
+                }
+
+                case GridBinaryMarshaller.NULL:
+                    val = null;
+
+                    break;
+
+                default:
+                    // Restore buffer position.
+                    buf.position(pos);
+
+                    val = BinaryUtils.unmarshal(BinaryByteBufferInputStream.create(buf), ctx, null);
+
+                    break;
+            }
+
+            return (F)val;
+        }
+        finally {
+            buf.order(oldOrder);
+        }
+    }
+
     /**
      * Get relative field offset.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/ddfe632e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
index ddf17f0..4ea5935 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.binary;
 
 import java.math.BigDecimal;
+import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.IdentityHashMap;
 import org.apache.ignite.IgniteException;
@@ -82,6 +83,15 @@ public abstract class BinaryObjectExImpl implements BinaryObjectEx {
     @Nullable protected abstract <F> F fieldByOrder(int fieldOffset);
 
     /**
+     * Writes field value defined by the given field offset to the given byte buffer.
+     *
+     * @param fieldOffset Field offset.
+     * @return Boolean flag indicating whether the field was successfully written to the buffer, {@code false}
+     *      if there is no enough space for the field in the buffer.
+     */
+    protected abstract boolean writeFieldByOrder(int fieldOffset, ByteBuffer buf);
+
+    /**
      * @param ctx Reader context.
      * @param fieldName Field name.
      * @return Field value.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ddfe632e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
index 44c7a08..a500ceb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
@@ -385,6 +385,122 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("IfMayBeConditional")
+    @Override public boolean writeFieldByOrder(int order, ByteBuffer buf) {
+        // Calculate field position.
+        int schemaOffset = BinaryPrimitives.readInt(arr, start + GridBinaryMarshaller.SCHEMA_OR_RAW_OFF_POS);
+
+        short flags = BinaryPrimitives.readShort(arr, start + GridBinaryMarshaller.FLAGS_POS);
+
+        int fieldIdLen = BinaryUtils.isCompactFooter(flags) ? 0 : BinaryUtils.FIELD_ID_LEN;
+        int fieldOffsetLen = BinaryUtils.fieldOffsetLength(flags);
+
+        int fieldOffsetPos = start + schemaOffset + order * (fieldIdLen + fieldOffsetLen) + fieldIdLen;
+
+        int fieldPos;
+
+        if (fieldOffsetLen == BinaryUtils.OFFSET_1)
+            fieldPos = start + ((int)BinaryPrimitives.readByte(arr, fieldOffsetPos) & 0xFF);
+        else if (fieldOffsetLen == BinaryUtils.OFFSET_2)
+            fieldPos = start + ((int)BinaryPrimitives.readShort(arr, fieldOffsetPos) & 0xFFFF);
+        else
+            fieldPos = start + BinaryPrimitives.readInt(arr, fieldOffsetPos);
+
+        // Read header and try performing fast lookup for well-known types (the most common types go first).
+        byte hdr = BinaryPrimitives.readByte(arr, fieldPos);
+
+        int totalLen;
+
+        switch (hdr) {
+            case GridBinaryMarshaller.NULL:
+                totalLen = 1;
+
+                break;
+
+            case GridBinaryMarshaller.INT:
+            case GridBinaryMarshaller.FLOAT:
+                totalLen = 5;
+
+                break;
+
+            case GridBinaryMarshaller.LONG:
+            case GridBinaryMarshaller.DOUBLE:
+            case GridBinaryMarshaller.DATE:
+                totalLen = 9;
+
+                break;
+
+            case GridBinaryMarshaller.BOOLEAN:
+                totalLen = 2;
+
+                break;
+
+            case GridBinaryMarshaller.SHORT:
+                totalLen = 3;
+
+                break;
+
+            case GridBinaryMarshaller.BYTE:
+                totalLen = 2;
+
+                break;
+
+            case GridBinaryMarshaller.CHAR:
+                totalLen = 3;
+
+                break;
+
+            case GridBinaryMarshaller.STRING: {
+                int dataLen = BinaryPrimitives.readInt(arr, fieldPos + 1);
+
+                totalLen = dataLen + 5;
+
+                break;
+            }
+
+            case GridBinaryMarshaller.TIMESTAMP:
+                totalLen = 13;
+
+                break;
+
+            case GridBinaryMarshaller.UUID:
+                totalLen = 17;
+
+                break;
+
+            case GridBinaryMarshaller.DECIMAL: {
+                int dataLen = BinaryPrimitives.readInt(arr, fieldPos + 5);
+
+                totalLen = dataLen + 9;
+
+                break;
+            }
+
+            case GridBinaryMarshaller.OBJ:
+                totalLen = BinaryPrimitives.readInt(arr, fieldPos + GridBinaryMarshaller.TOTAL_LEN_POS);
+
+                break;
+
+            case GridBinaryMarshaller.OPTM_MARSH:
+                totalLen = BinaryPrimitives.readInt(arr, fieldPos + 1);
+
+                break;
+
+            default:
+                throw new UnsupportedOperationException("Failed to write field of the given type " +
+                    "(field type is not supported): " + hdr);
+
+        }
+
+        if (buf.remaining() < totalLen)
+            return false;
+
+        buf.put(arr, fieldPos, totalLen);
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Nullable @Override protected <F> F field(BinaryReaderHandles rCtx, String fieldName) {
         return (F)reader(rCtx).unmarshalField(fieldName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ddfe632e/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
index 07ab4d3..ef6dac8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
@@ -299,6 +299,11 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
     }
 
     /** {@inheritDoc} */
+    @Override protected boolean writeFieldByOrder(int fieldOffset, ByteBuffer buf) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Nullable @Override protected <F> F field(BinaryReaderHandles rCtx, String fieldName) {
         return (F)reader(rCtx).unmarshalField(fieldName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ddfe632e/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryByteBufferInputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryByteBufferInputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryByteBufferInputStream.java
new file mode 100644
index 0000000..d4eba83
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/streams/BinaryByteBufferInputStream.java
@@ -0,0 +1,291 @@
+/*
+ * 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.binary.streams;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.binary.BinaryObjectException;
+
+/**
+ *
+ */
+public class BinaryByteBufferInputStream implements BinaryInputStream {
+    /** */
+    private ByteBuffer buf;
+
+    /**
+     * @param buf Buffer to wrap.
+     * @return Stream.
+     */
+    public static BinaryByteBufferInputStream create(ByteBuffer buf) {
+        return new BinaryByteBufferInputStream(buf);
+    }
+
+    /**
+     * @param buf Buffer to get data from.
+     */
+    BinaryByteBufferInputStream(ByteBuffer buf) {
+        this.buf = buf;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte readByte() {
+        ensureHasData(1);
+
+        return buf.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] readByteArray(int cnt) {
+        ensureHasData(cnt);
+
+        byte[] data = new byte[cnt];
+
+        buf.get(data);
+
+        return data;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int read(byte[] arr, int off, int cnt) {
+        ensureHasData(cnt);
+
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readBoolean() {
+        ensureHasData(1);
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean[] readBooleanArray(int cnt) {
+        ensureHasData(cnt);
+
+        boolean[] res = new boolean[cnt];
+
+        for (int i = 0; i < cnt; i++)
+            res[i] = buf.get() != 0;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public short readShort() {
+        ensureHasData(2);
+
+        return buf.getShort();
+    }
+
+    /** {@inheritDoc} */
+    @Override public short[] readShortArray(int cnt) {
+        ensureHasData(2 * cnt);
+
+        short[] res = new short[cnt];
+
+        for (int i = 0; i < cnt; i++)
+            res[i] = buf.getShort();
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public char readChar() {
+        ensureHasData(2);
+
+        return buf.getChar();
+    }
+
+    /** {@inheritDoc} */
+    @Override public char[] readCharArray(int cnt) {
+        ensureHasData(2 * cnt);
+
+        char[] res = new char[cnt];
+
+        for (int i = 0; i < cnt; i++)
+            res[i] = buf.getChar();
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int readInt() {
+        ensureHasData(4);
+
+        return buf.getInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int[] readIntArray(int cnt) {
+        ensureHasData(4 * cnt);
+
+        int[] res = new int[cnt];
+
+        for (int i = 0; i < cnt; i++)
+            res[i] = buf.getInt();
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float readFloat() {
+        ensureHasData(4);
+
+        return buf.getFloat();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float[] readFloatArray(int cnt) {
+        ensureHasData(4 * cnt);
+
+        float[] res = new float[cnt];
+
+        for (int i = 0; i < cnt; i++)
+            res[i] = buf.getFloat();
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long readLong() {
+        ensureHasData(8);
+
+        return buf.getLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long[] readLongArray(int cnt) {
+        ensureHasData(8 * cnt);
+
+        long[] res = new long[cnt];
+
+        for (int i = 0; i < cnt; i++)
+            res[i] = buf.getLong();
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public double readDouble() {
+        ensureHasData(8);
+
+        return buf.getDouble();
+    }
+
+    /** {@inheritDoc} */
+    @Override public double[] readDoubleArray(int cnt) {
+        ensureHasData(8 * cnt);
+
+        double[] res = new double[cnt];
+
+        for (int i = 0; i < cnt; i++)
+            res[i] = buf.getDouble();
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int remaining() {
+        return buf.remaining();
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte readBytePositioned(int pos) {
+        int oldPos = buf.position();
+
+        buf.position(pos);
+
+        ensureHasData(1);
+
+        byte res = buf.get();
+
+        buf.position(oldPos);
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public short readShortPositioned(int pos) {
+        int oldPos = buf.position();
+
+        buf.position(pos);
+
+        ensureHasData(2);
+
+        short res = buf.getShort();
+
+        buf.position(oldPos);
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int readIntPositioned(int pos) {
+        int oldPos = buf.position();
+
+        buf.position(pos);
+
+        ensureHasData(4);
+
+        byte res = buf.get();
+
+        buf.position(oldPos);
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int position() {
+        return buf.position();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void position(int pos) {
+        buf.position(pos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] array() {
+        return buf.array();
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] arrayCopy() {
+        return buf.array();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long offheapPointer() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasArray() {
+        return false;
+    }
+
+    /**
+     * @param cnt Remaining bytes.
+     */
+    private void ensureHasData(int cnt) {
+        if (buf.remaining() < cnt)
+            throw new BinaryObjectException("Not enough data to read the value " +
+                "[requiredBytes=" + cnt + ", remainingBytes=" + buf.remaining() + ']');
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ddfe632e/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldExtractionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldExtractionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldExtractionSelfTest.java
new file mode 100644
index 0000000..5721128
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldExtractionSelfTest.java
@@ -0,0 +1,161 @@
+/*
+ * 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.binary;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.marshaller.MarshallerContextTestImpl;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class BinaryFieldExtractionSelfTest extends GridCommonAbstractTest {
+    /**
+     * Create marshaller.
+     *
+     * @return Binary marshaller.
+     * @throws Exception If failed.
+     */
+    protected BinaryMarshaller createMarshaller() throws Exception {
+        BinaryContext ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration(),
+            log());
+
+        BinaryMarshaller marsh = new BinaryMarshaller();
+
+        BinaryConfiguration bCfg = new BinaryConfiguration();
+
+        IgniteConfiguration iCfg = new IgniteConfiguration();
+
+        iCfg.setBinaryConfiguration(bCfg);
+
+        marsh.setContext(new MarshallerContextTestImpl(null));
+
+        IgniteUtils.invoke(BinaryMarshaller.class, marsh, "setBinaryContext", ctx, iCfg);
+
+        return marsh;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPrimitiveMarshalling() throws Exception {
+        BinaryMarshaller marsh = createMarshaller();
+
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        TestObject obj = new TestObject(0);
+
+        BinaryObjectImpl binObj = toBinary(obj, marsh);
+
+        BinaryFieldEx[] fields = new BinaryFieldEx[] {
+            (BinaryFieldEx)binObj.type().field("bVal"),
+            (BinaryFieldEx)binObj.type().field("cVal"),
+            (BinaryFieldEx)binObj.type().field("sVal"),
+            (BinaryFieldEx)binObj.type().field("iVal"),
+            (BinaryFieldEx)binObj.type().field("lVal"),
+            (BinaryFieldEx)binObj.type().field("fVal"),
+            (BinaryFieldEx)binObj.type().field("dVal")
+        };
+
+        ByteBuffer buf = ByteBuffer.allocate(1024 * 1024);
+
+        for (int i = 0; i < 100; i++) {
+            TestObject to = new TestObject(rnd.nextLong());
+
+            BinaryObjectImpl bObj = toBinary(to, marsh);
+
+            for (BinaryFieldEx field : fields)
+                field.writeField(bObj, buf);
+
+            buf.flip();
+
+            for (BinaryFieldEx field : fields)
+                assertEquals(field.value(bObj), field.readField(buf));
+
+            buf.flip();
+        }
+    }
+
+    /**
+     * @param obj Object to transform to a binary object.
+     * @param marsh Binary marshaller.
+     * @return Binary object.
+     */
+    protected BinaryObjectImpl toBinary(Object obj, BinaryMarshaller marsh) throws Exception {
+        byte[] bytes = marsh.marshal(obj);
+
+        return new BinaryObjectImpl(binaryContext(marsh), bytes, 0);
+    }
+
+    /**
+     * Get binary context for the current marshaller.
+     *
+     * @param marsh Marshaller.
+     * @return Binary context.
+     */
+    protected static BinaryContext binaryContext(BinaryMarshaller marsh) {
+        GridBinaryMarshaller impl = U.field(marsh, "impl");
+
+        return impl.context();
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private static class TestObject {
+        /** */
+        private byte bVal;
+
+        /** */
+        private char cVal;
+
+        /** */
+        private short sVal;
+
+        /** */
+        private int iVal;
+
+        /** */
+        private long lVal;
+
+        /** */
+        private float fVal;
+
+        /** */
+        private double dVal;
+
+        /**
+         * @param seed Seed.
+         */
+        private TestObject(long seed) {
+            bVal = (byte)seed;
+            cVal = (char)seed;
+            sVal = (short)seed;
+            iVal = (int)seed;
+            lVal = seed;
+            fVal = seed;
+            dVal = seed;
+        }
+    }
+}