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/23 15:53:16 UTC

[33/33] incubator-ignite git commit: ignite-950: implementing marshal aware

ignite-950: implementing marshal aware


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

Branch: refs/heads/ignite-950
Commit: 5749b068d623072299b46eb1fb099c8e3636d5ee
Parents: d9f85e6
Author: Denis Magda <dm...@gridgain.com>
Authored: Tue Jun 23 16:51:33 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue Jun 23 16:51:33 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgnitionEx.java  |   1 -
 .../cache/CacheIndexedObjectImpl.java           |   2 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |   2 +-
 .../IgniteCacheObjectProcessorImpl.java         |   3 +-
 .../optimized/OptimizedClassDescriptor.java     |  38 +-
 .../optimized/OptimizedFieldsReader.java        | 196 ++++++++++
 .../optimized/OptimizedFieldsWriter.java        | 196 ++++++++++
 .../optimized/OptimizedMarshalAware.java        |  52 +++
 .../OptimizedMarshalAwareMetaCollector.java     | 179 +++++++++
 .../optimized/OptimizedMarshallerExt.java       | 357 +++++++++++++++++
 .../OptimizedMarshallerMetaHandler.java         |  40 ++
 .../optimized/OptimizedMarshallerUtils.java     |   3 +
 .../optimized/OptimizedObjectInputStream.java   | 392 ++++++++++++++++++-
 .../OptimizedObjectInputStreamExt.java          |  51 +++
 .../optimized/OptimizedObjectMetadata.java      | 172 ++++++++
 .../optimized/OptimizedObjectMetadataKey.java   |  70 ++++
 .../optimized/OptimizedObjectOutputStream.java  | 209 +++++++++-
 .../OptimizedObjectOutputStreamExt.java         | 152 +++++++
 .../OptimizedObjectStreamExtRegistry.java       | 225 +++++++++++
 .../optimized/ext/OptimizedMarshallerExt.java   | 353 -----------------
 .../ext/OptimizedMarshallerExtMetaHandler.java  |  40 --
 .../ext/OptimizedObjectInputStreamExt.java      | 241 ------------
 .../optimized/ext/OptimizedObjectMetadata.java  | 157 --------
 .../ext/OptimizedObjectMetadataKey.java         |  70 ----
 .../ext/OptimizedObjectOutputStreamExt.java     | 179 ---------
 .../ext/OptimizedObjectStreamExtRegistry.java   | 226 -----------
 .../marshaller/optimized/ext/package-info.java  |  21 -
 .../ext/OptimizedMarshallerExtSelfTest.java     |  75 +++-
 .../junits/IgniteTestResources.java             |   1 -
 ...acheOptimizedMarshallerExtQuerySelfTest.java |   4 +-
 30 files changed, 2405 insertions(+), 1302 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 422f517..7356d85 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -32,7 +32,6 @@ import org.apache.ignite.logger.java.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.jdk.*;
 import org.apache.ignite.marshaller.optimized.*;
-import org.apache.ignite.marshaller.optimized.ext.*;
 import org.apache.ignite.mxbean.*;
 import org.apache.ignite.plugin.segmentation.*;
 import org.apache.ignite.resources.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheIndexedObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheIndexedObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheIndexedObjectImpl.java
index 58e9c97..47b83ae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheIndexedObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheIndexedObjectImpl.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.marshaller.optimized.ext.*;
+import org.apache.ignite.marshaller.optimized.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 import org.jetbrains.annotations.*;
 import sun.misc.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
index 42ccbce..a4964b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
@@ -22,7 +22,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.marshaller.optimized.ext.*;
+import org.apache.ignite.marshaller.optimized.*;
 import org.jetbrains.annotations.*;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index 6ae958a..d29b64a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -29,7 +29,6 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.optimized.*;
-import org.apache.ignite.marshaller.optimized.ext.*;
 import org.jetbrains.annotations.*;
 
 import java.math.*;
@@ -90,7 +89,7 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
         if (marsh instanceof OptimizedMarshallerExt) {
             optMarshExt = (OptimizedMarshallerExt)marsh;
 
-            OptimizedMarshallerExtMetaHandler metaHandler = new OptimizedMarshallerExtMetaHandler() {
+            OptimizedMarshallerMetaHandler metaHandler = new OptimizedMarshallerMetaHandler() {
                 @Override public void addMeta(int typeId, OptimizedObjectMetadata meta) {
                     if (metaBuf.contains(typeId))
                         return;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
index d77551f..126568e 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
@@ -329,6 +329,21 @@ public class OptimizedClassDescriptor {
                         throw new IOException("Externalizable class doesn't have default constructor: " + cls, e);
                     }
                 }
+                else if (OptimizedMarshalAware.class.isAssignableFrom(cls)) {
+                    type = MARSHAL_AWARE;
+
+                    try {
+                        constructor = !Modifier.isStatic(cls.getModifiers()) && cls.getDeclaringClass() != null ?
+                            cls.getDeclaredConstructor(cls.getDeclaringClass()) :
+                            cls.getDeclaredConstructor();
+
+                        constructor.setAccessible(true);
+                    }
+                    catch (NoSuchMethodException e) {
+                        throw new IOException("OptimizedMarshalAware class doesn't have default constructor: " + cls,
+                            e);
+                    }
+                }
                 else {
                     type = SERIALIZABLE;
 
@@ -408,8 +423,6 @@ public class OptimizedClassDescriptor {
                                 isPrivate(mod) && isStatic(mod) && isFinal(mod)) {
                                 hasSerialPersistentFields = true;
 
-                                fieldsIndexingSupported = false;
-
                                 serFieldsDesc.setAccessible(true);
 
                                 ObjectStreamField[] serFields = (ObjectStreamField[]) serFieldsDesc.get(null);
@@ -718,6 +731,22 @@ public class OptimizedClassDescriptor {
 
                 break;
 
+            case MARSHAL_AWARE:
+                writeTypeData(out);
+
+                out.writeShort(checksum);
+                out.writeMarshalAware(obj);
+
+                if (out.metaHandler.metadata(typeId) == null) {
+                    OptimizedMarshalAwareMetaCollector collector = new OptimizedMarshalAwareMetaCollector();
+
+                    ((OptimizedMarshalAware)obj).writeFields(collector);
+
+                    out.metaHandler.addMeta(typeId, collector.meta());
+                }
+
+                break;
+
             case SERIALIZABLE:
                 if (out.requireSerializable() && !isSerial)
                     throw new NotSerializableException("Must implement java.io.Serializable or " +
@@ -770,6 +799,11 @@ public class OptimizedClassDescriptor {
 
                 return in.readSerializable(cls, readObjMtds, readResolveMtd, fields);
 
+            case MARSHAL_AWARE:
+                verifyChecksum(in.readShort());
+
+                return in.readMarshalAware(constructor, readResolveMtd);
+
             default:
                 assert false : "Unexpected type: " + type;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedFieldsReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedFieldsReader.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedFieldsReader.java
new file mode 100644
index 0000000..57a59ed
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedFieldsReader.java
@@ -0,0 +1,196 @@
+/*
+ * 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;
+
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * TODO: IGNITE-950
+ */
+public interface OptimizedFieldsReader {
+    /**
+     * @param fieldName Field name.
+     * @return Byte value.
+     * @throws IOException In case of error.
+     */
+    public byte readByte(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Short value.
+     * @throws IOException In case of error.
+     */
+    public short readShort(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Integer value.
+     * @throws IOException In case of error.
+     */
+    public int readInt(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Long value.
+     * @throws IOException In case of error.
+     */
+    public long readLong(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @throws IOException In case of error.
+     * @return Float value.
+     */
+    public float readFloat(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Double value.
+     * @throws IOException In case of error.
+     */
+    public double readDouble(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Char value.
+     * @throws IOException In case of error.
+     */
+    public char readChar(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Boolean value.
+     * @throws IOException In case of error.
+     */
+    public boolean readBoolean(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return String value.
+     * @throws IOException In case of error.
+     */
+    @Nullable public String readString(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Object.
+     * @throws IOException In case of error.
+     */
+    @Nullable public <T> T readObject(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Byte array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public byte[] readByteArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Short array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public short[] readShortArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Integer array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public int[] readIntArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Long array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public long[] readLongArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Float array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public float[] readFloatArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Byte array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public double[] readDoubleArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Char array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public char[] readCharArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Boolean array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public boolean[] readBooleanArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return String array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public String[] readStringArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Object array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public Object[] readObjectArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Collection.
+     * @throws IOException In case of error.
+     */
+    @Nullable public <T> Collection<T> readCollection(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Map.
+     * @throws IOException In case of error.
+     */
+    @Nullable public <K, V> Map<K, V> readMap(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Value.
+     * @throws IOException In case of error.
+     */
+    @Nullable public <T extends Enum<?>> T readEnum(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Value.
+     * @throws IOException In case of error.
+     */
+    @Nullable public <T extends Enum<?>> T[] readEnumArray(String fieldName) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedFieldsWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedFieldsWriter.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedFieldsWriter.java
new file mode 100644
index 0000000..f104fba
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedFieldsWriter.java
@@ -0,0 +1,196 @@
+/*
+ * 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;
+
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * TODO: IGNITE-950
+ */
+public interface OptimizedFieldsWriter {
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeByte(String fieldName, byte val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeShort(String fieldName, short val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeInt(String fieldName, int val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeLong(String fieldName, long val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeFloat(String fieldName, float val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeDouble(String fieldName, double val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeChar(String fieldName, char val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeBoolean(String fieldName, boolean val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeString(String fieldName, @Nullable String val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param obj Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeObject(String fieldName, @Nullable Object obj) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeByteArray(String fieldName, @Nullable byte[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeShortArray(String fieldName, @Nullable short[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeIntArray(String fieldName, @Nullable int[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeLongArray(String fieldName, @Nullable long[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeFloatArray(String fieldName, @Nullable float[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeDoubleArray(String fieldName, @Nullable double[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeCharArray(String fieldName, @Nullable char[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeStringArray(String fieldName, @Nullable String[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeObjectArray(String fieldName, @Nullable Object[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param col Collection to write.
+     * @throws IOException In case of error.
+     */
+    public <T> void writeCollection(String fieldName, @Nullable Collection<T> col) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param map Map to write.
+     * @throws IOException In case of error.
+     */
+    public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshalAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshalAware.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshalAware.java
new file mode 100644
index 0000000..cc90cff
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshalAware.java
@@ -0,0 +1,52 @@
+/*
+ * 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;
+
+import java.io.*;
+
+/**
+ * TODO: IGNITE-950
+ *
+ * Interface that allows to implement custom serialization
+ * logic for portable objects. Portable objects are not required
+ * to implement this interface, in which case Ignite will automatically
+ * serialize portable objects using reflection.
+ * <p>
+ * This interface, in a way, is analogous to {@link java.io.Externalizable}
+ * interface, which allows users to override default serialization logic,
+ * usually for performance reasons. The only difference here is that portable
+ * serialization is already very fast and implementing custom serialization
+ * logic for portables does not provide significant performance gains.
+ */
+public interface OptimizedMarshalAware {
+    /**
+     * Writes fields to provided writer.
+     *
+     * @param writer Fields writer.
+     * @throws IOException In case of error.
+     */
+    public void writeFields(OptimizedFieldsWriter writer) throws IOException;
+
+    /**
+     * Reads fields from provided reader.
+     *
+     * @param reader Fields reader.
+     * @throws IOException In case of error.
+     */
+    public void readFields(OptimizedFieldsReader reader) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshalAwareMetaCollector.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshalAwareMetaCollector.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshalAwareMetaCollector.java
new file mode 100644
index 0000000..c4fb9d4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshalAwareMetaCollector.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;
+
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+import static org.apache.ignite.marshaller.optimized.OptimizedFieldType.*;
+
+/**
+ * TODO: IGNITE-950
+ */
+class OptimizedMarshalAwareMetaCollector implements OptimizedFieldsWriter {
+    /** */
+    private OptimizedObjectMetadata meta;
+
+    /**
+     * Constructor.
+     */
+    public OptimizedMarshalAwareMetaCollector() {
+        meta = new OptimizedObjectMetadata();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeByte(String fieldName, byte val) throws IOException {
+        putFieldToMeta(fieldName, BYTE);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeShort(String fieldName, short val) throws IOException {
+        putFieldToMeta(fieldName, SHORT);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeInt(String fieldName, int val) throws IOException {
+        putFieldToMeta(fieldName, INT);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeLong(String fieldName, long val) throws IOException {
+        putFieldToMeta(fieldName, LONG);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeFloat(String fieldName, float val) throws IOException {
+        putFieldToMeta(fieldName, FLOAT);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDouble(String fieldName, double val) throws IOException {
+        putFieldToMeta(fieldName, DOUBLE);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeChar(String fieldName, char val) throws IOException {
+        putFieldToMeta(fieldName, CHAR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBoolean(String fieldName, boolean val) throws IOException {
+        putFieldToMeta(fieldName, BOOLEAN);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeString(String fieldName, @Nullable String val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeObject(String fieldName, @Nullable Object obj) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeByteArray(String fieldName, @Nullable byte[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeShortArray(String fieldName, @Nullable short[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeIntArray(String fieldName, @Nullable int[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeLongArray(String fieldName, @Nullable long[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeFloatArray(String fieldName, @Nullable float[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDoubleArray(String fieldName, @Nullable double[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeCharArray(String fieldName, @Nullable char[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeStringArray(String fieldName, @Nullable String[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeObjectArray(String fieldName, @Nullable Object[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> void writeCollection(String fieldName, @Nullable Collection<T> col) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /**
+     * Returns gather metadata.
+     *
+     * @return Metadata.
+     */
+    OptimizedObjectMetadata meta() {
+        return meta;
+    }
+
+    /**
+     * Adds field to the metadata.
+     *
+     * @param fieldName Field name.
+     * @param type Field type.
+     */
+    private void putFieldToMeta(String fieldName, OptimizedFieldType type) {
+        meta.addMeta(OptimizedMarshallerUtils.resolveFieldId(fieldName), type);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerExt.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerExt.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerExt.java
new file mode 100644
index 0000000..7d90581
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerExt.java
@@ -0,0 +1,357 @@
+/*
+ * 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;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+import static org.apache.ignite.marshaller.optimized.OptimizedClassDescriptor.*;
+
+/**
+ * TODO
+ */
+public class OptimizedMarshallerExt extends OptimizedMarshaller {
+    /** */
+    static final byte EMPTY_FOOTER = -1;
+
+    /** */
+    static final byte FOOTER_LEN_OFF = 2;
+
+    /** */
+    static final int FOOTER_BODY_LEN_MASK = 0x3FFFFFFF;
+
+    /** */
+    static final int FOOTER_BODY_IS_HANDLE_MASK = 0x40000000;
+
+    /** */
+    static final byte FOOTER_BODY_HANDLE_MASK_BIT = 30;
+
+    /** */
+    public static final byte VARIABLE_LEN = -1;
+
+    /** */
+    private volatile OptimizedMarshallerMetaHandler metaHandler;
+
+    /**
+     * Creates new marshaller will all defaults.
+     *
+     * @throws IgniteException If this marshaller is not supported on the current JVM.
+     */
+    public OptimizedMarshallerExt() {
+        // No-op
+    }
+
+    /**
+     * Creates new marshaller providing whether it should
+     * require {@link Serializable} interface or not.
+     *
+     * @param requireSer Whether to require {@link Serializable}.
+     */
+    public OptimizedMarshallerExt(boolean requireSer) {
+        super(requireSer);
+    }
+
+    /**
+     * Sets metadata handler.
+     *
+     * @param metaHandler Metadata handler.
+     */
+    public void setMetadataHandler(OptimizedMarshallerMetaHandler metaHandler) {
+        this.metaHandler = metaHandler;
+    }
+
+    /**
+     * Returns currently set ID mapper.
+     *
+     * @return ID mapper.
+     */
+    public OptimizedMarshallerIdMapper idMapper() {
+        return mapper;
+    }
+
+    /**
+     * Enables fields indexing for the object of the given {@code cls}.
+     *
+     * If enabled then a footer will be added during marshalling of an object of the given {@code cls} to the end of
+     * its serialized form.
+     *
+     * @param cls Class.
+     * @return {@code true} if fields indexing is enabled.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public boolean enableFieldsIndexing(Class<?> cls) throws IgniteCheckedException {
+        assert metaHandler != null;
+
+        if (ctx.isSystemType(cls.getName()))
+            return false;
+
+        if (OptimizedMarshalAware.class.isAssignableFrom(cls))
+            return true;
+
+        try {
+            OptimizedClassDescriptor desc = OptimizedMarshallerUtils.classDescriptor(clsMap, cls, ctx, mapper);
+
+            if (desc.fields() != null && desc.fields().fieldsIndexingSupported()) {
+                //The function is called on kernel startup, calling metaHandler.metadata() will hang the grid,
+                //because the underlying cache is not ready.
+                //if (metaHandler.metadata(desc.typeId()) != null)
+                //    return true;
+
+                OptimizedObjectMetadata meta = new OptimizedObjectMetadata();
+
+                for (ClassFields clsFields : desc.fields().fieldsList())
+                    for (FieldInfo info : clsFields.fieldInfoList())
+                        meta.addMeta(info.id(), info.type());
+
+                metaHandler.addMeta(desc.typeId(), meta);
+
+                return true;
+            }
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to put meta for class: " + cls.getName(), e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether fields indexing is enabled for objects of the given {@code cls}.
+     *
+     * @param cls Class.
+     * @return {@code true} if fields indexing is enabled.
+     */
+    public boolean fieldsIndexingEnabled(Class<?> cls) {
+        assert metaHandler != null;
+
+        if (ctx.isSystemType(cls.getName()))
+            return false;
+
+        if (OptimizedMarshalAware.class.isAssignableFrom(cls))
+            return true;
+
+        try {
+            OptimizedClassDescriptor desc = OptimizedMarshallerUtils.classDescriptor(clsMap, cls, ctx, mapper);
+
+            return desc.fields() != null && desc.fields().fieldsIndexingSupported() &&
+                metaHandler.metadata(desc.typeId()) != null;
+        }
+        catch (IOException e) {
+            throw new IgniteException("Failed to load class description: " + cls);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setPoolSize(int poolSize) {
+        OptimizedObjectStreamExtRegistry.poolSize(poolSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void marshal(@Nullable Object obj, OutputStream out) throws IgniteCheckedException {
+        assert out != null;
+
+        OptimizedObjectOutputStreamExt objOut = null;
+
+        try {
+            objOut = OptimizedObjectStreamExtRegistry.out();
+
+            objOut.context(clsMap, ctx, mapper, requireSer, metaHandler);
+
+            objOut.out().outputStream(out);
+
+            objOut.writeObject(obj);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to serialize object: " + obj, e);
+        }
+        finally {
+            OptimizedObjectStreamExtRegistry.closeOut(objOut);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] marshal(@Nullable Object obj) throws IgniteCheckedException {
+        OptimizedObjectOutputStreamExt objOut = null;
+
+        try {
+            objOut = OptimizedObjectStreamExtRegistry.out();
+
+            objOut.context(clsMap, ctx, mapper, requireSer, metaHandler);
+
+            objOut.writeObject(obj);
+
+            return objOut.out().array();
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to serialize object: " + obj, e);
+        }
+        finally {
+            OptimizedObjectStreamExtRegistry.closeOut(objOut);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <T> T unmarshal(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
+        assert in != null;
+
+        OptimizedObjectInputStreamExt objIn = null;
+
+        try {
+            objIn = OptimizedObjectStreamExtRegistry.in();
+
+            objIn.context(clsMap, ctx, mapper, clsLdr != null ? clsLdr : dfltClsLdr, metaHandler);
+
+            objIn.in().inputStream(in);
+
+            return (T)objIn.readObject();
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e);
+        }
+        catch (ClassNotFoundException e) {
+            throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " +
+                                             "(make sure same versions of all classes are available on all nodes or " +
+                                             "enable peer-class-loading): " + clsLdr, e);
+        }
+        finally {
+            OptimizedObjectStreamExtRegistry.closeIn(objIn);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <T> T unmarshal(byte[] arr, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
+        return unmarshal(arr, 0, arr.length, clsLdr);
+    }
+
+    /**
+     * Unmarshals object from byte array using given class loader and offset with len.
+     *
+     * @param <T> Type of unmarshalled object.
+     * @param arr Byte array.
+     * @param off Object's offset in the array.
+     * @param len Object's length in the array.
+     * @param clsLdr Class loader to use.
+     * @return Unmarshalled object.
+     * @throws IgniteCheckedException If unmarshalling failed.
+     */
+     public <T> T unmarshal(byte[] arr, int off, int len, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
+        assert arr != null;
+
+        OptimizedObjectInputStreamExt objIn = null;
+
+        try {
+            objIn = OptimizedObjectStreamExtRegistry.in();
+
+            objIn.context(clsMap, ctx, mapper, clsLdr != null ? clsLdr : dfltClsLdr, metaHandler);
+
+            objIn.in().bytes(arr, off, len);
+
+            return (T)objIn.readObject();
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e);
+        }
+        catch (ClassNotFoundException e) {
+            throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " +
+                                                 "(make sure same version of all classes are available on all nodes or" +
+                                                 " enable peer-class-loading): " + clsLdr, e);
+        }
+        finally {
+            OptimizedObjectStreamExtRegistry.closeIn(objIn);
+        }
+    }
+
+    /**
+     * Checks whether object, serialized to byte array {@code arr}, has a field with name {@code fieldName}.
+     *
+     * @param fieldName Field name.
+     * @param arr Object's serialized form.
+     * @param off Object's start off.
+     * @param len Object's len.
+     * @return {@code true} if field exists.
+     */
+    public boolean hasField(String fieldName, byte[] arr, int off, int len) throws IgniteCheckedException {
+        assert arr != null && fieldName != null;
+
+        OptimizedObjectInputStreamExt objIn = null;
+
+        try {
+            objIn = OptimizedObjectStreamExtRegistry.in();
+
+            objIn.context(clsMap, ctx, mapper, dfltClsLdr, metaHandler);
+
+            objIn.in().bytes(arr, off, len);
+
+            return objIn.hasField(fieldName);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to find field with name: " + fieldName, e);
+        }
+        finally {
+            OptimizedObjectStreamExtRegistry.closeIn(objIn);
+        }
+    }
+
+    /**
+     * Looks up field with the given name and returns it in one of the following representations. If the field is
+     * serializable and has a footer then it's not deserialized but rather returned wrapped by {@link CacheObjectImpl}
+     * for future processing. In all other cases the field is fully deserialized.
+     *
+     * @param fieldName Field name.
+     * @param arr Object's serialized form.
+     * @param off Object's start offset.
+     * @param len Object's len.
+     * @param clsLdr Class loader.
+     * @param <T> Expected field class.
+     * @return Field.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public <T> T readField(String fieldName, byte[] arr, int off, int len, @Nullable ClassLoader clsLdr)
+        throws IgniteCheckedException {
+
+        assert arr != null && fieldName != null;
+
+        OptimizedObjectInputStreamExt objIn = null;
+
+        try {
+            objIn = OptimizedObjectStreamExtRegistry.in();
+
+            objIn.context(clsMap, ctx, mapper, clsLdr != null ? clsLdr : dfltClsLdr, metaHandler);
+
+            objIn.in().bytes(arr, off, len);
+
+            return objIn.readField(fieldName);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to find field with name: " + fieldName, e);
+        }
+        catch (ClassNotFoundException e) {
+            throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " +
+                                             "(make sure same version of all classes are available on all nodes or" +
+                                             " enable peer-class-loading): " + clsLdr, e);
+        }
+        finally {
+            OptimizedObjectStreamExtRegistry.closeIn(objIn);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerMetaHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerMetaHandler.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerMetaHandler.java
new file mode 100644
index 0000000..5fad57c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerMetaHandler.java
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * Metadata handler for optimized objects.
+ */
+public interface OptimizedMarshallerMetaHandler {
+    /**
+     * Adds meta data.
+     *
+     * @param typeId Type ID.
+     * @param meta Meta data.
+     */
+    void addMeta(int typeId, OptimizedObjectMetadata meta);
+
+
+    /**
+     * Gets meta data for provided type ID.
+     *
+     * @param typeId Type ID.
+     * @return Meta data.
+     */
+    OptimizedObjectMetadata metadata(int typeId);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
index 7a7ee69..cd25f1c 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
@@ -144,6 +144,9 @@ public class OptimizedMarshallerUtils {
     /** */
     public static final byte SERIALIZABLE = 102;
 
+    /** */
+    public static final byte MARSHAL_AWARE = 103;
+
     /** UTF-8 character name. */
     static final Charset UTF_8 = Charset.forName("UTF-8");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStream.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStream.java
index c8a85b2..6eeadc8 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStream.java
@@ -18,10 +18,12 @@
 package org.apache.ignite.marshaller.optimized;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.io.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.marshaller.*;
+import org.jetbrains.annotations.*;
 import sun.misc.*;
 
 import java.io.*;
@@ -30,11 +32,12 @@ import java.util.*;
 import java.util.concurrent.*;
 
 import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.*;
+import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerExt.*;
 
 /**
  * Optimized object input stream.
  */
-public class OptimizedObjectInputStream extends ObjectInputStream {
+public class OptimizedObjectInputStream extends ObjectInputStream implements OptimizedFieldsReader {
     /** Unsafe. */
     private static final Unsafe UNSAFE = GridUnsafe.unsafe();
 
@@ -51,6 +54,9 @@ public class OptimizedObjectInputStream extends ObjectInputStream {
     protected ClassLoader clsLdr;
 
     /** */
+    protected OptimizedMarshallerMetaHandler metaHandler;
+
+    /** */
     protected GridDataInput in;
 
     /** */
@@ -95,6 +101,25 @@ public class OptimizedObjectInputStream extends ObjectInputStream {
     }
 
     /**
+     * @param clsMap Class descriptors by class map.
+     * @param ctx Context.
+     * @param mapper ID mapper.
+     * @param clsLdr Class loader.
+     * @param metaHandler Metadata handler.
+     */
+    protected void context(
+        ConcurrentMap<Class, OptimizedClassDescriptor> clsMap,
+        MarshallerContext ctx,
+        OptimizedMarshallerIdMapper mapper,
+        ClassLoader clsLdr,
+        OptimizedMarshallerMetaHandler metaHandler)
+    {
+        context(clsMap, ctx, mapper, clsLdr);
+
+        this.metaHandler = metaHandler;
+    }
+
+    /**
      * @return Input.
      */
     public GridDataInput in() {
@@ -115,6 +140,7 @@ public class OptimizedObjectInputStream extends ObjectInputStream {
         ctx = null;
         clsLdr = null;
         clsMap = null;
+        metaHandler = null;
     }
 
     /** {@inheritDoc} */
@@ -241,6 +267,7 @@ public class OptimizedObjectInputStream extends ObjectInputStream {
             case ENUM:
             case EXTERNALIZABLE:
             case SERIALIZABLE:
+            case MARSHAL_AWARE:
                 int typeId = readInt();
 
                 OptimizedClassDescriptor desc = typeId == 0 ?
@@ -481,6 +508,46 @@ public class OptimizedObjectInputStream extends ObjectInputStream {
     }
 
     /**
+     * Reads {@link OptimizedMarshalAware} object.
+     *
+     * @param constructor Constructor.
+     * @param readResolveMtd {@code readResolve} method.
+     * @return Object.
+     * @throws ClassNotFoundException If class not found.
+     * @throws IOException In case of error.
+     */
+    Object readMarshalAware(Constructor<?> constructor, Method readResolveMtd)
+        throws ClassNotFoundException, IOException {
+        Object obj;
+
+        try {
+            obj = constructor.newInstance();
+        }
+        catch (InstantiationException | InvocationTargetException | IllegalAccessException e) {
+            throw new IOException(e);
+        }
+
+        int handle = handles.assign(obj);
+
+        OptimizedMarshalAware extObj = ((OptimizedMarshalAware)obj);
+
+        extObj.readFields(this);
+
+        if (readResolveMtd != null) {
+            try {
+                obj = readResolveMtd.invoke(obj);
+
+                handles.set(handle, obj);
+            }
+            catch (IllegalAccessException | InvocationTargetException e) {
+                throw new IOException(e);
+            }
+        }
+
+        return obj;
+    }
+
+    /**
      * Reads serializable object.
      *
      * @param cls Class.
@@ -935,6 +1002,126 @@ public class OptimizedObjectInputStream extends ObjectInputStream {
         return new GetFieldImpl(this);
     }
 
+    /** {@inheritDoc} */
+    @Override public byte readByte(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public short readShort(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int readInt(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long readLong(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public float readFloat(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public double readDouble(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public char readChar(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readBoolean(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String readString(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T> T readObject(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public byte[] readByteArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public short[] readShortArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public int[] readIntArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public long[] readLongArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public float[] readFloatArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public double[] readDoubleArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public char[] readCharArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public boolean[] readBooleanArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String[] readStringArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object[] readObjectArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T> Collection<T> readCollection(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <K, V> Map<K, V> readMap(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T extends Enum<?>> T readEnum(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T extends Enum<?>> T[] readEnumArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
     /**
      * Skips object footer from the underlying stream.
      *
@@ -955,6 +1142,209 @@ public class OptimizedObjectInputStream extends ObjectInputStream {
         return 0;
     }
 
+    /**
+     * Checks whether the object has a field with name {@code fieldName}.
+     *
+     * @param fieldName Field name.
+     * @return {@code true} if field exists, {@code false} otherwise.
+     * @throws IOException in case of error.
+     */
+    public boolean hasField(String fieldName) throws IOException {
+        int pos = in.position();
+
+        byte type = in.readByte();
+
+        if (type != SERIALIZABLE && type != MARSHAL_AWARE) {
+            in.position(pos);
+            return false;
+        }
+
+        FieldRange range = fieldRange(fieldName, pos);
+
+        in.position(pos);
+
+        return range != null && range.start > 0;
+    }
+
+    /**
+     * Looks up field with the given name and returns it in one of the following representations. If the field is
+     * serializable and has a footer then it's not deserialized but rather returned wrapped by {@link CacheObjectImpl}
+     * for future processing. In all other cases the field is fully deserialized.
+     *
+     * @param fieldName Field name.
+     * @return Field.
+     * @throws IOException In case of error.
+     * @throws ClassNotFoundException In case of error.
+     */
+    public <F> F readField(String fieldName) throws IOException, ClassNotFoundException {
+        return doReadField(fieldName, false);
+    }
+
+    /**
+     * Reads the field using footer.
+     *
+     * @param fieldName Field name.
+     * @param deserialize Deserialize field if it supports footer.
+     * @param <F> Field type.
+     * @return Field.
+     */
+    private <F> F doReadField(String fieldName, boolean deserialize) throws IOException, ClassNotFoundException {
+        int pos = in.position();
+
+        byte type = in.readByte();
+
+        if (type != SERIALIZABLE && type != MARSHAL_AWARE) {
+            in.position(pos);
+            return null;
+        }
+
+        FieldRange range = fieldRange(fieldName, pos);
+
+        F field = null;
+
+        if (range != null && range.start >= 0) {
+            in.position(range.start);
+
+            if (deserialize)
+                field = (F)readObject();
+            else {
+                byte fieldType = in.readByte();
+
+                if ((fieldType == SERIALIZABLE && metaHandler.metadata(in.readInt()) != null) ||
+                    fieldType == MARSHAL_AWARE)
+                    //Do we need to make a copy of array?
+                    field = (F)new CacheIndexedObjectImpl(in.array(), range.start, range.len);
+                else {
+                    in.position(range.start);
+                    field = (F)readObject();
+                }
+            }
+        }
+
+        in.position(pos);
+
+        return field;
+    }
+
+    /**
+     * Reads the field and deserializes it.
+     *
+     * @param fieldName Field name.
+     * @param <F> Field type.
+     * @return Field.
+     * @throws IOException In case of error.
+     */
+    private <F> F doReadField(String fieldName) throws IOException {
+        try {
+            return doReadField(fieldName, true);
+        }
+        catch (ClassNotFoundException e) {
+            throw new IOException("Failed to read the field, class definition has not" +
+                " been found [field=" + fieldName + "]");
+        }
+    }
+
+    /**
+     * Returns field offset in the byte stream.
+     *
+     * @param fieldName Field name.
+     * @param start Object's start offset.
+     * @return positive range or {@code null} if the object doesn't have such a field.
+     * @throws IOException in case of error.
+     */
+    private FieldRange fieldRange(String fieldName, int start) throws IOException {
+        int fieldId = resolveFieldId(fieldName);
+
+        int typeId = readInt();
+
+        int clsNameLen = 0;
+
+        if (typeId == 0) {
+            int pos = in.position();
+
+            typeId = OptimizedMarshallerUtils.resolveTypeId(readUTF(), mapper);
+
+            clsNameLen = in.position() - pos;
+        }
+
+        OptimizedObjectMetadata meta = metaHandler.metadata(typeId);
+
+        if (meta == null)
+            // TODO: IGNITE-950 add warning!
+            return null;
+
+        int end = in.size();
+
+        in.position(end - FOOTER_LEN_OFF);
+
+        short footerLen = in.readShort();
+
+        if (footerLen == EMPTY_FOOTER)
+            return null;
+
+        // +2 - skipping length at the beginning
+        int footerOff = (end - footerLen) + 2;
+        in.position(footerOff);
+
+        int fieldOff = 0;
+
+        for (OptimizedObjectMetadata.FieldInfo info : meta.getMeta()) {
+            int len;
+            boolean isHandle;
+
+            if (info.length() == VARIABLE_LEN) {
+                int fieldInfo = in.readInt();
+
+                len = fieldInfo & FOOTER_BODY_LEN_MASK;
+                isHandle = ((fieldInfo & FOOTER_BODY_IS_HANDLE_MASK) >> FOOTER_BODY_HANDLE_MASK_BIT) == 1;
+            }
+            else {
+                len = info.length();
+                isHandle = false;
+            }
+
+            if (info.id() == fieldId) {
+                if (!isHandle) {
+                    //object header len: 1 - for type, 4 - for type ID, 2 - for checksum.
+                    fieldOff += 1 + 4 + clsNameLen + 2;
+
+                    return new FieldRange(start + fieldOff, len);
+                }
+                else
+                    return new FieldRange(in.readInt(), in.readInt());
+            }
+            else {
+                fieldOff += len;
+
+                if (isHandle) {
+                    in.skipBytes(8);
+                    fieldOff += 8;
+                }
+            }
+        }
+
+        return null;
+    }
+
+    /**
+     *
+     */
+    private static class FieldRange {
+        /** */
+        private int start;
+
+        /** */
+        private int len;
+
+        /**
+         * @param start Start.
+         * @param len   Length.
+         */
+        public FieldRange(int start, int len) {
+            this.start = start;
+            this.len = len;
+        }
+    }
     /** {@inheritDoc} */
     @Override public void registerValidation(ObjectInputValidation obj, int pri) {
         // No-op.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStreamExt.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStreamExt.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStreamExt.java
new file mode 100644
index 0000000..34ca279
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStreamExt.java
@@ -0,0 +1,51 @@
+/*
+ * 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;
+
+import org.apache.ignite.internal.util.io.*;
+
+import java.io.*;
+
+import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.*;
+import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerExt.*;
+
+
+/**
+ * TODO: IGNITE-950
+ */
+public class OptimizedObjectInputStreamExt extends OptimizedObjectInputStream {
+    /** {@inheritDoc} */
+    public OptimizedObjectInputStreamExt(GridDataInput in) throws IOException {
+        super(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void skipFooter(Class<?> cls) throws IOException {
+        if (!ctx.isSystemType(cls.getName()) && metaHandler != null &&
+            metaHandler.metadata(resolveTypeId(cls.getName(), mapper)) != null) {
+            short footerLen = in.readShort();
+
+            if (footerLen != EMPTY_FOOTER)
+                in.skipBytes(footerLen - 2);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int readFieldType() throws IOException {
+        return in.readByte();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectMetadata.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectMetadata.java
new file mode 100644
index 0000000..9a5463b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectMetadata.java
@@ -0,0 +1,172 @@
+/*
+ * 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;
+
+import org.apache.ignite.*;
+
+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 static final long serialVersionUID = 0L;
+
+    /** */
+    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<>();
+
+
+
+        fieldsInfo.add(new FieldInfo(fieldId, fieldType));
+    }
+
+    /**
+     * 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.writeByte(fieldInfo.type.ordinal());
+        }
+    }
+
+    /** {@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(), OptimizedFieldType.values()[in.readByte()]));
+    }
+
+    /**
+     * Field info.
+     */
+    public static class FieldInfo {
+        /** Field ID. */
+        int id;
+
+        /** Field len. */
+        int len;
+
+        /** Field type. */
+        OptimizedFieldType type;
+
+        /**
+         * Constructor.
+         *
+         * @param id Field ID.
+         * @param type Field len.
+         */
+        public FieldInfo(int id, OptimizedFieldType type) {
+            this.id = id;
+            this.type = type;
+
+            len = 1;
+
+            switch (type) {
+                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: " + type);
+            }
+
+            assert len != 1;
+        }
+
+        /**
+         * Returns ID.
+         * @return ID.
+         */
+        public int id() {
+            return id;
+        }
+
+        /**
+         * Returns length.
+         * @return Lenght.
+         */
+        public int length() {
+            return len;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectMetadataKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectMetadataKey.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectMetadataKey.java
new file mode 100644
index 0000000..ee85754
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/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;
+
+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/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStream.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStream.java
index 84b8351..1702b11 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStream.java
@@ -23,10 +23,12 @@ import org.apache.ignite.internal.util.io.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
+import org.jetbrains.annotations.*;
 
 import java.io.*;
 import java.lang.reflect.*;
 import java.util.*;
+import java.util.Date;
 import java.util.concurrent.*;
 
 import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.*;
@@ -34,7 +36,7 @@ import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.*;
 /**
  * Optimized object output stream.
  */
-public class OptimizedObjectOutputStream extends ObjectOutputStream {
+public class OptimizedObjectOutputStream extends ObjectOutputStream implements OptimizedFieldsWriter {
     /** */
     private static final Collection<String> CONVERTED_ERR = F.asList(
         "weblogic/management/ManagementException",
@@ -55,6 +57,9 @@ public class OptimizedObjectOutputStream extends ObjectOutputStream {
     protected ConcurrentMap<Class, OptimizedClassDescriptor> clsMap;
 
     /** */
+    protected OptimizedMarshallerMetaHandler metaHandler;
+
+    /** */
     protected boolean requireSer;
 
     /** */
@@ -72,6 +77,9 @@ public class OptimizedObjectOutputStream extends ObjectOutputStream {
     /** */
     private PutFieldImpl curPut;
 
+    /** */
+    private Stack<Footer> marshalAwareFooters;
+
     /**
      * @param out Output.
      * @throws IOException In case of error.
@@ -97,6 +105,23 @@ public class OptimizedObjectOutputStream extends ObjectOutputStream {
     }
 
     /**
+     * @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,
+        OptimizedMarshallerMetaHandler metaHandler) {
+        context(clsMap, ctx, mapper, requireSer);
+
+        this.metaHandler = metaHandler;
+    }
+
+    /**
      * @return Require {@link Serializable} flag.
      */
     boolean requireSerializable() {
@@ -116,6 +141,7 @@ public class OptimizedObjectOutputStream extends ObjectOutputStream {
 
         ctx = null;
         clsMap = null;
+        metaHandler = null;
     }
 
     /** {@inheritDoc} */
@@ -308,6 +334,36 @@ public class OptimizedObjectOutputStream extends ObjectOutputStream {
     }
 
     /**
+     * Writes marshal aware object with footer injected to the end of the stream.
+     *
+     * @param obj Object.
+     * @throws IOException In case of error.
+     */
+    void writeMarshalAware(Object obj) throws IOException {
+        Footer footer = createFooter(obj.getClass());
+
+        if (footer == null)
+            throw new IOException("Failed to marshal OptimizedMarshalAware object. OptimizedMarshallerExt must be " +
+                "set to IgniteConfiguration [obj=" + obj.getClass().getName() + "]");
+
+        if (marshalAwareFooters == null)
+            marshalAwareFooters = new Stack<>();
+
+        marshalAwareFooters.push(footer);
+
+        OptimizedMarshalAware marshalAwareObj = (OptimizedMarshalAware)obj;
+
+        marshalAwareObj.writeFields(this);
+
+        footer.write();
+
+        marshalAwareFooters.pop();
+
+        if (marshalAwareFooters.empty())
+            marshalAwareFooters = null;
+    }
+
+    /**
      * Writes serializable object.
      *
      * @param obj Object.
@@ -840,6 +896,7 @@ public class OptimizedObjectOutputStream extends ObjectOutputStream {
         curObj = null;
         curFields = null;
         curPut = null;
+        marshalAwareFooters = null;
     }
 
     /** {@inheritDoc} */
@@ -862,6 +919,156 @@ public class OptimizedObjectOutputStream extends ObjectOutputStream {
         // No-op
     }
 
+    /** {@inheritDoc} */
+    @Override public void writeByte(String fieldName, byte val) throws IOException {
+        writeFieldType(BYTE);
+        out.writeByte(val);
+        putFieldToFooter(fieldName, OptimizedFieldType.BYTE, 1);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeShort(String fieldName, short val) throws IOException {
+        writeFieldType(SHORT);
+        out.writeShort(val);
+        putFieldToFooter(fieldName, OptimizedFieldType.SHORT, 2);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeInt(String fieldName, int val) throws IOException {
+        writeFieldType(INT);
+        out.writeInt(val);
+        putFieldToFooter(fieldName, OptimizedFieldType.INT, 4);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeLong(String fieldName, long val) throws IOException {
+        writeFieldType(LONG);
+        out.writeLong(val);
+        putFieldToFooter(fieldName, OptimizedFieldType.LONG, 8);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeFloat(String fieldName, float val) throws IOException {
+        writeFieldType(FLOAT);
+        out.writeFloat(val);
+        putFieldToFooter(fieldName, OptimizedFieldType.FLOAT, 4);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDouble(String fieldName, double val) throws IOException {
+        writeFieldType(DOUBLE);
+        out.writeDouble(val);
+        putFieldToFooter(fieldName, OptimizedFieldType.DOUBLE, 8);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeChar(String fieldName, char val) throws IOException {
+        writeFieldType(CHAR);
+        out.writeChar(val);
+        putFieldToFooter(fieldName, OptimizedFieldType.CHAR, 2);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBoolean(String fieldName, boolean val) throws IOException {
+        writeFieldType(BOOLEAN);
+        out.writeBoolean(val);
+        putFieldToFooter(fieldName, OptimizedFieldType.BOOLEAN, 1);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeString(String fieldName, @Nullable String val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeObject(String fieldName, @Nullable Object obj) throws IOException {
+        int pos = out.offset();
+
+        writeObject(obj);
+        putFieldToFooter(fieldName, OptimizedFieldType.OTHER, out.offset() - pos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeByteArray(String fieldName, @Nullable byte[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeShortArray(String fieldName, @Nullable short[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeIntArray(String fieldName, @Nullable int[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeLongArray(String fieldName, @Nullable long[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeFloatArray(String fieldName, @Nullable float[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDoubleArray(String fieldName, @Nullable double[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeCharArray(String fieldName, @Nullable char[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeStringArray(String fieldName, @Nullable String[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeObjectArray(String fieldName, @Nullable Object[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> void writeCollection(String fieldName, @Nullable Collection<T> col) throws IOException {
+        writeObject(fieldName, col);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map) throws IOException {
+        writeObject(fieldName, map);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /**
+     * Puts field to the footer.
+     *
+     * @param fieldName Field name.
+     * @param type Field type.
+     * @param len Field length.
+     */
+    private void putFieldToFooter(String fieldName, OptimizedFieldType type, int len) {
+        marshalAwareFooters.peek().put(OptimizedMarshallerUtils.resolveFieldId(fieldName), type, len);
+    }
+
     /**
      * Creates new instance of {@code Footer}.
      *