You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/06/16 15:45:14 UTC

[2/3] incubator-ignite git commit: ignite-950: moved all footer related logic to dedicated class, unmarshalling enclosed objects, wrapping serializable fields with CacheObjectImpl

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e2943db/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectMetadata.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectMetadata.java
new file mode 100644
index 0000000..7154a25
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectMetadata.java
@@ -0,0 +1,149 @@
+/*
+ * 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.marshaller.optimized.ext;
+
+import org.apache.ignite.*;
+import org.apache.ignite.marshaller.optimized.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Metadata that keeps fields information. Used in conjunction with the footer that is added to some objects during
+ * marshalling.
+ */
+public class OptimizedObjectMetadata implements Externalizable {
+    /** */
+    private List<FieldInfo> fieldsInfo;
+
+    /** Constructor. */
+    public OptimizedObjectMetadata() {
+        // No-op
+    }
+
+    /**
+     * Adds meta for a new field.
+     *
+     * @param fieldId Field ID.
+     * @param fieldType Field type.
+     */
+    public void addMeta(int fieldId, OptimizedFieldType fieldType) {
+        if (fieldsInfo == null)
+            fieldsInfo = new ArrayList<>();
+
+        int len = 1;
+
+        switch (fieldType) {
+            case BYTE:
+            case BOOLEAN:
+                len += 1;
+                break;
+
+            case SHORT:
+            case CHAR:
+                len += 2;
+                break;
+
+            case INT:
+            case FLOAT:
+                len += 4;
+                break;
+
+            case LONG:
+            case DOUBLE:
+                len += 8;
+                break;
+
+            case OTHER:
+                len = OptimizedMarshallerExt.VARIABLE_LEN;
+                break;
+
+            default:
+                throw new IgniteException("Unknown field type: " + fieldType);
+        }
+
+        assert len != 1;
+
+        fieldsInfo.add(new FieldInfo(fieldId, len));
+    }
+
+    /**
+     * Gets {@link OptimizedObjectMetadata.FieldInfo} at the {@code index}.
+     *
+     * @param index Position.
+     * @return Field meta info.
+     */
+    public FieldInfo getMeta(int index) {
+        return fieldsInfo.get(index);
+    }
+    /**
+     * Returns all the metadata stored for the object.
+     *
+     * @return Metadata collection.
+     */
+    public List<FieldInfo> getMeta() {
+        return Collections.unmodifiableList(fieldsInfo);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        if (fieldsInfo == null) {
+            out.writeInt(0);
+            return;
+        }
+
+        out.writeInt(fieldsInfo.size());
+
+        for (FieldInfo fieldInfo : fieldsInfo) {
+            out.writeInt(fieldInfo.id);
+            out.writeInt(fieldInfo.len);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        int size = in.readInt();
+
+        fieldsInfo = new ArrayList<>(size);
+
+        for (int i = 0; i < size; i++)
+            fieldsInfo.add(new FieldInfo(in.readInt(), in.readInt()));
+    }
+
+    /**
+     * Field info.
+     */
+    public static class FieldInfo {
+        /** Field ID. */
+        int id;
+
+        /** Field type. */
+        int len;
+
+        /**
+         * Constructor.
+         *
+         * @param id Field ID.
+         * @param len Field len.
+         */
+        public FieldInfo(int id, int len) {
+            this.id = id;
+            this.len = len;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e2943db/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectMetadataKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectMetadataKey.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectMetadataKey.java
new file mode 100644
index 0000000..b9fcd58
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectMetadataKey.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.marshaller.optimized.ext;
+
+import org.apache.ignite.internal.processors.cache.*;
+
+import java.io.*;
+
+/**
+ * Optimized object metadata key.
+ */
+public class OptimizedObjectMetadataKey extends GridCacheUtilityKey<OptimizedObjectMetadataKey>
+    implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private int typeId;
+
+    /**
+     * For {@link Externalizable}.
+     */
+    public OptimizedObjectMetadataKey() {
+        // No-op
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param typeId Type id.
+     */
+    public OptimizedObjectMetadataKey(int typeId) {
+        this.typeId = typeId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeInt(typeId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        typeId = in.readInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean equalsx(OptimizedObjectMetadataKey key) {
+        return typeId == key.typeId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return typeId;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e2943db/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectOutputStreamExt.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectOutputStreamExt.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectOutputStreamExt.java
new file mode 100644
index 0000000..823caf5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectOutputStreamExt.java
@@ -0,0 +1,135 @@
+/*
+ * 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.marshaller.optimized.ext;
+
+import org.apache.ignite.internal.util.io.*;
+import org.apache.ignite.marshaller.*;
+import org.apache.ignite.marshaller.optimized.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.marshaller.optimized.ext.OptimizedMarshallerExt.*;
+import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.*;
+
+/**
+ * TODO: IGNITE-950
+ */
+public class OptimizedObjectOutputStreamExt extends OptimizedObjectOutputStream {
+    /** */
+    private OptimizedMarshallerExtMetaHandler metaHandler;
+
+    /** {@inheritDoc} */
+    protected OptimizedObjectOutputStreamExt(GridDataOutput out) throws IOException {
+        super(out);
+    }
+
+    /**
+     * @param clsMap Class descriptors by class map.
+     * @param ctx Context.
+     * @param mapper ID mapper.
+     * @param requireSer Require {@link Serializable} flag.
+     * @param metaHandler Metadata handler.
+     */
+    protected void context(ConcurrentMap<Class, OptimizedClassDescriptor> clsMap, MarshallerContext ctx,
+        OptimizedMarshallerIdMapper mapper, boolean requireSer, OptimizedMarshallerExtMetaHandler metaHandler) {
+        context(clsMap, ctx, mapper, requireSer);
+
+        this.metaHandler = metaHandler;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeFieldType(byte type) throws IOException {
+        out.writeByte(type);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Footer createFooter(Class<?> cls) {
+        if (metaHandler.metadata(resolveTypeId(cls.getName(), mapper)) != null)
+            return new FooterImpl();
+        else
+            return null;
+    }
+
+    /**
+     *
+     */
+    private class FooterImpl implements OptimizedObjectOutputStream.Footer {
+        /** */
+        private ArrayList<Short> data;
+
+        /** */
+        private int headerPos;
+
+        /** {@inheritDoc} */
+        @Override public void fields(OptimizedClassDescriptor.Fields fields) {
+            if (fields.fieldsIndexingSupported())
+                data = new ArrayList<>();
+            else
+                data = null;
+        }
+
+        /** {@inheritDoc} */
+        public void headerPos(int pos) {
+            headerPos = pos;
+        }
+
+        /** {@inheritDoc} */
+        public void put(int fieldId, OptimizedFieldType fieldType, int len) {
+            if (data == null)
+                return;
+
+            // Considering that field's length will be no longer 2^15 (32 MB)
+            if (fieldType == OptimizedFieldType.OTHER)
+                data.add((short)len);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void putHandle(int fieldId, int handleId) {
+            disable();
+        }
+
+        /** {@inheritDoc} */
+        public void write() throws IOException {
+            if (data == null)
+                writeInt(EMPTY_FOOTER);
+            else {
+                //12 - 4 bytes for len at the beginning, 4 bytes for len at the end, 4 bytes for object len.
+                int footerLen = data.size() * 2 + 12;
+
+                writeInt(footerLen);
+
+                for (short fieldLen : data)
+                    writeShort(fieldLen);
+
+                // object total len
+                writeInt((out.size() - headerPos) + 8);
+
+                writeInt(footerLen);
+            }
+        }
+
+        /**
+         * Disable footer and indexing for the given Object.
+         */
+        private void disable() {
+            data = null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e2943db/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectStreamExtRegistry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectStreamExtRegistry.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectStreamExtRegistry.java
new file mode 100644
index 0000000..e07b4de
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectStreamExtRegistry.java
@@ -0,0 +1,226 @@
+/*
+ * 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.marshaller.optimized.ext;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.io.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.marshaller.optimized.*;
+
+import java.io.*;
+import java.util.concurrent.*;
+
+/**
+ * Storage for object streams.
+ */
+class OptimizedObjectStreamExtRegistry {
+    /** Holders. */
+    private static final ThreadLocal<StreamHolder> holders = new ThreadLocal<>();
+
+    /** Holders pool. */
+    private static BlockingQueue<StreamHolder> pool;
+
+    /**
+     * Ensures singleton.
+     */
+    private OptimizedObjectStreamExtRegistry() {
+        // No-op.
+    }
+
+    /**
+     * Sets streams pool size.
+     *
+     * @param size Streams pool size.
+     */
+    static void poolSize(int size) {
+        if (size > 0) {
+            pool = new LinkedBlockingQueue<>(size);
+
+            for (int i = 0; i < size; i++) {
+                boolean b = pool.offer(new StreamHolder());
+
+                assert b;
+            }
+        }
+        else
+            pool = null;
+    }
+
+    /**
+     * Gets output stream.
+     *
+     * @return Object output stream.
+     * @throws IgniteInterruptedCheckedException If thread is interrupted while trying to take holder from pool.
+     */
+    static OptimizedObjectOutputStreamExt out() throws IgniteInterruptedCheckedException {
+        return holder().acquireOut();
+    }
+
+    /**
+     * Gets input stream.
+     *
+     * @return Object input stream.
+     * @throws IgniteInterruptedCheckedException If thread is interrupted while trying to take holder from pool.
+     */
+    static OptimizedObjectInputStreamExt in() throws IgniteInterruptedCheckedException {
+        return holder().acquireIn();
+    }
+
+    /**
+     * Closes and releases output stream.
+     *
+     * @param out Object output stream.
+     */
+    static void closeOut(OptimizedObjectOutputStream out) {
+        U.close(out, null);
+
+        StreamHolder holder = holders.get();
+
+        holder.releaseOut();
+
+        if (pool != null) {
+            holders.remove();
+
+            boolean b = pool.offer(holder);
+
+            assert b;
+        }
+    }
+
+    /**
+     * Closes and releases input stream.
+     *
+     * @param in Object input stream.
+     */
+    @SuppressWarnings("TypeMayBeWeakened")
+    static void closeIn(OptimizedObjectInputStream in) {
+        U.close(in, null);
+
+        StreamHolder holder = holders.get();
+
+        holder.releaseIn();
+
+        if (pool != null) {
+            holders.remove();
+
+            boolean b = pool.offer(holder);
+
+            assert b;
+        }
+    }
+
+    /**
+     * Gets holder from pool or thread local.
+     *
+     * @return Stream holder.
+     * @throws IgniteInterruptedCheckedException If thread is interrupted while trying to take holder from pool.
+     */
+    private static StreamHolder holder() throws IgniteInterruptedCheckedException {
+        StreamHolder holder = holders.get();
+
+        if (holder == null) {
+            try {
+                holders.set(holder = pool != null ? pool.take() : new StreamHolder());
+            }
+            catch (InterruptedException e) {
+                throw new IgniteInterruptedCheckedException(
+                    "Failed to take object stream from pool (thread interrupted).", e);
+            }
+        }
+
+        return holder;
+    }
+
+    /**
+     * Streams holder.
+     */
+    private static class StreamHolder {
+        /** Output stream. */
+        private final OptimizedObjectOutputStreamExt out = createOut();
+
+        /** Input stream. */
+        private final OptimizedObjectInputStreamExt in = createIn();
+
+        /** Output streams counter. */
+        private int outAcquireCnt;
+
+        /** Input streams counter. */
+        private int inAcquireCnt;
+
+        /**
+         * Gets output stream.
+         *
+         * @return Object output stream.
+         */
+        OptimizedObjectOutputStreamExt acquireOut() {
+            return outAcquireCnt++ > 0 ? createOut() : out;
+        }
+
+        /**
+         * Gets input stream.
+         *
+         * @return Object input stream.
+         */
+        OptimizedObjectInputStreamExt acquireIn() {
+            return inAcquireCnt++ > 0 ? createIn() : in;
+        }
+
+        /**
+         * Releases output stream.
+         */
+        void releaseOut() {
+            outAcquireCnt--;
+        }
+
+        /**
+         * Releases input stream.
+         */
+        void releaseIn() {
+            inAcquireCnt--;
+        }
+
+        /**
+         * Creates output stream.
+         *
+         * @return Object output stream.
+         */
+        private OptimizedObjectOutputStreamExt createOut() {
+            try {
+                return new OptimizedObjectOutputStreamExt(new GridUnsafeDataOutput(4 * 1024));
+            }
+            catch (IOException e) {
+                throw new IgniteException("Failed to create object output stream.", e);
+            }
+        }
+
+        /**
+         * Creates input stream.
+         *
+         * @return Object input stream.
+         */
+        private OptimizedObjectInputStreamExt createIn() {
+            try {
+                return new OptimizedObjectInputStreamExt(new GridUnsafeDataInput());
+            }
+            catch (IOException e) {
+                throw new IgniteException("Failed to create object input stream.", e);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e2943db/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/package-info.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/package-info.java
new file mode 100644
index 0000000..294ef7f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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 description. -->
+ * Contains <b>extended</b> Optimized marshaller.
+ */
+package org.apache.ignite.marshaller.optimized.ext;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e2943db/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerSelfTest.java
index 6c78506..7d09070 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerSelfTest.java
@@ -111,19 +111,6 @@ public class OptimizedMarshallerSelfTest extends GridMarshallerAbstractTest {
     }
 
     /**
-     * @throws Exception If failed.
-     */
-    public void testFieldUnmarshalling() throws Exception {
-        TestObject2 obj = new TestObject2(5);
-
-        byte[] data = marshal(obj);
-
-        Integer i = ((OptimizedMarshaller)marsh).unmarshal("i", data, Thread.currentThread().getContextClassLoader());
-
-        assertEquals(obj.i, (int)i);
-    }
-
-    /**
      * Class for nested execution test.
      */
     private static class NestedTestObject implements Serializable {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e2943db/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedObjectStreamSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedObjectStreamSelfTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedObjectStreamSelfTest.java
index 0324e2f..20391eb 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedObjectStreamSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedObjectStreamSelfTest.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.marshaller.optimized;
 
 import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.io.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -47,17 +46,6 @@ public class OptimizedObjectStreamSelfTest extends GridCommonAbstractTest {
     /** */
     private ConcurrentMap<Class, OptimizedClassDescriptor> clsMap = new ConcurrentHashMap8<>();
 
-    /** */
-    private static final OptimizedObjectMetadataHandler META_HANDLER = new OptimizedObjectMetadataHandler() {
-        @Override public void addMeta(int typeId, OptimizedObjectMetadata meta) {
-
-        }
-
-        @Override public OptimizedObjectMetadata metadata(int typeId) {
-            return null;
-        }
-    };
-
     /**
      * @throws Exception If failed.
      */
@@ -1033,7 +1021,7 @@ public class OptimizedObjectStreamSelfTest extends GridCommonAbstractTest {
         try {
             out = OptimizedObjectStreamRegistry.out();
 
-            out.context(clsMap, CTX, null, META_HANDLER, true);
+            out.context(clsMap, CTX, null, true);
 
             out.writeObject(obj);
 
@@ -1041,7 +1029,7 @@ public class OptimizedObjectStreamSelfTest extends GridCommonAbstractTest {
 
             in = OptimizedObjectStreamRegistry.in();
 
-            in.context(clsMap, CTX, null, META_HANDLER, getClass().getClassLoader());
+            in.context(clsMap, CTX, null, getClass().getClassLoader());
 
             in.in().bytes(arr, arr.length);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e2943db/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtSelfTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtSelfTest.java
new file mode 100644
index 0000000..d663c55
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtSelfTest.java
@@ -0,0 +1,179 @@
+/*
+ * 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.marshaller.optimized.ext;
+
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.marshaller.*;
+import org.apache.ignite.marshaller.optimized.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.concurrent.*;
+
+/**
+ * Optimized marshaller self test.
+ */
+@GridCommonTest(group = "Marshaller")
+public class OptimizedMarshallerExtSelfTest extends OptimizedMarshallerSelfTest {
+    /** {@inheritDoc} */
+    @Override protected Marshaller marshaller() {
+        return new OptimizedMarshallerExt(false);
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testHasField() throws Exception {
+        OptimizedMarshallerExt marsh = (OptimizedMarshallerExt)OptimizedMarshallerExtSelfTest.marsh;
+
+        assertTrue(marsh.putMetaForClass(TestObject.class));
+
+        TestObject testObj = new TestObject("World", 50);
+
+        byte[] arr = marsh.marshal(testObj);
+
+        assertTrue(marsh.hasField("o2", arr, 0, arr.length));
+        assertTrue(marsh.hasField("str", arr, 0, arr.length));
+
+        assertFalse(marsh.hasField("m", arr, 0, arr.length));
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testReadField() throws Exception {
+        OptimizedMarshallerExt marsh = (OptimizedMarshallerExt)OptimizedMarshallerExtSelfTest.marsh;
+
+        assertTrue(marsh.putMetaForClass(TestObject.class));
+
+        TestObject testObj = new TestObject("World", 50);
+
+        byte[] arr = marsh.marshal(testObj);
+
+        // Simple field extraction.
+
+        String text = marsh.readField("str", arr, 0, arr.length, null);
+
+        assertEquals(testObj.str, text);
+
+        // Serializable extraction (doesn't have meta, thus doesn't have footer)
+        TestObject2 o2 = marsh.readField("o2", arr, 0, arr.length, null);
+
+        assertEquals(testObj.o2, o2);
+
+        // Add metadata for the enclosed object.
+        assertTrue(marsh.putMetaForClass(TestObject2.class));
+
+        arr = marsh.marshal(testObj);
+
+        // Must be returned in a wrapped form, since metadata was added enabling the footer.
+        CacheObjectImpl cacheObject = marsh.readField("o2", arr, 0, arr.length, null);
+
+        arr = cacheObject.valueBytes(null);
+
+        // Check enclosed objects fields
+        assertTrue(marsh.hasField("i", arr, 0, arr.length));
+        assertEquals(testObj.o2.i, (int)marsh.readField("i", arr, 0, arr.length, null));
+    }
+
+    /** */
+    private static class TestObject2 {
+        /** */
+        private final int i;
+
+        /**
+         * Constructor for TestObject2 instances.
+         *
+         * @param i Integer value to hold.
+         */
+        private TestObject2(int i) {
+            this.i = i;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            return i == ((TestObject2)o).i;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return i;
+        }
+    }
+
+    /**
+     * Static nested class.
+     */
+    private static class TestObject {
+        /** */
+        private final TestObject2 o2;
+
+        /** The only meaningful field in the class, used for {@link #equals(Object o)} and {@link #hashCode()}. */
+        private final String str;
+
+        /**
+         * @param str String to hold.
+         * @param i Integer.
+         */
+        TestObject(String str, int i) {
+            this.str = str;
+
+            o2 = new TestObject2(i);
+        }
+
+        /**
+         * Method for accessing value of the hold string after the object is created.
+         *
+         * @return Wrapped string.
+         */
+        public String string() {
+            return str;
+        }
+
+        /**
+         * @return Object held in this wrapped.
+         */
+        public TestObject2 obj() {
+            return o2;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return 31 * o2.hashCode() + str.hashCode();
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("RedundantIfStatement")
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestObject obj = (TestObject)o;
+
+            if (o2 != null ? !o2.equals(obj.o2) : obj.o2 != null)
+                return false;
+
+            if (str != null ? !str.equals(obj.str) : obj.str != null)
+                return false;
+
+            return true;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e2943db/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/package-info.java b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/package-info.java
new file mode 100644
index 0000000..eabdbce
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 description. -->
+ * Extended optimized marshaller test package.
+ */
+package org.apache.ignite.marshaller.optimized.ext;
\ No newline at end of file