You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2017/03/09 13:49:52 UTC

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

Repository: ignite
Updated Branches:
  refs/heads/ignite-4565-ddl 323f77545 -> be88fa26f


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-4565-ddl
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();
+    }
+}


[2/2] ignite git commit: Merge branch 'ignite-2.0' into ignite-4565-ddl

Posted by vo...@apache.org.
Merge branch 'ignite-2.0' into ignite-4565-ddl

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java


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

Branch: refs/heads/ignite-4565-ddl
Commit: be88fa26f86e31c0ef562f9484412dde684327ef
Parents: 323f775 9e4c2cf
Author: devozerov <vo...@gridgain.com>
Authored: Thu Mar 9 16:49:26 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Mar 9 16:49:26 2017 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java    | 754 ++-----------------
 .../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, 902 insertions(+), 684 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/be88fa26/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index ae281b3,b35a506..1e73bbb
--- 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
@@@ -341,12 -324,10 +341,12 @@@ public class GridQueryProcessor extend
                                  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()));
                      }
  
 +                    desc.onInitialStateReady();
 +
                      addTypeByName(ccfg, desc);
                      types.put(typeId, desc);
  
@@@ -436,12 -415,10 +436,12 @@@
                      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()));
                      }
  
 +                    desc.onInitialStateReady();
 +
                      addTypeByName(ccfg, desc);
                      types.put(typeId, desc);
  
@@@ -1501,23 -1458,19 +1501,23 @@@
  
                  LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>> idxFields = entry.getValue();
  
 -                int order = 0;
 +                if (!idxFields.isEmpty()) {
 +                    d.addIndex(idxName, QueryIndexType.SORTED);
  
 -                for (Map.Entry<String, IgniteBiTuple<Class<?>, Boolean>> idxField : idxFields.entrySet()) {
 -                    QueryBinaryProperty prop = buildBinaryProperty(idxField.getKey(), idxField.getValue().get1(), aliases,
 -                        null);
 +                    int order = 0;
  
 -                    d.addProperty(prop, false);
 +                    for (Map.Entry<String, IgniteBiTuple<Class<?>, Boolean>> idxField : idxFields.entrySet()) {
-                         BinaryProperty prop = buildBinaryProperty(idxField.getKey(), idxField.getValue().get1(), aliases,
-                             null);
++                        QueryBinaryProperty prop = buildBinaryProperty(idxField.getKey(), idxField.getValue().get1(),
++                            aliases, null);
  
 -                    Boolean descending = idxField.getValue().get2();
 +                        d.addProperty(prop, false);
  
 -                    d.addFieldToIndex(idxName, prop.name(), order, descending != null && descending);
 +                        Boolean descending = idxField.getValue().get2();
  
 -                    order++;
 +                        d.addFieldToIndex(idxName, prop.name(), order, descending != null && descending);
 +
 +                        order++;
 +                    }
                  }
              }
          }
@@@ -1983,337 -1936,9 +1983,12 @@@
      }
  
      /**
-      * 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 {
 +    private class TypeDescriptor implements GridQueryTypeDescriptor {
 +        /** Space. */
 +        private String space;
 +
          /** */
          private String name;
  
@@@ -2732,181 -2310,6 +2407,68 @@@
      }
  
      /**
 +     * Index state manager.
 +     */
 +    private class IndexStateManager {
 +        /** Indexes. */
 +        private final Map<String, IndexDescriptor> idxs = new ConcurrentHashMap<>();
 +
 +        /** Client futures. */
 +        private final Map<UUID, GridFutureAdapter> cliFuts = new ConcurrentHashMap<>();
 +
 +        /** RW lock. */
 +        private final ReadWriteLock lock = new ReentrantReadWriteLock();
 +
 +        /**
 +         * Handle initial index state.
 +         *
 +         * @param idxs Indexes.
 +         */
 +        public void onInitialStateReady(Map<String, IndexDescriptor> idxs) {
 +            this.idxs.putAll(idxs);
 +        }
 +
 +        /**
 +         * Handle dynamic index creation.
 +         *
 +         * @param idx Index.
 +         * @param ifNotExists IF-NOT-EXISTS flag.
 +         * @return Future completed when index is created.
 +         */
 +        public IgniteInternalFuture<?> onCreateIndex(QueryIndex idx, boolean ifNotExists) {
 +            lock.writeLock().lock();
 +
 +            try {
 +                String idxName = idx.getName() != null ? idx.getName() : QueryEntity.defaultIndexName(idx);
 +
 +                IndexDescriptor oldIdx = idxs.get(idxName);
 +
 +                if (oldIdx != null) {
 +                    if (ifNotExists)
 +                        return new GridFinishedFuture<>();
 +                    else
 +                        return new GridFinishedFuture<>(new IgniteException("Index already exists [idxName=" +
 +                            idxName + ']'));
 +                }
 +
 +                UUID opId = UUID.randomUUID();
 +                GridFutureAdapter fut = new GridFutureAdapter();
 +
 +                GridFutureAdapter oldFut = cliFuts.put(opId, fut);
 +
 +                assert oldFut == null;
 +
 +                // TODO: Start discovery.
 +
 +                return fut;
 +            }
 +            finally {
 +                lock.writeLock().unlock();
 +            }
 +        }
 +    }
 +
 +    /**
-      * 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 {