You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2015/10/30 11:44:47 UTC

[19/19] ignite git commit: ignite-950-new WIP

ignite-950-new WIP


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

Branch: refs/heads/ignite-950-new
Commit: 35b6d61fad1963dfdc6fcfc4fac748acf7974830
Parents: e33fa63
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Fri Oct 30 13:44:08 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Oct 30 13:44:10 2015 +0300

----------------------------------------------------------------------
 .../ignite/examples/portable/Address.java       |   16 +-
 ...mputeClientPortableTaskExecutionExample.java |    4 +-
 .../portable/computegrid/ComputeClientTask.java |   16 +-
 .../CacheClientPortablePutGetExample.java       |   12 +-
 .../CacheClientPortableQueryExample.java        |   30 +-
 .../src/main/java/org/apache/ignite/Ignite.java |    6 +-
 .../java/org/apache/ignite/IgniteCache.java     |    4 +-
 .../java/org/apache/ignite/IgniteObjects.java   |  366 ++
 .../java/org/apache/ignite/IgnitePortables.java |  370 --
 .../ignite/cache/CacheKeyConfiguration.java     |   92 +
 .../org/apache/ignite/cache/QueryEntity.java    |   26 +-
 .../apache/ignite/cache/QueryEntityIndex.java   |  100 -
 .../org/apache/ignite/cache/QueryIndex.java     |  173 +
 .../org/apache/ignite/cache/QueryIndexType.java |   24 +
 .../configuration/CacheConfiguration.java       |   37 +-
 .../configuration/IgniteConfiguration.java      |   24 +
 .../ignite/igniteobject/IgniteObject.java       |  153 +
 .../igniteobject/IgniteObjectBuilder.java       |  136 +
 .../igniteobject/IgniteObjectConfiguration.java |  155 +
 .../igniteobject/IgniteObjectException.java     |   57 +
 .../igniteobject/IgniteObjectIdMapper.java      |   56 +
 .../IgniteObjectInvalidClassException.java      |   58 +
 .../igniteobject/IgniteObjectMarshalAware.java  |   48 +
 .../igniteobject/IgniteObjectMetadata.java      |   60 +
 .../igniteobject/IgniteObjectRawReader.java     |  240 ++
 .../igniteobject/IgniteObjectRawWriter.java     |  225 ++
 .../ignite/igniteobject/IgniteObjectReader.java |  291 ++
 .../igniteobject/IgniteObjectSerializer.java    |   49 +
 .../ignite/igniteobject/IgniteObjectWriter.java |  273 ++
 .../ignite/igniteobject/package-info.java       |   22 +
 .../apache/ignite/internal/IgniteKernal.java    |    5 +-
 .../communication/GridIoMessageFactory.java     |    4 +-
 .../portable/GridPortableMarshaller.java        |   32 +-
 .../internal/portable/IgniteObjectEx.java       |  214 ++
 .../internal/portable/IgniteObjectImpl.java     |  411 +++
 .../portable/IgniteObjectMetaDataCollector.java |  263 ++
 .../portable/IgniteObjectMetaDataImpl.java      |  150 +
 .../portable/IgniteObjectOffheapImpl.java       |  255 ++
 .../portable/IgniteObjectRawReaderEx.java       |   33 +
 .../portable/IgniteObjectRawWriterEx.java       |   60 +
 .../portable/IgniteObjectReaderExImpl.java      | 3230 ++++++++++++++++++
 .../portable/IgniteObjectWriterExImpl.java      | 1892 ++++++++++
 .../portable/PortableClassDescriptor.java       |   88 +-
 .../internal/portable/PortableContext.java      |  147 +-
 .../portable/PortableMetaDataCollector.java     |  263 --
 .../portable/PortableMetaDataHandler.java       |   12 +-
 .../internal/portable/PortableMetaDataImpl.java |  150 -
 .../internal/portable/PortableObjectEx.java     |  214 --
 .../internal/portable/PortableObjectImpl.java   |  411 ---
 .../portable/PortableObjectOffheapImpl.java     |  255 --
 .../internal/portable/PortableRawReaderEx.java  |   33 -
 .../internal/portable/PortableRawWriterEx.java  |   60 -
 .../portable/PortableReaderContext.java         |    8 +-
 .../internal/portable/PortableReaderExImpl.java | 3230 ------------------
 .../ignite/internal/portable/PortableUtils.java |   10 +-
 .../internal/portable/PortableWriterExImpl.java | 1892 ----------
 .../builder/IgniteObjectBuilderImpl.java        |  540 +++
 .../portable/builder/PortableBuilderEnum.java   |    8 +-
 .../portable/builder/PortableBuilderImpl.java   |  544 ---
 .../portable/builder/PortableBuilderReader.java |   56 +-
 .../PortableBuilderSerializationAware.java      |    2 +-
 .../builder/PortableBuilderSerializer.java      |   29 +-
 .../builder/PortableEnumArrayLazyValue.java     |   12 +-
 .../portable/builder/PortableLazyArrayList.java |    2 +-
 .../builder/PortableLazyLinkedList.java         |    2 +-
 .../portable/builder/PortableLazyMap.java       |    2 +-
 .../portable/builder/PortableLazyMapEntry.java  |    2 +-
 .../portable/builder/PortableLazySet.java       |    4 +-
 .../builder/PortableModifiableLazyValue.java    |    2 +-
 .../builder/PortableObjectArrayLazyValue.java   |    8 +-
 .../builder/PortablePlainLazyValue.java         |    2 +-
 .../builder/PortablePlainPortableObject.java    |   22 +-
 .../portable/builder/PortableValueWithType.java |    4 +-
 .../streams/PortableAbstractInputStream.java    |    6 +-
 .../internal/processors/cache/CacheObject.java  |    2 +-
 .../processors/cache/IgniteCacheProxy.java      |    2 +-
 .../CacheDefaultPortableAffinityKeyMapper.java  |    6 +-
 .../portable/CacheObjectPortableContext.java    |    8 -
 .../portable/CacheObjectPortableProcessor.java  |   24 +-
 .../CacheObjectPortableProcessorImpl.java       |  202 +-
 .../cache/portable/IgniteObjectsImpl.java       |  177 +
 .../cache/portable/IgnitePortablesImpl.java     |  177 -
 .../platform/PlatformAbstractTarget.java        |   30 +-
 .../processors/platform/PlatformContext.java    |   26 +-
 .../platform/PlatformContextImpl.java           |   47 +-
 .../platform/PlatformExtendedException.java     |    4 +-
 .../platform/PlatformProcessorImpl.java         |    4 +-
 .../platform/cache/PlatformCache.java           |   40 +-
 .../cache/PlatformCacheEntryFilterImpl.java     |    6 +-
 .../cache/PlatformCacheEntryProcessorImpl.java  |   12 +-
 .../platform/cache/PlatformCacheIterator.java   |    4 +-
 .../PlatformCachePartialUpdateException.java    |    4 +-
 .../cache/affinity/PlatformAffinity.java        |    8 +-
 .../query/PlatformAbstractQueryCursor.java      |   10 +-
 .../PlatformContinuousQueryRemoteFilter.java    |    4 +-
 .../cache/query/PlatformFieldsQueryCursor.java  |    4 +-
 .../cache/query/PlatformQueryCursor.java        |    4 +-
 .../cache/store/PlatformCacheStoreCallback.java |    6 +-
 .../platform/cluster/PlatformClusterGroup.java  |   12 +-
 .../cluster/PlatformClusterNodeFilterImpl.java  |    4 +-
 .../platform/compute/PlatformAbstractJob.java   |    4 +-
 .../platform/compute/PlatformAbstractTask.java  |    6 +-
 .../platform/compute/PlatformClosureJob.java    |    4 +-
 .../platform/compute/PlatformCompute.java       |   24 +-
 .../platform/compute/PlatformFullJob.java       |    6 +-
 .../platform/compute/PlatformFullTask.java      |   12 +-
 .../datastreamer/PlatformDataStreamer.java      |    4 +-
 .../PlatformStreamReceiverImpl.java             |    4 +-
 .../dotnet/PlatformDotNetCacheStore.java        |   48 +-
 .../PlatformDotNetConfigurationClosure.java     |   14 +-
 .../events/PlatformEventFilterListenerImpl.java |    6 +-
 .../platform/events/PlatformEvents.java         |   18 +-
 .../messaging/PlatformMessageFilterImpl.java    |    6 +-
 .../messaging/PlatformMessageLocalFilter.java   |    4 +-
 .../platform/messaging/PlatformMessaging.java   |    8 +-
 .../services/PlatformAbstractService.java       |   16 +-
 .../platform/services/PlatformServices.java     |   20 +-
 .../transactions/PlatformTransactions.java      |    5 +-
 .../platform/utils/PlatformFutureUtils.java     |   10 +-
 .../platform/utils/PlatformReaderBiClosure.java |    4 +-
 .../platform/utils/PlatformReaderClosure.java   |    4 +-
 .../platform/utils/PlatformUtils.java           |   60 +-
 .../platform/utils/PlatformWriterBiClosure.java |    4 +-
 .../platform/utils/PlatformWriterClosure.java   |    4 +-
 .../processors/query/GridQueryProcessor.java    |   28 +-
 .../marshaller/portable/PortableMarshaller.java |   42 +-
 .../apache/ignite/portable/PortableBuilder.java |  137 -
 .../ignite/portable/PortableException.java      |   57 -
 .../ignite/portable/PortableIdMapper.java       |   56 -
 .../portable/PortableInvalidClassException.java |   58 -
 .../ignite/portable/PortableMarshalAware.java   |   48 -
 .../ignite/portable/PortableMetadata.java       |   61 -
 .../apache/ignite/portable/PortableObject.java  |  154 -
 .../apache/ignite/portable/PortableReader.java  |  291 --
 .../ignite/portable/PortableSerializer.java     |   49 -
 .../portable/PortableTypeConfiguration.java     |  177 -
 .../apache/ignite/portable/PortableWriter.java  |  273 --
 .../apache/ignite/portable/package-info.java    |   22 -
 ...idIgniteObjectBuilderAdditionalSelfTest.java | 1289 +++++++
 .../GridIgniteObjectBuilderSelfTest.java        | 1069 ++++++
 ...tBuilderStringAsCharsAdditionalSelfTest.java |   28 +
 ...gniteObjectBuilderStringAsCharsSelfTest.java |   28 +
 .../GridPortableAffinityKeySelfTest.java        |   10 +-
 .../GridPortableBuilderAdditionalSelfTest.java  | 1289 -------
 .../portable/GridPortableBuilderSelfTest.java   | 1069 ------
 ...eBuilderStringAsCharsAdditionalSelfTest.java |   28 -
 ...ridPortableBuilderStringAsCharsSelfTest.java |   28 -
 ...idPortableMarshallerCtxDisabledSelfTest.java |   24 +-
 .../GridPortableMarshallerSelfTest.java         |  444 +--
 .../GridPortableMetaDataDisabledSelfTest.java   |   40 +-
 .../portable/GridPortableMetaDataSelfTest.java  |   40 +-
 .../portable/GridPortableWildcardsSelfTest.java |   70 +-
 ...GridIgniteObjectMarshalerAwareTestClass.java |   67 +
 .../GridPortableMarshalerAwareTestClass.java    |   67 -
 .../mutabletest/GridPortableTestClasses.java    |    6 +-
 ...ntNodeIgniteObjectMetadataMultinodeTest.java |  295 ++
 ...CacheClientNodeIgniteObjectMetadataTest.java |  290 ++
 ...ClientNodePortableMetadataMultinodeTest.java |  295 --
 ...GridCacheClientNodePortableMetadataTest.java |  286 --
 ...niteObjectsAbstractDataStreamerSelfTest.java |  190 ++
 ...iteObjectsAbstractMultiThreadedSelfTest.java |  231 ++
 .../GridCacheIgniteObjectsAbstractSelfTest.java |  981 ++++++
 ...ableObjectsAbstractDataStreamerSelfTest.java |  190 --
 ...bleObjectsAbstractMultiThreadedSelfTest.java |  231 --
 ...ridCachePortableObjectsAbstractSelfTest.java |  981 ------
 ...GridCachePortableStorePortablesSelfTest.java |    6 +-
 ...ridPortableCacheEntryMemorySizeSelfTest.java |    8 +-
 ...leDuplicateIndexObjectsAbstractSelfTest.java |    8 +-
 .../DataStreamProcessorPortableSelfTest.java    |    6 +-
 .../GridDataStreamerImplSelfTest.java           |   26 +-
 ...ridCacheAffinityRoutingPortableSelfTest.java |   10 +-
 ...rtableDataStreamerMultithreadedSelfTest.java |    4 +-
 ...tionedOnlyPortableMultithreadedSelfTest.java |    4 +-
 ...AtomicNearDisabledOffheapTieredSelfTest.java |   29 +
 ...IgniteObjectsAtomicNearDisabledSelfTest.java |   51 +
 ...gniteObjectsAtomicOffheapTieredSelfTest.java |   29 +
 .../GridCacheIgniteObjectsAtomicSelfTest.java   |   51 +
 ...tionedNearDisabledOffheapTieredSelfTest.java |   30 +
 ...eObjectsPartitionedNearDisabledSelfTest.java |   51 +
 ...ObjectsPartitionedOffheapTieredSelfTest.java |   30 +
 ...idCacheIgniteObjectsPartitionedSelfTest.java |   51 +
 ...eapTieredEvictionAtomicPortableSelfTest.java |    6 +-
 ...heOffHeapTieredEvictionPortableSelfTest.java |    6 +-
 ...AtomicNearDisabledOffheapTieredSelfTest.java |   29 -
 ...rtableObjectsAtomicNearDisabledSelfTest.java |   51 -
 ...tableObjectsAtomicOffheapTieredSelfTest.java |   29 -
 .../GridCachePortableObjectsAtomicSelfTest.java |   51 -
 ...tionedNearDisabledOffheapTieredSelfTest.java |   30 -
 ...eObjectsPartitionedNearDisabledSelfTest.java |   51 -
 ...ObjectsPartitionedOffheapTieredSelfTest.java |   30 -
 ...CachePortableObjectsPartitionedSelfTest.java |   51 -
 ...ridCacheIgniteObjectsReplicatedSelfTest.java |   51 +
 ...dCachePortableObjectsReplicatedSelfTest.java |   51 -
 ...idCacheIgniteObjectsAtomicLocalSelfTest.java |   32 +
 ...IgniteObjectsLocalOffheapTieredSelfTest.java |   29 +
 .../GridCacheIgniteObjectsLocalSelfTest.java    |   51 +
 ...CachePortableObjectsAtomicLocalSelfTest.java |   32 -
 ...rtableObjectsLocalOffheapTieredSelfTest.java |   29 -
 .../GridCachePortableObjectsLocalSelfTest.java  |   51 -
 .../PlatformComputePortableArgTask.java         |    8 +-
 .../platform/PlatformEventsWriteEventTask.java  |    6 +-
 .../ignite/testframework/junits/IgniteMock.java |    4 +-
 .../multijvm/IgniteCacheProcessProxy.java       |    2 +-
 .../junits/multijvm/IgniteProcessProxy.java     |    4 +-
 .../IgnitePortableObjectsTestSuite.java         |   72 +-
 .../org/apache/ignite/IgniteSpringBean.java     |    2 +-
 206 files changed, 15821 insertions(+), 15164 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/examples/src/main/java/org/apache/ignite/examples/portable/Address.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/Address.java b/examples/src/main/java/org/apache/ignite/examples/portable/Address.java
index cb08b25..1c23e41 100644
--- a/examples/src/main/java/org/apache/ignite/examples/portable/Address.java
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/Address.java
@@ -17,19 +17,19 @@
 
 package org.apache.ignite.examples.portable;
 
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMarshalAware;
+import org.apache.ignite.igniteobject.IgniteObjectReader;
+import org.apache.ignite.igniteobject.IgniteObjectWriter;
 
 /**
  * Employee address.
  * <p>
- * This class implements {@link PortableMarshalAware} only for example purposes,
+ * This class implements {@link org.apache.ignite.igniteobject.IgniteObjectMarshalAware} only for example purposes,
  * in order to show how to customize serialization and deserialization of
  * portable objects.
  */
-public class Address implements PortableMarshalAware {
+public class Address implements IgniteObjectMarshalAware {
     /** Street. */
     private String street;
 
@@ -53,13 +53,13 @@ public class Address implements PortableMarshalAware {
     }
 
     /** {@inheritDoc} */
-    @Override public void writePortable(PortableWriter writer) throws PortableException {
+    @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
         writer.writeString("street", street);
         writer.writeInt("zip", zip);
     }
 
     /** {@inheritDoc} */
-    @Override public void readPortable(PortableReader reader) throws PortableException {
+    @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
         street = reader.readString("street");
         zip = reader.readInt("zip");
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientPortableTaskExecutionExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientPortableTaskExecutionExample.java b/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientPortableTaskExecutionExample.java
index 34d9cde..15eed07 100644
--- a/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientPortableTaskExecutionExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientPortableTaskExecutionExample.java
@@ -25,7 +25,7 @@ import org.apache.ignite.Ignition;
 import org.apache.ignite.examples.portable.Address;
 import org.apache.ignite.examples.portable.Employee;
 import org.apache.ignite.examples.portable.ExamplePortableNodeStartup;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObject;
 
 /**
  * This example demonstrates use of portable objects with task execution.
@@ -74,7 +74,7 @@ public class ComputeClientPortableTaskExecutionExample {
             // Convert collection of employees to collection of portable objects.
             // This allows to send objects across nodes without requiring to have
             // Employee class on classpath of these nodes.
-            Collection<PortableObject> portables = ignite.portables().toPortable(employees);
+            Collection<IgniteObject> portables = ignite.portables().toPortable(employees);
 
             // Execute task and get average salary.
             Long avgSalary = ignite.compute(ignite.cluster().forRemotes()).execute(new ComputeClientTask(), portables);

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientTask.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientTask.java b/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientTask.java
index 0eee8c6..94b4e0c 100644
--- a/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientTask.java
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientTask.java
@@ -25,7 +25,7 @@ import org.apache.ignite.compute.ComputeJobAdapter;
 import org.apache.ignite.compute.ComputeJobResult;
 import org.apache.ignite.compute.ComputeTaskSplitAdapter;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObject;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -37,19 +37,19 @@ import org.jetbrains.annotations.Nullable;
  * for each batch. After all jobs are executed, there results are reduced to
  * get the average salary.
  */
-public class ComputeClientTask extends ComputeTaskSplitAdapter<Collection<PortableObject>, Long> {
+public class ComputeClientTask extends ComputeTaskSplitAdapter<Collection<IgniteObject>, Long> {
     /** {@inheritDoc} */
     @Override protected Collection<? extends ComputeJob> split(
         int gridSize,
-        Collection<PortableObject> arg
+        Collection<IgniteObject> arg
     ) {
         Collection<ComputeClientJob> jobs = new ArrayList<>();
 
-        Collection<PortableObject> employees = new ArrayList<>();
+        Collection<IgniteObject> employees = new ArrayList<>();
 
         // Split provided collection into batches and
         // create a job for each batch.
-        for (PortableObject employee : arg) {
+        for (IgniteObject employee : arg) {
             employees.add(employee);
 
             if (employees.size() == 3) {
@@ -85,12 +85,12 @@ public class ComputeClientTask extends ComputeTaskSplitAdapter<Collection<Portab
      */
     private static class ComputeClientJob extends ComputeJobAdapter {
         /** Collection of employees. */
-        private final Collection<PortableObject> employees;
+        private final Collection<IgniteObject> employees;
 
         /**
          * @param employees Collection of employees.
          */
-        private ComputeClientJob(Collection<PortableObject> employees) {
+        private ComputeClientJob(Collection<IgniteObject> employees) {
             this.employees = employees;
         }
 
@@ -99,7 +99,7 @@ public class ComputeClientTask extends ComputeTaskSplitAdapter<Collection<Portab
             long sum = 0;
             int cnt = 0;
 
-            for (PortableObject employee : employees) {
+            for (IgniteObject employee : employees) {
                 System.out.println(">>> Processing employee: " + employee.field("name"));
 
                 // Get salary from portable object. Note that object

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortablePutGetExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortablePutGetExample.java b/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortablePutGetExample.java
index 77c5d95..793bfbf 100644
--- a/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortablePutGetExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortablePutGetExample.java
@@ -32,7 +32,7 @@ import org.apache.ignite.examples.portable.Address;
 import org.apache.ignite.examples.portable.ExamplePortableNodeStartup;
 import org.apache.ignite.examples.portable.Organization;
 import org.apache.ignite.examples.portable.OrganizationType;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObject;
 
 /**
  * This example demonstrates use of portable objects with Ignite cache.
@@ -133,10 +133,10 @@ public class CacheClientPortablePutGetExample {
         cache.put(1, org);
 
         // Get cache that will get values as portable objects.
-        IgniteCache<Integer, PortableObject> portableCache = cache.withKeepPortable();
+        IgniteCache<Integer, IgniteObject> portableCache = cache.withKeepBinary();
 
         // Get recently created organization as a portable object.
-        PortableObject po = portableCache.get(1);
+        IgniteObject po = portableCache.get(1);
 
         // Get organization's name from portable object (note that
         // object doesn't need to be fully deserialized).
@@ -212,16 +212,16 @@ public class CacheClientPortablePutGetExample {
         cache.putAll(map);
 
         // Get cache that will get values as portable objects.
-        IgniteCache<Integer, PortableObject> portableCache = cache.withKeepPortable();
+        IgniteCache<Integer, IgniteObject> portableCache = cache.withKeepBinary();
 
         // Get recently created organizations as portable objects.
-        Map<Integer, PortableObject> poMap = portableCache.getAll(map.keySet());
+        Map<Integer, IgniteObject> poMap = portableCache.getAll(map.keySet());
 
         Collection<String> names = new ArrayList<>();
 
         // Get organizations' names from portable objects (note that
         // objects don't need to be fully deserialized).
-        for (PortableObject po : poMap.values())
+        for (IgniteObject po : poMap.values())
             names.add(po.<String>field("name"));
 
         System.out.println();

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java b/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java
index 3170864..cbeed12 100644
--- a/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java
@@ -39,7 +39,7 @@ import org.apache.ignite.examples.portable.EmployeeKey;
 import org.apache.ignite.examples.portable.ExamplePortableNodeStartup;
 import org.apache.ignite.examples.portable.Organization;
 import org.apache.ignite.examples.portable.OrganizationType;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObject;
 
 /**
  * This example demonstrates use of portable objects with cache queries.
@@ -98,7 +98,7 @@ public class CacheClientPortableQueryExample {
                 populateCache(orgCache, employeeCache);
 
                 // Get cache that will work with portable objects.
-                IgniteCache<PortableObject, PortableObject> portableCache = employeeCache.withKeepPortable();
+                IgniteCache<IgniteObject, IgniteObject> portableCache = employeeCache.withKeepBinary();
 
                 // Run SQL query example.
                 sqlQuery(portableCache);
@@ -180,17 +180,17 @@ public class CacheClientPortableQueryExample {
      *
      * @param cache Ignite cache.
      */
-    private static void sqlQuery(IgniteCache<PortableObject, PortableObject> cache) {
-        SqlQuery<PortableObject, PortableObject> query = new SqlQuery<>(Employee.class, "zip = ?");
+    private static void sqlQuery(IgniteCache<IgniteObject, IgniteObject> cache) {
+        SqlQuery<IgniteObject, IgniteObject> query = new SqlQuery<>(Employee.class, "zip = ?");
 
         int zip = 94109;
 
-        QueryCursor<Cache.Entry<PortableObject, PortableObject>> employees = cache.query(query.setArgs(zip));
+        QueryCursor<Cache.Entry<IgniteObject, IgniteObject>> employees = cache.query(query.setArgs(zip));
 
         System.out.println();
         System.out.println(">>> Employees with zip " + zip + ':');
 
-        for (Cache.Entry<PortableObject, PortableObject> e : employees.getAll())
+        for (Cache.Entry<IgniteObject, IgniteObject> e : employees.getAll())
             System.out.println(">>>     " + e.getValue().deserialize());
     }
 
@@ -199,20 +199,20 @@ public class CacheClientPortableQueryExample {
      *
      * @param cache Ignite cache.
      */
-    private static void sqlJoinQuery(IgniteCache<PortableObject, PortableObject> cache) {
-        SqlQuery<PortableObject, PortableObject> query = new SqlQuery<>(Employee.class,
+    private static void sqlJoinQuery(IgniteCache<IgniteObject, IgniteObject> cache) {
+        SqlQuery<IgniteObject, IgniteObject> query = new SqlQuery<>(Employee.class,
             "from Employee, \"" + ORGANIZATION_CACHE_NAME + "\".Organization as org " +
                 "where Employee.organizationId = org._key and org.name = ?");
 
         String organizationName = "GridGain";
 
-        QueryCursor<Cache.Entry<PortableObject, PortableObject>> employees =
+        QueryCursor<Cache.Entry<IgniteObject, IgniteObject>> employees =
             cache.query(query.setArgs(organizationName));
 
         System.out.println();
         System.out.println(">>> Employees working for " + organizationName + ':');
 
-        for (Cache.Entry<PortableObject, PortableObject> e : employees.getAll())
+        for (Cache.Entry<IgniteObject, IgniteObject> e : employees.getAll())
             System.out.println(">>>     " + e.getValue());
     }
 
@@ -221,7 +221,7 @@ public class CacheClientPortableQueryExample {
      *
      * @param cache Ignite cache.
      */
-    private static void sqlFieldsQuery(IgniteCache<PortableObject, PortableObject> cache) {
+    private static void sqlFieldsQuery(IgniteCache<IgniteObject, IgniteObject> cache) {
         SqlFieldsQuery query = new SqlFieldsQuery("select name, salary from Employee");
 
         QueryCursor<List<?>> employees = cache.query(query);
@@ -238,15 +238,15 @@ public class CacheClientPortableQueryExample {
      *
      * @param cache Ignite cache.
      */
-    private static void textQuery(IgniteCache<PortableObject, PortableObject> cache) {
-        TextQuery<PortableObject, PortableObject> query = new TextQuery<>(Employee.class, "TX");
+    private static void textQuery(IgniteCache<IgniteObject, IgniteObject> cache) {
+        TextQuery<IgniteObject, IgniteObject> query = new TextQuery<>(Employee.class, "TX");
 
-        QueryCursor<Cache.Entry<PortableObject, PortableObject>> employees = cache.query(query);
+        QueryCursor<Cache.Entry<IgniteObject, IgniteObject>> employees = cache.query(query);
 
         System.out.println();
         System.out.println(">>> Employees living in Texas:");
 
-        for (Cache.Entry<PortableObject, PortableObject> e : employees.getAll())
+        for (Cache.Entry<IgniteObject, IgniteObject> e : employees.getAll())
             System.out.println(">>>     " + e.getValue().deserialize());
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index 62fd020..4125f8b 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -459,11 +459,11 @@ public interface Ignite extends AutoCloseable {
     public <T extends IgnitePlugin> T plugin(String name) throws PluginNotFoundException;
 
     /**
-     * Gets an instance of {@link IgnitePortables} interface.
+     * Gets an instance of {@link IgniteObjects} interface.
      *
-     * @return Instance of {@link IgnitePortables} interface.
+     * @return Instance of {@link IgniteObjects} interface.
      */
-    public IgnitePortables portables();
+    public IgniteObjects portables();
 
     /**
      * Closes {@code this} instance of grid. This method is identical to calling

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index e0f9f55..d58ba30 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -156,7 +156,7 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      * (which will be stored in portable format), you should acquire following projection
      * to avoid deserialization:
      * <pre>
-     * IgniteCache<Integer, PortableObject> prj = cache.withKeepPortable();
+     * IgniteCache<Integer, PortableObject> prj = cache.withKeepBinary();
      *
      * // Value is not deserialized and returned in portable format.
      * PortableObject po = prj.get(1);
@@ -167,7 +167,7 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      *
      * @return New cache instance for portable objects.
      */
-    public <K1, V1> IgniteCache<K1, V1> withKeepPortable();
+    public <K1, V1> IgniteCache<K1, V1> withKeepBinary();
 
     /**
      * Executes {@link #localLoadCache(IgniteBiPredicate, Object...)} on all cache nodes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/IgniteObjects.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteObjects.java b/modules/core/src/main/java/org/apache/ignite/IgniteObjects.java
new file mode 100644
index 0000000..4afd15b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteObjects.java
@@ -0,0 +1,366 @@
+/*
+ * 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;
+
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.TreeMap;
+import java.util.UUID;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.igniteobject.IgniteObjectBuilder;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObject;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Defines portable objects functionality. With portable objects you are able to:
+ * <ul>
+ * <li>Seamlessly interoperate between Java, .NET, and C++.</li>
+ * <li>Make any object portable with zero code change to your existing code.</li>
+ * <li>Nest portable objects within each other.</li>
+ * <li>Automatically handle {@code circular} or {@code null} references.</li>
+ * <li>Automatically convert collections and maps between Java, .NET, and C++.</li>
+ * <li>
+ *      Optionally avoid deserialization of objects on the server side
+ *      (objects are stored in {@link org.apache.ignite.igniteobject.IgniteObject} format).
+ * </li>
+ * <li>Avoid need to have concrete class definitions on the server side.</li>
+ * <li>Dynamically change structure of the classes without having to restart the cluster.</li>
+ * <li>Index into portable objects for querying purposes.</li>
+ * </ul>
+ * <h1 class="header">Working With Portables Directly</h1>
+ * Once an object is defined as portable,
+ * Ignite will always store it in memory in the portable (i.e. binary) format.
+ * User can choose to work either with the portable format or with the deserialized form
+ * (assuming that class definitions are present in the classpath).
+ * <p>
+ * To work with the portable format directly, user should create a special cache projection
+ * using IgniteCache.withKeepBinary() method and then retrieve individual fields as needed:
+ * <pre name=code class=java>
+ * IgniteCache&lt;PortableObject, PortableObject&gt; prj = cache.withKeepBinary();
+ *
+ * // Convert instance of MyKey to portable format.
+ * // We could also use PortableBuilder to create the key in portable format directly.
+ * PortableObject key = grid.portables().toPortable(new MyKey());
+ *
+ * PortableObject val = prj.get(key);
+ *
+ * String field = val.field("myFieldName");
+ * </pre>
+ * Alternatively, if we have class definitions in the classpath, we may choose to work with deserialized
+ * typed objects at all times. In this case we do incur the deserialization cost. However, if
+ * {@link PortableMarshaller#isKeepDeserialized()} is {@code true} then Ignite will only deserialize on the first access
+ * and will cache the deserialized object, so it does not have to be deserialized again:
+ * <pre name=code class=java>
+ * IgniteCache&lt;MyKey.class, MyValue.class&gt; cache = grid.cache(null);
+ *
+ * MyValue val = cache.get(new MyKey());
+ *
+ * // Normal java getter.
+ * String fieldVal = val.getMyFieldName();
+ * </pre>
+ * If we used, for example, one of the automatically handled portable types for a key, like integer,
+ * and still wanted to work with binary portable format for values, then we would declare cache projection
+ * as follows:
+ * <pre name=code class=java>
+ * IgniteCache&lt;Integer.class, PortableObject&gt; prj = cache.withKeepBinary();
+ * </pre>
+ * <h1 class="header">Automatic Portable Types</h1>
+ * Note that only portable classes are converted to {@link org.apache.ignite.igniteobject.IgniteObject} format. Following
+ * classes are never converted (e.g., {@link #toPortable(Object)} method will return original
+ * object, and instances of these classes will be stored in cache without changes):
+ * <ul>
+ *     <li>All primitives (byte, int, ...) and there boxed versions (Byte, Integer, ...)</li>
+ *     <li>Arrays of primitives (byte[], int[], ...)</li>
+ *     <li>{@link String} and array of {@link String}s</li>
+ *     <li>{@link UUID} and array of {@link UUID}s</li>
+ *     <li>{@link Date} and array of {@link Date}s</li>
+ *     <li>{@link Timestamp} and array of {@link Timestamp}s</li>
+ *     <li>Enums and array of enums</li>
+ *     <li>
+ *         Maps, collections and array of objects (but objects inside
+ *         them will still be converted if they are portable)
+ *     </li>
+ * </ul>
+ * <h1 class="header">Working With Maps and Collections</h1>
+ * All maps and collections in the portable objects are serialized automatically. When working
+ * with different platforms, e.g. C++ or .NET, Ignite will automatically pick the most
+ * adequate collection or map in either language. For example, {@link ArrayList} in Java will become
+ * {@code List} in C#, {@link LinkedList} in Java is {@link LinkedList} in C#, {@link HashMap}
+ * in Java is {@code Dictionary} in C#, and {@link TreeMap} in Java becomes {@code SortedDictionary}
+ * in C#, etc.
+ * <h1 class="header">Building Portable Objects</h1>
+ * Ignite comes with {@link org.apache.ignite.igniteobject.IgniteObjectBuilder} which allows to build portable objects dynamically:
+ * <pre name=code class=java>
+ * PortableBuilder builder = Ignition.ignite().portables().builder();
+ *
+ * builder.typeId("MyObject");
+ *
+ * builder.stringField("fieldA", "A");
+ * build.intField("fieldB", "B");
+ *
+ * PortableObject portableObj = builder.build();
+ * </pre>
+ * For the cases when class definition is present
+ * in the class path, it is also possible to populate a standard POJO and then
+ * convert it to portable format, like so:
+ * <pre name=code class=java>
+ * MyObject obj = new MyObject();
+ *
+ * obj.setFieldA("A");
+ * obj.setFieldB(123);
+ *
+ * PortableObject portableObj = Ignition.ignite().portables().toPortable(obj);
+ * </pre>
+ * NOTE: you don't need to convert typed objects to portable format before storing
+ * them in cache, Ignite will do that automatically.
+ * <h1 class="header">Portable Metadata</h1>
+ * Even though Ignite portable protocol only works with hash codes for type and field names
+ * to achieve better performance, Ignite provides metadata for all portable types which
+ * can be queried ar runtime via any of the {@link IgniteObjects#metadata(Class)}
+ * methods. Having metadata also allows for proper formatting of {@code PortableObject#toString()} method,
+ * even when portable objects are kept in binary format only, which may be necessary for audit reasons.
+ * <h1 class="header">Dynamic Structure Changes</h1>
+ * Since objects are always cached in the portable binary format, server does not need to
+ * be aware of the class definitions. Moreover, if class definitions are not present or not
+ * used on the server, then clients can continuously change the structure of the portable
+ * objects without having to restart the cluster. For example, if one client stores a
+ * certain class with fields A and B, and another client stores the same class with
+ * fields B and C, then the server-side portable object will have the fields A, B, and C.
+ * As the structure of a portable object changes, the new fields become available for SQL queries
+ * automatically.
+ * <h1 class="header">Configuration</h1>
+ * By default all your objects are considered as portables and no specific configuration is needed.
+ * However, in some cases, like when an object is used by both Java and .Net, you may need to specify portable objects
+ * explicitly by calling {@link PortableMarshaller#setClassNames(Collection)}.
+ * The only requirement Ignite imposes is that your object has an empty
+ * constructor. Note, that since server side does not have to know the class definition,
+ * you only need to list portable objects in configuration on the client side. However, if you
+ * list them on the server side as well, then you get the ability to deserialize portable objects
+ * into concrete types on the server as well as on the client.
+ * <p>
+ * Here is an example of portable configuration (note that star (*) notation is supported):
+ * <pre name=code class=xml>
+ * ...
+ * &lt;!-- Explicit portable objects configuration. --&gt;
+ * &lt;property name="marshaller"&gt;
+ *     &lt;bean class="org.apache.ignite.marshaller.portable.PortableMarshaller"&gt;
+ *         &lt;property name="classNames"&gt;
+ *             &lt;list&gt;
+ *                 &lt;value&gt;my.package.for.portable.objects.*&lt;/value&gt;
+ *                 &lt;value&gt;org.apache.ignite.examples.client.portable.Employee&lt;/value&gt;
+ *             &lt;/list&gt;
+ *         &lt;/property&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * ...
+ * </pre>
+ * or from code:
+ * <pre name=code class=java>
+ * IgniteConfiguration cfg = new IgniteConfiguration();
+ *
+ * PortableMarshaller marsh = new PortableMarshaller();
+ *
+ * marsh.setClassNames(Arrays.asList(
+ *     Employee.class.getName(),
+ *     Address.class.getName())
+ * );
+ *
+ * cfg.setMarshaller(marsh);
+ * </pre>
+ * You can also specify class name for a portable object via {@link org.apache.ignite.igniteobject.IgniteObjectConfiguration}.
+ * Do it in case if you need to override other configuration properties on per-type level, like
+ * ID-mapper, or serializer.
+ * <h1 class="header">Custom Affinity Keys</h1>
+ * Often you need to specify an alternate key (not the cache key) for affinity routing whenever
+ * storing objects in cache. For example, if you are caching {@code Employee} object with
+ * {@code Organization}, and want to colocate employees with organization they work for,
+ * so you can process them together, you need to specify an alternate affinity key.
+ * With portable objects you would have to do it as following:
+ * <pre name=code class=xml>
+ * &lt;property name="marshaller"&gt;
+ *     &lt;bean class="org.gridgain.grid.marshaller.portable.PortableMarshaller"&gt;
+ *         ...
+ *         &lt;property name="typeConfigurations"&gt;
+ *             &lt;list&gt;
+ *                 &lt;bean class="org.apache.ignite.portable.PortableTypeConfiguration"&gt;
+ *                     &lt;property name="className" value="org.apache.ignite.examples.client.portable.EmployeeKey"/&gt;
+ *                     &lt;property name="affinityKeyFieldName" value="organizationId"/&gt;
+ *                 &lt;/bean&gt;
+ *             &lt;/list&gt;
+ *         &lt;/property&gt;
+ *         ...
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ * <h1 class="header">Serialization</h1>
+ * Serialization and deserialization works out-of-the-box in Ignite. However, you can provide your own custom
+ * serialization logic by optionally implementing {@link org.apache.ignite.igniteobject.IgniteObjectMarshalAware} interface, like so:
+ * <pre name=code class=java>
+ * public class Address implements PortableMarshalAware {
+ *     private String street;
+ *     private int zip;
+ *
+ *     // Empty constructor required for portable deserialization.
+ *     public Address() {}
+ *
+ *     &#64;Override public void writePortable(PortableWriter writer) throws PortableException {
+ *         writer.writeString("street", street);
+ *         writer.writeInt("zip", zip);
+ *     }
+ *
+ *     &#64;Override public void readPortable(PortableReader reader) throws PortableException {
+ *         street = reader.readString("street");
+ *         zip = reader.readInt("zip");
+ *     }
+ * }
+ * </pre>
+ * Alternatively, if you cannot change class definitions, you can provide custom serialization
+ * logic in {@link org.apache.ignite.igniteobject.IgniteObjectSerializer} either globally in {@link PortableMarshaller} or
+ * for a specific type via {@link org.apache.ignite.igniteobject.IgniteObjectConfiguration} instance.
+ * <p>
+ * Similar to java serialization you can use {@code writeReplace()} and {@code readResolve()} methods.
+ * <ul>
+ *     <li>
+ *         {@code readResolve} is defined as follows: {@code ANY-ACCESS-MODIFIER Object readResolve()}.
+ *         It may be used to replace the de-serialized object by another one of your choice.
+ *     </li>
+ *     <li>
+ *          {@code writeReplace} is defined as follows: {@code ANY-ACCESS-MODIFIER Object writeReplace()}. This method
+ *          allows the developer to provide a replacement object that will be serialized instead of the original one.
+ *     </li>
+ * </ul>
+ *
+ * <h1 class="header">Custom ID Mappers</h1>
+ * Ignite implementation uses name hash codes to generate IDs for class names or field names
+ * internally. However, in cases when you want to provide your own ID mapping schema,
+ * you can provide your own {@link org.apache.ignite.igniteobject.IgniteObjectIdMapper} implementation.
+ * <p>
+ * ID-mapper may be provided either globally in {@link PortableMarshaller},
+ * or for a specific type via {@link org.apache.ignite.igniteobject.IgniteObjectConfiguration} instance.
+ * <h1 class="header">Query Indexing</h1>
+ * Portable objects can be indexed for querying by specifying index fields in
+ * {@link org.apache.ignite.cache.CacheTypeMetadata} inside of specific
+ * {@link org.apache.ignite.configuration.CacheConfiguration} instance,
+ * like so:
+ * <pre name=code class=xml>
+ * ...
+ * &lt;bean class="org.apache.ignite.cache.CacheConfiguration"&gt;
+ *     ...
+ *     &lt;property name="typeMetadata"&gt;
+ *         &lt;list&gt;
+ *             &lt;bean class="CacheTypeMetadata"&gt;
+ *                 &lt;property name="type" value="Employee"/&gt;
+ *
+ *                 &lt;!-- Fields to index in ascending order. --&gt;
+ *                 &lt;property name="ascendingFields"&gt;
+ *                     &lt;map&gt;
+ *                     &lt;entry key="name" value="java.lang.String"/&gt;
+ *
+ *                         &lt;!-- Nested portable objects can also be indexed. --&gt;
+ *                         &lt;entry key="address.zip" value="java.lang.Integer"/&gt;
+ *                     &lt;/map&gt;
+ *                 &lt;/property&gt;
+ *             &lt;/bean&gt;
+ *         &lt;/list&gt;
+ *     &lt;/property&gt;
+ * &lt;/bean&gt;
+ * </pre>
+ */
+public interface IgniteObjects {
+    /**
+     * Gets type ID for given type name.
+     *
+     * @param typeName Type name.
+     * @return Type ID.
+     */
+    public int typeId(String typeName);
+
+    /**
+     * Converts provided object to instance of {@link org.apache.ignite.igniteobject.IgniteObject}.
+     *
+     * @param obj Object to convert.
+     * @return Converted object.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    public <T> T toPortable(@Nullable Object obj) throws IgniteObjectException;
+
+    /**
+     * Creates new portable builder.
+     *
+     * @param typeId ID of the type.
+     * @return Newly portable builder.
+     */
+    public IgniteObjectBuilder builder(int typeId);
+
+    /**
+     * Creates new portable builder.
+     *
+     * @param typeName Type name.
+     * @return Newly portable builder.
+     */
+    public IgniteObjectBuilder builder(String typeName);
+
+    /**
+     * Creates portable builder initialized by existing portable object.
+     *
+     * @param portableObj Portable object to initialize builder.
+     * @return Portable builder.
+     */
+    public IgniteObjectBuilder builder(IgniteObject portableObj);
+
+    /**
+     * Gets metadata for provided class.
+     *
+     * @param cls Class.
+     * @return Metadata.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable public IgniteObjectMetadata metadata(Class<?> cls) throws IgniteObjectException;
+
+    /**
+     * Gets metadata for provided class name.
+     *
+     * @param typeName Type name.
+     * @return Metadata.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable public IgniteObjectMetadata metadata(String typeName) throws IgniteObjectException;
+
+    /**
+     * Gets metadata for provided type ID.
+     *
+     * @param typeId Type ID.
+     * @return Metadata.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable public IgniteObjectMetadata metadata(int typeId) throws IgniteObjectException;
+
+    /**
+     * Gets metadata for all known types.
+     *
+     * @return Metadata.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    public Collection<IgniteObjectMetadata> metadata() throws IgniteObjectException;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java b/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java
deleted file mode 100644
index 1c63df7..0000000
--- a/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java
+++ /dev/null
@@ -1,370 +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;
-
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.TreeMap;
-import java.util.UUID;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableSerializer;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Defines portable objects functionality. With portable objects you are able to:
- * <ul>
- * <li>Seamlessly interoperate between Java, .NET, and C++.</li>
- * <li>Make any object portable with zero code change to your existing code.</li>
- * <li>Nest portable objects within each other.</li>
- * <li>Automatically handle {@code circular} or {@code null} references.</li>
- * <li>Automatically convert collections and maps between Java, .NET, and C++.</li>
- * <li>
- *      Optionally avoid deserialization of objects on the server side
- *      (objects are stored in {@link PortableObject} format).
- * </li>
- * <li>Avoid need to have concrete class definitions on the server side.</li>
- * <li>Dynamically change structure of the classes without having to restart the cluster.</li>
- * <li>Index into portable objects for querying purposes.</li>
- * </ul>
- * <h1 class="header">Working With Portables Directly</h1>
- * Once an object is defined as portable,
- * Ignite will always store it in memory in the portable (i.e. binary) format.
- * User can choose to work either with the portable format or with the deserialized form
- * (assuming that class definitions are present in the classpath).
- * <p>
- * To work with the portable format directly, user should create a special cache projection
- * using IgniteCache.withKeepPortable() method and then retrieve individual fields as needed:
- * <pre name=code class=java>
- * IgniteCache&lt;PortableObject, PortableObject&gt; prj = cache.withKeepPortable();
- *
- * // Convert instance of MyKey to portable format.
- * // We could also use PortableBuilder to create the key in portable format directly.
- * PortableObject key = grid.portables().toPortable(new MyKey());
- *
- * PortableObject val = prj.get(key);
- *
- * String field = val.field("myFieldName");
- * </pre>
- * Alternatively, if we have class definitions in the classpath, we may choose to work with deserialized
- * typed objects at all times. In this case we do incur the deserialization cost. However, if
- * {@link PortableMarshaller#isKeepDeserialized()} is {@code true} then Ignite will only deserialize on the first access
- * and will cache the deserialized object, so it does not have to be deserialized again:
- * <pre name=code class=java>
- * IgniteCache&lt;MyKey.class, MyValue.class&gt; cache = grid.cache(null);
- *
- * MyValue val = cache.get(new MyKey());
- *
- * // Normal java getter.
- * String fieldVal = val.getMyFieldName();
- * </pre>
- * If we used, for example, one of the automatically handled portable types for a key, like integer,
- * and still wanted to work with binary portable format for values, then we would declare cache projection
- * as follows:
- * <pre name=code class=java>
- * IgniteCache&lt;Integer.class, PortableObject&gt; prj = cache.withKeepPortable();
- * </pre>
- * <h1 class="header">Automatic Portable Types</h1>
- * Note that only portable classes are converted to {@link PortableObject} format. Following
- * classes are never converted (e.g., {@link #toPortable(Object)} method will return original
- * object, and instances of these classes will be stored in cache without changes):
- * <ul>
- *     <li>All primitives (byte, int, ...) and there boxed versions (Byte, Integer, ...)</li>
- *     <li>Arrays of primitives (byte[], int[], ...)</li>
- *     <li>{@link String} and array of {@link String}s</li>
- *     <li>{@link UUID} and array of {@link UUID}s</li>
- *     <li>{@link Date} and array of {@link Date}s</li>
- *     <li>{@link Timestamp} and array of {@link Timestamp}s</li>
- *     <li>Enums and array of enums</li>
- *     <li>
- *         Maps, collections and array of objects (but objects inside
- *         them will still be converted if they are portable)
- *     </li>
- * </ul>
- * <h1 class="header">Working With Maps and Collections</h1>
- * All maps and collections in the portable objects are serialized automatically. When working
- * with different platforms, e.g. C++ or .NET, Ignite will automatically pick the most
- * adequate collection or map in either language. For example, {@link ArrayList} in Java will become
- * {@code List} in C#, {@link LinkedList} in Java is {@link LinkedList} in C#, {@link HashMap}
- * in Java is {@code Dictionary} in C#, and {@link TreeMap} in Java becomes {@code SortedDictionary}
- * in C#, etc.
- * <h1 class="header">Building Portable Objects</h1>
- * Ignite comes with {@link PortableBuilder} which allows to build portable objects dynamically:
- * <pre name=code class=java>
- * PortableBuilder builder = Ignition.ignite().portables().builder();
- *
- * builder.typeId("MyObject");
- *
- * builder.stringField("fieldA", "A");
- * build.intField("fieldB", "B");
- *
- * PortableObject portableObj = builder.build();
- * </pre>
- * For the cases when class definition is present
- * in the class path, it is also possible to populate a standard POJO and then
- * convert it to portable format, like so:
- * <pre name=code class=java>
- * MyObject obj = new MyObject();
- *
- * obj.setFieldA("A");
- * obj.setFieldB(123);
- *
- * PortableObject portableObj = Ignition.ignite().portables().toPortable(obj);
- * </pre>
- * NOTE: you don't need to convert typed objects to portable format before storing
- * them in cache, Ignite will do that automatically.
- * <h1 class="header">Portable Metadata</h1>
- * Even though Ignite portable protocol only works with hash codes for type and field names
- * to achieve better performance, Ignite provides metadata for all portable types which
- * can be queried ar runtime via any of the {@link IgnitePortables#metadata(Class)}
- * methods. Having metadata also allows for proper formatting of {@code PortableObject#toString()} method,
- * even when portable objects are kept in binary format only, which may be necessary for audit reasons.
- * <h1 class="header">Dynamic Structure Changes</h1>
- * Since objects are always cached in the portable binary format, server does not need to
- * be aware of the class definitions. Moreover, if class definitions are not present or not
- * used on the server, then clients can continuously change the structure of the portable
- * objects without having to restart the cluster. For example, if one client stores a
- * certain class with fields A and B, and another client stores the same class with
- * fields B and C, then the server-side portable object will have the fields A, B, and C.
- * As the structure of a portable object changes, the new fields become available for SQL queries
- * automatically.
- * <h1 class="header">Configuration</h1>
- * By default all your objects are considered as portables and no specific configuration is needed.
- * However, in some cases, like when an object is used by both Java and .Net, you may need to specify portable objects
- * explicitly by calling {@link PortableMarshaller#setClassNames(Collection)}.
- * The only requirement Ignite imposes is that your object has an empty
- * constructor. Note, that since server side does not have to know the class definition,
- * you only need to list portable objects in configuration on the client side. However, if you
- * list them on the server side as well, then you get the ability to deserialize portable objects
- * into concrete types on the server as well as on the client.
- * <p>
- * Here is an example of portable configuration (note that star (*) notation is supported):
- * <pre name=code class=xml>
- * ...
- * &lt;!-- Explicit portable objects configuration. --&gt;
- * &lt;property name="marshaller"&gt;
- *     &lt;bean class="org.apache.ignite.marshaller.portable.PortableMarshaller"&gt;
- *         &lt;property name="classNames"&gt;
- *             &lt;list&gt;
- *                 &lt;value&gt;my.package.for.portable.objects.*&lt;/value&gt;
- *                 &lt;value&gt;org.apache.ignite.examples.client.portable.Employee&lt;/value&gt;
- *             &lt;/list&gt;
- *         &lt;/property&gt;
- *     &lt;/bean&gt;
- * &lt;/property&gt;
- * ...
- * </pre>
- * or from code:
- * <pre name=code class=java>
- * IgniteConfiguration cfg = new IgniteConfiguration();
- *
- * PortableMarshaller marsh = new PortableMarshaller();
- *
- * marsh.setClassNames(Arrays.asList(
- *     Employee.class.getName(),
- *     Address.class.getName())
- * );
- *
- * cfg.setMarshaller(marsh);
- * </pre>
- * You can also specify class name for a portable object via {@link PortableTypeConfiguration}.
- * Do it in case if you need to override other configuration properties on per-type level, like
- * ID-mapper, or serializer.
- * <h1 class="header">Custom Affinity Keys</h1>
- * Often you need to specify an alternate key (not the cache key) for affinity routing whenever
- * storing objects in cache. For example, if you are caching {@code Employee} object with
- * {@code Organization}, and want to colocate employees with organization they work for,
- * so you can process them together, you need to specify an alternate affinity key.
- * With portable objects you would have to do it as following:
- * <pre name=code class=xml>
- * &lt;property name="marshaller"&gt;
- *     &lt;bean class="org.gridgain.grid.marshaller.portable.PortableMarshaller"&gt;
- *         ...
- *         &lt;property name="typeConfigurations"&gt;
- *             &lt;list&gt;
- *                 &lt;bean class="org.apache.ignite.portable.PortableTypeConfiguration"&gt;
- *                     &lt;property name="className" value="org.apache.ignite.examples.client.portable.EmployeeKey"/&gt;
- *                     &lt;property name="affinityKeyFieldName" value="organizationId"/&gt;
- *                 &lt;/bean&gt;
- *             &lt;/list&gt;
- *         &lt;/property&gt;
- *         ...
- *     &lt;/bean&gt;
- * &lt;/property&gt;
- * </pre>
- * <h1 class="header">Serialization</h1>
- * Serialization and deserialization works out-of-the-box in Ignite. However, you can provide your own custom
- * serialization logic by optionally implementing {@link PortableMarshalAware} interface, like so:
- * <pre name=code class=java>
- * public class Address implements PortableMarshalAware {
- *     private String street;
- *     private int zip;
- *
- *     // Empty constructor required for portable deserialization.
- *     public Address() {}
- *
- *     &#64;Override public void writePortable(PortableWriter writer) throws PortableException {
- *         writer.writeString("street", street);
- *         writer.writeInt("zip", zip);
- *     }
- *
- *     &#64;Override public void readPortable(PortableReader reader) throws PortableException {
- *         street = reader.readString("street");
- *         zip = reader.readInt("zip");
- *     }
- * }
- * </pre>
- * Alternatively, if you cannot change class definitions, you can provide custom serialization
- * logic in {@link PortableSerializer} either globally in {@link PortableMarshaller} or
- * for a specific type via {@link PortableTypeConfiguration} instance.
- * <p>
- * Similar to java serialization you can use {@code writeReplace()} and {@code readResolve()} methods.
- * <ul>
- *     <li>
- *         {@code readResolve} is defined as follows: {@code ANY-ACCESS-MODIFIER Object readResolve()}.
- *         It may be used to replace the de-serialized object by another one of your choice.
- *     </li>
- *     <li>
- *          {@code writeReplace} is defined as follows: {@code ANY-ACCESS-MODIFIER Object writeReplace()}. This method
- *          allows the developer to provide a replacement object that will be serialized instead of the original one.
- *     </li>
- * </ul>
- *
- * <h1 class="header">Custom ID Mappers</h1>
- * Ignite implementation uses name hash codes to generate IDs for class names or field names
- * internally. However, in cases when you want to provide your own ID mapping schema,
- * you can provide your own {@link PortableIdMapper} implementation.
- * <p>
- * ID-mapper may be provided either globally in {@link PortableMarshaller},
- * or for a specific type via {@link PortableTypeConfiguration} instance.
- * <h1 class="header">Query Indexing</h1>
- * Portable objects can be indexed for querying by specifying index fields in
- * {@link org.apache.ignite.cache.CacheTypeMetadata} inside of specific
- * {@link org.apache.ignite.configuration.CacheConfiguration} instance,
- * like so:
- * <pre name=code class=xml>
- * ...
- * &lt;bean class="org.apache.ignite.cache.CacheConfiguration"&gt;
- *     ...
- *     &lt;property name="typeMetadata"&gt;
- *         &lt;list&gt;
- *             &lt;bean class="CacheTypeMetadata"&gt;
- *                 &lt;property name="type" value="Employee"/&gt;
- *
- *                 &lt;!-- Fields to index in ascending order. --&gt;
- *                 &lt;property name="ascendingFields"&gt;
- *                     &lt;map&gt;
- *                     &lt;entry key="name" value="java.lang.String"/&gt;
- *
- *                         &lt;!-- Nested portable objects can also be indexed. --&gt;
- *                         &lt;entry key="address.zip" value="java.lang.Integer"/&gt;
- *                     &lt;/map&gt;
- *                 &lt;/property&gt;
- *             &lt;/bean&gt;
- *         &lt;/list&gt;
- *     &lt;/property&gt;
- * &lt;/bean&gt;
- * </pre>
- */
-public interface IgnitePortables {
-    /**
-     * Gets type ID for given type name.
-     *
-     * @param typeName Type name.
-     * @return Type ID.
-     */
-    public int typeId(String typeName);
-
-    /**
-     * Converts provided object to instance of {@link PortableObject}.
-     *
-     * @param obj Object to convert.
-     * @return Converted object.
-     * @throws PortableException In case of error.
-     */
-    public <T> T toPortable(@Nullable Object obj) throws PortableException;
-
-    /**
-     * Creates new portable builder.
-     *
-     * @param typeId ID of the type.
-     * @return Newly portable builder.
-     */
-    public PortableBuilder builder(int typeId);
-
-    /**
-     * Creates new portable builder.
-     *
-     * @param typeName Type name.
-     * @return Newly portable builder.
-     */
-    public PortableBuilder builder(String typeName);
-
-    /**
-     * Creates portable builder initialized by existing portable object.
-     *
-     * @param portableObj Portable object to initialize builder.
-     * @return Portable builder.
-     */
-    public PortableBuilder builder(PortableObject portableObj);
-
-    /**
-     * Gets metadata for provided class.
-     *
-     * @param cls Class.
-     * @return Metadata.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public PortableMetadata metadata(Class<?> cls) throws PortableException;
-
-    /**
-     * Gets metadata for provided class name.
-     *
-     * @param typeName Type name.
-     * @return Metadata.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public PortableMetadata metadata(String typeName) throws PortableException;
-
-    /**
-     * Gets metadata for provided type ID.
-     *
-     * @param typeId Type ID.
-     * @return Metadata.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public PortableMetadata metadata(int typeId) throws PortableException;
-
-    /**
-     * Gets metadata for all known types.
-     *
-     * @return Metadata.
-     * @throws PortableException In case of error.
-     */
-    public Collection<PortableMetadata> metadata() throws PortableException;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/cache/CacheKeyConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheKeyConfiguration.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheKeyConfiguration.java
new file mode 100644
index 0000000..39ec2a8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheKeyConfiguration.java
@@ -0,0 +1,92 @@
+/*
+ * 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.cache;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+public class CacheKeyConfiguration implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Type name. */
+    private String typeName;
+
+    /** Affinity key field name. */
+    private String affKeyFieldName;
+
+    /**
+     * Creates an empty cache key configuration that should be populated via setters.
+     */
+    public CacheKeyConfiguration() {
+        // Convenience no-op constructor.
+    }
+
+    /**
+     * Creates cache key configuration with given type name and affinity field name.
+     *
+     * @param typeName Type name.
+     * @param affKeyFieldName Affinity field name.
+     */
+    public CacheKeyConfiguration(String typeName, String affKeyFieldName) {
+        this.typeName = typeName;
+        this.affKeyFieldName = affKeyFieldName;
+    }
+
+    /**
+     * Sets type name for which affinity field name is being defined.
+     *
+     * @return Type name.
+     */
+    public String getTypeName() {
+        return typeName;
+    }
+
+    /**
+     * @param typeName Type name for which affinity field name is being defined.
+     */
+    public void setTypeName(String typeName) {
+        this.typeName = typeName;
+    }
+
+    /**
+     * Gets affinity key field name.
+     *
+     * @return Affinity key field name.
+     */
+    public String getAffinityKeyFieldName() {
+        return affKeyFieldName;
+    }
+
+    /**
+     * Sets affinity key field name.
+     *
+     * @param affKeyFieldName Affinity key field name.
+     */
+    public void setAffinityKeyFieldName(String affKeyFieldName) {
+        this.affKeyFieldName = affKeyFieldName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CacheKeyConfiguration.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
index 330f911..cb84c47 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
@@ -17,7 +17,6 @@
 package org.apache.ignite.cache;
 
 import java.io.Serializable;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
@@ -46,7 +45,7 @@ public class QueryEntity implements Serializable {
     private Map<String, String> aliases = new HashMap<>();
 
     /** Collection of query indexes. */
-    private Map<String, QueryEntityIndex> idxs = new HashMap<>();
+    private Map<String, QueryIndex> idxs = new HashMap<>();
 
     /**
      * Gets key type for this query pair.
@@ -109,7 +108,7 @@ public class QueryEntity implements Serializable {
      *
      * @return Collection of index entities.
      */
-    public Collection<QueryEntityIndex> getIndexes() {
+    public Collection<QueryIndex> getIndexes() {
         return idxs.values();
     }
 
@@ -137,8 +136,8 @@ public class QueryEntity implements Serializable {
      *
      * @param idxs Collection of index entities.
      */
-    public void setIndexes(Collection<QueryEntityIndex> idxs) {
-        for (QueryEntityIndex idx : idxs) {
+    public void setIndexes(Collection<QueryIndex> idxs) {
+        for (QueryIndex idx : idxs) {
             if (!F.isEmpty(idx.getFields())) {
                 if (idx.getName() == null)
                     idx.setName(defaultIndexName(idx));
@@ -170,20 +169,20 @@ public class QueryEntity implements Serializable {
      * @param idxName Index name.
      * @param idxType Index type.
      */
-    public void ensureIndex(String idxName, QueryEntityIndex.Type idxType) {
-        QueryEntityIndex idx = idxs.get(idxName);
+    public void ensureIndex(String idxName, QueryIndexType idxType) {
+        QueryIndex idx = idxs.get(idxName);
 
         if (idx == null) {
-            idx = new QueryEntityIndex();
+            idx = new QueryIndex();
 
             idx.setName(idxName);
-            idx.setType(idxType);
+            idx.setIndexType(idxType);
 
             idxs.put(idxName, idx);
         }
         else
             throw new IllegalArgumentException("An index with the same name and of a different type already exists " +
-                "[idxName=" + idxName + ", existingIdxType=" + idx.getType() + ", newIdxType=" + idxType + ']');
+                "[idxName=" + idxName + ", existingIdxType=" + idx.getIndexType() + ", newIdxType=" + idxType + ']');
     }
 
     /**
@@ -192,13 +191,14 @@ public class QueryEntity implements Serializable {
      * @param idx Index to build name for.
      * @return Index name.
      */
-    public static String defaultIndexName(QueryEntityIndex idx) {
+    public static String defaultIndexName(QueryIndex idx) {
         StringBuilder idxName = new StringBuilder();
 
-        for (String field : idx.getFields()) {
-            idxName.append(field);
+        for (Map.Entry<String, Boolean> field : idx.getFields().entrySet()) {
+            idxName.append(field.getKey());
 
             idxName.append('_');
+            idxName.append(field.getValue() ? "asc_" : "desc_");
         }
 
         for (int i = 0; i < idxName.length(); i++) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/cache/QueryEntityIndex.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntityIndex.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntityIndex.java
deleted file mode 100644
index 18e7572..0000000
--- a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntityIndex.java
+++ /dev/null
@@ -1,100 +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.cache;
-
-import java.io.Serializable;
-import java.util.Collection;
-
-/**
- * Contains list of fields to be indexed. It is possible to provide field name
- * suffixed with index specific extension, for example for {@link Type#SORTED sorted} index
- * the list can be provided as following {@code (id, name asc, age desc)}.
- */
-public class QueryEntityIndex implements Serializable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Index name. */
-    private String name;
-
-    /** */
-    private Collection<String> fields;
-
-    /** */
-    private Type type;
-
-    /**
-     * Index type.
-     */
-    public enum Type {
-        SORTED, FULLTEXT, GEOSPATIAL
-    }
-
-    /**
-     * Gets index name. Will be automatically set if not provided by a user.
-     *
-     * @return Index name.
-     */
-    public String getName() {
-        return name;
-    }
-
-    /**
-     * Sets index name.
-     *
-     * @param name Index name.
-     */
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * Gets fields included in the index.
-     *
-     * @return Collection of index fields.
-     */
-    public Collection<String> getFields() {
-        return fields;
-    }
-
-    /**
-     * Sets fields included in the index.
-     *
-     * @param fields Collection of index fields.
-     */
-    public void setFields(Collection<String> fields) {
-        this.fields = fields;
-    }
-
-    /**
-     * Gets index type.
-     *
-     * @return Index type.
-     */
-    public Type getType() {
-        return type;
-    }
-
-    /**
-     * Sets index type.
-     *
-     * @param type Index type.
-     */
-    public void setType(Type type) {
-        this.type = type;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/cache/QueryIndex.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryIndex.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryIndex.java
new file mode 100644
index 0000000..4868e9e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryIndex.java
@@ -0,0 +1,173 @@
+/*
+ * 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.cache;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.LinkedHashMap;
+
+/**
+ * Contains list of fields to be indexed. It is possible to provide field name
+ * suffixed with index specific extension, for example for {@link QueryIndexType#SORTED sorted} index
+ * the list can be provided as following {@code (id, name asc, age desc)}.
+ */
+@SuppressWarnings("TypeMayBeWeakened")
+public class QueryIndex implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Index name. */
+    private String name;
+
+    /** */
+    private LinkedHashMap<String, Boolean> fields;
+
+    /** */
+    private QueryIndexType type;
+
+    /**
+     * Creates an empty index. Should be populated via setters.
+     */
+    public QueryIndex() {
+        // Empty constructor.
+    }
+
+    /**
+     * Creates index for one field.
+     * If index is sorted, then ascending sorting is used by default.
+     * To specify sort order, use the next method.
+     * This constructor should also have a corresponding setter method.
+     */
+    public QueryIndex(String field, QueryIndexType type) {
+        this(Arrays.asList(field), type);
+    }
+
+    /**
+     * Creates index for one field. The last boolean parameter
+     * is ignored for non-sorted indexes.
+     */
+    public QueryIndex(String field, QueryIndexType type, boolean asc) {
+        fields = new LinkedHashMap<>();
+        fields.put(field, asc);
+
+        this.type = type;
+    }
+
+    /**
+     * Creates index for a collection of fields. If index is sorted, fields will be sorted in
+     * ascending order.
+     *
+     * @param fields Collection of fields to create an index.
+     * @param type Index type.
+     */
+    public QueryIndex(Collection<String> fields, QueryIndexType type) {
+        this.fields = new LinkedHashMap<>();
+
+        for (String field : fields)
+            this.fields.put(field, true);
+
+        this.type = type;
+    }
+
+    /**
+     * Creates index for a collection of fields. The order of fields in the created index will be the same
+     * as iteration order in the passed map. Map value defines whether the index will be ascending.
+     *
+     * @param fields Field name to field sort direction for sorted indexes.
+     * @param type Index type.
+     */
+    public QueryIndex(LinkedHashMap<String, Boolean> fields, QueryIndexType type) {
+        this.fields = fields;
+        this.type = type;
+    }
+
+    /**
+     * Gets index name. Will be automatically set if not provided by a user.
+     *
+     * @return Index name.
+     */
+    public String getName() {
+        return name;
+    }
+
+    /**
+     * Sets index name.
+     *
+     * @param name Index name.
+     */
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    /**
+     * Gets fields included in the index.
+     *
+     * @return Collection of index fields.
+     */
+    public LinkedHashMap<String, Boolean> getFields() {
+        return fields;
+    }
+
+    /**
+     * Sets fields included in the index.
+     *
+     * @param fields Collection of index fields.
+     */
+    public void setFields(LinkedHashMap<String, Boolean> fields) {
+        this.fields = fields;
+    }
+
+    /**
+     * @return Gets a collection of field names.
+     */
+    public Collection<String> getFieldNames() {
+        return fields.keySet();
+    }
+
+    /**
+     * Sets a collection of field names altogether with the field sorting direction. Sorting direction will be
+     * ignored for non-sorted indexes.
+     *
+     * @param fields Collection of fields.
+     * @param asc Ascending flag.
+     */
+    public void setFieldNames(Collection<String> fields, boolean asc) {
+        this.fields = new LinkedHashMap<>();
+
+        for (String field : fields)
+            this.fields.put(field, asc);
+    }
+
+    /**
+     * Gets index type.
+     *
+     * @return Index type.
+     */
+    public QueryIndexType getIndexType() {
+        return type;
+    }
+
+    /**
+     * Sets index type.
+     *
+     * @param type Index type.
+     */
+    public void setIndexType(QueryIndexType type) {
+        this.type = type;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/cache/QueryIndexType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryIndexType.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryIndexType.java
new file mode 100644
index 0000000..8df0fa1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryIndexType.java
@@ -0,0 +1,24 @@
+/*
+ * 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.cache;
+
+/**
+ * Index type.
+ */
+public enum QueryIndexType {
+    SORTED, FULLTEXT, GEOSPATIAL
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 3a80b83..91ad609 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -30,7 +30,6 @@ import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.TreeSet;
 import javax.cache.Cache;
@@ -51,7 +50,8 @@ import org.apache.ignite.cache.CacheRebalanceMode;
 import org.apache.ignite.cache.CacheTypeMetadata;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.QueryEntity;
-import org.apache.ignite.cache.QueryEntityIndex;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.cache.affinity.AffinityKeyMapper;
 import org.apache.ignite.cache.eviction.EvictionFilter;
@@ -72,7 +72,6 @@ import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.plugin.CachePluginConfiguration;
 import org.jetbrains.annotations.Nullable;
@@ -1951,9 +1950,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         for (ClassProperty prop : desc.props.values())
             entity.addQueryField(prop.fullName(), U.box(prop.type()).getName(), prop.alias());
 
-        QueryEntityIndex txtIdx = null;
+        QueryIndex txtIdx = null;
 
-        Collection<QueryEntityIndex> idxs = new ArrayList<>();
+        Collection<QueryIndex> idxs = new ArrayList<>();
 
         for (Map.Entry<String, GridQueryIndexDescriptor> idxEntry : desc.indexes().entrySet()) {
             GridQueryIndexDescriptor idx = idxEntry.getValue();
@@ -1961,10 +1960,11 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
             if (idx.type() == FULLTEXT) {
                 assert txtIdx == null;
 
-                txtIdx = new QueryEntityIndex();
+                txtIdx = new QueryIndex();
 
-                txtIdx.setType(QueryEntityIndex.Type.FULLTEXT);
-                txtIdx.setFields(new ArrayList<>(idx.fields()));
+                txtIdx.setIndexType(QueryIndexType.FULLTEXT);
+
+                txtIdx.setFieldNames(idx.fields(), true);
                 txtIdx.setName(idxEntry.getKey());
             }
             else {
@@ -1973,10 +1973,17 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
                 for (String fieldName : idx.fields())
                     grp.add(idx.descending(fieldName) ? fieldName + " desc" : fieldName);
 
-                QueryEntityIndex sortedIdx = new QueryEntityIndex();
+                QueryIndex sortedIdx = new QueryIndex();
+
+                sortedIdx.setIndexType(idx.type() == SORTED ? QueryIndexType.SORTED : QueryIndexType.GEOSPATIAL);
+
+                LinkedHashMap<String, Boolean> fields = new LinkedHashMap<>();
+
+                for (String f : idx.fields())
+                    fields.put(f, !idx.descending(f));
+
+                sortedIdx.setFields(fields);
 
-                sortedIdx.setType(idx.type() == SORTED ? QueryEntityIndex.Type.SORTED : QueryEntityIndex.Type.GEOSPATIAL);
-                sortedIdx.setFields(grp);
                 sortedIdx.setName(idxEntry.getKey());
 
                 idxs.add(sortedIdx);
@@ -1985,14 +1992,14 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
         if (desc.valueTextIndex()) {
             if (txtIdx == null) {
-                txtIdx = new QueryEntityIndex();
+                txtIdx = new QueryIndex();
 
-                txtIdx.setType(QueryEntityIndex.Type.FULLTEXT);
+                txtIdx.setIndexType(QueryIndexType.FULLTEXT);
 
-                txtIdx.setFields(Arrays.asList(_VAL));
+                txtIdx.setFieldNames(Arrays.asList(_VAL), true);
             }
             else
-                txtIdx.getFields().add(_VAL);
+                txtIdx.getFields().put(_VAL, true);
         }
 
         if (txtIdx != null)