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 2017/03/13 08:36:39 UTC

[1/8] ignite git commit: IGNITE-4805: Removed GridQueryIndexType.

Repository: ignite
Updated Branches:
  refs/heads/ignite-4768 763f1b0b9 -> bc5dbb081


IGNITE-4805: Removed GridQueryIndexType.


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

Branch: refs/heads/ignite-4768
Commit: 0ea88cd4cd28653534178e629354f6495a7fb148
Parents: 93e1996
Author: devozerov <vo...@gridgain.com>
Authored: Thu Mar 9 16:23:39 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Mar 9 16:23:39 2017 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       | 28 ++++++++---------
 .../cache/query/GridCacheQueryManager.java      |  4 +--
 .../query/GridQueryIndexDescriptor.java         |  4 ++-
 .../processors/query/GridQueryIndexType.java    | 32 --------------------
 .../processors/query/GridQueryProcessor.java    | 23 ++++++--------
 .../resources/META-INF/classnames.properties    |  1 -
 .../processors/query/h2/IgniteH2Indexing.java   | 10 +++---
 .../query/h2/opt/GridLuceneIndex.java           |  4 +--
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  6 ++--
 9 files changed, 36 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0ea88cd4/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 f0179ca..f6f1376 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
@@ -66,7 +66,6 @@ import org.apache.ignite.cache.store.CacheStore;
 import org.apache.ignite.cache.store.CacheStoreSessionListener;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
-import org.apache.ignite.internal.processors.query.GridQueryIndexType;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
@@ -78,9 +77,6 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.plugin.CachePluginConfiguration;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.FULLTEXT;
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.GEO_SPATIAL;
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.SORTED;
 import static org.apache.ignite.internal.processors.query.GridQueryProcessor._VAL;
 import static org.apache.ignite.internal.processors.query.GridQueryProcessor.isGeometryClass;
 import static org.apache.ignite.internal.processors.query.GridQueryProcessor.isSqlType;
@@ -2220,7 +2216,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         for (Map.Entry<String, GridQueryIndexDescriptor> idxEntry : desc.indexes().entrySet()) {
             GridQueryIndexDescriptor idx = idxEntry.getValue();
 
-            if (idx.type() == FULLTEXT) {
+            if (idx.type() == QueryIndexType.FULLTEXT) {
                 assert txtIdx == null;
 
                 txtIdx = new QueryIndex();
@@ -2238,7 +2234,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
                 QueryIndex sortedIdx = new QueryIndex();
 
-                sortedIdx.setIndexType(idx.type() == SORTED ? QueryIndexType.SORTED : QueryIndexType.GEOSPATIAL);
+                sortedIdx.setIndexType(idx.type());
 
                 LinkedHashMap<String, Boolean> fields = new LinkedHashMap<>();
 
@@ -2318,7 +2314,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
             if (parent == null && !key && isSqlType(cls)) { // We have to index primitive _val.
                 String idxName = _VAL + "_idx";
 
-                type.addIndex(idxName, isGeometryClass(cls) ? GEO_SPATIAL : SORTED);
+                type.addIndex(idxName, isGeometryClass(cls) ? QueryIndexType.GEOSPATIAL : QueryIndexType.SORTED);
 
                 type.addFieldToIndex(idxName, _VAL, 0, false);
             }
@@ -2338,13 +2334,13 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
             QueryGroupIndex grpIdx = cls.getAnnotation(QueryGroupIndex.class);
 
             if (grpIdx != null)
-                type.addIndex(grpIdx.name(), SORTED);
+                type.addIndex(grpIdx.name(), QueryIndexType.SORTED);
 
             QueryGroupIndex.List grpIdxList = cls.getAnnotation(QueryGroupIndex.List.class);
 
             if (grpIdxList != null && !F.isEmpty(grpIdxList.value())) {
                 for (QueryGroupIndex idx : grpIdxList.value())
-                    type.addIndex(idx.name(), SORTED);
+                    type.addIndex(idx.name(), QueryIndexType.SORTED);
             }
         }
 
@@ -2417,7 +2413,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
             if (sqlAnn.index()) {
                 String idxName = prop.alias() + "_idx";
 
-                desc.addIndex(idxName, isGeometryClass(prop.type()) ? GEO_SPATIAL : SORTED);
+                desc.addIndex(idxName, isGeometryClass(prop.type()) ? QueryIndexType.GEOSPATIAL : QueryIndexType.SORTED);
 
                 desc.addFieldToIndex(idxName, prop.fullName(), 0, sqlAnn.descending());
             }
@@ -2506,7 +2502,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
          * @param type Index type.
          * @return Index descriptor.
          */
-        public IndexDescriptor addIndex(String idxName, GridQueryIndexType type) {
+        public IndexDescriptor addIndex(String idxName, QueryIndexType type) {
             IndexDescriptor idx = new IndexDescriptor(type);
 
             if (indexes.put(idxName, idx) != null)
@@ -2528,7 +2524,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
             IndexDescriptor desc = indexes.get(idxName);
 
             if (desc == null)
-                desc = addIndex(idxName, SORTED);
+                desc = addIndex(idxName, QueryIndexType.SORTED);
 
             desc.addField(field, orderNum, descending);
         }
@@ -2540,7 +2536,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
          */
         public void addFieldToTextIndex(String field) {
             if (fullTextIdx == null) {
-                fullTextIdx = new IndexDescriptor(FULLTEXT);
+                fullTextIdx = new IndexDescriptor(QueryIndexType.FULLTEXT);
 
                 indexes.put(null, fullTextIdx);
             }
@@ -2640,12 +2636,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         private Collection<String> descendings;
 
         /** */
-        private final GridQueryIndexType type;
+        private final QueryIndexType type;
 
         /**
          * @param type Type.
          */
-        private IndexDescriptor(GridQueryIndexType type) {
+        private IndexDescriptor(QueryIndexType type) {
             assert type != null;
 
             this.type = type;
@@ -2685,7 +2681,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         }
 
         /** {@inheritDoc} */
-        @Override public GridQueryIndexType type() {
+        @Override public QueryIndexType type() {
             return type;
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ea88cd4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 8e3f2a9..b92ecaf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -46,6 +46,7 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.cache.query.QueryMetrics;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -81,7 +82,6 @@ import org.apache.ignite.internal.processors.datastructures.SetItemKey;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
-import org.apache.ignite.internal.processors.query.GridQueryIndexType;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.task.GridInternal;
@@ -2469,7 +2469,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                             GridQueryIndexDescriptor desc = e.getValue();
 
                             // Add only SQL indexes.
-                            if (desc.type() == GridQueryIndexType.SORTED) {
+                            if (desc.type() == QueryIndexType.SORTED) {
                                 Collection<String> idxFields = new LinkedList<>();
                                 Collection<String> descendings = new LinkedList<>();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ea88cd4/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexDescriptor.java
index fe58112..134a61c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexDescriptor.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.query;
 
+import org.apache.ignite.cache.QueryIndexType;
+
 import java.util.Collection;
 
 /**
@@ -46,5 +48,5 @@ public interface GridQueryIndexDescriptor {
      *
      * @return Type.
      */
-    public GridQueryIndexType type();
+    public QueryIndexType type();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ea88cd4/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexType.java
deleted file mode 100644
index f59301c..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexType.java
+++ /dev/null
@@ -1,32 +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.query;
-
-/**
- * Index types.
- */
-public enum GridQueryIndexType {
-    /** Sorted SQL index. */
-    SORTED,
-
-    /** Spatial SQL index. */
-    GEO_SPATIAL,
-
-    /** Fulltext index. */
-    FULLTEXT
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ea88cd4/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 e19fbcc..1ff2330 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
@@ -101,9 +101,6 @@ import org.jsr166.ConcurrentHashMap8;
 
 import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED;
 import static org.apache.ignite.internal.IgniteComponentType.INDEXING;
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.FULLTEXT;
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.GEO_SPATIAL;
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.SORTED;
 
 /**
  * Indexing processor.
@@ -1399,7 +1396,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 idxName = propName + "_idx";
 
             if (idxOrder == 0) // Add index only on the first field.
-                d.addIndex(idxName, isGeometryClass(propCls) ? GEO_SPATIAL : SORTED);
+                d.addIndex(idxName, isGeometryClass(propCls) ? QueryIndexType.GEOSPATIAL : QueryIndexType.SORTED);
 
             if (idxType == IndexType.TEXT)
                 d.addFieldToTextIndex(propName);
@@ -1429,7 +1426,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             String idxName = prop.name() + "_idx";
 
-            d.addIndex(idxName, isGeometryClass(prop.type()) ? GEO_SPATIAL : SORTED);
+            d.addIndex(idxName, isGeometryClass(prop.type()) ? QueryIndexType.GEOSPATIAL : QueryIndexType.SORTED);
 
             d.addFieldToIndex(idxName, prop.name(), 0, false);
         }
@@ -1441,7 +1438,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             String idxName = prop.name() + "_idx";
 
-            d.addIndex(idxName, isGeometryClass(prop.type()) ? GEO_SPATIAL : SORTED);
+            d.addIndex(idxName, isGeometryClass(prop.type()) ? QueryIndexType.GEOSPATIAL : QueryIndexType.SORTED);
 
             d.addFieldToIndex(idxName, prop.name(), 0, true);
         }
@@ -1591,7 +1588,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 QueryIndexType idxTyp = idx.getIndexType();
 
                 if (idxTyp == QueryIndexType.SORTED || idxTyp == QueryIndexType.GEOSPATIAL) {
-                    d.addIndex(idxName, idxTyp == QueryIndexType.SORTED ? SORTED : GEO_SPATIAL);
+                    d.addIndex(idxName, idxTyp);
 
                     int i = 0;
 
@@ -2413,7 +2410,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
          * @return Index descriptor.
          * @throws IgniteCheckedException In case of error.
          */
-        public IndexDescriptor addIndex(String idxName, GridQueryIndexType type) throws IgniteCheckedException {
+        public IndexDescriptor addIndex(String idxName, QueryIndexType type) throws IgniteCheckedException {
             IndexDescriptor idx = new IndexDescriptor(type);
 
             if (indexes.put(idxName, idx) != null)
@@ -2436,7 +2433,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             IndexDescriptor desc = indexes.get(idxName);
 
             if (desc == null)
-                desc = addIndex(idxName, SORTED);
+                desc = addIndex(idxName, QueryIndexType.SORTED);
 
             desc.addField(field, orderNum, descending);
         }
@@ -2448,7 +2445,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
          */
         public void addFieldToTextIndex(String field) {
             if (fullTextIdx == null) {
-                fullTextIdx = new IndexDescriptor(FULLTEXT);
+                fullTextIdx = new IndexDescriptor(QueryIndexType.FULLTEXT);
 
                 indexes.put(null, fullTextIdx);
             }
@@ -2583,12 +2580,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         private Collection<String> descendings;
 
         /** */
-        private final GridQueryIndexType type;
+        private final QueryIndexType type;
 
         /**
          * @param type Type.
          */
-        private IndexDescriptor(GridQueryIndexType type) {
+        private IndexDescriptor(QueryIndexType type) {
             assert type != null;
 
             this.type = type;
@@ -2628,7 +2625,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public GridQueryIndexType type() {
+        @Override public QueryIndexType type() {
             return type;
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ea88cd4/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index 02bad40..0ad74a9 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -1276,7 +1276,6 @@ org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$Interna
 org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionLockProcessor
 org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionSetAndUnlockProcessor
 org.apache.ignite.internal.processors.query.GridQueryFieldMetadata
-org.apache.ignite.internal.processors.query.GridQueryIndexType
 org.apache.ignite.internal.processors.query.GridQueryProcessor$3
 org.apache.ignite.internal.processors.query.GridQueryProcessor$4
 org.apache.ignite.internal.processors.query.GridQueryProcessor$5

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ea88cd4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 88cd89b..a26406d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -59,6 +59,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.cache.query.QueryCancelledException;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
@@ -184,9 +185,6 @@ import static org.apache.ignite.IgniteSystemProperties.getString;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL_FIELDS;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.TEXT;
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.FULLTEXT;
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.GEO_SPATIAL;
-import static org.apache.ignite.internal.processors.query.GridQueryIndexType.SORTED;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.KEY_COL;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.LOCAL;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.PREPARE;
@@ -2590,7 +2588,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 String name = e.getKey();
                 GridQueryIndexDescriptor idx = e.getValue();
 
-                if (idx.type() == FULLTEXT) {
+                if (idx.type() == QueryIndexType.FULLTEXT) {
                     try {
                         luceneIdx = new GridLuceneIndex(ctx, schema.offheap, schema.spaceName, type);
                     }
@@ -2612,7 +2610,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                             idx.descending(field) ? SortOrder.DESCENDING : SortOrder.ASCENDING));
                     }
 
-                    if (idx.type() == SORTED) {
+                    if (idx.type() == QueryIndexType.SORTED) {
                         // We don't care about number of fields in affinity index, just affinity key must be the first.
                         affIdxFound |= affCol != null && equal(cols.get(0), affCol);
 
@@ -2620,7 +2618,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
                         idxs.add(new GridH2TreeIndex(name, tbl, false, cols));
                     }
-                    else if (idx.type() == GEO_SPATIAL)
+                    else if (idx.type() == QueryIndexType.GEOSPATIAL)
                         idxs.add(createH2SpatialIndex(tbl, name, cols.toArray(new IndexColumn[cols.size()])));
                     else
                         throw new IllegalStateException("Index type: " + idx.type());

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ea88cd4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
index 716c9cb..c35eebb 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridLuceneIndex.java
@@ -21,11 +21,11 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
-import org.apache.ignite.internal.processors.query.GridQueryIndexType;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.util.GridAtomicLong;
 import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
@@ -118,7 +118,7 @@ public class GridLuceneIndex implements AutoCloseable {
         GridQueryIndexDescriptor idx = null;
 
         for (GridQueryIndexDescriptor descriptor : type.indexes().values()) {
-            if (descriptor.type() == GridQueryIndexType.FULLTEXT) {
+            if (descriptor.type() == QueryIndexType.FULLTEXT) {
                 idx = descriptor;
 
                 break;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ea88cd4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index 54c6019..3b31051 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -27,12 +27,12 @@ import java.util.List;
 import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
 import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
-import org.apache.ignite.internal.processors.query.GridQueryIndexType;
 import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.util.typedef.F;
@@ -502,8 +502,8 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         }
 
         /** {@inheritDoc} */
-        @Override public GridQueryIndexType type() {
-            return GridQueryIndexType.FULLTEXT;
+        @Override public QueryIndexType type() {
+            return QueryIndexType.FULLTEXT;
         }
     }
 


[4/8] ignite git commit: IGNITE-4810: Removed index rebuild methods.

Posted by sb...@apache.org.
IGNITE-4810: Removed index rebuild methods.


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

Branch: refs/heads/ignite-4768
Commit: 6fda2c203b82314acff190304168b40043f587c3
Parents: 5899cc3
Author: devozerov <vo...@gridgain.com>
Authored: Fri Mar 10 13:10:07 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Mar 10 13:10:07 2017 +0300

----------------------------------------------------------------------
 .../cache/query/GridCacheQueryManager.java      |  18 --
 .../processors/query/GridQueryIndexing.java     |   8 -
 .../processors/query/GridQueryProcessor.java    | 102 --------
 .../processors/query/h2/IgniteH2Indexing.java   |  13 -
 .../query/h2/opt/GridH2IndexBase.java           |  10 -
 .../processors/query/h2/opt/GridH2Table.java    |  33 ---
 .../query/h2/opt/GridH2TreeIndex.java           |  23 --
 .../query/h2/GridH2IndexRebuildTest.java        | 261 -------------------
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  17 --
 .../query/h2/opt/GridH2TableSelfTest.java       |  54 ----
 .../IgniteCacheQuerySelfTestSuite.java          |   2 -
 11 files changed, 541 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6fda2c20/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index b92ecaf..93dee54 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -324,24 +324,6 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     }
 
     /**
-     * Rebuilds all search indexes of given value type.
-     *
-     * @param typeName Value type name.
-     * @return Future that will be completed when rebuilding of all indexes is finished.
-     */
-    public IgniteInternalFuture<?> rebuildIndexes(String typeName) {
-        if (!enterBusy())
-            throw new IllegalStateException("Failed to rebuild indexes (grid is stopping).");
-
-        try {
-            return qryProc.rebuildIndexes(space, typeName);
-        }
-        finally {
-            leaveBusy();
-        }
-    }
-
-    /**
      * Marks this request as canceled.
      *
      * @param reqId Request id.

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fda2c20/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index ca04724..629de78 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -208,14 +208,6 @@ public interface GridQueryIndexing {
     public void onUnswap(@Nullable String spaceName, CacheObject key, CacheObject val) throws IgniteCheckedException;
 
     /**
-     * Rebuilds all indexes of given type.
-     *
-     * @param spaceName Space name.
-     * @param type Type descriptor.
-     */
-    public void rebuildIndexes(@Nullable String spaceName, GridQueryTypeDescriptor type);
-
-    /**
      * Returns backup filter.
      *
      * @param topVer Topology version.

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fda2c20/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 1c5b822..176178c 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
@@ -35,7 +35,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
 import javax.cache.Cache;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
@@ -52,7 +51,6 @@ import org.apache.ignite.cache.query.SqlQuery;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.events.CacheQueryExecutedEvent;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -75,16 +73,12 @@ import org.apache.ignite.internal.processors.query.property.QueryPropertyAccesso
 import org.apache.ignite.internal.processors.query.property.QueryReadOnlyMethodsAccessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
-import org.apache.ignite.internal.util.future.GridCompoundFuture;
-import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 import org.apache.ignite.internal.util.lang.GridClosureException;
 import org.apache.ignite.internal.util.lang.IgniteOutClosureX;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.internal.util.worker.GridWorker;
-import org.apache.ignite.internal.util.worker.GridWorkerFuture;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
@@ -139,9 +133,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     private final ConcurrentMap<QueryTypeNameKey, QueryTypeDescriptorImpl> typesByName = new ConcurrentHashMap8<>();
 
     /** */
-    private ExecutorService execSvc;
-
-    /** */
     private final GridQueryIndexing idx;
 
     /** */
@@ -172,8 +163,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         if (idx != null) {
             ctx.resource().injectGeneric(idx);
 
-            execSvc = ctx.getExecutorService();
-
             idx.start(ctx, busyLock);
         }
 
@@ -556,97 +545,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Rebuilds all search indexes of given value type for given space of spi.
-     *
-     * @param space Space.
-     * @param valTypeName Value type name.
-     * @return Future that will be completed when rebuilding of all indexes is finished.
-     */
-    public IgniteInternalFuture<?> rebuildIndexes(@Nullable final String space, String valTypeName) {
-        if (!busyLock.enterBusy())
-            throw new IllegalStateException("Failed to rebuild indexes (grid is stopping).");
-
-        try {
-            return rebuildIndexes(
-                space,
-                typesByName.get(
-                    new QueryTypeNameKey(
-                        space,
-                        valTypeName)));
-        }
-        finally {
-            busyLock.leaveBusy();
-        }
-    }
-
-    /**
-     * @param space Space.
-     * @param desc Type descriptor.
-     * @return Future that will be completed when rebuilding of all indexes is finished.
-     */
-    private IgniteInternalFuture<?> rebuildIndexes(@Nullable final String space, @Nullable final QueryTypeDescriptorImpl desc) {
-        if (idx == null)
-            return new GridFinishedFuture<>(new IgniteCheckedException("Indexing is disabled."));
-
-        if (desc == null || !desc.registered())
-            return new GridFinishedFuture<Void>();
-
-        final GridWorkerFuture<?> fut = new GridWorkerFuture<Void>();
-
-        GridWorker w = new GridWorker(ctx.gridName(), "index-rebuild-worker", log) {
-            @Override protected void body() {
-                try {
-                    idx.rebuildIndexes(space, desc);
-
-                    fut.onDone();
-                }
-                catch (Exception e) {
-                    fut.onDone(e);
-                }
-                catch (Throwable e) {
-                    log.error("Failed to rebuild indexes for type: " + desc.name(), e);
-
-                    fut.onDone(e);
-
-                    if (e instanceof Error)
-                        throw e;
-                }
-            }
-        };
-
-        fut.setWorker(w);
-
-        execSvc.execute(w);
-
-        return fut;
-    }
-
-    /**
-     * Rebuilds all search indexes for given spi.
-     *
-     * @return Future that will be completed when rebuilding of all indexes is finished.
-     */
-    @SuppressWarnings("unchecked")
-    public IgniteInternalFuture<?> rebuildAllIndexes() {
-        if (!busyLock.enterBusy())
-            throw new IllegalStateException("Failed to get space size (grid is stopping).");
-
-        try {
-            GridCompoundFuture<?, ?> fut = new GridCompoundFuture<Object, Object>();
-
-            for (Map.Entry<QueryTypeIdKey, QueryTypeDescriptorImpl> e : types.entrySet())
-                fut.add((IgniteInternalFuture)rebuildIndexes(e.getKey().space(), e.getValue()));
-
-            fut.markInitialized();
-
-            return fut;
-        }
-        finally {
-            busyLock.leaveBusy();
-        }
-    }
-
-    /**
      * @param space Space name.
      * @return Cache object context.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fda2c20/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index a26406d..2f8faa4 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -1699,19 +1699,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         return schema.spaceName;
     }
 
-    /** {@inheritDoc} */
-    @Override public void rebuildIndexes(@Nullable String spaceName, GridQueryTypeDescriptor type) {
-        TableDescriptor tbl = tableDescriptor(spaceName, type);
-
-        if (tbl == null)
-            return;
-
-        if (tbl.schema.offheap != null)
-            throw new UnsupportedOperationException("Index rebuilding is not supported when off-heap memory is used");
-
-        tbl.tbl.rebuildIndexes();
-    }
-
     /**
      * Gets size (for tests only).
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fda2c20/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
index bab219c..5669fd0 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
@@ -178,16 +178,6 @@ public abstract class GridH2IndexBase extends BaseIndex {
     }
 
     /**
-     * If the index supports rebuilding it has to creates its own copy.
-     *
-     * @return Rebuilt copy.
-     * @throws InterruptedException If interrupted.
-     */
-    public GridH2IndexBase rebuild() throws InterruptedException {
-        return this;
-    }
-
-    /**
      * Put row if absent.
      *
      * @param row Row.

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fda2c20/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index 8d080ae..ca1bcb0 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -639,39 +639,6 @@ public class GridH2Table extends TableBase {
         return res;
     }
 
-    /**
-     * Rebuilds all indexes of this table.
-     */
-    public void rebuildIndexes() {
-        if (!snapshotEnabled)
-            return;
-
-        Lock l = lock(true, Long.MAX_VALUE);
-
-        ArrayList<Index> idxs0 = new ArrayList<>(idxs);
-
-        try {
-            snapshotIndexes(null); // Allow read access while we are rebuilding indexes.
-
-            for (int i = 1, len = idxs.size(); i < len; i++) {
-                GridH2IndexBase newIdx = index(i).rebuild();
-
-                idxs.set(i, newIdx);
-
-                if (i == 1) // ScanIndex at 0 and actualSnapshot can contain references to old indexes, reset them.
-                    idxs.set(0, new ScanIndex(newIdx));
-            }
-        }
-        catch (InterruptedException e) {
-            throw new IgniteInterruptedException(e);
-        }
-        finally {
-            releaseSnapshots0(idxs0);
-
-            unlock(l);
-        }
-    }
-
     /** {@inheritDoc} */
     @Override public Index addIndex(Session ses, String s, int i, IndexColumn[] idxCols, IndexType idxType,
         boolean b, String s1) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fda2c20/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index 914e0da..aa0846f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -27,7 +27,6 @@ import org.apache.ignite.internal.util.GridEmptyIterator;
 import org.apache.ignite.internal.util.offheap.unsafe.GridOffHeapSnapTreeMap;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeGuard;
 import org.apache.ignite.internal.util.snaptree.SnapTreeMap;
-import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
@@ -457,26 +456,4 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
             throw new IllegalStateException();
         }
     }
-
-    /** {@inheritDoc} */
-    @Override public GridH2TreeIndex rebuild() throws InterruptedException {
-        IndexColumn[] cols = getIndexColumns();
-
-        GridH2TreeIndex idx = new GridH2TreeIndex(getName(), getTable(),
-            getIndexType().isUnique(), F.asList(cols));
-
-        Thread thread = Thread.currentThread();
-
-        long i = 0;
-
-        for (GridH2Row row : tree.values()) {
-            // Check for interruptions every 1000 iterations.
-            if (++i % 1000 == 0 && thread.isInterrupted())
-                throw new InterruptedException();
-
-            idx.tree.put(row, row);
-        }
-
-        return idx;
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fda2c20/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexRebuildTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexRebuildTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexRebuildTest.java
deleted file mode 100644
index 2dec475..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexRebuildTest.java
+++ /dev/null
@@ -1,261 +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.query.h2;
-
-import java.util.ArrayList;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.query.SqlQuery;
-import org.apache.ignite.cache.query.annotations.QuerySqlField;
-import org.apache.ignite.cache.query.annotations.QueryTextField;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteFutureCancelledCheckedException;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
-import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
-import org.apache.ignite.internal.processors.query.GridQueryProcessor;
-import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.jetbrains.annotations.Nullable;
-
-/**
- */
-public class GridH2IndexRebuildTest extends GridCacheAbstractSelfTest {
-    /** */
-    private static final int GRID_CNT = 1;
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return GRID_CNT;
-    }
-
-    /**
-     * Overrides rebuildIndexes to check it can be interrupted.
-     */
-    private static class SleepingH2Indexing extends IgniteH2Indexing {
-        /** */
-        private volatile boolean sleepInRebuild;
-
-        /** */
-        private volatile CountDownLatch interrupted;
-
-        /**
-         * Constructor.
-         */
-        public SleepingH2Indexing() {
-            spi = this;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void rebuildIndexes(@Nullable String spaceName, GridQueryTypeDescriptor type) {
-            if (sleepInRebuild) {
-                try {
-                    U.sleep(Long.MAX_VALUE);
-                }
-                catch (IgniteInterruptedCheckedException ignored) {
-                    interrupted.countDown();
-                }
-            }
-
-            super.rebuildIndexes(spaceName, type);
-        }
-    }
-
-    /** */
-    private static SleepingH2Indexing spi;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        GridQueryProcessor.idxCls = SleepingH2Indexing.class;
-
-        return cfg;
-    }
-
-    /**
-     * Value class with regular and compound indexes.
-     */
-    @SuppressWarnings("UnusedDeclaration")
-    private static class TestValue1 {
-        /** */
-        @QuerySqlField(index = true)
-        private long val1;
-
-        /** */
-        @QuerySqlField(index = true)
-        private String val2;
-
-        /** */
-        @QuerySqlField(groups = "group1")
-        private int val3;
-
-        /** */
-        @QuerySqlField(groups = "group1")
-        private int val4;
-
-        /**
-         */
-        TestValue1(long val1, String val2, int val3, int val4) {
-            this.val1 = val1;
-            this.val2 = val2;
-            this.val3 = val3;
-            this.val4 = val4;
-        }
-    }
-
-    /**
-     * Value class with regular and text indexes.
-     */
-    @SuppressWarnings("UnusedDeclaration")
-    private static class TestValue2 {
-        /** */
-        @QuerySqlField(index = true)
-        private long val1;
-
-        /** */
-        @QueryTextField
-        private String val2;
-
-        /**
-         */
-        TestValue2(long val1, String val2) {
-            this.val1 = val1;
-            this.val2 = val2;
-        }
-    }
-
-    /** */
-    private static final int ENTRY_CNT = 10000;
-
-    /** {@inheritDoc} */
-    @Override protected Class<?>[] indexedTypes() {
-        return new Class<?>[] {
-            Integer.class, TestValue1.class,
-            Integer.class, TestValue2.class
-        };
-    }
-
-    /**
-     * @throws Exception if failed.
-     */
-    public void testRebuildIndexes() throws Exception {
-        grid(0).context().query().rebuildIndexes(null, ArrayList.class.getName()).get();
-
-        grid(0).context().query().rebuildAllIndexes().get();
-
-        IgniteCache<Integer, TestValue1> cache1 = grid(0).cache(null);
-        IgniteCache<Integer, TestValue2> cache2 = grid(0).cache(null);
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            cache1.put(i, new TestValue1(i, "val2-" + i, i, i));
-            cache2.put(ENTRY_CNT * 2 + i, new TestValue2(i, "val2-" + i));
-        }
-
-        SqlQuery<Integer, TestValue1> qry1 = new SqlQuery(TestValue1.class, "val1 = 9000");
-
-        SqlQuery<Integer, TestValue1> qry2 = new SqlQuery(TestValue1.class, "val2 = 'val2-9000'");
-
-        SqlQuery<Integer, TestValue1> qry3 = new SqlQuery(TestValue1.class, "val3 = 9000 and val4 = 9000");
-
-        SqlQuery<Integer, TestValue2> qry4 = new SqlQuery(TestValue2.class, "val1 = 9000");
-
-        SqlQuery<Integer, TestValue2> qry5 = new SqlQuery(TestValue2.class, "val2 = 'val2-9000'");
-
-        assertEquals(1, cache1.query(qry1).getAll().size());
-        assertEquals(1, cache1.query(qry2).getAll().size());
-        assertEquals(1, cache1.query(qry3).getAll().size());
-        assertEquals(1, cache2.query(qry4).getAll().size());
-        assertEquals(1, cache2.query(qry5).getAll().size());
-
-        for (int i = 0; i < ENTRY_CNT / 2; i++) {
-            cache1.remove(i);
-            cache2.remove(ENTRY_CNT * 2 + i);
-        }
-
-        grid(0).context().query().rebuildIndexes(null, TestValue1.class.getName()).get();
-        grid(0).context().query().rebuildIndexes(null, TestValue2.class.getName()).get();
-
-        assertEquals(1, cache1.query(qry1).getAll().size());
-        assertEquals(1, cache1.query(qry2).getAll().size());
-        assertEquals(1, cache1.query(qry3).getAll().size());
-        assertEquals(1, cache2.query(qry4).getAll().size());
-        assertEquals(1, cache2.query(qry5).getAll().size());
-
-        grid(0).context().query().rebuildAllIndexes().get();
-
-        assertEquals(1, cache1.query(qry1).getAll().size());
-        assertEquals(1, cache1.query(qry2).getAll().size());
-        assertEquals(1, cache1.query(qry3).getAll().size());
-        assertEquals(1, cache2.query(qry4).getAll().size());
-        assertEquals(1, cache2.query(qry5).getAll().size());
-    }
-
-    /**
-     * @throws Exception if failed.
-     */
-    public void testRebuildInterrupted() throws Exception {
-        spi.sleepInRebuild = true;
-
-        IgniteCache<Integer, TestValue1> cache1 = grid(0).cache(null);
-        IgniteCache<Integer, TestValue2> cache2 = grid(0).cache(null);
-
-        cache1.put(0, new TestValue1(0, "val0", 0 ,0));
-        cache2.put(1, new TestValue2(0, "val0"));
-
-        checkCancel(grid(0).context().query().rebuildIndexes(null, "TestValue1"));
-
-        checkCancel(grid(0).context().query().rebuildAllIndexes());
-
-        spi.sleepInRebuild = false;
-
-        final IgniteInternalFuture<?> fut1 = grid(0).context().query().rebuildIndexes(null, TestValue1.class.getName());
-
-        assertFalse(fut1.isCancelled());
-
-        fut1.get();
-
-        final IgniteInternalFuture<?> fut2 = grid(0).context().query().rebuildAllIndexes();;
-
-        assertFalse(fut2.isCancelled());
-
-        fut2.get();
-    }
-
-    /**
-     * @param fut Future.
-     * @throws Exception if failed.
-     */
-    private void checkCancel(final IgniteInternalFuture<?> fut) throws Exception {
-        spi.interrupted = new CountDownLatch(1);
-
-        assertTrue(fut.cancel());
-
-        GridTestUtils.assertThrows(log, new Callable<Void>() {
-            @Override public Void call() throws Exception {
-                fut.get();
-                return null;
-            }
-        }, IgniteFutureCancelledCheckedException.class, null);
-
-        assertTrue(spi.interrupted.await(5, TimeUnit.SECONDS));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fda2c20/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index 3b31051..49914c5 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -392,23 +392,6 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         assertEquals(2, spi.size(typeAB.space(), typeAB));
         assertEquals(0, spi.size(typeBA.space(), typeBA));
 
-        boolean h2IdxOffheap = offheap();
-
-        // At the time of this writing index rebuilding is not supported for GridH2Indexing with off-heap storage.
-        if (!h2IdxOffheap) {
-            // Rebuild
-
-            spi.rebuildIndexes(typeAB.space(), typeAB);
-
-            assertEquals(1, spi.size(typeAA.space(), typeAA));
-            assertEquals(2, spi.size(typeAB.space(), typeAB));
-            assertEquals(0, spi.size(typeBA.space(), typeBA));
-
-            // For invalid space name/type should not fail.
-            spi.rebuildIndexes("not_existing_space", typeAA);
-            spi.rebuildIndexes(typeAA.space(), new TypeDesc("C", "C", fieldsAA, null));
-        }
-
         // Unregister.
         spi.unregisterType(typeAA.space(), typeAA);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fda2c20/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java
index 8408ba0..a3baad4 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableSelfTest.java
@@ -46,7 +46,6 @@ import org.h2.value.ValueString;
 import org.h2.value.ValueTimestamp;
 import org.h2.value.ValueUuid;
 import org.jetbrains.annotations.Nullable;
-import org.junit.Assert;
 
 /**
  * Tests H2 Table.
@@ -274,18 +273,6 @@ public class GridH2TableSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Dumps all table rows for index.
-     *
-     * @param idx Index.
-     */
-    private void dumpRows(GridH2TreeIndex idx) {
-        Iterator<GridH2Row> iter = idx.rows();
-
-        while (iter.hasNext())
-            System.out.println(iter.next().toString());
-    }
-
-    /**
      * Multithreaded indexes consistency test.
      *
      * @throws Exception If failed.
@@ -512,47 +499,6 @@ public class GridH2TableSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @throws Exception If failed.
-     */
-    public void testRebuildIndexes() throws Exception {
-        ArrayList<GridH2IndexBase> idxsBefore = tbl.indexes();
-
-        assertEquals(3, idxsBefore.size());
-
-        Random rnd = new Random();
-
-        for (int i = 0; i < MAX_X; i++) {
-            UUID id = UUID.randomUUID();
-
-            GridH2Row row = row(id, System.currentTimeMillis(), rnd.nextBoolean() ? id.toString() :
-                    UUID.randomUUID().toString(), rnd.nextInt(100));
-
-            tbl.doUpdate(row, false);
-        }
-
-        for (GridH2IndexBase idx : idxsBefore)
-            assertEquals(MAX_X, idx.getRowCountApproximation());
-
-        tbl.rebuildIndexes();
-
-        ArrayList<GridH2IndexBase> idxsAfter = tbl.indexes();
-
-        assertEquals(3, idxsAfter.size());
-
-        for (int i = 0; i < 3; i++) {
-            GridH2IndexBase idxBefore = idxsBefore.get(i);
-            GridH2IndexBase idxAfter = idxsAfter.get(i);
-
-            assertNotSame(idxBefore, idxAfter);
-            assertEquals(idxBefore.getName(), idxAfter.getName());
-            assertSame(idxBefore.getTable(), idxAfter.getTable());
-            assertEquals(idxBefore.getRowCountApproximation(), idxAfter.getRowCountApproximation());
-            assertEquals(idxBefore.getIndexType().isUnique(), idxAfter.getIndexType().isUnique());
-            Assert.assertArrayEquals(idxBefore.getColumns(), idxAfter.getColumns());
-        }
-    }
-
-    /**
      * Check query plan to correctly select index.
      *
      * @param conn Connection.

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fda2c20/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 3ef331f..7bf3887 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -100,7 +100,6 @@ import org.apache.ignite.internal.processors.query.IgniteSqlQueryDedicatedPoolTe
 import org.apache.ignite.internal.processors.query.IgniteSqlEntryCacheModeAgnosticTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSchemaIndexingTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest;
-import org.apache.ignite.internal.processors.query.h2.GridH2IndexRebuildTest;
 import org.apache.ignite.internal.processors.query.h2.GridH2IndexingInMemSelfTest;
 import org.apache.ignite.internal.processors.query.h2.GridH2IndexingOffheapSelfTest;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2TableSelfTest;
@@ -125,7 +124,6 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridH2TableSelfTest.class));
         suite.addTest(new TestSuite(GridH2IndexingInMemSelfTest.class));
         suite.addTest(new TestSuite(GridH2IndexingOffheapSelfTest.class));
-        suite.addTest(new TestSuite(GridH2IndexRebuildTest.class));
 
         // Parsing
         suite.addTestSuite(GridQueryParsingTest.class);


[5/8] ignite git commit: Minor refactoring in GridQueryProcessor.

Posted by sb...@apache.org.
Minor refactoring in GridQueryProcessor.


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

Branch: refs/heads/ignite-4768
Commit: 86fad9823dff03d486abe86b9040f809bd7be25b
Parents: 6fda2c2
Author: devozerov <vo...@gridgain.com>
Authored: Fri Mar 10 15:28:41 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Mar 10 15:28:41 2017 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java    | 38 ++++++--------------
 1 file changed, 11 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/86fad982/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 176178c..8223fef 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
@@ -663,12 +663,10 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         try {
             final GridCacheContext<?, ?> cctx = ctx.cache().internalCache(space).context();
 
-            return executeQuery(GridCacheQueryType.SQL_FIELDS, clause, cctx, new IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, V>>>() {
+            return executeQuery(GridCacheQueryType.SQL_FIELDS, clause, cctx,
+                new IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, V>>>() {
                 @Override public GridCloseableIterator<IgniteBiTuple<K, V>> applyx() throws IgniteCheckedException {
-                    QueryTypeDescriptorImpl type = typesByName.get(new QueryTypeNameKey(space, resType));
-
-                    if (type == null || !type.registered())
-                        throw new CacheException("Failed to find SQL table for type: " + resType);
+                    QueryTypeDescriptorImpl type = type(space, resType);
 
                     return idx.queryLocalSql(space, clause, null, params, type, filters);
                 }
@@ -754,13 +752,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         String sqlQry = qry.getSql();
                         Object[] params = qry.getArgs();
 
-                        QueryTypeDescriptorImpl typeDesc = typesByName.get(
-                            new QueryTypeNameKey(
-                                space,
-                                type));
-
-                        if (typeDesc == null || !typeDesc.registered())
-                            throw new CacheException("Failed to find SQL table for type: " + type);
+                        QueryTypeDescriptorImpl typeDesc = type(space, type);
 
                         final GridCloseableIterator<IgniteBiTuple<K, V>> i = idx.queryLocalSql(
                             space,
@@ -1056,16 +1048,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             return executeQuery(GridCacheQueryType.TEXT, clause, cctx,
                 new IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, V>>>() {
                     @Override public GridCloseableIterator<IgniteBiTuple<K, V>> applyx() throws IgniteCheckedException {
-                        QueryTypeDescriptorImpl type = typesByName.get(new QueryTypeNameKey(space, resType));
+                        QueryTypeDescriptorImpl type = type(space, resType);
 
-                        if (type == null || !type.registered())
-                            throw new CacheException("Failed to find SQL table for type: " + resType);
-
-                        return idx.queryLocalText(
-                            space,
-                            clause,
-                            type,
-                            filters);
+                        return idx.queryLocalText(space, clause, type, filters);
                     }
                 }, true);
         }
@@ -1629,8 +1614,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @return Descriptors.
      */
     public Collection<GridQueryTypeDescriptor> types(@Nullable String space) {
-        Collection<GridQueryTypeDescriptor> spaceTypes = new ArrayList<>(
-            Math.min(10, types.size()));
+        Collection<GridQueryTypeDescriptor> spaceTypes = new ArrayList<>();
 
         for (Map.Entry<QueryTypeIdKey, QueryTypeDescriptorImpl> e : types.entrySet()) {
             QueryTypeDescriptorImpl desc = e.getValue();
@@ -1650,11 +1634,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @return Type descriptor.
      * @throws IgniteCheckedException If failed.
      */
-    public GridQueryTypeDescriptor type(@Nullable String space, String typeName) throws IgniteCheckedException {
+    public QueryTypeDescriptorImpl type(@Nullable String space, String typeName) throws IgniteCheckedException {
         QueryTypeDescriptorImpl type = typesByName.get(new QueryTypeNameKey(space, typeName));
 
         if (type == null || !type.registered())
-            throw new IgniteCheckedException("Failed to find type descriptor for type name: " + typeName);
+            throw new IgniteCheckedException("Failed to find SQL table for type: " + typeName);
 
         return type;
     }
@@ -1666,8 +1650,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param clo Closure.
      * @param complete Complete.
      */
-    public <R> R executeQuery(GridCacheQueryType qryType, String qry, GridCacheContext<?, ?> cctx, IgniteOutClosureX<R> clo, boolean complete)
-        throws IgniteCheckedException {
+    public <R> R executeQuery(GridCacheQueryType qryType, String qry, GridCacheContext<?, ?> cctx,
+        IgniteOutClosureX<R> clo, boolean complete) throws IgniteCheckedException {
         final long startTime = U.currentTimeMillis();
 
         Throwable err = null;


[7/8] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-2.0' into ignite-4768

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-2.0' into ignite-4768


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

Branch: refs/heads/ignite-4768
Commit: 850e4dc03df64863a7ae0473362a42e97c8a2041
Parents: e4e8253 86fad98
Author: sboikov <sb...@gridgain.com>
Authored: Mon Mar 13 11:22:00 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Mar 13 11:22:00 2017 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |   28 +-
 .../cache/query/GridCacheQueryManager.java      |   22 +-
 .../query/GridQueryIndexDescriptor.java         |    4 +-
 .../processors/query/GridQueryIndexType.java    |   32 -
 .../processors/query/GridQueryIndexing.java     |    8 -
 .../processors/query/GridQueryProcessor.java    | 1307 ++----------------
 .../query/QueryIndexDescriptorImpl.java         |  102 ++
 .../query/QueryTypeDescriptorImpl.java          |  337 +++++
 .../processors/query/QueryTypeIdKey.java        |   94 ++
 .../processors/query/QueryTypeNameKey.java      |   68 +
 .../query/property/QueryBinaryProperty.java     |  267 ++++
 .../query/property/QueryClassProperty.java      |  130 ++
 .../query/property/QueryFieldAccessor.java      |   67 +
 .../query/property/QueryMethodsAccessor.java    |   82 ++
 .../query/property/QueryPropertyAccessor.java   |   53 +
 .../property/QueryReadOnlyMethodsAccessor.java  |   71 +
 .../resources/META-INF/classnames.properties    |    1 -
 .../processors/query/h2/IgniteH2Indexing.java   |   23 +-
 .../query/h2/opt/GridH2IndexBase.java           |   10 -
 .../processors/query/h2/opt/GridH2Table.java    |   33 -
 .../query/h2/opt/GridH2TreeIndex.java           |   23 -
 .../query/h2/opt/GridLuceneIndex.java           |    4 +-
 .../query/h2/GridH2IndexRebuildTest.java        |  261 ----
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   23 +-
 .../query/h2/opt/GridH2TableSelfTest.java       |   54 -
 .../IgniteCacheQuerySelfTestSuite.java          |    2 -
 26 files changed, 1392 insertions(+), 1714 deletions(-)
----------------------------------------------------------------------



[3/8] ignite git commit: IGNITE-4807: Moved type descriptor and index descriptor to top-level.

Posted by sb...@apache.org.
IGNITE-4807: Moved type descriptor and index descriptor to top-level.


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

Branch: refs/heads/ignite-4768
Commit: 5899cc3e61e56fb34a8e8dba51295c55e5385da2
Parents: 9e4c2cf
Author: devozerov <vo...@gridgain.com>
Authored: Thu Mar 9 16:58:44 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Mar 9 16:58:44 2017 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java    | 443 ++-----------------
 .../query/QueryIndexDescriptorImpl.java         | 102 +++++
 .../query/QueryTypeDescriptorImpl.java          | 337 ++++++++++++++
 3 files changed, 470 insertions(+), 412 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5899cc3e/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 b35a506..1c5b822 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
@@ -27,15 +27,12 @@ import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutorService;
@@ -83,12 +80,8 @@ import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 import org.apache.ignite.internal.util.lang.GridClosureException;
 import org.apache.ignite.internal.util.lang.IgniteOutClosureX;
-import org.apache.ignite.internal.util.tostring.GridToStringExclude;
-import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
-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.internal.util.worker.GridWorker;
 import org.apache.ignite.internal.util.worker.GridWorkerFuture;
@@ -140,10 +133,10 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
 
     /** Type descriptors. */
-    private final Map<QueryTypeIdKey, TypeDescriptor> types = new ConcurrentHashMap8<>();
+    private final Map<QueryTypeIdKey, QueryTypeDescriptorImpl> types = new ConcurrentHashMap8<>();
 
     /** Type descriptors. */
-    private final ConcurrentMap<QueryTypeNameKey, TypeDescriptor> typesByName = new ConcurrentHashMap8<>();
+    private final ConcurrentMap<QueryTypeNameKey, QueryTypeDescriptorImpl> typesByName = new ConcurrentHashMap8<>();
 
     /** */
     private ExecutorService execSvc;
@@ -231,7 +224,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     if (F.isEmpty(qryEntity.getValueType()))
                         throw new IgniteCheckedException("Value type is not set: " + qryEntity);
 
-                    TypeDescriptor desc = new TypeDescriptor();
+                    QueryTypeDescriptorImpl desc = new QueryTypeDescriptorImpl();
 
                     // Key and value classes still can be available if they are primitive or JDK part.
                     // We need that to set correct types for _key and _val columns.
@@ -318,7 +311,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                         if (keyMapper instanceof GridCacheDefaultAffinityKeyMapper) {
                             String affField =
-                                ((GridCacheDefaultAffinityKeyMapper)keyMapper).affinityKeyPropertyName(desc.keyCls);
+                                ((GridCacheDefaultAffinityKeyMapper)keyMapper).affinityKeyPropertyName(desc.keyClass());
 
                             if (affField != null)
                                 desc.affinityKey(affField);
@@ -347,7 +340,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         meta.getDescendingFields().isEmpty() && meta.getGroups().isEmpty())
                         continue;
 
-                    TypeDescriptor desc = new TypeDescriptor();
+                    QueryTypeDescriptorImpl desc = new QueryTypeDescriptorImpl();
 
                     // Key and value classes still can be available if they are primitive or JDK part.
                     // We need that to set correct types for _key and _val columns.
@@ -468,7 +461,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param desc Type descriptor.
      * @throws IgniteCheckedException If failed.
      */
-    private void addTypeByName(CacheConfiguration<?, ?> ccfg, TypeDescriptor desc) throws IgniteCheckedException {
+    private void addTypeByName(CacheConfiguration<?, ?> ccfg, QueryTypeDescriptorImpl desc) throws IgniteCheckedException {
         if (typesByName.putIfAbsent(new QueryTypeNameKey(ccfg.getName(), desc.name()), desc) != null)
             throw new IgniteCheckedException("Type with name '" + desc.name() + "' already indexed " +
                 "in cache '" + ccfg.getName() + "'.");
@@ -542,10 +535,10 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         try {
             idx.unregisterCache(cctx.config());
 
-            Iterator<Map.Entry<QueryTypeIdKey, TypeDescriptor>> it = types.entrySet().iterator();
+            Iterator<Map.Entry<QueryTypeIdKey, QueryTypeDescriptorImpl>> it = types.entrySet().iterator();
 
             while (it.hasNext()) {
-                Map.Entry<QueryTypeIdKey, TypeDescriptor> entry = it.next();
+                Map.Entry<QueryTypeIdKey, QueryTypeDescriptorImpl> entry = it.next();
 
                 if (F.eq(cctx.name(), entry.getKey().space())) {
                     it.remove();
@@ -591,7 +584,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param desc Type descriptor.
      * @return Future that will be completed when rebuilding of all indexes is finished.
      */
-    private IgniteInternalFuture<?> rebuildIndexes(@Nullable final String space, @Nullable final TypeDescriptor desc) {
+    private IgniteInternalFuture<?> rebuildIndexes(@Nullable final String space, @Nullable final QueryTypeDescriptorImpl desc) {
         if (idx == null)
             return new GridFinishedFuture<>(new IgniteCheckedException("Indexing is disabled."));
 
@@ -641,7 +634,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         try {
             GridCompoundFuture<?, ?> fut = new GridCompoundFuture<Object, Object>();
 
-            for (Map.Entry<QueryTypeIdKey, TypeDescriptor> e : types.entrySet())
+            for (Map.Entry<QueryTypeIdKey, QueryTypeDescriptorImpl> e : types.entrySet())
                 fut.add((IgniteInternalFuture)rebuildIndexes(e.getKey().space(), e.getValue()));
 
             fut.markInitialized();
@@ -709,7 +702,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 id = new QueryTypeIdKey(space, valCls);
             }
 
-            TypeDescriptor desc = types.get(id);
+            QueryTypeDescriptorImpl desc = types.get(id);
 
             if (desc == null || !desc.registered())
                 return;
@@ -774,7 +767,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             return executeQuery(GridCacheQueryType.SQL_FIELDS, clause, cctx, new IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, V>>>() {
                 @Override public GridCloseableIterator<IgniteBiTuple<K, V>> applyx() throws IgniteCheckedException {
-                    TypeDescriptor type = typesByName.get(new QueryTypeNameKey(space, resType));
+                    QueryTypeDescriptorImpl type = typesByName.get(new QueryTypeNameKey(space, resType));
 
                     if (type == null || !type.registered())
                         throw new CacheException("Failed to find SQL table for type: " + resType);
@@ -863,7 +856,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         String sqlQry = qry.getSql();
                         Object[] params = qry.getArgs();
 
-                        TypeDescriptor typeDesc = typesByName.get(
+                        QueryTypeDescriptorImpl typeDesc = typesByName.get(
                             new QueryTypeNameKey(
                                 space,
                                 type));
@@ -1165,7 +1158,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             return executeQuery(GridCacheQueryType.TEXT, clause, cctx,
                 new IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, V>>>() {
                     @Override public GridCloseableIterator<IgniteBiTuple<K, V>> applyx() throws IgniteCheckedException {
-                        TypeDescriptor type = typesByName.get(new QueryTypeNameKey(space, resType));
+                        QueryTypeDescriptorImpl type = typesByName.get(new QueryTypeNameKey(space, resType));
 
                         if (type == null || !type.registered())
                             throw new CacheException("Failed to find SQL table for type: " + resType);
@@ -1255,17 +1248,18 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to process undeploy event (grid is stopping).");
 
         try {
-            Iterator<Map.Entry<QueryTypeIdKey, TypeDescriptor>> it = types.entrySet().iterator();
+            Iterator<Map.Entry<QueryTypeIdKey, QueryTypeDescriptorImpl>> it = types.entrySet().iterator();
 
             while (it.hasNext()) {
-                Map.Entry<QueryTypeIdKey, TypeDescriptor> e = it.next();
+                Map.Entry<QueryTypeIdKey, QueryTypeDescriptorImpl> e = it.next();
 
                 if (!F.eq(e.getKey().space(), space))
                     continue;
 
-                TypeDescriptor desc = e.getValue();
+                QueryTypeDescriptorImpl desc = e.getValue();
 
-                if (ldr.equals(U.detectClassLoader(desc.valCls)) || ldr.equals(U.detectClassLoader(desc.keyCls))) {
+                if (ldr.equals(U.detectClassLoader(desc.valueClass())) ||
+                    ldr.equals(U.detectClassLoader(desc.keyClass()))) {
                     idx.unregisterType(e.getKey().space(), desc);
 
                     it.remove();
@@ -1285,7 +1279,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param coCtx Cache object context.
      * @throws IgniteCheckedException If failed.
      */
-    private void processClassMeta(CacheTypeMetadata meta, TypeDescriptor d, CacheObjectContext coCtx)
+    private void processClassMeta(CacheTypeMetadata meta, QueryTypeDescriptorImpl d, CacheObjectContext coCtx)
         throws IgniteCheckedException {
         Map<String,String> aliases = meta.getAliases();
 
@@ -1357,7 +1351,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     private void addToIndex(
-        TypeDescriptor d,
+        QueryTypeDescriptorImpl d,
         Class<?> keyCls,
         Class<?> valCls,
         String pathStr,
@@ -1411,7 +1405,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param d Type descriptor.
      * @throws IgniteCheckedException If failed.
      */
-    private void processBinaryMeta(CacheTypeMetadata meta, TypeDescriptor d)
+    private void processBinaryMeta(CacheTypeMetadata meta, QueryTypeDescriptorImpl d)
         throws IgniteCheckedException {
         Map<String,String> aliases = meta.getAliases();
 
@@ -1478,7 +1472,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
             QueryBinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null);
 
-            if (!d.props.containsKey(prop.name()))
+            if (!d.properties().containsKey(prop.name()))
                 d.addProperty(prop, false);
         }
     }
@@ -1490,7 +1484,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param d Type descriptor.
      * @throws IgniteCheckedException If failed.
      */
-    private void processBinaryMeta(QueryEntity qryEntity, TypeDescriptor d) throws IgniteCheckedException {
+    private void processBinaryMeta(QueryEntity qryEntity, QueryTypeDescriptorImpl d) throws IgniteCheckedException {
         Map<String,String> aliases = qryEntity.getAliases();
 
         if (aliases == null)
@@ -1523,8 +1517,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             else
                 isKeyField = (hasKeyFields ? keyFields.contains(entry.getKey()) : null);
 
-            QueryBinaryProperty prop = buildBinaryProperty(entry.getKey(), U.classForName(entry.getValue(), Object.class, true),
-                aliases, isKeyField);
+            QueryBinaryProperty prop = buildBinaryProperty(entry.getKey(),
+                U.classForName(entry.getValue(), Object.class, true), aliases, isKeyField);
 
             d.addProperty(prop, false);
         }
@@ -1541,7 +1535,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      */
     private void processClassMeta(
         QueryEntity qryEntity,
-        TypeDescriptor d,
+        QueryTypeDescriptorImpl d,
         CacheObjectContext coCtx
     ) throws IgniteCheckedException {
         Map<String,String> aliases = qryEntity.getAliases();
@@ -1571,7 +1565,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param d Type descriptor to populate.
      * @throws IgniteCheckedException If failed to build index information.
      */
-    private void processIndexes(QueryEntity qryEntity, TypeDescriptor d) throws IgniteCheckedException {
+    private void processIndexes(QueryEntity qryEntity, QueryTypeDescriptorImpl d) throws IgniteCheckedException {
         if (!F.isEmpty(qryEntity.getIndexes())) {
             Map<String, String> aliases = qryEntity.getAliases();
 
@@ -1740,8 +1734,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         Collection<GridQueryTypeDescriptor> spaceTypes = new ArrayList<>(
             Math.min(10, types.size()));
 
-        for (Map.Entry<QueryTypeIdKey, TypeDescriptor> e : types.entrySet()) {
-            TypeDescriptor desc = e.getValue();
+        for (Map.Entry<QueryTypeIdKey, QueryTypeDescriptorImpl> e : types.entrySet()) {
+            QueryTypeDescriptorImpl desc = e.getValue();
 
             if (desc.registered() && F.eq(e.getKey().space(), space))
                 spaceTypes.add(desc);
@@ -1759,7 +1753,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     public GridQueryTypeDescriptor type(@Nullable String space, String typeName) throws IgniteCheckedException {
-        TypeDescriptor type = typesByName.get(new QueryTypeNameKey(space, typeName));
+        QueryTypeDescriptorImpl type = typesByName.get(new QueryTypeNameKey(space, typeName));
 
         if (type == null || !type.registered())
             throw new IgniteCheckedException("Failed to find type descriptor for type name: " + typeName);
@@ -1936,380 +1930,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Descriptor of type.
-     */
-    private static class TypeDescriptor implements GridQueryTypeDescriptor {
-        /** */
-        private String name;
-
-        /** */
-        private String tblName;
-
-        /** Value field names and types with preserved order. */
-        @GridToStringInclude
-        private final Map<String, Class<?>> fields = new LinkedHashMap<>();
-
-        /** */
-        @GridToStringExclude
-        private final Map<String, GridQueryProperty> props = new HashMap<>();
-
-        /** Map with upper cased property names to help find properties based on SQL INSERT and MERGE queries. */
-        private final Map<String, GridQueryProperty> uppercaseProps = new HashMap<>();
-
-        /** */
-        @GridToStringInclude
-        private final Map<String, IndexDescriptor> indexes = new HashMap<>();
-
-        /** */
-        private IndexDescriptor fullTextIdx;
-
-        /** */
-        private Class<?> keyCls;
-
-        /** */
-        private Class<?> valCls;
-
-        /** */
-        private String keyTypeName;
-
-        /** */
-        private String valTypeName;
-
-        /** */
-        private boolean valTextIdx;
-
-        /** */
-        private String affKey;
-
-        /** SPI can decide not to register this type. */
-        private boolean registered;
-
-        /**
-         * @return {@code True} if type registration in SPI was finished and type was not rejected.
-         */
-        boolean registered() {
-            return registered;
-        }
-
-        /**
-         * @param registered Sets registered flag.
-         */
-        void registered(boolean registered) {
-            this.registered = registered;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String name() {
-            return name;
-        }
-
-        /**
-         * Sets type name.
-         *
-         * @param name Name.
-         */
-        void name(String name) {
-            this.name = name;
-        }
-
-        /**
-         * Gets table name for type.
-         * @return Table name.
-         */
-        public String tableName() {
-            return tblName;
-        }
-
-        /**
-         * Sets table name for type.
-         *
-         * @param tblName Table name.
-         */
-        public void tableName(String tblName) {
-            this.tblName = tblName;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Map<String, Class<?>> fields() {
-            return fields;
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridQueryProperty property(String name) {
-            GridQueryProperty res = props.get(name);
-
-            if (res == null)
-                res = uppercaseProps.get(name.toUpperCase());
-
-            return res;
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("unchecked")
-        @Override public <T> T value(String field, Object key, Object val) throws IgniteCheckedException {
-            assert field != null;
-
-            GridQueryProperty prop = property(field);
-
-            if (prop == null)
-                throw new IgniteCheckedException("Failed to find field '" + field + "' in type '" + name + "'.");
-
-            return (T)prop.value(key, val);
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("unchecked")
-        @Override public void setValue(String field, Object key, Object val, Object propVal)
-            throws IgniteCheckedException {
-            assert field != null;
-
-            GridQueryProperty prop = property(field);
-
-            if (prop == null)
-                throw new IgniteCheckedException("Failed to find field '" + field + "' in type '" + name + "'.");
-
-            prop.setValue(key, val, propVal);
-        }
-
-        /** {@inheritDoc} */
-        @Override public Map<String, GridQueryIndexDescriptor> indexes() {
-            return Collections.<String, GridQueryIndexDescriptor>unmodifiableMap(indexes);
-        }
-
-        /**
-         * Adds index.
-         *
-         * @param idxName Index name.
-         * @param type Index type.
-         * @return Index descriptor.
-         * @throws IgniteCheckedException In case of error.
-         */
-        public IndexDescriptor addIndex(String idxName, QueryIndexType type) throws IgniteCheckedException {
-            IndexDescriptor idx = new IndexDescriptor(type);
-
-            if (indexes.put(idxName, idx) != null)
-                throw new IgniteCheckedException("Index with name '" + idxName + "' already exists.");
-
-            return idx;
-        }
-
-        /**
-         * Adds field to index.
-         *
-         * @param idxName Index name.
-         * @param field Field name.
-         * @param orderNum Fields order number in index.
-         * @param descending Sorting order.
-         * @throws IgniteCheckedException If failed.
-         */
-        public void addFieldToIndex(String idxName, String field, int orderNum,
-            boolean descending) throws IgniteCheckedException {
-            IndexDescriptor desc = indexes.get(idxName);
-
-            if (desc == null)
-                desc = addIndex(idxName, QueryIndexType.SORTED);
-
-            desc.addField(field, orderNum, descending);
-        }
-
-        /**
-         * Adds field to text index.
-         *
-         * @param field Field name.
-         */
-        public void addFieldToTextIndex(String field) {
-            if (fullTextIdx == null) {
-                fullTextIdx = new IndexDescriptor(QueryIndexType.FULLTEXT);
-
-                indexes.put(null, fullTextIdx);
-            }
-
-            fullTextIdx.addField(field, 0, false);
-        }
-
-        /** {@inheritDoc} */
-        @Override public Class<?> valueClass() {
-            return valCls;
-        }
-
-        /**
-         * Sets value class.
-         *
-         * @param valCls Value class.
-         */
-        void valueClass(Class<?> valCls) {
-            A.notNull(valCls, "Value class must not be null");
-            this.valCls = valCls;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Class<?> keyClass() {
-            return keyCls;
-        }
-
-        /**
-         * Set key class.
-         *
-         * @param keyCls Key class.
-         */
-        void keyClass(Class<?> keyCls) {
-            this.keyCls = keyCls;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String keyTypeName() {
-            return keyTypeName;
-        }
-
-        /**
-         * Set key type name.
-         *
-         * @param keyTypeName Key type name.
-         */
-        public void keyTypeName(String keyTypeName) {
-            this.keyTypeName = keyTypeName;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String valueTypeName() {
-            return valTypeName;
-        }
-
-        /**
-         * Set value type name.
-         *
-         * @param valTypeName Value type name.
-         */
-        public void valueTypeName(String valTypeName) {
-            this.valTypeName = valTypeName;
-        }
-
-        /**
-         * Adds property to the type descriptor.
-         *
-         * @param prop Property.
-         * @param failOnDuplicate Fail on duplicate flag.
-         * @throws IgniteCheckedException In case of error.
-         */
-        public void addProperty(GridQueryProperty prop, boolean failOnDuplicate) throws IgniteCheckedException {
-            String name = prop.name();
-
-            if (props.put(name, prop) != null && failOnDuplicate)
-                throw new IgniteCheckedException("Property with name '" + name + "' already exists.");
-
-            if (uppercaseProps.put(name.toUpperCase(), prop) != null && failOnDuplicate)
-                throw new IgniteCheckedException("Property with upper cased name '" + name + "' already exists.");
-
-            fields.put(name, prop.type());
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean valueTextIndex() {
-            return valTextIdx;
-        }
-
-        /**
-         * Sets if this value should be text indexed.
-         *
-         * @param valTextIdx Flag value.
-         */
-        public void valueTextIndex(boolean valTextIdx) {
-            this.valTextIdx = valTextIdx;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String affinityKey() {
-            return affKey;
-        }
-
-        /**
-         * @param affKey Affinity key field.
-         */
-        void affinityKey(String affKey) {
-            this.affKey = affKey;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(TypeDescriptor.class, this);
-        }
-    }
-
-    /**
-     * Index descriptor.
-     */
-    private static class IndexDescriptor implements GridQueryIndexDescriptor {
-        /** Fields sorted by order number. */
-        private final Collection<T2<String, Integer>> fields = new TreeSet<>(
-            new Comparator<T2<String, Integer>>() {
-                @Override public int compare(T2<String, Integer> o1, T2<String, Integer> o2) {
-                    if (o1.get2().equals(o2.get2())) // Order is equal, compare field names to avoid replace in Set.
-                        return o1.get1().compareTo(o2.get1());
-
-                    return o1.get2() < o2.get2() ? -1 : 1;
-                }
-            });
-
-        /** Fields which should be indexed in descending order. */
-        private Collection<String> descendings;
-
-        /** */
-        private final QueryIndexType type;
-
-        /**
-         * @param type Type.
-         */
-        private IndexDescriptor(QueryIndexType type) {
-            assert type != null;
-
-            this.type = type;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Collection<String> fields() {
-            Collection<String> res = new ArrayList<>(fields.size());
-
-            for (T2<String, Integer> t : fields)
-                res.add(t.get1());
-
-            return res;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean descending(String field) {
-            return descendings != null && descendings.contains(field);
-        }
-
-        /**
-         * Adds field to this index.
-         *
-         * @param field Field name.
-         * @param orderNum Field order number in this index.
-         * @param descending Sort order.
-         */
-        public void addField(String field, int orderNum, boolean descending) {
-            fields.add(new T2<>(field, orderNum));
-
-            if (descending) {
-                if (descendings == null)
-                    descendings  = new HashSet<>();
-
-                descendings.add(field);
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public QueryIndexType type() {
-            return type;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(IndexDescriptor.class, this);
-        }
-    }
-
-    /**
      * The way to index.
      */
     private enum IndexType {
@@ -2322,5 +1942,4 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         /** Text index. */
         TEXT
     }
-
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5899cc3e/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexDescriptorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexDescriptorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexDescriptorImpl.java
new file mode 100644
index 0000000..53a0dfe
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryIndexDescriptorImpl.java
@@ -0,0 +1,102 @@
+/*
+ * 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.query;
+
+import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.TreeSet;
+
+/**
+ * Index descriptor.
+ */
+public class QueryIndexDescriptorImpl implements GridQueryIndexDescriptor {
+    /** Fields sorted by order number. */
+    private final Collection<T2<String, Integer>> fields = new TreeSet<>(
+        new Comparator<T2<String, Integer>>() {
+            @Override public int compare(T2<String, Integer> o1, T2<String, Integer> o2) {
+                if (o1.get2().equals(o2.get2())) // Order is equal, compare field names to avoid replace in Set.
+                    return o1.get1().compareTo(o2.get1());
+
+                return o1.get2() < o2.get2() ? -1 : 1;
+            }
+        });
+
+    /** Fields which should be indexed in descending order. */
+    private Collection<String> descendings;
+
+    /** */
+    private final QueryIndexType type;
+
+    /**
+     * @param type Type.
+     */
+    public QueryIndexDescriptorImpl(QueryIndexType type) {
+        assert type != null;
+
+        this.type = type;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<String> fields() {
+        Collection<String> res = new ArrayList<>(fields.size());
+
+        for (T2<String, Integer> t : fields)
+            res.add(t.get1());
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean descending(String field) {
+        return descendings != null && descendings.contains(field);
+    }
+
+    /**
+     * Adds field to this index.
+     *
+     * @param field Field name.
+     * @param orderNum Field order number in this index.
+     * @param descending Sort order.
+     */
+    public void addField(String field, int orderNum, boolean descending) {
+        fields.add(new T2<>(field, orderNum));
+
+        if (descending) {
+            if (descendings == null)
+                descendings  = new HashSet<>();
+
+            descendings.add(field);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public QueryIndexType type() {
+        return type;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(QueryIndexDescriptorImpl.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5899cc3e/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
new file mode 100644
index 0000000..f22cc11
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
@@ -0,0 +1,337 @@
+/*
+ * 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.query;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Descriptor of type.
+ */
+public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
+    /** */
+    private String name;
+
+    /** */
+    private String tblName;
+
+    /** Value field names and types with preserved order. */
+    @GridToStringInclude
+    private final Map<String, Class<?>> fields = new LinkedHashMap<>();
+
+    /** */
+    @GridToStringExclude
+    private final Map<String, GridQueryProperty> props = new HashMap<>();
+
+    /** Map with upper cased property names to help find properties based on SQL INSERT and MERGE queries. */
+    private final Map<String, GridQueryProperty> uppercaseProps = new HashMap<>();
+
+    /** */
+    @GridToStringInclude
+    private final Map<String, QueryIndexDescriptorImpl> indexes = new HashMap<>();
+
+    /** */
+    private QueryIndexDescriptorImpl fullTextIdx;
+
+    /** */
+    private Class<?> keyCls;
+
+    /** */
+    private Class<?> valCls;
+
+    /** */
+    private String keyTypeName;
+
+    /** */
+    private String valTypeName;
+
+    /** */
+    private boolean valTextIdx;
+
+    /** */
+    private String affKey;
+
+    /** SPI can decide not to register this type. */
+    private boolean registered;
+
+    /**
+     * @return {@code True} if type registration in SPI was finished and type was not rejected.
+     */
+    public boolean registered() {
+        return registered;
+    }
+
+    /**
+     * @param registered Sets registered flag.
+     */
+    public void registered(boolean registered) {
+        this.registered = registered;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String name() {
+        return name;
+    }
+
+    /**
+     * Sets type name.
+     *
+     * @param name Name.
+     */
+    public void name(String name) {
+        this.name = name;
+    }
+
+    /**
+     * Gets table name for type.
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /**
+     * Sets table name for type.
+     *
+     * @param tblName Table name.
+     */
+    public void tableName(String tblName) {
+        this.tblName = tblName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<String, Class<?>> fields() {
+        return fields;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridQueryProperty property(String name) {
+        GridQueryProperty res = props.get(name);
+
+        if (res == null)
+            res = uppercaseProps.get(name.toUpperCase());
+
+        return res;
+    }
+
+    /**
+     * @return Properties.
+     */
+    public Map<String, GridQueryProperty> properties() {
+        return props;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <T> T value(String field, Object key, Object val) throws IgniteCheckedException {
+        assert field != null;
+
+        GridQueryProperty prop = property(field);
+
+        if (prop == null)
+            throw new IgniteCheckedException("Failed to find field '" + field + "' in type '" + name + "'.");
+
+        return (T)prop.value(key, val);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void setValue(String field, Object key, Object val, Object propVal)
+        throws IgniteCheckedException {
+        assert field != null;
+
+        GridQueryProperty prop = property(field);
+
+        if (prop == null)
+            throw new IgniteCheckedException("Failed to find field '" + field + "' in type '" + name + "'.");
+
+        prop.setValue(key, val, propVal);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<String, GridQueryIndexDescriptor> indexes() {
+        return Collections.<String, GridQueryIndexDescriptor>unmodifiableMap(indexes);
+    }
+
+    /**
+     * Adds index.
+     *
+     * @param idxName Index name.
+     * @param type Index type.
+     * @return Index descriptor.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public QueryIndexDescriptorImpl addIndex(String idxName, QueryIndexType type) throws IgniteCheckedException {
+        QueryIndexDescriptorImpl idx = new QueryIndexDescriptorImpl(type);
+
+        if (indexes.put(idxName, idx) != null)
+            throw new IgniteCheckedException("Index with name '" + idxName + "' already exists.");
+
+        return idx;
+    }
+
+    /**
+     * Adds field to index.
+     *
+     * @param idxName Index name.
+     * @param field Field name.
+     * @param orderNum Fields order number in index.
+     * @param descending Sorting order.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void addFieldToIndex(String idxName, String field, int orderNum,
+        boolean descending) throws IgniteCheckedException {
+        QueryIndexDescriptorImpl desc = indexes.get(idxName);
+
+        if (desc == null)
+            desc = addIndex(idxName, QueryIndexType.SORTED);
+
+        desc.addField(field, orderNum, descending);
+    }
+
+    /**
+     * Adds field to text index.
+     *
+     * @param field Field name.
+     */
+    public void addFieldToTextIndex(String field) {
+        if (fullTextIdx == null) {
+            fullTextIdx = new QueryIndexDescriptorImpl(QueryIndexType.FULLTEXT);
+
+            indexes.put(null, fullTextIdx);
+        }
+
+        fullTextIdx.addField(field, 0, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Class<?> valueClass() {
+        return valCls;
+    }
+
+    /**
+     * Sets value class.
+     *
+     * @param valCls Value class.
+     */
+    public void valueClass(Class<?> valCls) {
+        A.notNull(valCls, "Value class must not be null");
+        this.valCls = valCls;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Class<?> keyClass() {
+        return keyCls;
+    }
+
+    /**
+     * Set key class.
+     *
+     * @param keyCls Key class.
+     */
+    public void keyClass(Class<?> keyCls) {
+        this.keyCls = keyCls;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String keyTypeName() {
+        return keyTypeName;
+    }
+
+    /**
+     * Set key type name.
+     *
+     * @param keyTypeName Key type name.
+     */
+    public void keyTypeName(String keyTypeName) {
+        this.keyTypeName = keyTypeName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String valueTypeName() {
+        return valTypeName;
+    }
+
+    /**
+     * Set value type name.
+     *
+     * @param valTypeName Value type name.
+     */
+    public void valueTypeName(String valTypeName) {
+        this.valTypeName = valTypeName;
+    }
+
+    /**
+     * Adds property to the type descriptor.
+     *
+     * @param prop Property.
+     * @param failOnDuplicate Fail on duplicate flag.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public void addProperty(GridQueryProperty prop, boolean failOnDuplicate) throws IgniteCheckedException {
+        String name = prop.name();
+
+        if (props.put(name, prop) != null && failOnDuplicate)
+            throw new IgniteCheckedException("Property with name '" + name + "' already exists.");
+
+        if (uppercaseProps.put(name.toUpperCase(), prop) != null && failOnDuplicate)
+            throw new IgniteCheckedException("Property with upper cased name '" + name + "' already exists.");
+
+        fields.put(name, prop.type());
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean valueTextIndex() {
+        return valTextIdx;
+    }
+
+    /**
+     * Sets if this value should be text indexed.
+     *
+     * @param valTextIdx Flag value.
+     */
+    public void valueTextIndex(boolean valTextIdx) {
+        this.valTextIdx = valTextIdx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String affinityKey() {
+        return affKey;
+    }
+
+    /**
+     * @param affKey Affinity key field.
+     */
+    public void affinityKey(String affKey) {
+        this.affKey = affKey;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(QueryTypeDescriptorImpl.class, this);
+    }
+}


[2/8] ignite git commit: IGNITE-4807: Moved several GridQueryProcessor classes to top-level.

Posted by sb...@apache.org.
IGNITE-4807: Moved several GridQueryProcessor classes to top-level.


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

Branch: refs/heads/ignite-4768
Commit: 9e4c2cfdc0d527b439a212f2c347ea4356e7c941
Parents: 0ea88cd
Author: devozerov <vo...@gridgain.com>
Authored: Thu Mar 9 16:47:24 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Mar 9 16:47:24 2017 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java    | 753 ++-----------------
 .../processors/query/QueryTypeIdKey.java        |  94 +++
 .../processors/query/QueryTypeNameKey.java      |  68 ++
 .../query/property/QueryBinaryProperty.java     | 267 +++++++
 .../query/property/QueryClassProperty.java      | 130 ++++
 .../query/property/QueryFieldAccessor.java      |  67 ++
 .../query/property/QueryMethodsAccessor.java    |  82 ++
 .../query/property/QueryPropertyAccessor.java   |  53 ++
 .../property/QueryReadOnlyMethodsAccessor.java  |  71 ++
 9 files changed, 901 insertions(+), 684 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9e4c2cfd/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 1ff2330..b35a506 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
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.query;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.util.concurrent.TimeUnit;
-import java.lang.reflect.Field;
 import java.lang.reflect.Method;
 import java.math.BigDecimal;
 import java.sql.Time;
@@ -44,10 +43,6 @@ import javax.cache.Cache;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.binary.BinaryField;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.binary.BinaryObjectBuilder;
-import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.cache.CacheTypeMetadata;
 import org.apache.ignite.cache.QueryEntity;
@@ -62,8 +57,6 @@ import org.apache.ignite.events.CacheQueryExecutedEvent;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
-import org.apache.ignite.internal.binary.BinaryObjectEx;
-import org.apache.ignite.internal.binary.BinaryObjectExImpl;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
@@ -77,6 +70,12 @@ import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProce
 import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture;
 import org.apache.ignite.internal.processors.cache.query.CacheQueryType;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.query.property.QueryBinaryProperty;
+import org.apache.ignite.internal.processors.query.property.QueryClassProperty;
+import org.apache.ignite.internal.processors.query.property.QueryFieldAccessor;
+import org.apache.ignite.internal.processors.query.property.QueryMethodsAccessor;
+import org.apache.ignite.internal.processors.query.property.QueryPropertyAccessor;
+import org.apache.ignite.internal.processors.query.property.QueryReadOnlyMethodsAccessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
@@ -141,10 +140,10 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
 
     /** Type descriptors. */
-    private final Map<TypeId, TypeDescriptor> types = new ConcurrentHashMap8<>();
+    private final Map<QueryTypeIdKey, TypeDescriptor> types = new ConcurrentHashMap8<>();
 
     /** Type descriptors. */
-    private final ConcurrentMap<TypeName, TypeDescriptor> typesByName = new ConcurrentHashMap8<>();
+    private final ConcurrentMap<QueryTypeNameKey, TypeDescriptor> typesByName = new ConcurrentHashMap8<>();
 
     /** */
     private ExecutorService execSvc;
@@ -293,16 +292,16 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                             mustDeserializeClss.add(valCls);
                     }
 
-                    TypeId typeId;
-                    TypeId altTypeId = null;
+                    QueryTypeIdKey typeId;
+                    QueryTypeIdKey altTypeId = null;
 
                     if (valCls == null || (binaryEnabled && !keyOrValMustDeserialize)) {
                         processBinaryMeta(qryEntity, desc);
 
-                        typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(qryEntity.getValueType()));
+                        typeId = new QueryTypeIdKey(ccfg.getName(), ctx.cacheObjects().typeId(qryEntity.getValueType()));
 
                         if (valCls != null)
-                            altTypeId = new TypeId(ccfg.getName(), valCls);
+                            altTypeId = new QueryTypeIdKey(ccfg.getName(), valCls);
 
                         if (!cctx.customAffinityMapper() && qryEntity.getKeyType() != null) {
                             // Need to setup affinity key for distributed joins.
@@ -325,8 +324,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                                 desc.affinityKey(affField);
                         }
 
-                        typeId = new TypeId(ccfg.getName(), valCls);
-                        altTypeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(qryEntity.getValueType()));
+                        typeId = new QueryTypeIdKey(ccfg.getName(), valCls);
+                        altTypeId = new QueryTypeIdKey(ccfg.getName(), ctx.cacheObjects().typeId(qryEntity.getValueType()));
                     }
 
                     addTypeByName(ccfg, desc);
@@ -402,22 +401,22 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                             mustDeserializeClss.add(valCls);
                     }
 
-                    TypeId typeId;
-                    TypeId altTypeId = null;
+                    QueryTypeIdKey typeId;
+                    QueryTypeIdKey altTypeId = null;
 
                     if (valCls == null || (binaryEnabled && !keyOrValMustDeserialize)) {
                         processBinaryMeta(meta, desc);
 
-                        typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType()));
+                        typeId = new QueryTypeIdKey(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType()));
 
                         if (valCls != null)
-                            altTypeId = new TypeId(ccfg.getName(), valCls);
+                            altTypeId = new QueryTypeIdKey(ccfg.getName(), valCls);
                     }
                     else {
                         processClassMeta(meta, desc, coCtx);
 
-                        typeId = new TypeId(ccfg.getName(), valCls);
-                        altTypeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType()));
+                        typeId = new QueryTypeIdKey(ccfg.getName(), valCls);
+                        altTypeId = new QueryTypeIdKey(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType()));
                     }
 
                     addTypeByName(ccfg, desc);
@@ -470,7 +469,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     private void addTypeByName(CacheConfiguration<?, ?> ccfg, TypeDescriptor desc) throws IgniteCheckedException {
-        if (typesByName.putIfAbsent(new TypeName(ccfg.getName(), desc.name()), desc) != null)
+        if (typesByName.putIfAbsent(new QueryTypeNameKey(ccfg.getName(), desc.name()), desc) != null)
             throw new IgniteCheckedException("Type with name '" + desc.name() + "' already indexed " +
                 "in cache '" + ccfg.getName() + "'.");
     }
@@ -543,15 +542,15 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         try {
             idx.unregisterCache(cctx.config());
 
-            Iterator<Map.Entry<TypeId, TypeDescriptor>> it = types.entrySet().iterator();
+            Iterator<Map.Entry<QueryTypeIdKey, TypeDescriptor>> it = types.entrySet().iterator();
 
             while (it.hasNext()) {
-                Map.Entry<TypeId, TypeDescriptor> entry = it.next();
+                Map.Entry<QueryTypeIdKey, TypeDescriptor> entry = it.next();
 
-                if (F.eq(cctx.name(), entry.getKey().space)) {
+                if (F.eq(cctx.name(), entry.getKey().space())) {
                     it.remove();
 
-                    typesByName.remove(new TypeName(cctx.name(), entry.getValue().name()));
+                    typesByName.remove(new QueryTypeNameKey(cctx.name(), entry.getValue().name()));
                 }
             }
         }
@@ -578,7 +577,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             return rebuildIndexes(
                 space,
                 typesByName.get(
-                    new TypeName(
+                    new QueryTypeNameKey(
                         space,
                         valTypeName)));
         }
@@ -642,8 +641,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         try {
             GridCompoundFuture<?, ?> fut = new GridCompoundFuture<Object, Object>();
 
-            for (Map.Entry<TypeId, TypeDescriptor> e : types.entrySet())
-                fut.add((IgniteInternalFuture)rebuildIndexes(e.getKey().space, e.getValue()));
+            for (Map.Entry<QueryTypeIdKey, TypeDescriptor> e : types.entrySet())
+                fut.add((IgniteInternalFuture)rebuildIndexes(e.getKey().space(), e.getValue()));
 
             fut.markInitialized();
 
@@ -695,19 +694,19 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             Class<?> valCls = null;
 
-            TypeId id;
+            QueryTypeIdKey id;
 
             boolean binaryVal = ctx.cacheObjects().isBinaryObject(val);
 
             if (binaryVal) {
                 int typeId = ctx.cacheObjects().typeId(val);
 
-                id = new TypeId(space, typeId);
+                id = new QueryTypeIdKey(space, typeId);
             }
             else {
                 valCls = val.value(coctx, false).getClass();
 
-                id = new TypeId(space, valCls);
+                id = new QueryTypeIdKey(space, valCls);
             }
 
             TypeDescriptor desc = types.get(id);
@@ -775,7 +774,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             return executeQuery(GridCacheQueryType.SQL_FIELDS, clause, cctx, new IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, V>>>() {
                 @Override public GridCloseableIterator<IgniteBiTuple<K, V>> applyx() throws IgniteCheckedException {
-                    TypeDescriptor type = typesByName.get(new TypeName(space, resType));
+                    TypeDescriptor type = typesByName.get(new QueryTypeNameKey(space, resType));
 
                     if (type == null || !type.registered())
                         throw new CacheException("Failed to find SQL table for type: " + resType);
@@ -865,7 +864,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         Object[] params = qry.getArgs();
 
                         TypeDescriptor typeDesc = typesByName.get(
-                            new TypeName(
+                            new QueryTypeNameKey(
                                 space,
                                 type));
 
@@ -1166,7 +1165,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             return executeQuery(GridCacheQueryType.TEXT, clause, cctx,
                 new IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, V>>>() {
                     @Override public GridCloseableIterator<IgniteBiTuple<K, V>> applyx() throws IgniteCheckedException {
-                        TypeDescriptor type = typesByName.get(new TypeName(space, resType));
+                        TypeDescriptor type = typesByName.get(new QueryTypeNameKey(space, resType));
 
                         if (type == null || !type.registered())
                             throw new CacheException("Failed to find SQL table for type: " + resType);
@@ -1256,18 +1255,18 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to process undeploy event (grid is stopping).");
 
         try {
-            Iterator<Map.Entry<TypeId, TypeDescriptor>> it = types.entrySet().iterator();
+            Iterator<Map.Entry<QueryTypeIdKey, TypeDescriptor>> it = types.entrySet().iterator();
 
             while (it.hasNext()) {
-                Map.Entry<TypeId, TypeDescriptor> e = it.next();
+                Map.Entry<QueryTypeIdKey, TypeDescriptor> e = it.next();
 
-                if (!F.eq(e.getKey().space, space))
+                if (!F.eq(e.getKey().space(), space))
                     continue;
 
                 TypeDescriptor desc = e.getValue();
 
                 if (ldr.equals(U.detectClassLoader(desc.valCls)) || ldr.equals(U.detectClassLoader(desc.keyCls))) {
-                    idx.unregisterType(e.getKey().space, desc);
+                    idx.unregisterType(e.getKey().space(), desc);
 
                     it.remove();
                 }
@@ -1333,7 +1332,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
-            ClassProperty prop = buildClassProperty(
+            QueryClassProperty prop = buildClassProperty(
                 keyCls,
                 valCls,
                 entry.getKey(),
@@ -1377,7 +1376,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             propCls = valCls;
         }
         else {
-            ClassProperty prop = buildClassProperty(
+            QueryClassProperty prop = buildClassProperty(
                 keyCls,
                 valCls,
                 pathStr,
@@ -1420,7 +1419,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             aliases = Collections.emptyMap();
 
         for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet()) {
-            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null);
+            QueryBinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null);
 
             d.addProperty(prop, false);
 
@@ -1432,7 +1431,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (Map.Entry<String, Class<?>> entry : meta.getDescendingFields().entrySet()) {
-            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null);
+            QueryBinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null);
 
             d.addProperty(prop, false);
 
@@ -1444,7 +1443,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (String txtIdx : meta.getTextFields()) {
-            BinaryProperty prop = buildBinaryProperty(txtIdx, String.class, aliases, null);
+            QueryBinaryProperty prop = buildBinaryProperty(txtIdx, String.class, aliases, null);
 
             d.addProperty(prop, false);
 
@@ -1462,7 +1461,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 int order = 0;
 
                 for (Map.Entry<String, IgniteBiTuple<Class<?>, Boolean>> idxField : idxFields.entrySet()) {
-                    BinaryProperty prop = buildBinaryProperty(idxField.getKey(), idxField.getValue().get1(), aliases,
+                    QueryBinaryProperty prop = buildBinaryProperty(idxField.getKey(), idxField.getValue().get1(), aliases,
                         null);
 
                     d.addProperty(prop, false);
@@ -1477,7 +1476,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
-            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null);
+            QueryBinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null);
 
             if (!d.props.containsKey(prop.name()))
                 d.addProperty(prop, false);
@@ -1524,7 +1523,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             else
                 isKeyField = (hasKeyFields ? keyFields.contains(entry.getKey()) : null);
 
-            BinaryProperty prop = buildBinaryProperty(entry.getKey(), U.classForName(entry.getValue(), Object.class, true),
+            QueryBinaryProperty prop = buildBinaryProperty(entry.getKey(), U.classForName(entry.getValue(), Object.class, true),
                 aliases, isKeyField);
 
             d.addProperty(prop, false);
@@ -1551,7 +1550,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             aliases = Collections.emptyMap();
 
         for (Map.Entry<String, String> entry : qryEntity.getFields().entrySet()) {
-            ClassProperty prop = buildClassProperty(
+            QueryClassProperty prop = buildClassProperty(
                 d.keyClass(),
                 d.valueClass(),
                 entry.getKey(),
@@ -1634,11 +1633,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      *      to key, {@code false} if it belongs to value, {@code null} if QueryEntity#keyFields is null.
      * @return Binary property.
      */
-    private BinaryProperty buildBinaryProperty(String pathStr, Class<?> resType, Map<String, String> aliases,
+    private QueryBinaryProperty buildBinaryProperty(String pathStr, Class<?> resType, Map<String, String> aliases,
         @Nullable Boolean isKeyField) throws IgniteCheckedException {
         String[] path = pathStr.split("\\.");
 
-        BinaryProperty res = null;
+        QueryBinaryProperty res = null;
 
         StringBuilder fullName = new StringBuilder();
 
@@ -1651,7 +1650,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             String alias = aliases.get(fullName.toString());
 
             // The key flag that we've found out is valid for the whole path.
-            res = new BinaryProperty(prop, res, resType, isKeyField, alias);
+            res = new QueryBinaryProperty(ctx, log, prop, res, resType, isKeyField, alias);
         }
 
         return res;
@@ -1666,9 +1665,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @return Class property.
      * @throws IgniteCheckedException If failed.
      */
-    private static ClassProperty buildClassProperty(Class<?> keyCls, Class<?> valCls, String pathStr, Class<?> resType,
+    private static QueryClassProperty buildClassProperty(Class<?> keyCls, Class<?> valCls, String pathStr, Class<?> resType,
         Map<String,String> aliases, CacheObjectContext coCtx) throws IgniteCheckedException {
-        ClassProperty res = buildClassProperty(
+        QueryClassProperty res = buildClassProperty(
             true,
             keyCls,
             pathStr,
@@ -1695,11 +1694,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param aliases Aliases.
      * @return Property instance corresponding to the given path.
      */
-    private static ClassProperty buildClassProperty(boolean key, Class<?> cls, String pathStr, Class<?> resType,
+    private static QueryClassProperty buildClassProperty(boolean key, Class<?> cls, String pathStr, Class<?> resType,
         Map<String,String> aliases, CacheObjectContext coCtx) {
         String[] path = pathStr.split("\\.");
 
-        ClassProperty res = null;
+        QueryClassProperty res = null;
 
         StringBuilder fullName = new StringBuilder();
 
@@ -1711,12 +1710,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             String alias = aliases.get(fullName.toString());
 
-            PropertyAccessor accessor = findProperty(prop, cls);
+            QueryPropertyAccessor accessor = findProperty(prop, cls);
 
             if (accessor == null)
                 return null;
 
-            ClassProperty tmp = new ClassProperty(accessor, key, alias, coCtx);
+            QueryClassProperty tmp = new QueryClassProperty(accessor, key, alias, coCtx);
 
             tmp.parent(res);
 
@@ -1741,10 +1740,10 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         Collection<GridQueryTypeDescriptor> spaceTypes = new ArrayList<>(
             Math.min(10, types.size()));
 
-        for (Map.Entry<TypeId, TypeDescriptor> e : types.entrySet()) {
+        for (Map.Entry<QueryTypeIdKey, TypeDescriptor> e : types.entrySet()) {
             TypeDescriptor desc = e.getValue();
 
-            if (desc.registered() && F.eq(e.getKey().space, space))
+            if (desc.registered() && F.eq(e.getKey().space(), space))
                 spaceTypes.add(desc);
         }
 
@@ -1760,7 +1759,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     public GridQueryTypeDescriptor type(@Nullable String space, String typeName) throws IgniteCheckedException {
-        TypeDescriptor type = typesByName.get(new TypeName(space, typeName));
+        TypeDescriptor type = typesByName.get(new QueryTypeNameKey(space, typeName));
 
         if (type == null || !type.registered())
             throw new IgniteCheckedException("Failed to find type descriptor for type name: " + typeName);
@@ -1830,7 +1829,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param cls Class to search for a member in.
      * @return Member for given name.
      */
-    @Nullable private static PropertyAccessor findProperty(String prop, Class<?> cls) {
+    @Nullable private static QueryPropertyAccessor findProperty(String prop, Class<?> cls) {
         StringBuilder getBldr = new StringBuilder("get");
         getBldr.append(prop);
         getBldr.setCharAt(3, Character.toUpperCase(getBldr.charAt(3)));
@@ -1851,10 +1850,10 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             }
             catch (NoSuchMethodException ignore) {
                 // Have getter, but no setter - return read-only accessor.
-                return new ReadOnlyMethodsAccessor(getter, prop);
+                return new QueryReadOnlyMethodsAccessor(getter, prop);
             }
 
-            return new MethodsAccessor(getter, setter, prop);
+            return new QueryMethodsAccessor(getter, setter, prop);
         }
         catch (NoSuchMethodException ignore) {
             // No-op.
@@ -1878,10 +1877,10 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             }
             catch (NoSuchMethodException ignore) {
                 // Have getter, but no setter - return read-only accessor.
-                return new ReadOnlyMethodsAccessor(getter, prop);
+                return new QueryReadOnlyMethodsAccessor(getter, prop);
             }
 
-            return new MethodsAccessor(getter, setter, prop);
+            return new QueryMethodsAccessor(getter, setter, prop);
         }
         catch (NoSuchMethodException ignore) {
             // No-op.
@@ -1891,7 +1890,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         while (cls0 != null)
             try {
-                return new FieldAccessor(cls0.getDeclaredField(prop));
+                return new QueryFieldAccessor(cls0.getDeclaredField(prop));
             }
             catch (NoSuchFieldException ignored) {
                 cls0 = cls0.getSuperclass();
@@ -1909,10 +1908,10 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             }
             catch (NoSuchMethodException ignore) {
                 // Have getter, but no setter - return read-only accessor.
-                return new ReadOnlyMethodsAccessor(getter, prop);
+                return new QueryReadOnlyMethodsAccessor(getter, prop);
             }
 
-            return new MethodsAccessor(getter, setter, prop);
+            return new QueryMethodsAccessor(getter, setter, prop);
         }
         catch (NoSuchMethodException ignored) {
             // No-op.
@@ -1937,331 +1936,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Description of type property.
-     */
-    private static class ClassProperty implements GridQueryProperty {
-        /** */
-        private final PropertyAccessor accessor;
-
-        /** */
-        private final boolean key;
-
-        /** */
-        private ClassProperty parent;
-
-        /** */
-        private final String name;
-
-        /** */
-        private final CacheObjectContext coCtx;
-
-        /**
-         * Constructor.
-         *
-         * @param accessor Way of accessing the property.
-         */
-        ClassProperty(PropertyAccessor accessor, boolean key, String name, @Nullable CacheObjectContext coCtx) {
-            this.accessor = accessor;
-
-            this.key = key;
-
-            this.name = !F.isEmpty(name) ? name : accessor.getPropertyName();
-
-            this.coCtx = coCtx;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object value(Object key, Object val) throws IgniteCheckedException {
-            Object x = unwrap(this.key ? key : val);
-
-            if (parent != null)
-                x = parent.value(key, val);
-
-            if (x == null)
-                return null;
-
-            return accessor.getValue(x);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void setValue(Object key, Object val, Object propVal) throws IgniteCheckedException {
-            Object x = unwrap(this.key ? key : val);
-
-            if (parent != null)
-                x = parent.value(key, val);
-
-            if (x == null)
-                return;
-
-            accessor.setValue(x, propVal);
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean key() {
-            return key;
-        }
-
-        /**
-         * Unwraps cache object, if needed.
-         *
-         * @param o Object to unwrap.
-         * @return Unwrapped object.
-         */
-        private Object unwrap(Object o) {
-            return coCtx == null ? o : o instanceof CacheObject ? ((CacheObject)o).value(coCtx, false) : o;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String name() {
-            return name;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Class<?> type() {
-            return accessor.getType();
-        }
-
-        /**
-         * @param parent Parent property if this is embeddable element.
-         */
-        public void parent(ClassProperty parent) {
-            this.parent = parent;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(ClassProperty.class, this);
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridQueryProperty parent() {
-            return parent;
-        }
-    }
-
-    /**
-     *
-     */
-    private class BinaryProperty implements GridQueryProperty {
-        /** Property name. */
-        private String propName;
-
-        /** */
-        private String alias;
-
-        /** Parent property. */
-        private BinaryProperty parent;
-
-        /** Result class. */
-        private Class<?> type;
-
-        /** */
-        private volatile int isKeyProp;
-
-        /** Binary field to speed-up deserialization. */
-        private volatile BinaryField field;
-
-        /** Flag indicating that we already tried to take a field. */
-        private volatile boolean fieldTaken;
-
-        /** Whether user was warned about missing property. */
-        private volatile boolean warned;
-
-        /**
-         * Constructor.
-         *
-         * @param propName Property name.
-         * @param parent Parent property.
-         * @param type Result type.
-         * @param key {@code true} if key property, {@code false} otherwise, {@code null}  if unknown.
-         * @param alias Field alias.
-         */
-        private BinaryProperty(String propName, BinaryProperty parent, Class<?> type, @Nullable Boolean key, String alias) {
-            super();
-            this.propName = propName;
-            this.alias = F.isEmpty(alias) ? propName : alias;
-            this.parent = parent;
-            this.type = type;
-
-            if (key != null)
-                this.isKeyProp = key ? 1 : -1;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object value(Object key, Object val) throws IgniteCheckedException {
-            Object obj;
-
-            if (parent != null) {
-                obj = parent.value(key, val);
-
-                if (obj == null)
-                    return null;
-
-                if (!ctx.cacheObjects().isBinaryObject(obj))
-                    throw new IgniteCheckedException("Non-binary object received as a result of property extraction " +
-                        "[parent=" + parent + ", propName=" + propName + ", obj=" + obj + ']');
-            }
-            else {
-                int isKeyProp0 = isKeyProp;
-
-                if (isKeyProp0 == 0) {
-                    // Key is allowed to be a non-binary object here.
-                    // We check key before value consistently with ClassProperty.
-                    if (key instanceof BinaryObject && ((BinaryObject)key).hasField(propName))
-                        isKeyProp = isKeyProp0 = 1;
-                    else if (val instanceof BinaryObject && ((BinaryObject)val).hasField(propName))
-                        isKeyProp = isKeyProp0 = -1;
-                    else {
-                        if (!warned) {
-                            U.warn(log, "Neither key nor value have property \"" + propName + "\" " +
-                                "(is cache indexing configured correctly?)");
-
-                            warned = true;
-                        }
-
-                        return null;
-                    }
-                }
-
-                obj = isKeyProp0 == 1 ? key : val;
-            }
-
-            if (obj instanceof BinaryObject) {
-                BinaryObject obj0 = (BinaryObject) obj;
-                return fieldValue(obj0);
-            }
-            else if (obj instanceof BinaryObjectBuilder) {
-                BinaryObjectBuilder obj0 = (BinaryObjectBuilder)obj;
-
-                return obj0.getField(name());
-            }
-            else
-                throw new IgniteCheckedException("Unexpected binary object class [type=" + obj.getClass() + ']');
-        }
-
-        /** {@inheritDoc} */
-        @Override public void setValue(Object key, Object val, Object propVal) throws IgniteCheckedException {
-            Object obj = key() ? key : val;
-
-            if (obj == null)
-                return;
-
-            Object srcObj = obj;
-
-            if (!(srcObj instanceof BinaryObjectBuilder))
-                throw new UnsupportedOperationException("Individual properties can be set for binary builders only");
-
-            if (parent != null)
-                obj = parent.value(key, val);
-
-            boolean needsBuild = false;
-
-            if (obj instanceof BinaryObjectExImpl) {
-                if (parent == null)
-                    throw new UnsupportedOperationException("Individual properties can be set for binary builders only");
-
-                needsBuild = true;
-
-                obj = ((BinaryObjectExImpl)obj).toBuilder();
-            }
-
-            if (!(obj instanceof BinaryObjectBuilder))
-                throw new UnsupportedOperationException("Individual properties can be set for binary builders only");
-
-            setValue0((BinaryObjectBuilder) obj, propName, propVal, type());
-
-            if (needsBuild) {
-                obj = ((BinaryObjectBuilder) obj).build();
-
-                assert parent != null;
-
-                // And now let's set this newly constructed object to parent
-                setValue0((BinaryObjectBuilder) srcObj, parent.propName, obj, obj.getClass());
-            }
-        }
-
-        /**
-         * @param builder Object builder.
-         * @param field Field name.
-         * @param val Value to set.
-         * @param valType Type of {@code val}.
-         * @param <T> Value type.
-         */
-        private <T> void setValue0(BinaryObjectBuilder builder, String field, Object val, Class<T> valType) {
-            //noinspection unchecked
-            builder.setField(field, (T)val, valType);
-        }
-
-        /**
-         * Get binary field for the property.
-         *
-         * @param obj Target object.
-         * @return Binary field.
-         */
-        private BinaryField binaryField(BinaryObject obj) {
-            BinaryField field0 = field;
-
-            if (field0 == null && !fieldTaken) {
-                BinaryType type = obj instanceof BinaryObjectEx ? ((BinaryObjectEx)obj).rawType() : obj.type();
-
-                if (type != null) {
-                    field0 = type.field(propName);
-
-                    assert field0 != null;
-
-                    field = field0;
-                }
-
-                fieldTaken = true;
-            }
-
-            return field0;
-        }
-
-        /**
-         * Gets field value for the given binary object.
-         *
-         * @param obj Binary object.
-         * @return Field value.
-         */
-        @SuppressWarnings("IfMayBeConditional")
-        private Object fieldValue(BinaryObject obj) {
-            BinaryField field = binaryField(obj);
-
-            if (field != null)
-                return field.value(obj);
-            else
-                return obj.field(propName);
-        }
-
-        /** {@inheritDoc} */
-        @Override public String name() {
-            return alias;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Class<?> type() {
-            return type;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean key() {
-            int isKeyProp0 = isKeyProp;
-
-            if (isKeyProp0 == 0)
-                throw new IllegalStateException("Ownership flag not set for binary property. Have you set 'keyFields'" +
-                    " property of QueryEntity in programmatic or XML configuration?");
-
-            return isKeyProp0 == 1;
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridQueryProperty parent() {
-            return parent;
-        }
-    }
-
-    /**
      * Descriptor of type.
      */
     private static class TypeDescriptor implements GridQueryTypeDescriptor {
@@ -2636,306 +2310,17 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Identifying TypeDescriptor by space and value class.
-     */
-    private static class TypeId {
-        /** */
-        private final String space;
-
-        /** Value type. */
-        private final Class<?> valType;
-
-        /** Value type ID. */
-        private final int valTypeId;
-
-        /**
-         * Constructor.
-         *
-         * @param space Space name.
-         * @param valType Value type.
-         */
-        private TypeId(String space, Class<?> valType) {
-            assert valType != null;
-
-            this.space = space;
-            this.valType = valType;
-
-            valTypeId = 0;
-        }
-
-        /**
-         * Constructor.
-         *
-         * @param space Space name.
-         * @param valTypeId Value type ID.
-         */
-        private TypeId(String space, int valTypeId) {
-            this.space = space;
-            this.valTypeId = valTypeId;
-
-            valType = null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            TypeId typeId = (TypeId)o;
-
-            return (valTypeId == typeId.valTypeId) &&
-                (valType != null ? valType == typeId.valType : typeId.valType == null) &&
-                (space != null ? space.equals(typeId.space) : typeId.space == null);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return 31 * (space != null ? space.hashCode() : 0) + (valType != null ? valType.hashCode() : valTypeId);
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(TypeId.class, this);
-        }
-    }
-
-    /**
-     *
-     */
-    private static class TypeName {
-        /** */
-        private final String space;
-
-        /** */
-        private final String typeName;
-
-        /**
-         * @param space Space name.
-         * @param typeName Type name.
-         */
-        private TypeName(@Nullable String space, String typeName) {
-            assert !F.isEmpty(typeName) : typeName;
-
-            this.space = space;
-            this.typeName = typeName;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            TypeName other = (TypeName)o;
-
-            return (space != null ? space.equals(other.space) : other.space == null) &&
-                typeName.equals(other.typeName);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return 31 * (space != null ? space.hashCode() : 0) + typeName.hashCode();
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(TypeName.class, this);
-        }
-    }
-
-    /**
      * The way to index.
      */
     private enum IndexType {
         /** Ascending index. */
         ASC,
+
         /** Descending index. */
         DESC,
+
         /** Text index. */
         TEXT
     }
 
-    /** Way of accessing a property - either via field or getter and setter methods. */
-    private interface PropertyAccessor {
-        /**
-         * Get property value from given object.
-         *
-         * @param obj Object to retrieve property value from.
-         * @return Property value.
-         * @throws IgniteCheckedException if failed.
-         */
-        public Object getValue(Object obj) throws IgniteCheckedException;
-
-        /**
-         * Set property value on given object.
-         *
-         * @param obj Object to set property value on.
-         * @param newVal Property value.
-         * @throws IgniteCheckedException if failed.
-         */
-        public void setValue(Object obj, Object newVal)throws IgniteCheckedException;
-
-        /**
-         * @return Name of this property.
-         */
-        public String getPropertyName();
-
-        /**
-         * @return Type of the value of this property.
-         */
-        public Class<?> getType();
-    }
-
-    /** Accessor that deals with fields. */
-    private static final class FieldAccessor implements PropertyAccessor {
-        /** Field to access. */
-        private final Field fld;
-
-        /** */
-        private FieldAccessor(Field fld) {
-            fld.setAccessible(true);
-
-            this.fld = fld;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object getValue(Object obj) throws IgniteCheckedException {
-            try {
-                return fld.get(obj);
-            }
-            catch (Exception e) {
-                throw new IgniteCheckedException("Failed to get field value", e);
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public void setValue(Object obj, Object newVal) throws IgniteCheckedException {
-            try {
-                fld.set(obj, newVal);
-            }
-            catch (Exception e) {
-                throw new IgniteCheckedException("Failed to set field value", e);
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public String getPropertyName() {
-            return fld.getName();
-        }
-
-        /** {@inheritDoc} */
-        @Override public Class<?> getType() {
-            return fld.getType();
-        }
-    }
-
-    /** Getter and setter methods based accessor. */
-    private static final class MethodsAccessor implements PropertyAccessor {
-        /** */
-        private final Method getter;
-
-        /** */
-        private final Method setter;
-
-        /** */
-        private final String propName;
-
-        /**
-         * @param getter Getter method.
-         * @param setter Setter method.
-         * @param propName Property name.
-         */
-        private MethodsAccessor(Method getter, Method setter, String propName) {
-            getter.setAccessible(true);
-            setter.setAccessible(true);
-
-            this.getter = getter;
-            this.setter = setter;
-            this.propName = propName;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object getValue(Object obj) throws IgniteCheckedException {
-            try {
-                return getter.invoke(obj);
-            }
-            catch (Exception e) {
-                throw new IgniteCheckedException("Failed to invoke getter method " +
-                    "[type=" + getType() + ", property=" + propName + ']', e);
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public void setValue(Object obj, Object newVal) throws IgniteCheckedException {
-            try {
-                setter.invoke(obj, newVal);
-            }
-            catch (Exception e) {
-                throw new IgniteCheckedException("Failed to invoke setter method " +
-                    "[type=" + getType() + ", property=" + propName + ']', e);
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public String getPropertyName() {
-            return propName;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Class<?> getType() {
-            return getter.getReturnType();
-        }
-    }
-
-    /** Accessor with getter only. */
-    private static final class ReadOnlyMethodsAccessor implements PropertyAccessor {
-        /** */
-        private final Method getter;
-
-        /** */
-        private final String propName;
-
-        /**
-         * @param getter Getter method.
-         * @param propName Property name.
-         */
-        private ReadOnlyMethodsAccessor(Method getter, String propName) {
-            getter.setAccessible(true);
-
-            this.getter = getter;
-            this.propName = propName;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object getValue(Object obj) throws IgniteCheckedException {
-            try {
-                return getter.invoke(obj);
-            }
-            catch (Exception e) {
-                throw new IgniteCheckedException("Failed to invoke getter method " +
-                    "[type=" + getType() + ", property=" + propName + ']', e);
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public void setValue(Object obj, Object newVal) throws IgniteCheckedException {
-            throw new UnsupportedOperationException("Property is read-only [type=" + getType() +
-                ", property=" + propName + ']');
-        }
-
-        /** {@inheritDoc} */
-        @Override public String getPropertyName() {
-            return propName;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Class<?> getType() {
-            return getter.getReturnType();
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e4c2cfd/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeIdKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeIdKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeIdKey.java
new file mode 100644
index 0000000..4d486f9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeIdKey.java
@@ -0,0 +1,94 @@
+/*
+ * 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.query;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Identifying TypeDescriptor by space and value class.
+ */
+public class QueryTypeIdKey {
+    /** */
+    private final String space;
+
+    /** Value type. */
+    private final Class<?> valType;
+
+    /** Value type ID. */
+    private final int valTypeId;
+
+    /**
+     * Constructor.
+     *
+     * @param space Space name.
+     * @param valType Value type.
+     */
+    public  QueryTypeIdKey(String space, Class<?> valType) {
+        assert valType != null;
+
+        this.space = space;
+        this.valType = valType;
+
+        valTypeId = 0;
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param space Space name.
+     * @param valTypeId Value type ID.
+     */
+    public QueryTypeIdKey(String space, int valTypeId) {
+        this.space = space;
+        this.valTypeId = valTypeId;
+
+        valType = null;
+    }
+
+    /**
+     * @return Space.
+     */
+    public String space() {
+        return space;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        QueryTypeIdKey typeId = (QueryTypeIdKey)o;
+
+        return (valTypeId == typeId.valTypeId) &&
+            (valType != null ? valType == typeId.valType : typeId.valType == null) &&
+            (space != null ? space.equals(typeId.space) : typeId.space == null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return 31 * (space != null ? space.hashCode() : 0) + (valType != null ? valType.hashCode() : valTypeId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(QueryTypeIdKey.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e4c2cfd/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeNameKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeNameKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeNameKey.java
new file mode 100644
index 0000000..8a36a03
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeNameKey.java
@@ -0,0 +1,68 @@
+/*
+ * 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.query;
+
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Query type name key.
+ */
+public class QueryTypeNameKey {
+    /** */
+    private final String space;
+
+    /** */
+    private final String typeName;
+
+    /**
+     * @param space Space name.
+     * @param typeName Type name.
+     */
+    public QueryTypeNameKey(@Nullable String space, String typeName) {
+        assert !F.isEmpty(typeName) : typeName;
+
+        this.space = space;
+        this.typeName = typeName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        QueryTypeNameKey other = (QueryTypeNameKey)o;
+
+        return (space != null ? space.equals(other.space) : other.space == null) &&
+            typeName.equals(other.typeName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return 31 * (space != null ? space.hashCode() : 0) + typeName.hashCode();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(QueryTypeNameKey.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e4c2cfd/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
new file mode 100644
index 0000000..ef914c9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
@@ -0,0 +1,267 @@
+/*
+ * 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.query.property;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.binary.BinaryField;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.binary.BinaryObjectEx;
+import org.apache.ignite.internal.binary.BinaryObjectExImpl;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Binary property.
+ */
+public class QueryBinaryProperty implements GridQueryProperty {
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Property name. */
+    private String propName;
+
+    /** */
+    private String alias;
+
+    /** Parent property. */
+    private QueryBinaryProperty parent;
+
+    /** Result class. */
+    private Class<?> type;
+
+    /** */
+    private volatile int isKeyProp;
+
+    /** Binary field to speed-up deserialization. */
+    private volatile BinaryField field;
+
+    /** Flag indicating that we already tried to take a field. */
+    private volatile boolean fieldTaken;
+
+    /** Whether user was warned about missing property. */
+    private volatile boolean warned;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Kernal context.
+     * @param log Logger.
+     * @param propName Property name.
+     * @param parent Parent property.
+     * @param type Result type.
+     * @param key {@code true} if key property, {@code false} otherwise, {@code null}  if unknown.
+     * @param alias Field alias.
+     */
+    public QueryBinaryProperty(GridKernalContext ctx, IgniteLogger log, String propName, QueryBinaryProperty parent,
+        Class<?> type, @Nullable Boolean key, String alias) {
+        super();
+
+        this.ctx = ctx;
+        this.log = log;
+
+        this.propName = propName;
+        this.alias = F.isEmpty(alias) ? propName : alias;
+        this.parent = parent;
+        this.type = type;
+
+        if (key != null)
+            this.isKeyProp = key ? 1 : -1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object value(Object key, Object val) throws IgniteCheckedException {
+        Object obj;
+
+        if (parent != null) {
+            obj = parent.value(key, val);
+
+            if (obj == null)
+                return null;
+
+            if (!ctx.cacheObjects().isBinaryObject(obj))
+                throw new IgniteCheckedException("Non-binary object received as a result of property extraction " +
+                    "[parent=" + parent + ", propName=" + propName + ", obj=" + obj + ']');
+        }
+        else {
+            int isKeyProp0 = isKeyProp;
+
+            if (isKeyProp0 == 0) {
+                // Key is allowed to be a non-binary object here.
+                // We check key before value consistently with ClassProperty.
+                if (key instanceof BinaryObject && ((BinaryObject)key).hasField(propName))
+                    isKeyProp = isKeyProp0 = 1;
+                else if (val instanceof BinaryObject && ((BinaryObject)val).hasField(propName))
+                    isKeyProp = isKeyProp0 = -1;
+                else {
+                    if (!warned) {
+                        U.warn(log, "Neither key nor value have property \"" + propName + "\" " +
+                            "(is cache indexing configured correctly?)");
+
+                        warned = true;
+                    }
+
+                    return null;
+                }
+            }
+
+            obj = isKeyProp0 == 1 ? key : val;
+        }
+
+        if (obj instanceof BinaryObject) {
+            BinaryObject obj0 = (BinaryObject) obj;
+            return fieldValue(obj0);
+        }
+        else if (obj instanceof BinaryObjectBuilder) {
+            BinaryObjectBuilder obj0 = (BinaryObjectBuilder)obj;
+
+            return obj0.getField(name());
+        }
+        else
+            throw new IgniteCheckedException("Unexpected binary object class [type=" + obj.getClass() + ']');
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setValue(Object key, Object val, Object propVal) throws IgniteCheckedException {
+        Object obj = key() ? key : val;
+
+        if (obj == null)
+            return;
+
+        Object srcObj = obj;
+
+        if (!(srcObj instanceof BinaryObjectBuilder))
+            throw new UnsupportedOperationException("Individual properties can be set for binary builders only");
+
+        if (parent != null)
+            obj = parent.value(key, val);
+
+        boolean needsBuild = false;
+
+        if (obj instanceof BinaryObjectExImpl) {
+            if (parent == null)
+                throw new UnsupportedOperationException("Individual properties can be set for binary builders only");
+
+            needsBuild = true;
+
+            obj = ((BinaryObjectExImpl)obj).toBuilder();
+        }
+
+        if (!(obj instanceof BinaryObjectBuilder))
+            throw new UnsupportedOperationException("Individual properties can be set for binary builders only");
+
+        setValue0((BinaryObjectBuilder) obj, propName, propVal, type());
+
+        if (needsBuild) {
+            obj = ((BinaryObjectBuilder) obj).build();
+
+            assert parent != null;
+
+            // And now let's set this newly constructed object to parent
+            setValue0((BinaryObjectBuilder) srcObj, parent.propName, obj, obj.getClass());
+        }
+    }
+
+    /**
+     * @param builder Object builder.
+     * @param field Field name.
+     * @param val Value to set.
+     * @param valType Type of {@code val}.
+     * @param <T> Value type.
+     */
+    private <T> void setValue0(BinaryObjectBuilder builder, String field, Object val, Class<T> valType) {
+        //noinspection unchecked
+        builder.setField(field, (T)val, valType);
+    }
+
+    /**
+     * Get binary field for the property.
+     *
+     * @param obj Target object.
+     * @return Binary field.
+     */
+    private BinaryField binaryField(BinaryObject obj) {
+        BinaryField field0 = field;
+
+        if (field0 == null && !fieldTaken) {
+            BinaryType type = obj instanceof BinaryObjectEx ? ((BinaryObjectEx)obj).rawType() : obj.type();
+
+            if (type != null) {
+                field0 = type.field(propName);
+
+                assert field0 != null;
+
+                field = field0;
+            }
+
+            fieldTaken = true;
+        }
+
+        return field0;
+    }
+
+    /**
+     * Gets field value for the given binary object.
+     *
+     * @param obj Binary object.
+     * @return Field value.
+     */
+    @SuppressWarnings("IfMayBeConditional")
+    private Object fieldValue(BinaryObject obj) {
+        BinaryField field = binaryField(obj);
+
+        if (field != null)
+            return field.value(obj);
+        else
+            return obj.field(propName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String name() {
+        return alias;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Class<?> type() {
+        return type;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean key() {
+        int isKeyProp0 = isKeyProp;
+
+        if (isKeyProp0 == 0)
+            throw new IllegalStateException("Ownership flag not set for binary property. Have you set 'keyFields'" +
+                " property of QueryEntity in programmatic or XML configuration?");
+
+        return isKeyProp0 == 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridQueryProperty parent() {
+        return parent;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e4c2cfd/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryClassProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryClassProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryClassProperty.java
new file mode 100644
index 0000000..80b9c87
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryClassProperty.java
@@ -0,0 +1,130 @@
+/*
+ * 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.query.property;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Description of type property.
+ */
+public class QueryClassProperty implements GridQueryProperty {
+    /** */
+    private final QueryPropertyAccessor accessor;
+
+    /** */
+    private final boolean key;
+
+    /** */
+    private QueryClassProperty parent;
+
+    /** */
+    private final String name;
+
+    /** */
+    private final CacheObjectContext coCtx;
+
+    /**
+     * Constructor.
+     *
+     * @param accessor Way of accessing the property.
+     */
+    public QueryClassProperty(QueryPropertyAccessor accessor, boolean key, String name,
+        @Nullable CacheObjectContext coCtx) {
+        this.accessor = accessor;
+
+        this.key = key;
+
+        this.name = !F.isEmpty(name) ? name : accessor.getPropertyName();
+
+        this.coCtx = coCtx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object value(Object key, Object val) throws IgniteCheckedException {
+        Object x = unwrap(this.key ? key : val);
+
+        if (parent != null)
+            x = parent.value(key, val);
+
+        if (x == null)
+            return null;
+
+        return accessor.getValue(x);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setValue(Object key, Object val, Object propVal) throws IgniteCheckedException {
+        Object x = unwrap(this.key ? key : val);
+
+        if (parent != null)
+            x = parent.value(key, val);
+
+        if (x == null)
+            return;
+
+        accessor.setValue(x, propVal);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean key() {
+        return key;
+    }
+
+    /**
+     * Unwraps cache object, if needed.
+     *
+     * @param o Object to unwrap.
+     * @return Unwrapped object.
+     */
+    private Object unwrap(Object o) {
+        return coCtx == null ? o : o instanceof CacheObject ? ((CacheObject)o).value(coCtx, false) : o;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String name() {
+        return name;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Class<?> type() {
+        return accessor.getType();
+    }
+
+    /**
+     * @param parent Parent property if this is embeddable element.
+     */
+    public void parent(QueryClassProperty parent) {
+        this.parent = parent;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(QueryClassProperty.class, this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridQueryProperty parent() {
+        return parent;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e4c2cfd/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryFieldAccessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryFieldAccessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryFieldAccessor.java
new file mode 100644
index 0000000..6ff557c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryFieldAccessor.java
@@ -0,0 +1,67 @@
+/*
+ * 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.query.property;
+
+import org.apache.ignite.IgniteCheckedException;
+
+import java.lang.reflect.Field;
+
+/**
+ * Accessor that deals with fields.
+ */
+public class QueryFieldAccessor implements QueryPropertyAccessor {
+    /** Field to access. */
+    private final Field fld;
+
+    /** */
+    public QueryFieldAccessor(Field fld) {
+        fld.setAccessible(true);
+
+        this.fld = fld;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object getValue(Object obj) throws IgniteCheckedException {
+        try {
+            return fld.get(obj);
+        }
+        catch (Exception e) {
+            throw new IgniteCheckedException("Failed to get field value", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setValue(Object obj, Object newVal) throws IgniteCheckedException {
+        try {
+            fld.set(obj, newVal);
+        }
+        catch (Exception e) {
+            throw new IgniteCheckedException("Failed to set field value", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getPropertyName() {
+        return fld.getName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Class<?> getType() {
+        return fld.getType();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e4c2cfd/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryMethodsAccessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryMethodsAccessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryMethodsAccessor.java
new file mode 100644
index 0000000..d5d3062
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryMethodsAccessor.java
@@ -0,0 +1,82 @@
+/*
+ * 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.query.property;
+
+import org.apache.ignite.IgniteCheckedException;
+
+import java.lang.reflect.Method;
+
+/**
+ * Getter and setter methods based accessor.
+ */
+public class QueryMethodsAccessor implements QueryPropertyAccessor {
+    /** */
+    private final Method getter;
+
+    /** */
+    private final Method setter;
+
+    /** */
+    private final String propName;
+
+    /**
+     * @param getter Getter method.
+     * @param setter Setter method.
+     * @param propName Property name.
+     */
+    public QueryMethodsAccessor(Method getter, Method setter, String propName) {
+        getter.setAccessible(true);
+        setter.setAccessible(true);
+
+        this.getter = getter;
+        this.setter = setter;
+        this.propName = propName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object getValue(Object obj) throws IgniteCheckedException {
+        try {
+            return getter.invoke(obj);
+        }
+        catch (Exception e) {
+            throw new IgniteCheckedException("Failed to invoke getter method " +
+                "[type=" + getType() + ", property=" + propName + ']', e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setValue(Object obj, Object newVal) throws IgniteCheckedException {
+        try {
+            setter.invoke(obj, newVal);
+        }
+        catch (Exception e) {
+            throw new IgniteCheckedException("Failed to invoke setter method " +
+                "[type=" + getType() + ", property=" + propName + ']', e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getPropertyName() {
+        return propName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Class<?> getType() {
+        return getter.getReturnType();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e4c2cfd/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryPropertyAccessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryPropertyAccessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryPropertyAccessor.java
new file mode 100644
index 0000000..cd4a444
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryPropertyAccessor.java
@@ -0,0 +1,53 @@
+/*
+ * 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.query.property;
+
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Way of accessing a property - either via field or getter and setter methods.
+ */
+public interface QueryPropertyAccessor {
+    /**
+     * Get property value from given object.
+     *
+     * @param obj Object to retrieve property value from.
+     * @return Property value.
+     * @throws IgniteCheckedException if failed.
+     */
+    public Object getValue(Object obj) throws IgniteCheckedException;
+
+    /**
+     * Set property value on given object.
+     *
+     * @param obj Object to set property value on.
+     * @param newVal Property value.
+     * @throws IgniteCheckedException if failed.
+     */
+    public void setValue(Object obj, Object newVal)throws IgniteCheckedException;
+
+    /**
+     * @return Name of this property.
+     */
+    public String getPropertyName();
+
+    /**
+     * @return Type of the value of this property.
+     */
+    public Class<?> getType();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e4c2cfd/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryReadOnlyMethodsAccessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryReadOnlyMethodsAccessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryReadOnlyMethodsAccessor.java
new file mode 100644
index 0000000..1b17a8a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryReadOnlyMethodsAccessor.java
@@ -0,0 +1,71 @@
+/*
+ * 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.query.property;
+
+import org.apache.ignite.IgniteCheckedException;
+
+import java.lang.reflect.Method;
+
+/**
+ * Accessor with getter only.
+ */
+public class QueryReadOnlyMethodsAccessor implements QueryPropertyAccessor {
+    /** */
+    private final Method getter;
+
+    /** */
+    private final String propName;
+
+    /**
+     * @param getter Getter method.
+     * @param propName Property name.
+     */
+    public QueryReadOnlyMethodsAccessor(Method getter, String propName) {
+        getter.setAccessible(true);
+
+        this.getter = getter;
+        this.propName = propName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object getValue(Object obj) throws IgniteCheckedException {
+        try {
+            return getter.invoke(obj);
+        }
+        catch (Exception e) {
+            throw new IgniteCheckedException("Failed to invoke getter method " +
+                "[type=" + getType() + ", property=" + propName + ']', e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setValue(Object obj, Object newVal) throws IgniteCheckedException {
+        throw new UnsupportedOperationException("Property is read-only [type=" + getType() +
+            ", property=" + propName + ']');
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getPropertyName() {
+        return propName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Class<?> getType() {
+        return getter.getReturnType();
+    }
+}


[8/8] ignite git commit: ignite-4768

Posted by sb...@apache.org.
ignite-4768


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

Branch: refs/heads/ignite-4768
Commit: bc5dbb08143dd6088bc5551da4e830be17e3d4f4
Parents: 850e4dc
Author: sboikov <sb...@gridgain.com>
Authored: Mon Mar 13 11:35:48 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Mar 13 11:35:48 2017 +0300

----------------------------------------------------------------------
 .../distributed/near/GridNearOptimisticTxPrepareFuture.java      | 1 -
 .../distributed/near/GridNearPessimisticTxPrepareFuture.java     | 2 +-
 .../cache/distributed/near/GridNearTxFinishFuture.java           | 4 ++--
 .../internal/processors/cache/transactions/IgniteTxHandler.java  | 1 -
 4 files changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bc5dbb08/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index 4233002..6189b38 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -58,7 +58,6 @@ import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiClosure;
-import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.transactions.TransactionDeadlockException;
 import org.apache.ignite.transactions.TransactionTimeoutException;
 import org.jetbrains.annotations.Nullable;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bc5dbb08/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
index ddee7b7..4a443a9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
@@ -200,7 +200,7 @@ public class GridNearPessimisticTxPrepareFuture extends GridNearTxPrepareFutureA
             if (!cacheCtx.isLocal()) {
                 GridDhtPartitionTopology top = cacheCtx.topology();
 
-                nodes = cacheCtx.topology().nodes(cacheCtx.affinity().partition(txEntry.key()), topVer);
+                nodes = top.nodes(cacheCtx.affinity().partition(txEntry.key()), topVer);
             }
             else
                 nodes = cacheCtx.affinity().nodesByKey(txEntry.key(), topVer);

http://git-wip-us.apache.org/repos/asf/ignite/blob/bc5dbb08/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
index 28f2ce1..7387501 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
@@ -303,7 +303,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
             }
 
             if (initialized() || err != null) {
-                if (tx.needCheckBackup() && !nodeStop) {
+                if (tx.needCheckBackup()) {
                     assert tx.onePhaseCommit();
 
                     if (err != null)
@@ -478,7 +478,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
      *
      */
     private void checkBackup() {
-        assert !hasFutures();
+        assert !hasFutures() : futures();
 
         GridDistributedTxMapping mapping = mappings.singleMapping();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bc5dbb08/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index a3f1356..6daec14 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -76,7 +76,6 @@ import org.apache.ignite.lang.IgniteFutureCancelledException;
 import org.apache.ignite.transactions.TransactionState;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.UTILITY_CACHE_POOL;


[6/8] ignite git commit: ignite-4768

Posted by sb...@apache.org.
ignite-4768


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

Branch: refs/heads/ignite-4768
Commit: e4e825380c960f5d539c73473a827a09052145fa
Parents: 763f1b0
Author: sboikov <sb...@gridgain.com>
Authored: Mon Mar 13 11:10:06 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Mar 13 11:20:04 2017 +0300

----------------------------------------------------------------------
 .../distributed/near/GridNearTxFinishFuture.java   | 17 ++++++++++++-----
 .../cache/distributed/near/GridNearTxLocal.java    |  4 +++-
 .../cache/transactions/IgniteTxLocalAdapter.java   |  4 +++-
 3 files changed, 18 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e4e82538/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
index 3e9069c..28f2ce1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
@@ -29,6 +29,7 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -286,9 +287,14 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
             if (isDone())
                 return false;
 
-            if (err != null)
+            boolean nodeStop = false;
+
+            if (err != null) {
                 tx.setRollbackOnly();
 
+                nodeStop = err instanceof NodeStoppingException;
+            }
+
             if (commit) {
                 if (tx.commitError() != null)
                     err = tx.commitError();
@@ -297,7 +303,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
             }
 
             if (initialized() || err != null) {
-                if (tx.needCheckBackup()) {
+                if (tx.needCheckBackup() && !nodeStop) {
                     assert tx.onePhaseCommit();
 
                     if (err != null)
@@ -317,7 +323,8 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
                 if (tx.onePhaseCommit()) {
                     boolean commit = this.commit && err == null;
 
-                    finishOnePhase(commit);
+                    if (!nodeStop)
+                        finishOnePhase(commit);
 
                     try {
                         tx.tmFinish(commit);
@@ -401,7 +408,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
                         GridDistributedTxMapping mapping = mappings.singleMapping();
 
                         if (mapping != null) {
-                            assert !hasFutures();
+                            assert !hasFutures() : futures();
 
                             finish(1, mapping, commit);
                         }
@@ -652,7 +659,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
      * @param commit Commit flag.
      */
     private void finish(Iterable<GridDistributedTxMapping> mappings, boolean commit) {
-        assert !hasFutures();
+        assert !hasFutures() : futures();
 
         int miniId = 0;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4e82538/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index 1954c34..aab115c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -28,6 +28,7 @@ import javax.cache.expiry.ExpiryPolicy;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
@@ -883,7 +884,8 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
                 catch (IgniteCheckedException e) {
                     COMMIT_ERR_UPD.compareAndSet(GridNearTxLocal.this, null, e);
 
-                    fut0.finish(false);
+                    if (!(e instanceof NodeStoppingException))
+                        fut0.finish(false);
                 }
             }
         });

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4e82538/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index cd4c55c..50197b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -37,6 +37,7 @@ import javax.cache.expiry.ExpiryPolicy;
 import javax.cache.processor.EntryProcessor;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
@@ -3257,7 +3258,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
                             return new GridCacheReturn(cacheCtx, true, keepBinary, res, implicitRes.success());
                         }
                         catch (IgniteCheckedException | RuntimeException e) {
-                            rollbackAsync();
+                            if (!(e instanceof NodeStoppingException))
+                                rollbackAsync();
 
                             throw e;
                         }