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:14 UTC

[31/33] incubator-ignite git commit: ignite-950: test fixes, refactoring

ignite-950: test fixes, refactoring


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

Branch: refs/heads/ignite-950
Commit: d9f85e69aee657b3fb976ccb757c714efe78c106
Parents: a088170
Author: Denis Magda <dm...@gridgain.com>
Authored: Tue Jun 23 10:46:24 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue Jun 23 10:46:24 2015 +0300

----------------------------------------------------------------------
 .../ignite/codegen/MessageCodeGenerator.java    |   2 +-
 .../communication/GridIoMessageFactory.java     |   4 +-
 .../cache/CacheIndexedObjectImpl.java           | 364 +++++++++++++++++++
 .../processors/cache/CacheObjectContext.java    |  12 +-
 .../cache/CacheOptimizedObjectImpl.java         | 364 -------------------
 .../cache/KeyCacheIndexedObjectImpl.java        | 104 ++++++
 .../cache/KeyCacheOptimizedObjectImpl.java      | 104 ------
 .../IgniteCacheObjectProcessorImpl.java         |  42 +--
 .../processors/query/GridQueryProcessor.java    | 119 ++----
 .../optimized/OptimizedMarshallerUtils.java     |   2 +-
 .../ext/OptimizedObjectInputStreamExt.java      |   2 +-
 .../ext/OptimizedMarshallerExtSelfTest.java     |  56 ++-
 12 files changed, 586 insertions(+), 589 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
----------------------------------------------------------------------
diff --git a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
index 4780ee4..e71e628 100644
--- a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
+++ b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
@@ -143,7 +143,7 @@ public class MessageCodeGenerator {
 
         MessageCodeGenerator gen = new MessageCodeGenerator(srcDir);
 
-        gen.generateAndWrite(CacheOptimizedObjectImpl.class);
+        gen.generateAndWrite(CacheIndexedObjectImpl.class);
 
 //        gen.generateAndWrite(GridDistributedLockRequest.class);
 //        gen.generateAndWrite(GridDistributedLockResponse.class);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index 4c8fec5..29e778d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -601,12 +601,12 @@ public class GridIoMessageFactory implements MessageFactory {
                 break;
 
             case 113:
-                msg = new CacheOptimizedObjectImpl();
+                msg = new CacheIndexedObjectImpl();
 
                 break;
 
             case 114:
-                msg = new KeyCacheOptimizedObjectImpl();
+                msg = new KeyCacheIndexedObjectImpl();
 
             // [-3..114] - this
             // [120..123] - DR

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/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
new file mode 100644
index 0000000..58e9c97
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheIndexedObjectImpl.java
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.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.plugin.extensions.communication.*;
+import org.jetbrains.annotations.*;
+import sun.misc.*;
+
+import java.io.*;
+import java.nio.*;
+
+/**
+ * Cache object implementation for classes that support footer injection is their serialized form thus enabling fields
+ * search and extraction without necessity to fully deserialize an object.
+ */
+public class CacheIndexedObjectImpl extends CacheObjectAdapter {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
+
+    /** */
+    private static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    protected int start;
+
+    /** */
+    protected int len;
+
+    /**
+     * For {@link Externalizable}.
+     */
+    public CacheIndexedObjectImpl() {
+       // No-op
+    }
+
+    /**
+     * Instantiates {@code CacheIndexedObjectImpl} with object.
+     * @param val Object.
+     */
+    public CacheIndexedObjectImpl(Object val) {
+        this(val, null, 0, 0);
+    }
+
+    /**
+     * Instantiates {@code CacheIndexedObjectImpl} with object's serialized form.
+     * @param valBytes Object serialized to byte array.
+     * @param start Object's start in the array.
+     * @param len Object's len in the array.
+     */
+    public CacheIndexedObjectImpl(byte[] valBytes, int start, int len) {
+        this(null, valBytes, start, len);
+    }
+
+    /**
+     * Instantiates {@code CacheIndexedObjectImpl} with object's serialized form and value.
+     * @param val Object.
+     * @param valBytes Object serialized to byte array.
+     */
+    public CacheIndexedObjectImpl(Object val, byte[] valBytes) {
+        this(val, valBytes, 0, valBytes != null ? valBytes.length : 0);
+    }
+
+    /**
+     * Instantiates {@code CacheIndexedObjectImpl}.
+     * @param val Object.
+     * @param valBytes Object in a serialized form.
+     * @param start Object's start in the array.
+     * @param len Object's len in the array.
+     */
+    public CacheIndexedObjectImpl(Object val, byte[] valBytes, int start, int len) {
+        assert val != null || (valBytes != null && start >= 0 && len > 0);
+
+        this.val = val;
+        this.valBytes = valBytes;
+        this.start = start;
+        this.len = len;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T> T value(CacheObjectContext ctx, boolean cpy) {
+        return (T)this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] valueBytes(CacheObjectContext ctx) throws IgniteCheckedException {
+        toMarshaledFormIfNeeded(ctx);
+
+        if (detached())
+            return valBytes;
+
+        byte[] arr = new byte[len];
+
+        U.arrayCopy(valBytes, start, arr, 0, len);
+
+        return arr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
+        if (detached())
+            return this;
+
+        return detach();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void finishUnmarshal(CacheObjectContext ctx, ClassLoader ldr) throws IgniteCheckedException {
+        assert val != null || valBytes != null;
+
+        if (val == null && ctx.storeValue())
+            val = ctx.processor().unmarshal(ctx, valBytes, start, len, ldr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void prepareMarshal(CacheObjectContext ctx) throws IgniteCheckedException {
+        assert val != null || valBytes != null;
+
+        toMarshaledFormIfNeeded(ctx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        // refer to GridIoMessageFactory.
+        return 113;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte type() {
+        return TYPE_OPTIMIZED;
+    }
+
+    /**
+     * Returns object's type ID.
+     *
+     * @return Type ID.
+     */
+    public int typeId() {
+        assert valBytes != null;
+
+        int typeId = UNSAFE.getInt(valBytes, BYTE_ARR_OFF + start + 1);
+
+        if (typeId == 0)
+            throw new IgniteException("Object's type ID wasn't written to cache.");
+
+        return typeId;
+    }
+
+    /**
+     * Checks whether a wrapped object has field with name {@code fieldName}.
+     *
+     * @param fieldName Field name.
+     * @param marsh Marshaller.
+     * @return {@code true} if has.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public boolean hasField(String fieldName, OptimizedMarshallerExt marsh) throws IgniteCheckedException {
+        assert valBytes != null;
+
+        return marsh.hasField(fieldName, valBytes, start, len);
+    }
+
+    /**
+     * Searches and returns field if it exists.
+     *
+     * @param fieldName Field name.
+     * @param marsh Marshaller.
+     * @return Field.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public Object field(String fieldName, OptimizedMarshallerExt marsh) throws IgniteCheckedException {
+        assert valBytes != null;
+
+        return marsh.readField(fieldName, valBytes, start, len, val != null ? val.getClass().getClassLoader() : null);
+    }
+
+    /**
+     * Deserializes wrapped object.
+     *
+     * @param ctx Cache context.
+     * @return Deserialized object.
+     */
+    public Object deserialize(CacheObjectContext ctx) {
+        if (val != null)
+            return val;
+
+        try {
+            assert valBytes != null;
+
+            Object val = ctx.processor().unmarshal(ctx, valBytes, start, len,
+                ctx.kernalContext().config().getClassLoader());
+
+            if (ctx.storeValue())
+                this.val = val;
+
+            return val;
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException("Failed to unmarshall object.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        super.writeExternal(out);
+
+        out.writeInt(start);
+        out.writeInt(len);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        super.readExternal(in);
+
+        start = in.readInt();
+        len = in.readInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 1:
+                len = reader.readInt("len");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 2:
+                start = reader.readInt("start");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 1:
+                if (!writer.writeInt("len", len))
+                    return false;
+
+                writer.incrementState();
+
+            case 2:
+                if (!writer.writeInt("start", start))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        assert false;
+
+        return super.hashCode();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object obj) {
+        assert false;
+
+        return super.equals(obj);
+    }
+
+    /**
+     * Detaches object.
+     *
+     * @return Detached object wrapped by {@code CacheIndexedObjectImpl}.
+     */
+    protected CacheIndexedObjectImpl detach() {
+        if (detached())
+            return this;
+
+        byte[] arr = new byte[len];
+
+        U.arrayCopy(valBytes, start, arr, 0, len);
+
+        return new CacheIndexedObjectImpl(arr, 0, len);
+    }
+
+    /**
+     * Checks whether the object is already detached or not.
+     *
+     * @return {@code true} if detached.
+     */
+    protected boolean detached() {
+        return start == 0 && len == valBytes.length;
+    }
+
+    /**
+     * Marshals {@link #val} to {@link #valBytes} if needed.
+     *
+     * @param ctx Cache object context.
+     * @throws IgniteCheckedException In case of error.
+     */
+    protected void toMarshaledFormIfNeeded(CacheObjectContext ctx) throws IgniteCheckedException {
+        if (valBytes == null) {
+            assert val != null;
+
+            valBytes = ctx.processor().marshal(ctx, val);
+
+            start = 0;
+            len = valBytes.length;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
index bf2cc07..4e5ca5f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
@@ -157,8 +157,8 @@ public class CacheObjectContext {
      * @return Unwrapped object.
      */
     private Object unwrapObject(Object obj) {
-        if (obj instanceof CacheOptimizedObjectImpl)
-            return ((CacheOptimizedObjectImpl)obj).deserialize(this);
+        if (obj instanceof CacheIndexedObjectImpl)
+            return ((CacheIndexedObjectImpl)obj).deserialize(this);
         else if (obj instanceof Map.Entry) {
             Map.Entry<Object, Object> entry = (Map.Entry<Object, Object>)obj;
 
@@ -166,16 +166,16 @@ public class CacheObjectContext {
 
             boolean unwrapped = false;
 
-            if (key instanceof CacheOptimizedObjectImpl) {
-                key = ((CacheOptimizedObjectImpl)key).deserialize(this);
+            if (key instanceof CacheIndexedObjectImpl) {
+                key = ((CacheIndexedObjectImpl)key).deserialize(this);
 
                 unwrapped = true;
             }
 
             Object val = entry.getValue();
 
-            if (val instanceof CacheOptimizedObjectImpl) {
-                val = ((CacheOptimizedObjectImpl)val).deserialize(this);
+            if (val instanceof CacheIndexedObjectImpl) {
+                val = ((CacheIndexedObjectImpl)val).deserialize(this);
 
                 unwrapped = true;
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOptimizedObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOptimizedObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOptimizedObjectImpl.java
deleted file mode 100644
index f07344c..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOptimizedObjectImpl.java
+++ /dev/null
@@ -1,364 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.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.plugin.extensions.communication.*;
-import org.jetbrains.annotations.*;
-import sun.misc.*;
-
-import java.io.*;
-import java.nio.*;
-
-/**
- * Cache object implementation for classes that support footer injection is their serialized form thus enabling fields
- * search and extraction without necessity to fully deserialize an object.
- */
-public class CacheOptimizedObjectImpl extends CacheObjectAdapter {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** */
-    private static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
-
-    /** */
-    protected int start;
-
-    /** */
-    protected int len;
-
-    /**
-     * For {@link Externalizable}.
-     */
-    public CacheOptimizedObjectImpl() {
-       // No-op
-    }
-
-    /**
-     * Instantiates {@code CacheOptimizedObjectImpl} with object.
-     * @param val Object.
-     */
-    public CacheOptimizedObjectImpl(Object val) {
-        this(val, null, 0, 0);
-    }
-
-    /**
-     * Instantiates {@code CacheOptimizedObjectImpl} with object's serialized form.
-     * @param valBytes Object serialized to byte array.
-     * @param start Object's start in the array.
-     * @param len Object's len in the array.
-     */
-    public CacheOptimizedObjectImpl(byte[] valBytes, int start, int len) {
-        this(null, valBytes, start, len);
-    }
-
-    /**
-     * Instantiates {@code CacheOptimizedObjectImpl} with object's serialized form and value.
-     * @param val Object.
-     * @param valBytes Object serialized to byte array.
-     */
-    public CacheOptimizedObjectImpl(Object val, byte[] valBytes) {
-        this(val, valBytes, 0, valBytes != null ? valBytes.length : 0);
-    }
-
-    /**
-     * Instantiates {@code CacheOptimizedObjectImpl}.
-     * @param val Object.
-     * @param valBytes Object in a serialized form.
-     * @param start Object's start in the array.
-     * @param len Object's len in the array.
-     */
-    public CacheOptimizedObjectImpl(Object val, byte[] valBytes, int start, int len) {
-        assert val != null || (valBytes != null && start >= 0 && len > 0);
-
-        this.val = val;
-        this.valBytes = valBytes;
-        this.start = start;
-        this.len = len;
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <T> T value(CacheObjectContext ctx, boolean cpy) {
-        return (T)this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte[] valueBytes(CacheObjectContext ctx) throws IgniteCheckedException {
-        toMarshaledFormIfNeeded(ctx);
-
-        if (detached())
-            return valBytes;
-
-        byte[] arr = new byte[len];
-
-        U.arrayCopy(valBytes, start, arr, 0, len);
-
-        return arr;
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
-        if (detached())
-            return this;
-
-        return detach();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void finishUnmarshal(CacheObjectContext ctx, ClassLoader ldr) throws IgniteCheckedException {
-        assert val != null || valBytes != null;
-
-        if (val == null && ctx.storeValue())
-            val = ctx.processor().unmarshal(ctx, valBytes, start, len, ldr);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void prepareMarshal(CacheObjectContext ctx) throws IgniteCheckedException {
-        assert val != null || valBytes != null;
-
-        toMarshaledFormIfNeeded(ctx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte directType() {
-        // refer to GridIoMessageFactory.
-        return 113;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte type() {
-        return TYPE_OPTIMIZED;
-    }
-
-    /**
-     * Returns object's type ID.
-     *
-     * @return Type ID.
-     */
-    public int typeId() {
-        assert valBytes != null;
-
-        int typeId = UNSAFE.getInt(valBytes, BYTE_ARR_OFF + start + 1);
-
-        if (typeId == 0)
-            throw new IgniteException("Object's type ID wasn't written to cache.");
-
-        return typeId;
-    }
-
-    /**
-     * Checks whether a wrapped object has field with name {@code fieldName}.
-     *
-     * @param fieldName Field name.
-     * @param marsh Marshaller.
-     * @return {@code true} if has.
-     * @throws IgniteCheckedException In case of error.
-     */
-    public boolean hasField(String fieldName, OptimizedMarshallerExt marsh) throws IgniteCheckedException {
-        assert valBytes != null;
-
-        return marsh.hasField(fieldName, valBytes, start, len);
-    }
-
-    /**
-     * Searches and returns field if it exists.
-     *
-     * @param fieldName Field name.
-     * @param marsh Marshaller.
-     * @return Field.
-     * @throws IgniteCheckedException In case of error.
-     */
-    public Object field(String fieldName, OptimizedMarshallerExt marsh) throws IgniteCheckedException {
-        assert valBytes != null;
-
-        return marsh.readField(fieldName, valBytes, start, len, val != null ? val.getClass().getClassLoader() : null);
-    }
-
-    /**
-     * Deserializes wrapped object.
-     *
-     * @param ctx Cache context.
-     * @return Deserialized object.
-     */
-    public Object deserialize(CacheObjectContext ctx) {
-        if (val != null)
-            return val;
-
-        try {
-            assert valBytes != null;
-
-            Object val = ctx.processor().unmarshal(ctx, valBytes, start, len,
-                ctx.kernalContext().config().getClassLoader());
-
-            if (ctx.storeValue())
-                this.val = val;
-
-            return val;
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteException("Failed to unmarshall object.", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        super.writeExternal(out);
-
-        out.writeInt(start);
-        out.writeInt(len);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        super.readExternal(in);
-
-        start = in.readInt();
-        len = in.readInt();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
-        reader.setBuffer(buf);
-
-        if (!reader.beforeMessageRead())
-            return false;
-
-        if (!super.readFrom(buf, reader))
-            return false;
-
-        switch (reader.state()) {
-            case 1:
-                len = reader.readInt("len");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 2:
-                start = reader.readInt("start");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-        }
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
-        writer.setBuffer(buf);
-
-        if (!super.writeTo(buf, writer))
-            return false;
-
-        if (!writer.isHeaderWritten()) {
-            if (!writer.writeHeader(directType(), fieldsCount()))
-                return false;
-
-            writer.onHeaderWritten();
-        }
-
-        switch (writer.state()) {
-            case 1:
-                if (!writer.writeInt("len", len))
-                    return false;
-
-                writer.incrementState();
-
-            case 2:
-                if (!writer.writeInt("start", start))
-                    return false;
-
-                writer.incrementState();
-
-        }
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte fieldsCount() {
-        return 3;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        assert false;
-
-        return super.hashCode();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object obj) {
-        assert false;
-
-        return super.equals(obj);
-    }
-
-    /**
-     * Detaches object.
-     *
-     * @return Detached object wrapped by {@code CacheOptimizedObjectImpl}.
-     */
-    protected CacheOptimizedObjectImpl detach() {
-        if (detached())
-            return this;
-
-        byte[] arr = new byte[len];
-
-        U.arrayCopy(valBytes, start, arr, 0, len);
-
-        return new CacheOptimizedObjectImpl(arr, 0, len);
-    }
-
-    /**
-     * Checks whether the object is already detached or not.
-     *
-     * @return {@code true} if detached.
-     */
-    protected boolean detached() {
-        return start == 0 && len == valBytes.length;
-    }
-
-    /**
-     * Marshals {@link #val} to {@link #valBytes} if needed.
-     *
-     * @param ctx Cache object context.
-     * @throws IgniteCheckedException In case of error.
-     */
-    protected void toMarshaledFormIfNeeded(CacheObjectContext ctx) throws IgniteCheckedException {
-        if (valBytes == null) {
-            assert val != null;
-
-            valBytes = ctx.processor().marshal(ctx, val);
-
-            start = 0;
-            len = valBytes.length;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheIndexedObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheIndexedObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheIndexedObjectImpl.java
new file mode 100644
index 0000000..b63bca6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheIndexedObjectImpl.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.jetbrains.annotations.*;
+
+/**
+ * Cache object implementation for classes that support footer injection is their serialized form thus enabling fields
+ * search and extraction without necessity to fully deserialize an object.
+ */
+public class KeyCacheIndexedObjectImpl extends CacheIndexedObjectImpl implements KeyCacheObject {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     *
+     */
+    public KeyCacheIndexedObjectImpl() {
+        // No-op
+    }
+
+    /**
+     * @param val Object.
+     * @param valBytes Object in a serialized form.
+     */
+    public KeyCacheIndexedObjectImpl(Object val, byte[] valBytes) {
+        super(val, valBytes);
+
+        assert val != null;
+    }
+
+    /**
+     * @param val Object.
+     * @param valBytes Object in a serialized form.
+     * @param start Object's start in the array.
+     * @param len Object's len in the array.
+     */
+    public KeyCacheIndexedObjectImpl(Object val, byte[] valBytes, int start, int len) {
+        super(val, valBytes, start, len);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Nullable @Override public <T> T value(CacheObjectContext ctx, boolean cpy) {
+        return (T)this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        // refer to GridIoMessageFactory.
+        return 113;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean internal() {
+        assert val != null;
+
+        return val instanceof GridCacheInternal;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object deserialize(CacheObjectContext ctx) {
+        assert val != null;
+
+        return val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        assert val != null;
+
+        return val.hashCode();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object obj) {
+        if (!(obj instanceof KeyCacheIndexedObjectImpl))
+            return false;
+
+        KeyCacheIndexedObjectImpl other = (KeyCacheIndexedObjectImpl)obj;
+
+        return val.equals(other.val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheOptimizedObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheOptimizedObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheOptimizedObjectImpl.java
deleted file mode 100644
index 56e6fe2..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheOptimizedObjectImpl.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import org.jetbrains.annotations.*;
-
-/**
- * Cache object implementation for classes that support footer injection is their serialized form thus enabling fields
- * search and extraction without necessity to fully deserialize an object.
- */
-public class KeyCacheOptimizedObjectImpl extends CacheOptimizedObjectImpl implements KeyCacheObject {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     *
-     */
-    public KeyCacheOptimizedObjectImpl() {
-        // No-op
-    }
-
-    /**
-     * @param val Object.
-     * @param valBytes Object in a serialized form.
-     */
-    public KeyCacheOptimizedObjectImpl(Object val, byte[] valBytes) {
-        super(val, valBytes);
-
-        assert val != null;
-    }
-
-    /**
-     * @param val Object.
-     * @param valBytes Object in a serialized form.
-     * @param start Object's start in the array.
-     * @param len Object's len in the array.
-     */
-    public KeyCacheOptimizedObjectImpl(Object val, byte[] valBytes, int start, int len) {
-        super(val, valBytes, start, len);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override public <T> T value(CacheObjectContext ctx, boolean cpy) {
-        return (T)this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte directType() {
-        // refer to GridIoMessageFactory.
-        return 113;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean internal() {
-        assert val != null;
-
-        return val instanceof GridCacheInternal;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object deserialize(CacheObjectContext ctx) {
-        assert val != null;
-
-        return val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        assert val != null;
-
-        return val.hashCode();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object obj) {
-        if (!(obj instanceof KeyCacheOptimizedObjectImpl))
-            return false;
-
-        KeyCacheOptimizedObjectImpl other = (KeyCacheOptimizedObjectImpl)obj;
-
-        return val.equals(other.val);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/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 748dd42..6ae958a 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
@@ -196,10 +196,10 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
     @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor")
     protected KeyCacheObject toCacheKeyObject0(Object obj, boolean userObj) {
         if (!userObj)
-            return isFieldsIndexingEnabled(obj.getClass()) ? new KeyCacheOptimizedObjectImpl(obj, null) :
+            return isFieldsIndexingEnabled(obj.getClass()) ? new KeyCacheIndexedObjectImpl(obj, null) :
                 new KeyCacheObjectImpl(obj, null);
 
-        return isFieldsIndexingEnabled(obj.getClass()) ? new UserKeyCacheOptimizedObjectImpl(obj) :
+        return isFieldsIndexingEnabled(obj.getClass()) ? new UserKeyCacheIndexedObjectImpl(obj) :
             new UserKeyCacheObjectImpl(obj);
     }
 
@@ -239,7 +239,7 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
                 return new CacheObjectImpl(null, bytes);
 
             case CacheObject.TYPE_OPTIMIZED:
-                return new CacheOptimizedObjectImpl(bytes, 0, bytes.length);
+                return new CacheIndexedObjectImpl(bytes, 0, bytes.length);
         }
 
         throw new IllegalArgumentException("Invalid object type: " + type);
@@ -271,10 +271,10 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
         }
 
         if (!userObj)
-            return isFieldsIndexingEnabled(obj.getClass()) ? new CacheOptimizedObjectImpl(obj) :
+            return isFieldsIndexingEnabled(obj.getClass()) ? new CacheIndexedObjectImpl(obj) :
                 new CacheObjectImpl(obj, null);
 
-        return isFieldsIndexingEnabled(obj.getClass()) ? new UserCacheOptimizedObjectImpl(obj, null) :
+        return isFieldsIndexingEnabled(obj.getClass()) ? new UserCacheIndexedObjectImpl(obj, null) :
             new UserCacheObjectImpl(obj, null);
     }
 
@@ -313,8 +313,8 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
 
     /** {@inheritDoc} */
     @Override public int typeId(Object obj) {
-        if (obj instanceof CacheOptimizedObjectImpl)
-            return ((CacheOptimizedObjectImpl)obj).typeId();
+        if (obj instanceof CacheIndexedObjectImpl)
+            return ((CacheIndexedObjectImpl)obj).typeId();
 
         return 0;
     }
@@ -336,11 +336,11 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
 
     /** {@inheritDoc} */
     @Override public Object field(Object obj, String fieldName) {
-        if (obj instanceof CacheOptimizedObjectImpl) {
+        if (obj instanceof CacheIndexedObjectImpl) {
             assert optMarshExt != null;
 
             try {
-                return ((CacheOptimizedObjectImpl)obj).field(fieldName, optMarshExt);
+                return ((CacheIndexedObjectImpl)obj).field(fieldName, optMarshExt);
             }
             catch (IgniteCheckedException e) {
                 throw new IgniteException(e);
@@ -352,11 +352,11 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
 
     /** {@inheritDoc} */
     @Override public boolean hasField(Object obj, String fieldName) {
-        if (obj instanceof CacheOptimizedObjectImpl) {
+        if (obj instanceof CacheIndexedObjectImpl) {
             assert optMarshExt != null;
 
             try {
-                return ((CacheOptimizedObjectImpl)obj).hasField(fieldName, optMarshExt);
+                return ((CacheIndexedObjectImpl)obj).hasField(fieldName, optMarshExt);
             }
             catch (IgniteCheckedException e) {
                 throw new IgniteException(e);
@@ -480,14 +480,14 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
      * Wraps value provided by user, must be serialized before stored in cache.
      * Used by classes that support fields indexing. Refer to {@link #isFieldsIndexingEnabled(Class)}.
      */
-    private static class UserCacheOptimizedObjectImpl extends CacheOptimizedObjectImpl {
+    private static class UserCacheIndexedObjectImpl extends CacheIndexedObjectImpl {
         /** */
         private static final long serialVersionUID = 0L;
 
         /**
          *
          */
-        public UserCacheOptimizedObjectImpl() {
+        public UserCacheIndexedObjectImpl() {
             //No-op.
         }
 
@@ -495,7 +495,7 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
          * @param val Value.
          * @param valBytes Value bytes.
          */
-        public UserCacheOptimizedObjectImpl(Object val, byte[] valBytes) {
+        public UserCacheIndexedObjectImpl(Object val, byte[] valBytes) {
             super(val, valBytes);
         }
 
@@ -516,10 +516,10 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
                     Object val = this.val != null && ctx.processor().immutable(this.val) ? this.val :
                         ctx.processor().unmarshal(ctx, valBytes, start, len, ldr);
 
-                    return new CacheOptimizedObjectImpl(val, valBytes, start, len);
+                    return new CacheIndexedObjectImpl(val, valBytes, start, len);
                 }
 
-                return new CacheOptimizedObjectImpl(null, valBytes, start, len);
+                return new CacheIndexedObjectImpl(null, valBytes, start, len);
             }
             catch (IgniteCheckedException e) {
                 throw new IgniteException("Failed to marshal object: " + val, e);
@@ -531,21 +531,21 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
      * Wraps key provided by user, must be serialized before stored in cache.
      * Used by classes that support fields indexing. Refer to {@link #isFieldsIndexingEnabled(Class)}.
      */
-    private static class UserKeyCacheOptimizedObjectImpl extends KeyCacheOptimizedObjectImpl {
+    private static class UserKeyCacheIndexedObjectImpl extends KeyCacheIndexedObjectImpl {
         /** */
         private static final long serialVersionUID = 0L;
 
         /**
          *
          */
-        public UserKeyCacheOptimizedObjectImpl() {
+        public UserKeyCacheIndexedObjectImpl() {
             //No-op.
         }
 
         /**
          * @param key Key.
          */
-        UserKeyCacheOptimizedObjectImpl(Object key) {
+        UserKeyCacheIndexedObjectImpl(Object key) {
             super(key, null);
         }
 
@@ -560,10 +560,10 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
 
                     Object val = ctx.processor().unmarshal(ctx, valBytes, start, len, ldr);
 
-                    return new KeyCacheOptimizedObjectImpl(val, valBytes, start, len);
+                    return new KeyCacheIndexedObjectImpl(val, valBytes, start, len);
                 }
 
-                return new KeyCacheOptimizedObjectImpl(val, valBytes, start, len);
+                return new KeyCacheIndexedObjectImpl(val, valBytes, start, len);
             }
             catch (IgniteCheckedException e) {
                 throw new IgniteException("Failed to marshal object: " + val, e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index e5800de..f51a0bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -47,6 +47,7 @@ import java.util.concurrent.*;
 import static org.apache.ignite.events.EventType.*;
 import static org.apache.ignite.internal.IgniteComponentType.*;
 import static org.apache.ignite.internal.processors.query.GridQueryIndexType.*;
+import static org.apache.ignite.internal.processors.query.GridQueryProcessor.PropertyType.*;
 
 /**
  * Indexing processor.
@@ -141,12 +142,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     TypeId typeId;
 
                     if (valCls == null || ctx.cacheObjects().isPortableEnabled()) {
-                        processPortableMeta(meta, desc);
+                        processCacheTypeMeta(meta, desc, PORTABLE_PROPERTY);
 
                         typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType()));
                     }
                     else if (ctx.cacheObjects().enableFieldsIndexing(valCls)) {
-                        processIndexedFieldsMeta(meta, desc);
+                        processCacheTypeMeta(meta, desc, INDEXED_FIELDS_PROPERTY);
 
                         typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(valCls.getName()));
                     }
@@ -459,7 +460,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             TypeId id;
 
             boolean portableVal = ctx.cacheObjects().isPortableObject(val);
-            boolean indexedFieldsVal = val instanceof CacheOptimizedObjectImpl;
+            boolean indexedFieldsVal = val instanceof CacheIndexedObjectImpl;
 
             if (portableVal || indexedFieldsVal) {
                 int typeId = ctx.cacheObjects().typeId(val);
@@ -482,7 +483,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     "(multiple classes with same simple name are stored in the same cache) " +
                     "[expCls=" + desc.valueClass().getName() + ", actualCls=" + valCls.getName() + ']');
 
-            if (!(key instanceof CacheOptimizedObjectImpl) && !ctx.cacheObjects().isPortableObject(key)) {
+            if (!(key instanceof CacheIndexedObjectImpl) && !ctx.cacheObjects().isPortableObject(key)) {
                 Class<?> keyCls = key.value(coctx, false).getClass();
 
                 if (!desc.keyClass().isAssignableFrom(keyCls))
@@ -1234,91 +1235,21 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Processes declarative metadata for portable object.
+     * Processes declarative metadata.
      *
      * @param meta Declared metadata.
      * @param d Type descriptor.
+     * @param propertyType PropertyType.
      * @throws IgniteCheckedException If failed.
      */
-    private void processPortableMeta(CacheTypeMetadata meta, TypeDescriptor d)
+    private void processCacheTypeMeta(CacheTypeMetadata meta, TypeDescriptor d, PropertyType propertyType)
         throws IgniteCheckedException {
-        for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet()) {
-            PortableProperty prop = buildPortableProperty(entry.getKey(), entry.getValue());
-
-            d.addProperty(prop, false);
-
-            String idxName = prop.name() + "_idx";
-
-            d.addIndex(idxName, idx.isGeometryClass(prop.type()) ? GEO_SPATIAL : SORTED);
-
-            d.addFieldToIndex(idxName, prop.name(), 0, false);
-        }
-
-        for (Map.Entry<String, Class<?>> entry : meta.getDescendingFields().entrySet()) {
-            PortableProperty prop = buildPortableProperty(entry.getKey(), entry.getValue());
-
-            d.addProperty(prop, false);
-
-            String idxName = prop.name() + "_idx";
-
-            d.addIndex(idxName, idx.isGeometryClass(prop.type()) ? GEO_SPATIAL : SORTED);
-
-            d.addFieldToIndex(idxName, prop.name(), 0, true);
-        }
-
-        for (String txtIdx : meta.getTextFields()) {
-            PortableProperty prop = buildPortableProperty(txtIdx, String.class);
-
-            d.addProperty(prop, false);
-
-            d.addFieldToTextIndex(prop.name());
-        }
-
-        Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> grps = meta.getGroups();
-
-        if (grps != null) {
-            for (Map.Entry<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> entry : grps.entrySet()) {
-                String idxName = entry.getKey();
-
-                LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>> idxFields = entry.getValue();
-
-                int order = 0;
-
-                for (Map.Entry<String, IgniteBiTuple<Class<?>, Boolean>> idxField : idxFields.entrySet()) {
-                    PortableProperty prop = buildPortableProperty(idxField.getKey(), idxField.getValue().get1());
-
-                    d.addProperty(prop, false);
-
-                    Boolean descending = idxField.getValue().get2();
-
-                    d.addFieldToIndex(idxName, prop.name(), order, descending != null && descending);
-
-                    order++;
-                }
-            }
-        }
-
-        for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
-            PortableProperty prop = buildPortableProperty(entry.getKey(), entry.getValue());
-
-            if (!d.props.containsKey(prop.name()))
-                d.addProperty(prop, false);
-        }
-    }
-
-    /**
-     * Processes declarative metadata for object that has fields information in its serialized form.
-     *
-     * @param meta Declared metadata.
-     * @param d Type descriptor.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void processIndexedFieldsMeta(CacheTypeMetadata meta, TypeDescriptor d)
-        throws IgniteCheckedException {
-        //TODO: IGNITE-950, refactor. The code is similar to portable properties ones.
+        assert propertyType != CLASS_PROPERTY;
 
         for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet()) {
-            IndexedFieldsProperty prop = buildIndexedFieldsProperty(entry.getKey(), entry.getValue());
+            Property prop = propertyType == PORTABLE_PROPERTY ?
+                buildPortableProperty(entry.getKey(), entry.getValue()) :
+                buildIndexedFieldsProperty(entry.getKey(), entry.getValue());
 
             d.addProperty(prop, false);
 
@@ -1330,7 +1261,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (Map.Entry<String, Class<?>> entry : meta.getDescendingFields().entrySet()) {
-            IndexedFieldsProperty prop = buildIndexedFieldsProperty(entry.getKey(), entry.getValue());
+            Property prop = propertyType == PORTABLE_PROPERTY ?
+                buildPortableProperty(entry.getKey(), entry.getValue()) :
+                buildIndexedFieldsProperty(entry.getKey(), entry.getValue());
 
             d.addProperty(prop, false);
 
@@ -1342,7 +1275,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (String txtIdx : meta.getTextFields()) {
-            IndexedFieldsProperty prop = buildIndexedFieldsProperty(txtIdx, String.class);
+            Property prop = propertyType == PORTABLE_PROPERTY ?
+                buildPortableProperty(txtIdx, String.class) :
+                buildIndexedFieldsProperty(txtIdx, String.class);
 
             d.addProperty(prop, false);
 
@@ -1360,8 +1295,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 int order = 0;
 
                 for (Map.Entry<String, IgniteBiTuple<Class<?>, Boolean>> idxField : idxFields.entrySet()) {
-                    IndexedFieldsProperty prop = buildIndexedFieldsProperty(idxField.getKey(),
-                        idxField.getValue().get1());
+                    Property prop = propertyType == PORTABLE_PROPERTY ?
+                        buildPortableProperty(idxField.getKey(), idxField.getValue().get1()) :
+                        buildIndexedFieldsProperty(idxField.getKey(), idxField.getValue().get1());
 
                     d.addProperty(prop, false);
 
@@ -1375,7 +1311,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
-            IndexedFieldsProperty prop = buildIndexedFieldsProperty(entry.getKey(), entry.getValue());
+            Property prop = propertyType == PORTABLE_PROPERTY ?
+                buildPortableProperty(entry.getKey(), entry.getValue()) :
+                buildIndexedFieldsProperty(entry.getKey(), entry.getValue());
 
             if (!d.props.containsKey(prop.name()))
                 d.addProperty(prop, false);
@@ -2247,4 +2185,13 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             return S.toString(TypeName.class, this);
         }
     }
+
+    /**
+     *
+     */
+    enum PropertyType {
+        CLASS_PROPERTY,
+        PORTABLE_PROPERTY,
+        INDEXED_FIELDS_PROPERTY
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/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 2f42e8d..7a7ee69 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
@@ -248,7 +248,7 @@ public class OptimizedMarshallerUtils {
         if (obj == null)
             return false;
 
-        if (obj instanceof CacheOptimizedObjectImpl ||
+        if (obj instanceof CacheIndexedObjectImpl ||
             obj instanceof Map.Entry ||
             obj instanceof Collection ||
             obj instanceof Map ||

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java
index 1b75f57..66544a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java
@@ -125,7 +125,7 @@ public class OptimizedObjectInputStreamExt extends OptimizedObjectInputStream {
 
             if (in.readByte() == SERIALIZABLE && metaHandler.metadata(in.readInt()) != null)
                 //Do we need to make a copy of array?
-                field = (F)new CacheOptimizedObjectImpl(in.array(), range.start, range.len);
+                field = (F)new CacheIndexedObjectImpl(in.array(), range.start, range.len);
             else {
                 in.position(range.start);
                 field = (F)readObject();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/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
index 2526de7..c1f07ce 100644
--- 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
@@ -22,24 +22,44 @@ 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 {
+    /** */
+    private static ConcurrentHashMap<Integer, OptimizedObjectMetadata> META_BUF = new ConcurrentHashMap<>();
+
+    /** */
+    private static final OptimizedMarshallerExtMetaHandler META_HANDLER = new OptimizedMarshallerExtMetaHandler() {
+        @Override public void addMeta(int typeId, OptimizedObjectMetadata meta) {
+            META_BUF.putIfAbsent(typeId, meta);
+        }
+
+        @Override public OptimizedObjectMetadata metadata(int typeId) {
+            return META_BUF.get(typeId);
+        }
+    };
+
     /** {@inheritDoc} */
     @Override protected Marshaller marshaller() {
-        return new OptimizedMarshallerExt(false);
+        return new InternalMarshaller(false);
     }
 
     /**
      * @throws Exception In case of error.
      */
     public void testHasField() throws Exception {
+        META_BUF.clear();
+
         OptimizedMarshallerExt marsh = (OptimizedMarshallerExt)OptimizedMarshallerExtSelfTest.marsh;
 
         assertTrue(marsh.enableFieldsIndexing(TestObject.class));
 
+        assertEquals(1, META_BUF.size());
+
         TestObject testObj = new TestObject("World", 50);
 
         byte[] arr = marsh.marshal(testObj);
@@ -54,9 +74,12 @@ public class OptimizedMarshallerExtSelfTest extends OptimizedMarshallerSelfTest
      * @throws Exception In case of error.
      */
     public void testReadField() throws Exception {
+        META_BUF.clear();
+
         OptimizedMarshallerExt marsh = (OptimizedMarshallerExt)OptimizedMarshallerExtSelfTest.marsh;
 
         assertTrue(marsh.enableFieldsIndexing(TestObject.class));
+        assertEquals(1, META_BUF.size());
 
         TestObject testObj = new TestObject("World", 50);
 
@@ -75,11 +98,12 @@ public class OptimizedMarshallerExtSelfTest extends OptimizedMarshallerSelfTest
 
         // Add metadata for the enclosed object.
         assertTrue(marsh.enableFieldsIndexing(TestObject2.class));
+        assertEquals(2, META_BUF.size());
 
         arr = marsh.marshal(testObj);
 
         // Must be returned in a wrapped form, since metadata was added enabling the footer.
-        CacheOptimizedObjectImpl cacheObject = marsh.readField("o2", arr, 0, arr.length, null);
+        CacheIndexedObjectImpl cacheObject = marsh.readField("o2", arr, 0, arr.length, null);
 
         arr = cacheObject.valueBytes(null);
 
@@ -92,9 +116,12 @@ public class OptimizedMarshallerExtSelfTest extends OptimizedMarshallerSelfTest
      * @throws Exception In case of error.
      */
     public void testHandles() throws Exception {
+        META_BUF.clear();
+
         OptimizedMarshallerExt marsh = (OptimizedMarshallerExt)OptimizedMarshallerExtSelfTest.marsh;
 
         assertTrue(marsh.enableFieldsIndexing(SelfLinkObject.class));
+        assertEquals(1, META_BUF.size());
 
         SelfLinkObject selfLinkObject = new SelfLinkObject();
         selfLinkObject.str1 = "Hello, world!";
@@ -107,7 +134,7 @@ public class OptimizedMarshallerExtSelfTest extends OptimizedMarshallerSelfTest
 
         assertEquals(selfLinkObject.str1, str2);
 
-        CacheOptimizedObjectImpl cacheObj = marsh.readField("link", arr, 0, arr.length, null);
+        CacheIndexedObjectImpl cacheObj = marsh.readField("link", arr, 0, arr.length, null);
 
         arr = cacheObj.valueBytes(null);
 
@@ -116,6 +143,29 @@ public class OptimizedMarshallerExtSelfTest extends OptimizedMarshallerSelfTest
         assertEquals(selfLinkObject, selfLinkObject2);
     }
 
+    private static class InternalMarshaller extends OptimizedMarshallerExt {
+        /**
+         * Constructor.
+         */
+        public InternalMarshaller() {
+        }
+
+        /**
+         * Constructor.
+         * @param requireSer Requires serialiazable.
+         */
+        public InternalMarshaller(boolean requireSer) {
+            super(requireSer);
+
+            super.setMetadataHandler(META_HANDLER);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void setMetadataHandler(OptimizedMarshallerExtMetaHandler metaHandler) {
+            // No-op
+        }
+    }
+
     /** */
     private static class TestObject2 {
         /** */