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 2016/12/05 12:38:10 UTC

[06/52] ignite git commit: IGNITE-2294: Implemented DML.

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
deleted file mode 100644
index 8dcba2f..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
+++ /dev/null
@@ -1,253 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.query;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-import org.apache.ignite.internal.util.tostring.GridToStringInclude;
-import org.apache.ignite.internal.util.typedef.internal.S;
-
-/**
- * Two step map-reduce style query.
- */
-public class GridCacheTwoStepQuery {
-    /** */
-    public static final int DFLT_PAGE_SIZE = 1000;
-
-    /** */
-    @GridToStringInclude
-    private List<GridCacheSqlQuery> mapQrys = new ArrayList<>();
-
-    /** */
-    @GridToStringInclude
-    private GridCacheSqlQuery rdc;
-
-    /** */
-    private int pageSize = DFLT_PAGE_SIZE;
-
-    /** */
-    private boolean explain;
-
-    /** */
-    private Collection<String> spaces;
-
-    /** */
-    private Set<String> schemas;
-
-    /** */
-    private Set<String> tbls;
-
-    /** */
-    private boolean distributedJoins;
-
-    /** */
-    private boolean skipMergeTbl;
-
-    /** */
-    private List<Integer> caches;
-
-    /** */
-    private List<Integer> extraCaches;
-
-    /**
-     * @param schemas Schema names in query.
-     * @param tbls Tables in query.
-     */
-    public GridCacheTwoStepQuery(Set<String> schemas, Set<String> tbls) {
-        this.schemas = schemas;
-        this.tbls = tbls;
-    }
-
-    /**
-     * Specify if distributed joins are enabled for this query.
-     *
-     * @param distributedJoins Distributed joins enabled.
-     */
-    public void distributedJoins(boolean distributedJoins) {
-        this.distributedJoins = distributedJoins;
-    }
-
-    /**
-     * Check if distributed joins are enabled for this query.
-     *
-     * @return {@code true} If distributed joind enabled.
-     */
-    public boolean distributedJoins() {
-        return distributedJoins;
-    }
-
-
-    /**
-     * @return {@code True} if reduce query can skip merge table creation and get data directly from merge index.
-     */
-    public boolean skipMergeTable() {
-        return skipMergeTbl;
-    }
-
-    /**
-     * @param skipMergeTbl Skip merge table.
-     */
-    public void skipMergeTable(boolean skipMergeTbl) {
-        this.skipMergeTbl = skipMergeTbl;
-    }
-
-    /**
-     * @return If this is explain query.
-     */
-    public boolean explain() {
-        return explain;
-    }
-
-    /**
-     * @param explain If this is explain query.
-     */
-    public void explain(boolean explain) {
-        this.explain = explain;
-    }
-
-    /**
-     * @param pageSize Page size.
-     */
-    public void pageSize(int pageSize) {
-        this.pageSize = pageSize;
-    }
-
-    /**
-     * @return Page size.
-     */
-    public int pageSize() {
-        return pageSize;
-    }
-
-    /**
-     * @param qry SQL Query.
-     * @return {@code this}.
-     */
-    public GridCacheTwoStepQuery addMapQuery(GridCacheSqlQuery qry) {
-        mapQrys.add(qry);
-
-        return this;
-    }
-
-    /**
-     * @return Reduce query.
-     */
-    public GridCacheSqlQuery reduceQuery() {
-        return rdc;
-    }
-
-    /**
-     * @param rdc Reduce query.
-     */
-    public void reduceQuery(GridCacheSqlQuery rdc) {
-        this.rdc = rdc;
-    }
-
-    /**
-     * @return Map queries.
-     */
-    public List<GridCacheSqlQuery> mapQueries() {
-        return mapQrys;
-    }
-
-    /**
-     * @return Caches.
-     */
-    public List<Integer> caches() {
-        return caches;
-    }
-
-    /**
-     * @param caches Caches.
-     */
-    public void caches(List<Integer> caches) {
-        this.caches = caches;
-    }
-
-    /**
-     * @return Caches.
-     */
-    public List<Integer> extraCaches() {
-        return extraCaches;
-    }
-
-    /**
-     * @param extraCaches Caches.
-     */
-    public void extraCaches(List<Integer> extraCaches) {
-        this.extraCaches = extraCaches;
-    }
-
-    /**
-     * @return Spaces.
-     */
-    public Collection<String> spaces() {
-        return spaces;
-    }
-
-    /**
-     * @param spaces Spaces.
-     */
-    public void spaces(Collection<String> spaces) {
-        this.spaces = spaces;
-    }
-
-    /**
-     * @return Schemas.
-     */
-    public Set<String> schemas() {
-        return schemas;
-    }
-
-    /**
-     * @param args New arguments to copy with.
-     * @return Copy.
-     */
-    public GridCacheTwoStepQuery copy(Object[] args) {
-        assert !explain;
-
-        GridCacheTwoStepQuery cp = new GridCacheTwoStepQuery(schemas, tbls);
-
-        cp.caches = caches;
-        cp.extraCaches = extraCaches;
-        cp.spaces = spaces;
-        cp.rdc = rdc.copy(args);
-        cp.skipMergeTbl = skipMergeTbl;
-        cp.pageSize = pageSize;
-        cp.distributedJoins = distributedJoins;
-
-        for (int i = 0; i < mapQrys.size(); i++)
-            cp.mapQrys.add(mapQrys.get(i).copy(args));
-
-        return cp;
-    }
-
-    /**
-     * @return Tables.
-     */
-    public Set<String> tables() {
-        return tbls;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridCacheTwoStepQuery.class, this);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java
new file mode 100644
index 0000000..93b8d47
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query;
+
+import java.sql.SQLException;
+import javax.cache.processor.EntryProcessor;
+import org.apache.ignite.IgniteCache;
+
+/**
+ * Error codes for query operations.
+ */
+public final class IgniteQueryErrorCode {
+    /** Unknown error, or the one without specific code. */
+    public final static int UNKNOWN = 1;
+
+    /* 1xxx - parsing errors */
+
+    /** General parsing error - for the cases when there's no more specific code available. */
+    public final static int PARSING = 1001;
+
+    /** Code encountered unexpected type of SQL operation - like {@code EXPLAIN MERGE}. */
+    public final static int UNSUPPORTED_OPERATION = 1002;
+
+    /* 2xxx - analysis errors */
+
+    /** Code encountered SQL statement of some type that it did not expect in current analysis context. */
+    public final static int UNEXPECTED_OPERATION = 2001;
+
+    /** Code encountered SQL expression of some type that it did not expect in current analysis context. */
+    public final static int UNEXPECTED_ELEMENT_TYPE = 2002;
+
+    /** Analysis detected that the statement is trying to directly {@code UPDATE} key or its fields. */
+    public final static int KEY_UPDATE = 2003;
+
+    /* 3xxx - database API related runtime errors */
+
+    /** Required table not found. */
+    public final static int TABLE_NOT_FOUND = 3001;
+
+    /** Required table does not have a descriptor set. */
+    public final static int NULL_TABLE_DESCRIPTOR = 3002;
+
+    /** Statement type does not match that declared by JDBC driver. */
+    public final static int STMT_TYPE_MISMATCH = 3003;
+
+    /** Statement type does not match that declared by JDBC driver. */
+    public final static int TABLE_DROP_FAILED = 3004;
+
+    /* 4xxx - cache related runtime errors */
+
+    /** Attempt to INSERT a key that is already in cache. */
+    public final static int DUPLICATE_KEY = 4001;
+
+    /** Attempt to UPDATE or DELETE a key whose value has been updated concurrently by someone else. */
+    public final static int CONCURRENT_UPDATE = 4002;
+
+    /** Attempt to INSERT or MERGE {@code null} key. */
+    public final static int NULL_KEY = 4003;
+
+    /** Attempt to INSERT or MERGE {@code null} value. */
+    public final static int NULL_VALUE = 4004;
+
+    /** {@link EntryProcessor} has thrown an exception during {@link IgniteCache#invokeAll}. */
+    public final static int ENTRY_PROCESSING = 4005;
+
+    /**
+     * Create a {@link SQLException} for given code and message with null state.
+     *
+     * @param msg Message.
+     * @param code Ignite status code.
+     * @return {@link SQLException} with given details.
+     */
+    public static SQLException createJdbcSqlException(String msg, int code) {
+        return new SQLException(msg, null, code);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java
index 5e26905..859de39 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformInputStreamImpl.java
@@ -259,6 +259,11 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
     }
 
     /** {@inheritDoc} */
+    @Override public int capacity() {
+        return len;
+    }
+
+    /** {@inheritDoc} */
     @Override public int position() {
         return pos;
     }
@@ -293,6 +298,11 @@ public class PlatformInputStreamImpl implements PlatformInputStream {
     }
 
     /** {@inheritDoc} */
+    @Override public long rawOffheapPointer() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean hasArray() {
         return false;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStreamImpl.java
index cb30336..1ece10b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/memory/PlatformOutputStreamImpl.java
@@ -208,6 +208,13 @@ public class PlatformOutputStreamImpl implements PlatformOutputStream {
     }
 
     /** {@inheritDoc} */
+    @Override public long rawOffheapPointer() {
+        assert false;
+
+        throw new UnsupportedOperationException("Should not be called.");
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean hasArray() {
         assert false;
 
@@ -331,4 +338,9 @@ public class PlatformOutputStreamImpl implements PlatformOutputStream {
 
         shift(len);
     }
+
+    /** {@inheritDoc} */
+    public int capacity() {
+        return cap;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index 6bffa5d..13c1b3d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.query;
 
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
 import java.util.Collection;
 import java.util.List;
 import javax.cache.Cache;
@@ -61,9 +63,12 @@ public interface GridQueryIndexing {
      *
      * @param cctx Cache context.
      * @param qry Query.
+     * @param cancel Query cancel.
      * @return Cursor.
+     * @throws IgniteCheckedException If failed.
      */
-    public QueryCursor<List<?>> queryTwoStep(GridCacheContext<?,?> cctx, SqlFieldsQuery qry);
+    public QueryCursor<List<?>> queryTwoStep(GridCacheContext<?, ?> cctx, SqlFieldsQuery qry, GridQueryCancel cancel)
+        throws IgniteCheckedException;
 
     /**
      * Parses SQL query into two step query and executes it.
@@ -71,8 +76,10 @@ public interface GridQueryIndexing {
      * @param cctx Cache context.
      * @param qry Query.
      * @return Cursor.
+     * @throws IgniteCheckedException If failed.
      */
-    public <K,V> QueryCursor<Cache.Entry<K,V>> queryTwoStep(GridCacheContext<?,?> cctx, SqlQuery qry);
+    public <K,V> QueryCursor<Cache.Entry<K,V>> queryTwoStep(GridCacheContext<?,?> cctx, SqlQuery qry)
+        throws IgniteCheckedException;
 
     /**
      * Queries individual fields (generally used by JDBC drivers).
@@ -88,7 +95,8 @@ public interface GridQueryIndexing {
      * @throws IgniteCheckedException If failed.
      */
     public GridQueryFieldsResult queryLocalSqlFields(@Nullable String spaceName, String qry,
-        Collection<Object> params, IndexingQueryFilter filter, boolean enforceJoinOrder, int timeout, GridQueryCancel cancel) throws IgniteCheckedException;
+        Collection<Object> params, IndexingQueryFilter filter, boolean enforceJoinOrder, int timeout,
+        GridQueryCancel cancel) throws IgniteCheckedException;
 
     /**
      * Executes regular query.
@@ -222,6 +230,15 @@ public interface GridQueryIndexing {
     public void onDisconnected(IgniteFuture<?> reconnectFut);
 
     /**
+     * Prepare native statement to retrieve JDBC metadata from.
+     *
+     * @param schema Schema.
+     * @param sql Query.
+     * @return {@link PreparedStatement} from underlying engine to supply metadata to Prepared - most likely H2.
+     */
+    public PreparedStatement prepareNativeStatement(String schema, String sql) throws SQLException;
+
+    /**
      * Cancels all executing queries.
      */
     public void cancelAllQueries();

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/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 8befa0e..fe773c1 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
@@ -17,10 +17,10 @@
 
 package org.apache.ignite.internal.processors.query;
 
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
 import java.util.concurrent.TimeUnit;
-import java.lang.reflect.AccessibleObject;
 import java.lang.reflect.Field;
-import java.lang.reflect.Member;
 import java.lang.reflect.Method;
 import java.math.BigDecimal;
 import java.sql.Time;
@@ -47,6 +47,7 @@ import org.apache.ignite.IgniteException;
 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.binary.Binarylizable;
 import org.apache.ignite.cache.CacheTypeMetadata;
@@ -279,6 +280,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         desc.keyClass(keyCls);
                     }
 
+                    desc.keyTypeName(qryEntity.getKeyType());
+                    desc.valueTypeName(qryEntity.getValueType());
+
                     if (binaryEnabled && keyOrValMustDeserialize) {
                         if (mustDeserializeClss == null)
                             mustDeserializeClss = new ArrayList<>();
@@ -385,6 +389,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         desc.keyClass(keyCls);
                     }
 
+                    desc.keyTypeName(meta.getKeyType());
+                    desc.valueTypeName(meta.getValueType());
+
                     if (binaryEnabled && keyOrValMustDeserialize) {
                         if (mustDeserializeClss == null)
                             mustDeserializeClss = new ArrayList<>();
@@ -802,8 +809,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         try {
             return executeQuery(GridCacheQueryType.SQL_FIELDS, qry.getSql(), cctx, new IgniteOutClosureX<QueryCursor<List<?>>>() {
-                @Override public QueryCursor<List<?>> applyx() {
-                    return idx.queryTwoStep(cctx, qry);
+                @Override public QueryCursor<List<?>> applyx() throws IgniteCheckedException {
+                    return idx.queryTwoStep(cctx, qry, null);
                 }
             }, true);
         }
@@ -938,6 +945,18 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     *
+     * @param schema Schema.
+     * @param sql Query.
+     * @return {@link PreparedStatement} from underlying engine to supply metadata to Prepared - most likely H2.
+     */
+    public PreparedStatement prepareNativeStatement(String schema, String sql) throws SQLException {
+        checkxEnabled();
+
+        return idx.prepareNativeStatement(schema, sql);
+    }
+
+    /**
      * @param timeout Timeout.
      * @param timeUnit Time unit.
      * @return Converted time.
@@ -967,7 +986,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param qry Query.
      * @return Iterator.
      */
-    public QueryCursor<List<?>> queryLocalFields(final GridCacheContext<?,?> cctx, final SqlFieldsQuery qry) {
+    public QueryCursor<List<?>> queryLocalFields(final GridCacheContext<?, ?> cctx, final SqlFieldsQuery qry) {
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
@@ -1406,7 +1425,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);
+            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null, d);
 
             d.addProperty(prop, false);
 
@@ -1418,7 +1437,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (Map.Entry<String, Class<?>> entry : meta.getDescendingFields().entrySet()) {
-            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases);
+            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null, d);
 
             d.addProperty(prop, false);
 
@@ -1430,7 +1449,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (String txtIdx : meta.getTextFields()) {
-            BinaryProperty prop = buildBinaryProperty(txtIdx, String.class, aliases);
+            BinaryProperty prop = buildBinaryProperty(txtIdx, String.class, aliases, null, d);
 
             d.addProperty(prop, false);
 
@@ -1448,7 +1467,8 @@ 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);
+                    BinaryProperty prop = buildBinaryProperty(idxField.getKey(), idxField.getValue().get1(), aliases,
+                        null, d);
 
                     d.addProperty(prop, false);
 
@@ -1462,7 +1482,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
-            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases);
+            BinaryProperty prop = buildBinaryProperty(entry.getKey(), entry.getValue(), aliases, null, d);
 
             if (!d.props.containsKey(prop.name()))
                 d.addProperty(prop, false);
@@ -1482,8 +1502,14 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         if (aliases == null)
             aliases = Collections.emptyMap();
 
+        Set<String> keyFields = qryEntity.getKeyFields();
+        boolean hasKeyFields = (keyFields != null);
+
         for (Map.Entry<String, String> entry : qryEntity.getFields().entrySet()) {
-            BinaryProperty prop = buildBinaryProperty(entry.getKey(), U.classForName(entry.getValue(), Object.class, true), aliases);
+            Boolean isKeyField = (hasKeyFields ? keyFields.contains(entry.getKey()) : null);
+
+            BinaryProperty prop = buildBinaryProperty(entry.getKey(), U.classForName(entry.getValue(), Object.class, true),
+                aliases, isKeyField, d);
 
             d.addProperty(prop, false);
         }
@@ -1583,9 +1609,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      *      nested fields.
      * @param resType Result type.
      * @param aliases Aliases.
-     * @return Binary property.
+     * @param isKeyField
+     *@param d Type descriptor.  @return Binary property.
      */
-    private BinaryProperty buildBinaryProperty(String pathStr, Class<?> resType, Map<String,String> aliases) {
+    private BinaryProperty buildBinaryProperty(String pathStr, Class<?> resType, Map<String, String> aliases,
+        @Nullable Boolean isKeyField, TypeDescriptor d) throws IgniteCheckedException {
+
         String[] path = pathStr.split("\\.");
 
         BinaryProperty res = null;
@@ -1600,7 +1629,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             String alias = aliases.get(fullName.toString());
 
-            res = new BinaryProperty(prop, res, resType, alias);
+            // The key flag that we've found out is valid for the whole path.
+            res = new BinaryProperty(prop, res, resType, isKeyField, alias);
         }
 
         return res;
@@ -1629,8 +1659,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             res = buildClassProperty(false, valCls, pathStr, resType, aliases, coCtx);
 
         if (res == null)
-            throw new IgniteCheckedException("Failed to initialize property '" + pathStr + "' for " +
-                "key class '" + keyCls + "' and value class '" + valCls + "'. " +
+            throw new IgniteCheckedException("Failed to initialize property '" + pathStr + "' of type '" +
+                resType.getName() + "' for key class '" + keyCls + "' and value class '" + valCls + "'. " +
                 "Make sure that one of these classes contains respective getter method or field.");
 
         return res;
@@ -1644,7 +1674,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param aliases Aliases.
      * @return Property instance corresponding to the given path.
      */
-    static ClassProperty buildClassProperty(boolean key, Class<?> cls, String pathStr, Class<?> resType,
+    private static ClassProperty buildClassProperty(boolean key, Class<?> cls, String pathStr, Class<?> resType,
         Map<String,String> aliases, CacheObjectContext coCtx) {
         String[] path = pathStr.split("\\.");
 
@@ -1660,58 +1690,13 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             String alias = aliases.get(fullName.toString());
 
-            StringBuilder bld = new StringBuilder("get");
-
-            bld.append(prop);
-
-            bld.setCharAt(3, Character.toUpperCase(bld.charAt(3)));
-
-            ClassProperty tmp = null;
-
-            try {
-                tmp = new ClassProperty(cls.getMethod(bld.toString()), key, alias, coCtx);
-            }
-            catch (NoSuchMethodException ignore) {
-                // No-op.
-            }
-
-            if (tmp == null) { // Boolean getter can be defined as is###().
-                bld = new StringBuilder("is");
-
-                bld.append(prop);
-
-                bld.setCharAt(2, Character.toUpperCase(bld.charAt(2)));
-
-                try {
-                    tmp = new ClassProperty(cls.getMethod(bld.toString()), key, alias, coCtx);
-                }
-                catch (NoSuchMethodException ignore) {
-                    // No-op.
-                }
-            }
-
-            Class cls0 = cls;
-
-            while (tmp == null && cls0 != null)
-                try {
-                    tmp = new ClassProperty(cls0.getDeclaredField(prop), key, alias, coCtx);
-                }
-                catch (NoSuchFieldException ignored) {
-                    cls0 = cls0.getSuperclass();
-                }
-
-            if (tmp == null) {
-                try {
-                    tmp = new ClassProperty(cls.getMethod(prop), key, alias, coCtx);
-                }
-                catch (NoSuchMethodException ignored) {
-                    // No-op.
-                }
-            }
+            PropertyAccessor accessor = findProperty(prop, cls);
 
-            if (tmp == null)
+            if (accessor == null)
                 return null;
 
+            ClassProperty tmp = new ClassProperty(accessor, key, alias, coCtx);
+
             tmp.parent(res);
 
             cls = tmp.type();
@@ -1819,6 +1804,104 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Find a member (either a getter method or a field) with given name of given class.
+     * @param prop Property name.
+     * @param cls Class to search for a member in.
+     * @return Member for given name.
+     */
+    @Nullable private static PropertyAccessor findProperty(String prop, Class<?> cls) {
+        StringBuilder getBldr = new StringBuilder("get");
+        getBldr.append(prop);
+        getBldr.setCharAt(3, Character.toUpperCase(getBldr.charAt(3)));
+
+        StringBuilder setBldr = new StringBuilder("set");
+        setBldr.append(prop);
+        setBldr.setCharAt(3, Character.toUpperCase(setBldr.charAt(3)));
+
+        try {
+            Method getter = cls.getMethod(getBldr.toString());
+
+            Method setter;
+
+            try {
+                // Setter has to have the same name like 'setXxx' and single param of the same type
+                // as the return type of the getter.
+                setter = cls.getMethod(setBldr.toString(), getter.getReturnType());
+            }
+            catch (NoSuchMethodException ignore) {
+                // Have getter, but no setter - return read-only accessor.
+                return new ReadOnlyMethodsAccessor(getter, prop);
+            }
+
+            return new MethodsAccessor(getter, setter, prop);
+        }
+        catch (NoSuchMethodException ignore) {
+            // No-op.
+        }
+
+        getBldr = new StringBuilder("is");
+        getBldr.append(prop);
+        getBldr.setCharAt(2, Character.toUpperCase(getBldr.charAt(2)));
+
+        // We do nothing about setBldr here as it corresponds to setProperty name which is what we need
+        // for boolean property setter as well
+        try {
+            Method getter = cls.getMethod(getBldr.toString());
+
+            Method setter;
+
+            try {
+                // Setter has to have the same name like 'setXxx' and single param of the same type
+                // as the return type of the getter.
+                setter = cls.getMethod(setBldr.toString(), getter.getReturnType());
+            }
+            catch (NoSuchMethodException ignore) {
+                // Have getter, but no setter - return read-only accessor.
+                return new ReadOnlyMethodsAccessor(getter, prop);
+            }
+
+            return new MethodsAccessor(getter, setter, prop);
+        }
+        catch (NoSuchMethodException ignore) {
+            // No-op.
+        }
+
+        Class cls0 = cls;
+
+        while (cls0 != null)
+            try {
+                return new FieldAccessor(cls0.getDeclaredField(prop));
+            }
+            catch (NoSuchFieldException ignored) {
+                cls0 = cls0.getSuperclass();
+            }
+
+        try {
+            Method getter = cls.getMethod(prop);
+
+            Method setter;
+
+            try {
+                // Setter has to have the same name and single param of the same type
+                // as the return type of the getter.
+                setter = cls.getMethod(prop, getter.getReturnType());
+            }
+            catch (NoSuchMethodException ignore) {
+                // Have getter, but no setter - return read-only accessor.
+                return new ReadOnlyMethodsAccessor(getter, prop);
+            }
+
+            return new MethodsAccessor(getter, setter, prop);
+        }
+        catch (NoSuchMethodException ignored) {
+            // No-op.
+        }
+
+        // No luck.
+        return null;
+    }
+
+    /**
      * @param ver Version.
      */
     public static void setRequestAffinityTopologyVersion(AffinityTopologyVersion ver) {
@@ -1837,39 +1920,31 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      */
     private static class ClassProperty extends GridQueryProperty {
         /** */
-        private final Member member;
+        private final PropertyAccessor accessor;
 
         /** */
-        private ClassProperty parent;
+        private final boolean key;
 
         /** */
-        private String name;
-
-        /** */
-        private boolean field;
+        private ClassProperty parent;
 
         /** */
-        private boolean key;
+        private final String name;
 
         /** */
-        private CacheObjectContext coCtx;
+        private final CacheObjectContext coCtx;
 
         /**
          * Constructor.
          *
-         * @param member Element.
+         * @param accessor Way of accessing the property.
          */
-        ClassProperty(Member member, boolean key, String name, @Nullable CacheObjectContext coCtx) {
-            this.member = member;
-            this.key = key;
-
-            this.name = !F.isEmpty(name) ? name :
-                member instanceof Method && member.getName().startsWith("get") && member.getName().length() > 3 ?
-                member.getName().substring(3) : member.getName();
+        ClassProperty(PropertyAccessor accessor, boolean key, String name, @Nullable CacheObjectContext coCtx) {
+            this.accessor = accessor;
 
-            ((AccessibleObject) member).setAccessible(true);
+            this.key = key;
 
-            field = member instanceof Field;
+            this.name = !F.isEmpty(name) ? name : accessor.getPropertyName();
 
             this.coCtx = coCtx;
         }
@@ -1884,21 +1959,25 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             if (x == null)
                 return null;
 
-            try {
-                if (field) {
-                    Field field = (Field)member;
+            return accessor.getValue(x);
+        }
 
-                    return field.get(x);
-                }
-                else {
-                    Method mtd = (Method)member;
+        /** {@inheritDoc} */
+        @Override public void setValue(Object key, Object val, Object propVal) throws IgniteCheckedException {
+            Object x = unwrap(this.key ? key : val);
 
-                    return mtd.invoke(x);
-                }
-            }
-            catch (Exception e) {
-                throw new IgniteCheckedException(e);
-            }
+            if (parent != null)
+                x = parent.value(key, val);
+
+            if (x == null)
+                return;
+
+            accessor.setValue(x, propVal);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean key() {
+            return key;
         }
 
         /**
@@ -1918,7 +1997,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public Class<?> type() {
-            return member instanceof Field ? ((Field)member).getType() : ((Method)member).getReturnType();
+            return accessor.getType();
         }
 
         /**
@@ -1932,14 +2011,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         @Override public String toString() {
             return S.toString(ClassProperty.class, this);
         }
-
-        /**
-         * @param cls Class.
-         * @return {@code true} If this property or some parent relates to member of the given class.
-         */
-        public boolean knowsClass(Class<?> cls) {
-            return member.getDeclaringClass() == cls || (parent != null && parent.knowsClass(cls));
-        }
     }
 
     /**
@@ -1976,12 +2047,18 @@ public class GridQueryProcessor extends GridProcessorAdapter {
          * @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, String 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} */
@@ -2030,6 +2107,31 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             return fieldValue(obj0);
         }
 
+        /** {@inheritDoc} */
+        @Override public void setValue(Object key, Object val, Object propVal) throws IgniteCheckedException {
+            Object obj = key() ? key : val;
+
+            if (obj == null)
+                return;
+
+            if (!(obj instanceof BinaryObjectBuilder))
+                throw new UnsupportedOperationException("Individual properties can be set for binary builders only");
+
+            setValue0((BinaryObjectBuilder) obj, name(), propVal, type());
+        }
+
+        /**
+         * @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.
          *
@@ -2081,6 +2183,17 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         @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 configuration XML?");
+
+            return isKeyProp0 == 1;
+        }
     }
 
     /**
@@ -2098,6 +2211,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         @GridToStringExclude
         private final Map<String, GridQueryProperty> props = new HashMap<>();
 
+        /** Map with upper cased property names to help find properties based on SQL INSERT and MERGE queries. */
+        private final Map<String, GridQueryProperty> uppercaseProps = new HashMap<>();
+
         /** */
         @GridToStringInclude
         private final Map<String, IndexDescriptor> indexes = new HashMap<>();
@@ -2112,6 +2228,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         private Class<?> valCls;
 
         /** */
+        private String keyTypeName;
+
+        /** */
+        private String valTypeName;
+
+        /** */
         private boolean valTextIdx;
 
         /** */
@@ -2155,7 +2277,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public GridQueryProperty property(String name) {
-            return props.get(name);
+            return getProperty(name);
         }
 
         /** {@inheritDoc} */
@@ -2163,7 +2285,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         @Override public <T> T value(String field, Object key, Object val) throws IgniteCheckedException {
             assert field != null;
 
-            GridQueryProperty prop = props.get(field);
+            GridQueryProperty prop = getProperty(field);
 
             if (prop == null)
                 throw new IgniteCheckedException("Failed to find field '" + field + "' in type '" + name + "'.");
@@ -2172,6 +2294,20 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
+        @Override public void setValue(String field, Object key, Object val, Object propVal)
+            throws IgniteCheckedException {
+            assert field != null;
+
+            GridQueryProperty prop = getProperty(field);
+
+            if (prop == null)
+                throw new IgniteCheckedException("Failed to find field '" + field + "' in type '" + name + "'.");
+
+            prop.setValue(key, val, propVal);
+        }
+
+        /** {@inheritDoc} */
         @Override public Map<String, GridQueryIndexDescriptor> indexes() {
             return Collections.<String, GridQueryIndexDescriptor>unmodifiableMap(indexes);
         }
@@ -2238,6 +2374,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
          * @param valCls Value class.
          */
         void valueClass(Class<?> valCls) {
+            A.notNull(valCls, "Value class must not be null");
             this.valCls = valCls;
         }
 
@@ -2255,6 +2392,34 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             this.keyCls = keyCls;
         }
 
+        /** {@inheritDoc} */
+        @Override public String keyTypeName() {
+            return keyTypeName;
+        }
+
+        /**
+         * Set key type name.
+         *
+         * @param keyTypeName Key type name.
+         */
+        public void keyTypeName(String keyTypeName) {
+            this.keyTypeName = keyTypeName;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String valueTypeName() {
+            return valTypeName;
+        }
+
+        /**
+         * Set value type name.
+         *
+         * @param valTypeName Value type name.
+         */
+        public void valueTypeName(String valTypeName) {
+            this.valTypeName = valTypeName;
+        }
+
         /**
          * Adds property to the type descriptor.
          *
@@ -2268,9 +2433,25 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             if (props.put(name, prop) != null && failOnDuplicate)
                 throw new IgniteCheckedException("Property with name '" + name + "' already exists.");
 
+            if (uppercaseProps.put(name.toUpperCase(), prop) != null && failOnDuplicate)
+                throw new IgniteCheckedException("Property with upper cased name '" + name + "' already exists.");
+
             fields.put(name, prop.type());
         }
 
+        /**
+         * @param field Property name.
+         * @return Property with given field name.
+         */
+        private GridQueryProperty getProperty(String field) {
+            GridQueryProperty res = props.get(field);
+
+            if (res == null)
+                res = uppercaseProps.get(field.toUpperCase());
+
+            return res;
+        }
+
         /** {@inheritDoc} */
         @Override public boolean valueTextIndex() {
             return valTextIdx;
@@ -2501,4 +2682,183 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         /** 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 final static 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 final static 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 final static 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/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
index d623d25..5d74a2e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
@@ -23,6 +23,10 @@ import org.apache.ignite.IgniteCheckedException;
  * Description and access method for query entity field.
  */
 public abstract class GridQueryProperty {
+    /** */
+    public GridQueryProperty() {
+    }
+
     /**
      * Gets this property value from the given object.
      *
@@ -34,6 +38,16 @@ public abstract class GridQueryProperty {
     public abstract Object value(Object key, Object val) throws IgniteCheckedException;
 
     /**
+     * Sets this property value for the given object.
+     *
+     * @param key Key.
+     * @param val Value.
+     * @param propVal Property value.
+     * @throws IgniteCheckedException If failed.
+     */
+    public abstract void setValue(Object key, Object val, Object propVal) throws IgniteCheckedException;
+
+    /**
      * @return Property name.
      */
     public abstract String name();
@@ -42,4 +56,10 @@ public abstract class GridQueryProperty {
      * @return Class member type.
      */
     public abstract Class<?> type();
+
+    /**
+     * Property ownership flag.
+     * @return {@code true} if this property belongs to key, {@code false} if it belongs to value.
+     */
+    public abstract boolean key();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
index b636841..dcc01af 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
@@ -50,6 +50,17 @@ public interface GridQueryTypeDescriptor {
     public <T> T value(String field, Object key, Object val) throws IgniteCheckedException;
 
     /**
+     * Sets field value for given key and value.
+     *
+     * @param field Field name.
+     * @param key Key.
+     * @param val Value.
+     * @param propVal Value for given field.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void setValue(String field, Object key, Object val, Object propVal) throws IgniteCheckedException;
+
+    /**
      * @param name Property name.
      * @return Property.
      */
@@ -77,6 +88,20 @@ public interface GridQueryTypeDescriptor {
     public Class<?> keyClass();
 
     /**
+     * Gets key type name.
+     *
+     * @return Key type name.
+     */
+    public String keyTypeName();
+
+    /**
+     * Gets value type name.
+     *
+     * @return Value type name.
+     */
+    public String valueTypeName();
+
+    /**
      * Returns {@code true} if string representation of value should be indexed as text.
      *
      * @return If string representation of value should be full-text indexed.

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/processors/query/IgniteSQLException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/IgniteSQLException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/IgniteSQLException.java
new file mode 100644
index 0000000..b15007e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/IgniteSQLException.java
@@ -0,0 +1,89 @@
+/*
+ * 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 java.sql.SQLException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Specific exception bearing information about query processing errors for more detailed
+ * errors in JDBC driver.
+ *
+ * @see IgniteQueryErrorCode
+ */
+public class IgniteSQLException extends IgniteException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** State to return as {@link SQLException#SQLState} */
+    private final String sqlState;
+
+    /** Code to return as {@link SQLException#vendorCode} */
+    private final int statusCode;
+
+    /** */
+    public IgniteSQLException(String msg) {
+        this(msg, null, 0);
+    }
+
+    /**
+     * Minimalistic ctor accepting only {@link SQLException} as the cause.
+     */
+    public IgniteSQLException(SQLException cause) {
+        super(cause);
+        this.sqlState = null;
+        this.statusCode = 0;
+    }
+
+    /** */
+    public IgniteSQLException(String msg, @Nullable Throwable cause) {
+        super(msg, cause);
+        this.sqlState = null;
+        this.statusCode = 0;
+    }
+
+    /** */
+    public IgniteSQLException(String msg, int statusCode, @Nullable Throwable cause) {
+        super(msg, cause);
+        this.sqlState = null;
+        this.statusCode = statusCode;
+    }
+
+    /** */
+    public IgniteSQLException(String msg, String sqlState, int statusCode) {
+        super(msg);
+        this.sqlState = sqlState;
+        this.statusCode = statusCode;
+    }
+
+    /** */
+    public IgniteSQLException(String msg, int statusCode) {
+        super(msg);
+        this.sqlState = null;
+        this.statusCode = statusCode;
+    }
+
+    /**
+     * @return JDBC exception containing details from this instance.
+     */
+    public SQLException toJdbcException() {
+        return new SQLException(getMessage(), sqlState, statusCode, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteSingletonIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteSingletonIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteSingletonIterator.java
new file mode 100644
index 0000000..0b0561b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/IgniteSingletonIterator.java
@@ -0,0 +1,56 @@
+/*
+ * 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.util.lang;
+
+import java.util.NoSuchElementException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
+
+/**
+ * Trivial iterator to return single item.
+ */
+public class IgniteSingletonIterator<T> extends GridCloseableIteratorAdapter<T> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final T val;
+
+    /** */
+    private boolean hasNext = true;
+
+    /** */
+    public IgniteSingletonIterator(T val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean onHasNext() throws IgniteCheckedException {
+        return hasNext;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected T onNext() throws IgniteCheckedException {
+        if (!hasNext)
+            throw new NoSuchElementException();
+
+        hasNext = false;
+
+        return val;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryArrayIdentityResolverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryArrayIdentityResolverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryArrayIdentityResolverSelfTest.java
new file mode 100644
index 0000000..4a16662
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryArrayIdentityResolverSelfTest.java
@@ -0,0 +1,300 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.binary.BinaryArrayIdentityResolver;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.eclipse.jetty.util.ConcurrentHashSet;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.assertNotEquals;
+
+/**
+ * Array identity resolver self test.
+ */
+public class BinaryArrayIdentityResolverSelfTest extends GridCommonAbstractTest {
+    /** Pointers to release. */
+    private final Set<Long> ptrs = new ConcurrentHashSet<>();
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrid();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        for (Long ptr : ptrs)
+            GridUnsafe.freeMemory(ptr);
+
+        super.afterTest();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg =  super.getConfiguration(gridName);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        BinaryConfiguration binCfg = new BinaryConfiguration();
+
+        BinaryTypeConfiguration binTypCfg1 = new BinaryTypeConfiguration();
+        BinaryTypeConfiguration binTypCfg2 = new BinaryTypeConfiguration();
+
+        binTypCfg1.setTypeName(InnerClass.class.getName());
+        binTypCfg2.setTypeName(InnerClassBinarylizable.class.getName());
+
+        binTypCfg1.setIdentityResolver(BinaryArrayIdentityResolver.instance());
+        binTypCfg2.setIdentityResolver(BinaryArrayIdentityResolver.instance());
+
+        List<BinaryTypeConfiguration> binTypCfgs = new ArrayList<>();
+
+        binTypCfgs.add(binTypCfg1);
+        binTypCfgs.add(binTypCfg2);
+
+        binCfg.setTypeConfigurations(binTypCfgs);
+
+        cfg.setBinaryConfiguration(binCfg);
+
+        return cfg;
+    }
+
+    /**
+     * Test hash code generation for simple object.
+     */
+    public void testHashCode() {
+        InnerClass obj = new InnerClass(1, "2", 3);
+
+        int expHash = BinaryArrayIdentityResolver.instance().hashCode(asBinary(obj));
+
+        assertEquals(expHash, build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c).hashCode());
+    }
+
+    /**
+     * Test hash code generation for simple object.
+     */
+    public void testHashCodeBinarylizable() {
+        InnerClassBinarylizable obj = new InnerClassBinarylizable(1, "2", 3);
+
+        int expHash = BinaryArrayIdentityResolver.instance().hashCode(asBinary(obj));
+
+        assertEquals(expHash, build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c).hashCode());
+    }
+
+    /**
+     * Test equals for simple object.
+     */
+    public void testEquals() {
+        InnerClass obj = new InnerClass(1, "2", 3);
+
+        // Positive cases.
+        compareTwo(asBinary(obj), asBinary(obj), true);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c), true);
+
+        // Negative cases.
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a, "b", obj.b), false);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c, "d", "d"), false);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a), false);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a, "b", obj.b + "1"), false);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a + 1, "b", obj.b), false);
+    }
+
+    /**
+     * Test equals for simple object.
+     */
+    public void testEqualsBinarilyzable() {
+        InnerClassBinarylizable obj = new InnerClassBinarylizable(1, "2", 3);
+
+        // Positive cases.
+        compareTwo(asBinary(obj), asBinary(obj), true);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b, "c", obj.c),
+            true);
+
+        // Negative cases.
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b), false);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b, "c", obj.c, "d", "d"),
+            false);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a), false);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b + "1"), false);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a + 1, "b", obj.b), false);
+    }
+
+    /**
+     * Test equals for different type IDs.
+     */
+    public void testEqualsDifferenTypes() {
+        InnerClass obj1 = new InnerClass(1, "2", 3);
+        InnerClassBinarylizable obj2 = new InnerClassBinarylizable(1, "2", 3);
+
+        compareTwo(asBinary(obj1), asBinary(obj2), false);
+    }
+
+    /**
+     * Compare two objects in different heap/offheap modes.
+     *
+     * @param obj1 Object 1.
+     * @param obj2 Object 2.
+     * @param expRes Expected result.
+     */
+    private void compareTwo(BinaryObject obj1, BinaryObject obj2, boolean expRes) {
+        if (expRes) {
+            assertEquals(convert(obj1, false), convert(obj2, false));
+            assertEquals(convert(obj1, false), convert(obj2, true));
+            assertEquals(convert(obj1, true), convert(obj2, false));
+            assertEquals(convert(obj1, true), convert(obj2, true));
+        }
+        else {
+            assertNotEquals(convert(obj1, false), convert(obj2, false));
+            assertNotEquals(convert(obj1, false), convert(obj2, true));
+            assertNotEquals(convert(obj1, true), convert(obj2, false));
+            assertNotEquals(convert(obj1, true), convert(obj2, true));
+        }
+    }
+
+    /**
+     * Convert to binary object.
+     *
+     * @param obj Original object.
+     * @return Binary object.
+     */
+    private BinaryObject asBinary(Object obj) {
+        return grid().binary().toBinary(obj);
+    }
+
+    /**
+     * Build object of the given type with provided fields.
+     *
+     * @param cls Class.
+     * @param parts Parts.
+     * @return Result.
+     */
+    private BinaryObject build(Class cls, Object... parts) {
+        BinaryObjectBuilder builder = grid().binary().builder(cls.getName());
+
+        if (!F.isEmpty(parts)) {
+            for (int i = 0; i < parts.length; )
+                builder.setField((String)parts[i++], parts[i++]);
+        }
+
+        return builder.build();
+    }
+
+    /**
+     * Inner class.
+     */
+    private static class InnerClass {
+        /** Field a. */
+        public int a;
+
+        /** Field b. */
+        public String b;
+
+        /** Field c. */
+        public long c;
+
+        /**
+         * Constructor.
+         *
+         * @param a Field a.
+         * @param b Field b.
+         * @param c Field c.
+         */
+        public InnerClass(int a, String b, long c) {
+            this.a = a;
+            this.b = b;
+            this.c = c;
+        }
+    }
+
+    /**
+     * Convert binary object to it's final state.
+     *
+     * @param obj Object.
+     * @param offheap Offheap flag.
+     * @return Result.
+     */
+    private BinaryObjectExImpl convert(BinaryObject obj, boolean offheap) {
+        BinaryObjectExImpl obj0 = (BinaryObjectExImpl)obj;
+
+        if (offheap) {
+            byte[] arr = obj0.array();
+
+            long ptr = GridUnsafe.allocateMemory(arr.length);
+
+            ptrs.add(ptr);
+
+            GridUnsafe.copyMemory(arr, GridUnsafe.BYTE_ARR_OFF, null, ptr, arr.length);
+
+            obj0 = new BinaryObjectOffheapImpl(obj0.context(), ptr, 0, obj0.array().length);
+        }
+
+        return obj0;
+    }
+
+    /**
+     * Inner class with Binarylizable interface.
+     */
+    private static class InnerClassBinarylizable extends InnerClass implements Binarylizable {
+        /**
+         * Constructor.
+         *
+         * @param a Field a.
+         * @param b Field b.
+         * @param c Field c.
+         */
+        public InnerClassBinarylizable(int a, String b, long c) {
+            super(a, b, c);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            writer.writeInt("a", a);
+            writer.writeString("b", b);
+            writer.writeLong("c", c);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            a = reader.readInt("a");
+            b = reader.readString("b");
+            c = reader.readLong("c");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldIdentityResolverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldIdentityResolverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldIdentityResolverSelfTest.java
new file mode 100644
index 0000000..ba58392
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldIdentityResolverSelfTest.java
@@ -0,0 +1,333 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.binary.BinaryFieldIdentityResolver;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.eclipse.jetty.util.ConcurrentHashSet;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.assertNotEquals;
+
+/**
+ * Field identity resolver self test.
+ */
+public class BinaryFieldIdentityResolverSelfTest extends GridCommonAbstractTest {
+    /** Pointers to release. */
+    private final Set<Long> ptrs = new ConcurrentHashSet<>();
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrid();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        for (Long ptr : ptrs)
+            GridUnsafe.freeMemory(ptr);
+
+        super.afterTest();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg =  super.getConfiguration(gridName);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        BinaryConfiguration binCfg = new BinaryConfiguration();
+
+        BinaryTypeConfiguration binTypCfg1 = new BinaryTypeConfiguration();
+        BinaryTypeConfiguration binTypCfg2 = new BinaryTypeConfiguration();
+
+        binTypCfg1.setTypeName(InnerClass.class.getName());
+        binTypCfg2.setTypeName(InnerClassBinarylizable.class.getName());
+
+        BinaryFieldIdentityResolver binTypIdentityRslvr = new BinaryFieldIdentityResolver();
+
+        binTypIdentityRslvr.setFieldNames("a", "b");
+
+        binTypCfg1.setIdentityResolver(binTypIdentityRslvr);
+        binTypCfg2.setIdentityResolver(binTypIdentityRslvr);
+
+        List<BinaryTypeConfiguration> binTypCfgs = new ArrayList<>();
+
+        binTypCfgs.add(binTypCfg1);
+        binTypCfgs.add(binTypCfg2);
+
+        binCfg.setTypeConfigurations(binTypCfgs);
+
+        cfg.setBinaryConfiguration(binCfg);
+
+        return cfg;
+    }
+
+    /**
+     * Test hash code generation for simple object.
+     */
+    public void testHashCode() {
+        // Simple case.
+        InnerClass obj = new InnerClass(1, "2", 3);
+
+        int expHash = 31 * obj.a + obj.b.hashCode();
+
+        assertEquals(expHash, asBinary(obj).hashCode());
+        assertEquals(expHash, build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c).hashCode());
+
+        // Different fields count.
+        assertEquals(expHash, build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c, "d", "d").hashCode());
+
+        // Null fields.
+        obj = new InnerClass(1, null, 3);
+
+        expHash = 31 * obj.a;
+
+        assertEquals(expHash, asBinary(obj).hashCode());
+        assertEquals(expHash, build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c).hashCode());
+        assertEquals(expHash, build(InnerClass.class, "a", obj.a).hashCode());
+    }
+
+    /**
+     * Test hash code generation for simple object.
+     */
+    public void testHashCodeBinarylizable() {
+        // Simple case.
+        InnerClassBinarylizable obj = new InnerClassBinarylizable(1, "2", 3);
+
+        int expHash = 31 * obj.a + obj.b.hashCode();
+
+        assertEquals(expHash, asBinary(obj).hashCode());
+        assertEquals(expHash, build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b, "c", obj.c).hashCode());
+
+        // Different fields count.
+        assertEquals(expHash, build(InnerClassBinarylizable.class,"a", obj.a, "b", obj.b, "c", obj.c, "d", "d")
+            .hashCode());
+
+        // Null fields.
+        obj = new InnerClassBinarylizable(1, null, 3);
+
+        expHash = 31 * obj.a;
+
+        assertEquals(expHash, asBinary(obj).hashCode());
+        assertEquals(expHash, build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b, "c", obj.c).hashCode());
+        assertEquals(expHash, build(InnerClassBinarylizable.class, "a", obj.a).hashCode());
+    }
+
+    /**
+     * Test equals for simple object.
+     */
+    public void testEquals() {
+        InnerClass obj = new InnerClass(1, "2", 3);
+
+        // Positive cases.
+        compareTwo(asBinary(obj), asBinary(obj), true);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a, "b", obj.b), true);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c), true);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a, "b", obj.b, "c", obj.c, "d", "d"), true);
+
+        // Negative cases.
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a), false);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a, "b", obj.b + "1"), false);
+        compareTwo(asBinary(obj), build(InnerClass.class, "a", obj.a + 1, "b", obj.b), false);
+    }
+
+    /**
+     * Test equals for simple object.
+     */
+    public void testEqualsBinarilyzable() {
+        InnerClassBinarylizable obj = new InnerClassBinarylizable(1, "2", 3);
+
+        // Positive cases.
+        compareTwo(asBinary(obj), asBinary(obj), true);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b), true);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b, "c", obj.c),
+            true);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b, "c", obj.c, "d", "d"),
+            true);
+
+        // Negative cases.
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a), false);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a, "b", obj.b + "1"), false);
+        compareTwo(asBinary(obj), build(InnerClassBinarylizable.class, "a", obj.a + 1, "b", obj.b), false);
+    }
+
+    /**
+     * Test equals for different type IDs.
+     */
+    public void testEqualsDifferenTypes() {
+        InnerClass obj1 = new InnerClass(1, "2", 3);
+        InnerClassBinarylizable obj2 = new InnerClassBinarylizable(1, "2", 3);
+
+        compareTwo(asBinary(obj1), asBinary(obj2), false);
+    }
+
+    /**
+     * Compare two objects in different heap/offheap modes.
+     *
+     * @param obj1 Object 1.
+     * @param obj2 Object 2.
+     * @param expRes Expected result.
+     */
+    private void compareTwo(BinaryObject obj1, BinaryObject obj2, boolean expRes) {
+        if (expRes) {
+            assertEquals(convert(obj1, false), convert(obj2, false));
+            assertEquals(convert(obj1, false), convert(obj2, true));
+            assertEquals(convert(obj1, true), convert(obj2, false));
+            assertEquals(convert(obj1, true), convert(obj2, true));
+        }
+        else {
+            assertNotEquals(convert(obj1, false), convert(obj2, false));
+            assertNotEquals(convert(obj1, false), convert(obj2, true));
+            assertNotEquals(convert(obj1, true), convert(obj2, false));
+            assertNotEquals(convert(obj1, true), convert(obj2, true));
+        }
+    }
+
+    /**
+     * Convert to binary object.
+     *
+     * @param obj Original object.
+     * @return Binary object.
+     */
+    private BinaryObject asBinary(Object obj) {
+        return grid().binary().toBinary(obj);
+    }
+
+    /**
+     * Build object of the given type with provided fields.
+     *
+     * @param cls Class.
+     * @param parts Parts.
+     * @return Result.
+     */
+    private BinaryObject build(Class cls, Object... parts) {
+        BinaryObjectBuilder builder = grid().binary().builder(cls.getName());
+
+        if (!F.isEmpty(parts)) {
+            for (int i = 0; i < parts.length; )
+                builder.setField((String)parts[i++], parts[i++]);
+        }
+
+        return builder.build();
+    }
+
+    /**
+     * Inner class.
+     */
+    private static class InnerClass {
+        /** Field a. */
+        public int a;
+
+        /** Field b. */
+        public String b;
+
+        /** Field c. */
+        public long c;
+
+        /**
+         * Constructor.
+         *
+         * @param a Field a.
+         * @param b Field b.
+         * @param c Field c.
+         */
+        public InnerClass(int a, String b, long c) {
+            this.a = a;
+            this.b = b;
+            this.c = c;
+        }
+    }
+
+    /**
+     * Convert binary object to it's final state.
+     *
+     * @param obj Object.
+     * @param offheap Offheap flag.
+     * @return Result.
+     */
+    private BinaryObjectExImpl convert(BinaryObject obj, boolean offheap) {
+        BinaryObjectExImpl obj0 = (BinaryObjectExImpl)obj;
+
+        if (offheap) {
+            byte[] arr = obj0.array();
+
+            long ptr = GridUnsafe.allocateMemory(arr.length);
+
+            ptrs.add(ptr);
+
+            GridUnsafe.copyMemory(arr, GridUnsafe.BYTE_ARR_OFF, null, ptr, arr.length);
+
+            obj0 = new BinaryObjectOffheapImpl(obj0.context(), ptr, 0, obj0.array().length);
+        }
+
+        return obj0;
+    }
+
+    /**
+     * Inner class with Binarylizable interface.
+     */
+    private static class InnerClassBinarylizable extends InnerClass implements Binarylizable {
+        /**
+         * Constructor.
+         *
+         * @param a Field a.
+         * @param b Field b.
+         * @param c Field c.
+         */
+        public InnerClassBinarylizable(int a, String b, long c) {
+            super(a, b, c);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            writer.writeLong("c", c);
+            writer.writeInt("a", a);
+            writer.writeString("b", b);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            c = reader.readLong("c");
+            a = reader.readInt("a");
+            b = reader.readString("b");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryIdentityResolverConfigurationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryIdentityResolverConfigurationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryIdentityResolverConfigurationSelfTest.java
new file mode 100644
index 0000000..3a2753b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryIdentityResolverConfigurationSelfTest.java
@@ -0,0 +1,138 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.IgniteBinary;
+import org.apache.ignite.binary.BinaryArrayIdentityResolver;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Test for identity resolver configuration.
+ */
+public class BinaryIdentityResolverConfigurationSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrid();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg =  super.getConfiguration(gridName);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        BinaryConfiguration binCfg = new BinaryConfiguration();
+
+        BinaryTypeConfiguration binTypCfg = new BinaryTypeConfiguration();
+
+        binTypCfg.setTypeName(MyClass.class.getName());
+        binTypCfg.setIdentityResolver(new CustomResolver());
+
+        List<BinaryTypeConfiguration> binTypCfgs = new ArrayList<>();
+
+        binTypCfgs.add(binTypCfg);
+
+        binCfg.setTypeConfigurations(binTypCfgs);
+
+        cfg.setBinaryConfiguration(binCfg);
+
+        return cfg;
+    }
+
+    /**
+     * Test type resolver.
+     */
+    public void testTypeResolver() {
+        MyClass obj = new MyClass(1, 2);
+
+        int expHash = hash(obj.a, obj.b);
+
+        BinaryObject binObj1 = binary().toBinary(obj);
+        BinaryObject binObj2 =
+            binary().builder(MyClass.class.getName()).setField("a", obj.a).setField("b", obj.b).build();
+
+        assertEquals(expHash, binObj1.hashCode());
+        assertEquals(expHash, binObj2.hashCode());
+    }
+
+    /**
+     * @return Binary interface for current Ignite instance.
+     */
+    public IgniteBinary binary() {
+        return grid().binary();
+    }
+
+    /**
+     * Second hash function.
+     *
+     * @param a First value.
+     * @param b Second value.
+     * @return Result.
+     */
+    public static int hash(Object a, Object b) {
+        return 31 * a.hashCode() + b.hashCode();
+    }
+
+    /**
+     * First class.
+     */
+    private static class MyClass {
+        /** Value 1. */
+        public int a;
+
+        /** Value 2. */
+        public int b;
+
+        /**
+         * Constructor.
+         *
+         * @param a Value 1.
+         * @param b Value 2.
+         */
+        public MyClass(int a, int b) {
+            this.a = a;
+            this.b = b;
+        }
+    }
+
+    /**
+     * First custom identity resolver.
+     */
+    private static class CustomResolver extends BinaryArrayIdentityResolver {
+        /** {@inheritDoc} */
+        @Override protected int hashCode0(BinaryObject obj) {
+            return hash(obj.field("a"), obj.field("b"));
+        }
+    }
+}