You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2015/12/18 07:46:19 UTC

[1/6] ignite git commit: IGNITE-2100: Fixed serilaization of Externalizable. Now queries work in all modes.

Repository: ignite
Updated Branches:
  refs/heads/ignite-843-rc2 23c3e5441 -> f888e54fc


http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/BinarySerializationQueryWithReflectiveSerializerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/BinarySerializationQueryWithReflectiveSerializerSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/BinarySerializationQueryWithReflectiveSerializerSelfTest.java
new file mode 100644
index 0000000..b905646
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/BinarySerializationQueryWithReflectiveSerializerSelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ * Test for query with BinaryMarshaller and different serialization modes and with reflective serializer.
+ */
+public class BinarySerializationQueryWithReflectiveSerializerSelfTest extends BinarySerializationQuerySelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean useReflectiveSerializer() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
index 64a1115..cae7f0c 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
@@ -54,10 +54,10 @@ public class IgniteCacheP2pUnmarshallingQueryErrorTest extends IgniteCacheP2pUnm
         try {
             jcache(0).query(new SqlQuery<TestKey, String>(String.class, "field like '" + key + "'")).getAll();
 
-            assertTrue("p2p marshalling failed, but error response was not sent", binaryMarshaller());
+            fail("p2p marshalling failed, but error response was not sent");
         }
         catch (CacheException e) {
-            assertFalse("Unexpected exception: " + e, binaryMarshaller());
+            // No-op.
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
index 786b5b8..6abc2d4 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
@@ -18,31 +18,85 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.BinarySerializationQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.BinarySerializationQueryWithReflectiveSerializerSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheLocalQueryMetricsSelfTest;
 import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsDistributedSelfTest;
 import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsLocalSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsDistributedSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLocalSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheScanPartitionQueryFallbackSelfTest;
+import org.apache.ignite.internal.processors.cache.GridCacheCrossCacheQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest;
+import org.apache.ignite.internal.processors.cache.GridCacheQueryInternalKeysSelfTest;
+import org.apache.ignite.internal.processors.cache.GridCacheQuerySerializationSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheReduceQueryMultithreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectFieldsQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheCollocatedQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheDuplicateEntityConfigurationSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheLargeResultSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheNoClassQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapEvictQueryTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapTieredMultithreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingQueryErrorTest;
 import org.apache.ignite.internal.processors.cache.IgniteCachePartitionedQueryMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryEvictsMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheQueryIndexSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheQueryLoadSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryOffheapMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheSqlQueryMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.SqlFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.binary.distributed.dht.GridCacheBinaryDuplicateIndexObjectPartitionedAtomicSelfTest;
 import org.apache.ignite.internal.processors.cache.binary.distributed.dht.GridCacheBinaryDuplicateIndexObjectPartitionedTransactionalSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicFieldsQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicNearEnabledFieldsQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicNearEnabledQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheClientQueryReplicatedNodeRestartSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedQueryP2PDisabledSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedSnapshotEnabledQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest2;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQueryP2PDisabledSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicPrimaryWriteOrderSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicReplicatedSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverTxReplicatedSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverTxSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicNearEnabledSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicP2PDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryLocalAtomicSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryLocalSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionedOnlySelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionedP2PDisabledSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionedSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedAtomicSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedOneNodeSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedP2PDisabledSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryTxSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientReconnectTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTxReconnectTest;
+import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryAtomicSelfTest;
+import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryLocalSelfTest;
+import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryPartitionedSelfTest;
+import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryReplicatedSelfTest;
+import org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest;
 import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest;
 import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest;
 import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryTest;
@@ -63,32 +117,90 @@ public class IgniteBinaryCacheQueryTestSuite extends TestSuite {
 
         TestSuite suite = new TestSuite("Grid Cache Query Test Suite using BinaryMarshaller");
 
-        // Parsing
+        // Serialization.
+        suite.addTestSuite(BinarySerializationQuerySelfTest.class);
+        suite.addTestSuite(BinarySerializationQueryWithReflectiveSerializerSelfTest.class);
+
+        // Parsing.
         suite.addTestSuite(GridQueryParsingTest.class);
 
+        // Config.
+        suite.addTestSuite(IgniteCacheDuplicateEntityConfigurationSelfTest.class);
+
         // Queries tests.
+        suite.addTestSuite(IgniteSqlSplitterSelfTest.class);
         suite.addTestSuite(GridCacheQueryIndexDisabledSelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryLoadSelfTest.class);
+        suite.addTestSuite(IgniteCacheLocalQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheLocalAtomicQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheReplicatedQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheReplicatedQueryP2PDisabledSelfTest.class);
+        suite.addTestSuite(IgniteCachePartitionedQuerySelfTest.class);
+        suite.addTestSuite(IgniteCachePartitionedSnapshotEnabledQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheAtomicQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheAtomicNearEnabledQuerySelfTest.class);
+        suite.addTestSuite(IgniteCachePartitionedQueryP2PDisabledSelfTest.class);
         suite.addTestSuite(IgniteCachePartitionedQueryMultiThreadedSelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryIndexSelfTest.class);
+        suite.addTestSuite(IgniteCacheCollocatedQuerySelfTest.class);
         suite.addTestSuite(IgniteCacheLargeResultSelfTest.class);
+        suite.addTestSuite(GridCacheQueryInternalKeysSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryMultiThreadedSelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
-
+        suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class);
+        suite.addTestSuite(IgniteCacheOffheapEvictQueryTest.class);
+        suite.addTestSuite(IgniteCacheSqlQueryMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest2.class);
+        suite.addTestSuite(IgniteCacheClientQueryReplicatedNodeRestartSelfTest.class);
         suite.addTestSuite(GridCacheReduceQueryMultithreadedSelfTest.class);
+        suite.addTestSuite(GridCacheCrossCacheQuerySelfTest.class);
+        suite.addTestSuite(GridCacheQuerySerializationSelfTest.class);
 
+        // Scan queries.
+        suite.addTestSuite(CacheScanPartitionQueryFallbackSelfTest.class);
 
         // Fields queries.
+        suite.addTestSuite(SqlFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheLocalFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheReplicatedFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest.class);
+        suite.addTestSuite(IgniteCachePartitionedFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheAtomicFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheAtomicNearEnabledFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest.class);
         suite.addTestSuite(IgniteCacheFieldsQueryNoDataSelfTest.class);
-        suite.addTestSuite(IgniteBinaryObjectFieldsQuerySelfTest.class);
 
         // Continuous queries.
+        suite.addTestSuite(GridCacheContinuousQueryLocalSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryLocalAtomicSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryReplicatedSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryReplicatedAtomicSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryReplicatedP2PDisabledSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryPartitionedSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryPartitionedOnlySelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryPartitionedP2PDisabledSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryTxSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryAtomicSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryAtomicNearEnabledSelfTest.class);
         suite.addTestSuite(GridCacheContinuousQueryAtomicP2PDisabledSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryReplicatedOneNodeSelfTest.class);
+        suite.addTestSuite(IgniteCacheContinuousQueryClientTest.class);
+        suite.addTestSuite(IgniteCacheContinuousQueryClientReconnectTest.class);
+        suite.addTestSuite(IgniteCacheContinuousQueryClientTxReconnectTest.class);
+        suite.addTestSuite(CacheContinuousQueryFailoverAtomicPrimaryWriteOrderSelfTest.class);
+        suite.addTestSuite(CacheContinuousQueryFailoverAtomicReplicatedSelfTest.class);
+        suite.addTestSuite(CacheContinuousQueryFailoverTxSelfTest.class);
+        suite.addTestSuite(CacheContinuousQueryFailoverTxReplicatedSelfTest.class);
+
+        // Reduce fields queries.
+        suite.addTestSuite(GridCacheReduceFieldsQueryLocalSelfTest.class);
+        suite.addTestSuite(GridCacheReduceFieldsQueryPartitionedSelfTest.class);
+        suite.addTestSuite(GridCacheReduceFieldsQueryAtomicSelfTest.class);
+        suite.addTestSuite(GridCacheReduceFieldsQueryReplicatedSelfTest.class);
 
         suite.addTestSuite(GridCacheQueryIndexingDisabledSelfTest.class);
 
@@ -110,6 +222,7 @@ public class IgniteBinaryCacheQueryTestSuite extends TestSuite {
 
         //Unmarshallig query test.
         suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class);
+        suite.addTestSuite(IgniteCacheNoClassQuerySelfTest.class);
 
         suite.addTestSuite(GridCacheBinaryDuplicateIndexObjectPartitionedAtomicSelfTest.class);
         suite.addTestSuite(GridCacheBinaryDuplicateIndexObjectPartitionedTransactionalSelfTest.class);


[2/6] ignite git commit: IGNITE-2100: Fixed serilaization of Externalizable. Now queries work in all modes.

Posted by ak...@apache.org.
IGNITE-2100: Fixed serilaization of Externalizable. Now queries work in all modes.


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

Branch: refs/heads/ignite-843-rc2
Commit: 057ad5bb4d054cba90932ff5614a928b623d44cc
Parents: 1f484ea
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Dec 17 17:00:29 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Dec 17 17:00:29 2015 +0300

----------------------------------------------------------------------
 .../binary/BinaryReflectiveSerializer.java      |  33 ++
 .../ignite/binary/BinaryTypeConfiguration.java  |  20 +-
 .../ignite/cache/affinity/AffinityKey.java      |   2 +-
 .../internal/binary/BinaryClassDescriptor.java  | 124 ++----
 .../ignite/internal/binary/BinaryContext.java   | 109 ++++-
 .../internal/binary/BinaryFieldAccessor.java    |   3 -
 .../ignite/internal/binary/BinaryUtils.java     |  49 ++-
 .../ignite/internal/binary/BinaryWriteMode.java |   3 -
 .../cache/binary/BinaryMetadataKey.java         |   2 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |   2 +-
 .../PlatformDotNetConfigurationClosure.java     |   4 +-
 .../processors/query/GridQueryProcessor.java    | 204 ++++++---
 .../binary/BinaryFieldsAbstractSelfTest.java    |   4 +-
 .../BinaryFooterOffsetsAbstractSelfTest.java    |   3 +-
 .../binary/BinaryMarshallerSelfTest.java        |  34 +-
 ...GridBinaryMarshallerCtxDisabledSelfTest.java |   3 +-
 .../binary/GridBinaryWildcardsSelfTest.java     |   3 +-
 .../GridCacheOnCopyFlagAbstractSelfTest.java    |  33 +-
 .../GridBinaryCacheEntryMemorySizeSelfTest.java |   3 +-
 .../session/GridSessionCheckpointSelfTest.java  |   3 +-
 .../ignite/testframework/junits/IgniteMock.java |   3 +-
 .../junits/IgniteTestResources.java             |   4 +-
 .../cache/BinarySerializationQuerySelfTest.java | 416 +++++++++++++++++++
 ...onQueryWithReflectiveSerializerSelfTest.java |  28 ++
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |   4 +-
 .../IgniteBinaryCacheQueryTestSuite.java        | 119 +++++-
 26 files changed, 995 insertions(+), 220 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/main/java/org/apache/ignite/binary/BinaryReflectiveSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryReflectiveSerializer.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryReflectiveSerializer.java
new file mode 100644
index 0000000..fd0f841
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryReflectiveSerializer.java
@@ -0,0 +1,33 @@
+/*
+ * 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.binary;
+
+/**
+ * Binary serializer which writes object fields using reflection. Transient fields are not written.
+ */
+public class BinaryReflectiveSerializer implements BinarySerializer {
+    /** {@inheritDoc} */
+    @Override public void writeBinary(Object obj, BinaryWriter writer) throws BinaryObjectException {
+        assert false : "Method should not be called directly.";
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(Object obj, BinaryReader reader) throws BinaryObjectException {
+        assert false : "Method should not be called directly.";
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java
index a694eaf..a00c061 100644
--- a/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java
@@ -17,9 +17,10 @@
 
 package org.apache.ignite.binary;
 
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
  * Defines configuration properties for a specific binary type. Providing per-type
@@ -42,12 +43,27 @@ public class BinaryTypeConfiguration {
     private boolean isEnum;
 
     /**
+     * Constructor.
      */
     public BinaryTypeConfiguration() {
         // No-op.
     }
 
     /**
+     * Copying constructor.
+     *
+     * @param other Other instance.
+     */
+    public BinaryTypeConfiguration(BinaryTypeConfiguration other) {
+        A.notNull(other, "other");
+
+        typeName = other.typeName;
+        idMapper = other.idMapper;
+        serializer = other.serializer;
+        isEnum = other.isEnum;
+    }
+
+    /**
      * @param typeName Class name.
      */
     public BinaryTypeConfiguration(String typeName) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKey.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKey.java
index ee6321c..c745ed8 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKey.java
@@ -64,6 +64,7 @@ public class AffinityKey<K> implements Externalizable {
     private K key;
 
     /** Affinity key. */
+    @AffinityKeyMapped
     @GridToStringInclude
     private Object affKey;
 
@@ -126,7 +127,6 @@ public class AffinityKey<K> implements Externalizable {
      *
      * @return Affinity key to use for affinity mapping.
      */
-    @AffinityKeyMapped
     @SuppressWarnings({"unchecked"})
     public <T> T affinityKey() {
         A.notNull(key, "key");

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
index 4a93bf6..eac1874 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
@@ -17,15 +17,24 @@
 
 package org.apache.ignite.internal.binary;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryIdMapper;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryReflectiveSerializer;
+import org.apache.ignite.binary.BinarySerializer;
+import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.internal.processors.cache.CacheObjectImpl;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.marshaller.MarshallerExclusions;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Field;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
 import java.math.BigDecimal;
 import java.sql.Timestamp;
 import java.util.ArrayList;
@@ -36,18 +45,6 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.UUID;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.binary.BinaryIdMapper;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinarySerializer;
-import org.apache.ignite.binary.Binarylizable;
-import org.apache.ignite.internal.processors.cache.CacheObjectImpl;
-import org.apache.ignite.internal.util.GridUnsafe;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.marshaller.MarshallerExclusions;
-import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
-import org.jetbrains.annotations.Nullable;
-import sun.misc.Unsafe;
 
 import static java.lang.reflect.Modifier.isStatic;
 import static java.lang.reflect.Modifier.isTransient;
@@ -65,7 +62,7 @@ public class BinaryClassDescriptor {
     /** */
     private final Class<?> cls;
 
-    /** */
+    /** Configured serializer. */
     private final BinarySerializer serializer;
 
     /** ID mapper. */
@@ -127,7 +124,6 @@ public class BinaryClassDescriptor {
      * @param serializer Serializer.
      * @param metaDataEnabled Metadata enabled flag.
      * @param registered Whether typeId has been successfully registered by MarshallerContext or not.
-     * @param predefined Whether the class is predefined or not.
      * @throws BinaryObjectException In case of error.
      */
     BinaryClassDescriptor(
@@ -140,13 +136,19 @@ public class BinaryClassDescriptor {
         @Nullable BinaryIdMapper idMapper,
         @Nullable BinarySerializer serializer,
         boolean metaDataEnabled,
-        boolean registered,
-        boolean predefined
+        boolean registered
     ) throws BinaryObjectException {
         assert ctx != null;
         assert cls != null;
         assert idMapper != null;
 
+        // If serializer is not defined at this point, then we have to user OptimizedMarshaller.
+        useOptMarshaller = serializer == null;
+
+        // Reset reflective serializer so that we rely on existing reflection-based serialization.
+        if (serializer instanceof BinaryReflectiveSerializer)
+            serializer = null;
+
         this.ctx = ctx;
         this.cls = cls;
         this.typeId = typeId;
@@ -161,10 +163,10 @@ public class BinaryClassDescriptor {
 
         excluded = MarshallerExclusions.isExcluded(cls);
 
-        useOptMarshaller = !predefined && initUseOptimizedMarshallerFlag();
-
         if (excluded)
             mode = BinaryWriteMode.EXCLUSION;
+        else if (useOptMarshaller)
+            mode = BinaryWriteMode.OBJECT; // Will not be used anywhere.
         else {
             if (cls == BinaryEnumObjectImpl.class)
                 mode = BinaryWriteMode.BINARY_ENUM;
@@ -172,6 +174,14 @@ public class BinaryClassDescriptor {
                 mode = serializer != null ? BinaryWriteMode.BINARY : BinaryUtils.mode(cls);
         }
 
+        if (useOptMarshaller && userType) {
+            U.quietAndWarn(ctx.log(), "Class \"" + cls.getName() + "\" cannot be written in binary format because " +
+                "it either implements Externalizable interface or have writeObject/readObject methods. Please " +
+                "ensure that all nodes have this class in classpath. To enable binary serialization either " +
+                "implement " + Binarylizable.class.getSimpleName() + " interface or set explicit serializer using " +
+                "BinaryTypeConfiguration.setSerializer() method.");
+        }
+
         switch (mode) {
             case P_BYTE:
             case P_BOOLEAN:
@@ -224,7 +234,6 @@ public class BinaryClassDescriptor {
                 break;
 
             case BINARY:
-            case EXTERNALIZABLE:
                 ctor = constructor(cls);
                 fields = null;
                 stableFieldsMeta = null;
@@ -267,8 +276,11 @@ public class BinaryClassDescriptor {
 
                             schemaBuilder.addField(fieldId);
 
-                            if (metaDataEnabled)
+                            if (metaDataEnabled) {
+                                assert stableFieldsMeta != null;
+
                                 stableFieldsMeta.put(name, fieldInfo.mode().typeId());
+                            }
                         }
                     }
                 }
@@ -284,8 +296,7 @@ public class BinaryClassDescriptor {
                 throw new BinaryObjectException("Invalid mode: " + mode);
         }
 
-        if (mode == BinaryWriteMode.BINARY || mode == BinaryWriteMode.EXTERNALIZABLE ||
-            mode == BinaryWriteMode.OBJECT) {
+        if (mode == BinaryWriteMode.BINARY || mode == BinaryWriteMode.OBJECT) {
             readResolveMtd = U.findNonPublicMethod(cls, "readResolve");
             writeReplaceMtd = U.findNonPublicMethod(cls, "writeReplace");
         }
@@ -608,25 +619,6 @@ public class BinaryClassDescriptor {
 
                 break;
 
-            case EXTERNALIZABLE:
-                if (preWrite(writer, obj)) {
-                    writer.rawWriter();
-
-                    try {
-                        ((Externalizable)obj).writeExternal(writer);
-
-                        postWrite(writer, obj);
-                    }
-                    catch (IOException e) {
-                        throw new BinaryObjectException("Failed to write Externalizable object: " + obj, e);
-                    }
-                    finally {
-                        writer.popSchema();
-                    }
-                }
-
-                break;
-
             case OBJECT:
                 if (preWrite(writer, obj)) {
                     try {
@@ -672,21 +664,6 @@ public class BinaryClassDescriptor {
 
                 break;
 
-            case EXTERNALIZABLE:
-                res = newInstance();
-
-                reader.setHandle(res);
-
-                try {
-                    ((Externalizable)res).readExternal(reader);
-                }
-                catch (IOException | ClassNotFoundException e) {
-                    throw new BinaryObjectException("Failed to read Externalizable object: " +
-                        res.getClass().getName(), e);
-                }
-
-                break;
-
             case OBJECT:
                 res = newInstance();
 
@@ -785,29 +762,4 @@ public class BinaryClassDescriptor {
             throw new BinaryObjectException("Failed to get constructor for class: " + cls.getName(), e);
         }
     }
-
-    /**
-     * Determines whether to use {@link OptimizedMarshaller} for serialization or
-     * not.
-     *
-     * @return {@code true} if to use, {@code false} otherwise.
-     */
-    @SuppressWarnings("unchecked")
-    private boolean initUseOptimizedMarshallerFlag() {
-        for (Class c = cls; c != null && !c.equals(Object.class); c = c.getSuperclass()) {
-            try {
-                Method writeObj = c.getDeclaredMethod("writeObject", ObjectOutputStream.class);
-                Method readObj = c.getDeclaredMethod("readObject", ObjectInputStream.class);
-
-                if (!Modifier.isStatic(writeObj.getModifiers()) && !Modifier.isStatic(readObj.getModifiers()) &&
-                    writeObj.getReturnType() == void.class && readObj.getReturnType() == void.class)
-                    return true;
-            }
-            catch (NoSuchMethodException ignored) {
-                // No-op.
-            }
-        }
-
-        return false;
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
index 7f9bacf..abe283e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
@@ -18,18 +18,22 @@
 package org.apache.ignite.internal.binary;
 
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.binary.BinaryIdMapper;
 import org.apache.ignite.binary.BinaryInvalidTypeException;
 import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryReflectiveSerializer;
 import org.apache.ignite.binary.BinarySerializer;
 import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.cache.CacheKeyConfiguration;
+import org.apache.ignite.cache.affinity.AffinityKey;
 import org.apache.ignite.cache.affinity.AffinityKeyMapped;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.IgnitionEx;
+import org.apache.ignite.internal.processors.cache.binary.BinaryMetadataKey;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.processors.datastructures.CollocatedQueueItemKey;
 import org.apache.ignite.internal.processors.datastructures.CollocatedSetItemKey;
@@ -126,8 +130,11 @@ public class BinaryContext implements Externalizable {
     /** */
     private IgniteConfiguration igniteCfg;
 
+    /** Logger. */
+    private IgniteLogger log;
+
     /** */
-    private final OptimizedMarshaller optmMarsh = new OptimizedMarshaller();
+    private final OptimizedMarshaller optmMarsh = new OptimizedMarshaller(false);
 
     /** Compact footer flag. */
     private boolean compactFooter;
@@ -145,13 +152,15 @@ public class BinaryContext implements Externalizable {
     /**
      * @param metaHnd Meta data handler.
      * @param igniteCfg Ignite configuration.
+     * @param log Logger.
      */
-    public BinaryContext(BinaryMetadataHandler metaHnd, IgniteConfiguration igniteCfg) {
+    public BinaryContext(BinaryMetadataHandler metaHnd, IgniteConfiguration igniteCfg, IgniteLogger log) {
         assert metaHnd != null;
         assert igniteCfg != null;
 
         this.metaHnd = metaHnd;
         this.igniteCfg = igniteCfg;
+        this.log = log;
 
         gridName = igniteCfg.getGridName();
 
@@ -194,22 +203,37 @@ public class BinaryContext implements Externalizable {
         registerPredefinedType(Timestamp[].class, GridBinaryMarshaller.TIMESTAMP_ARR);
         registerPredefinedType(Object[].class, GridBinaryMarshaller.OBJ_ARR);
 
+        // Special collections.
         registerPredefinedType(ArrayList.class, 0);
         registerPredefinedType(LinkedList.class, 0);
         registerPredefinedType(HashSet.class, 0);
         registerPredefinedType(LinkedHashSet.class, 0);
-
         registerPredefinedType(HashMap.class, 0);
         registerPredefinedType(LinkedHashMap.class, 0);
 
+        // Classes with overriden default serialization flag.
+        registerPredefinedType(AffinityKey.class, 0);
+
         registerPredefinedType(GridMapEntry.class, 60);
         registerPredefinedType(IgniteBiTuple.class, 61);
         registerPredefinedType(T2.class, 62);
 
+        registerPredefinedType(BinaryObjectImpl.class, 0);
+        registerPredefinedType(BinaryObjectOffheapImpl.class, 0);
+        registerPredefinedType(BinaryMetadataKey.class, 0);
+        registerPredefinedType(BinaryMetadata.class, 0);
+
         // IDs range [200..1000] is used by Ignite internal APIs.
     }
 
     /**
+     * @return Logger.
+     */
+    public IgniteLogger log() {
+        return log;
+    }
+
+    /**
      * @return Marshaller.
      */
     public BinaryMarshaller marshaller() {
@@ -217,6 +241,21 @@ public class BinaryContext implements Externalizable {
     }
 
     /**
+     * Check whether class must be deserialized anyway.
+     *
+     * @param cls Class.
+     * @return {@code True} if must be deserialized.
+     */
+    public boolean mustDeserialize(Class cls) {
+        BinaryClassDescriptor desc = descByCls.get(cls);
+
+        if (desc == null)
+            return marshCtx.isSystemType(cls.getName()) || serializerForClass(cls) == null;
+        else
+            return desc.useOptimizedMarshaller();
+    }
+
+    /**
      * @return Ignite configuration.
      */
     public IgniteConfiguration configuration(){
@@ -504,8 +543,7 @@ public class BinaryContext implements Externalizable {
                 BinaryInternalIdMapper.defaultInstance(),
                 null,
                 false,
-                true, /* registered */
-                false /* predefined */
+                true /* registered */
             );
 
             BinaryClassDescriptor old = descByCls.putIfAbsent(cls, desc);
@@ -541,6 +579,8 @@ public class BinaryContext implements Externalizable {
             throw new BinaryObjectException("Failed to register class.", e);
         }
 
+        BinarySerializer serializer = serializerForClass(cls);
+
         String affFieldName = affinityFieldName(cls);
 
         BinaryClassDescriptor desc = new BinaryClassDescriptor(this,
@@ -550,10 +590,9 @@ public class BinaryContext implements Externalizable {
             typeName,
             affFieldName,
             idMapper,
-            null,
+            serializer,
             true,
-            registered,
-            false /* predefined */
+            registered
         );
 
         if (!deserialize) {
@@ -576,6 +615,30 @@ public class BinaryContext implements Externalizable {
     }
 
     /**
+     * Get serializer for class taking in count default one.
+     *
+     * @param cls Class.
+     * @return Serializer for class or {@code null} if none exists.
+     */
+    private @Nullable BinarySerializer serializerForClass(Class cls) {
+        BinarySerializer serializer = defaultSerializer();
+
+        if (serializer == null && canUseReflectiveSerializer(cls))
+            serializer = new BinaryReflectiveSerializer();
+
+        return serializer;
+    }
+
+    /**
+     * @return Default serializer.
+     */
+    private BinarySerializer defaultSerializer() {
+        BinaryConfiguration binCfg = igniteCfg.getBinaryConfiguration();
+
+        return binCfg != null ? binCfg.getSerializer() : null;
+    }
+
+    /**
      * @param cls Collection class.
      * @return Collection type ID.
      */
@@ -700,6 +763,9 @@ public class BinaryContext implements Externalizable {
     public BinaryClassDescriptor registerPredefinedType(Class<?> cls, int id) {
         String typeName = typeName(cls.getName());
 
+        if (id == 0)
+            id = BinaryInternalIdMapper.defaultInstance().typeId(typeName);
+
         BinaryClassDescriptor desc = new BinaryClassDescriptor(
             this,
             cls,
@@ -708,10 +774,9 @@ public class BinaryContext implements Externalizable {
             typeName,
             null,
             BinaryInternalIdMapper.defaultInstance(),
-            null,
+            new BinaryReflectiveSerializer(),
             false,
-            true, /* registered */
-            true /* predefined */
+            true /* registered */
         );
 
         predefinedTypeNames.put(typeName, id);
@@ -770,6 +835,14 @@ public class BinaryContext implements Externalizable {
         Collection<BinarySchema> schemas = null;
 
         if (cls != null) {
+            if (serializer == null) {
+                // At this point we must decide whether to rely on Java serialization mechanics or not.
+                // If no serializer is provided, we examine the class and if it doesn't contain non-trivial
+                // serialization logic we are safe to fallback to reflective binary serialization.
+                if (canUseReflectiveSerializer(cls))
+                    serializer = new BinaryReflectiveSerializer();
+            }
+
             BinaryClassDescriptor desc = new BinaryClassDescriptor(
                 this,
                 cls,
@@ -780,8 +853,7 @@ public class BinaryContext implements Externalizable {
                 idMapper,
                 serializer,
                 true,
-                true, /* registered */
-                false /* predefined */
+                true /* registered */
             );
 
             fieldsMeta = desc.fieldsMeta();
@@ -797,6 +869,16 @@ public class BinaryContext implements Externalizable {
     }
 
     /**
+     * Check whether reflective serializer can be used for class.
+     *
+     * @param cls Class.
+     * @return {@code True} if reflective serializer can be used.
+     */
+    private static boolean canUseReflectiveSerializer(Class cls) {
+        return BinaryUtils.isBinarylizable(cls) || !BinaryUtils.isCustomJavaSerialization(cls);
+    }
+
+    /**
      * Create binary field.
      *
      * @param typeId Type ID.
@@ -1059,6 +1141,7 @@ public class BinaryContext implements Externalizable {
                 idMapper = other.idMapper;
                 serializer = other.serializer;
                 affKeyFieldName = other.affKeyFieldName;
+                isEnum = other.isEnum;
                 canOverride = other.canOverride;
             }
             else if (!other.canOverride)

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java
index 8050edd..2d69cbd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java
@@ -105,7 +105,6 @@ public abstract class BinaryFieldAccessor {
             case OBJECT_ARR:
             case BINARY_OBJ:
             case BINARY:
-            case EXTERNALIZABLE:
                 return new DefaultFinalClassAccessor(field, id, mode, false);
 
             default:
@@ -624,7 +623,6 @@ public abstract class BinaryFieldAccessor {
                     break;
 
                 case BINARY:
-                case EXTERNALIZABLE:
                 case OBJECT:
                     writer.writeObjectField(val);
 
@@ -825,7 +823,6 @@ public abstract class BinaryFieldAccessor {
                     break;
 
                 case BINARY:
-                case EXTERNALIZABLE:
                 case OBJECT:
                     val = reader.readObject(id);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
index 9e5260b..8cb4b38 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
@@ -34,7 +34,11 @@ import org.jsr166.ConcurrentHashMap8;
 
 import java.io.ByteArrayInputStream;
 import java.io.Externalizable;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
 import java.lang.reflect.Array;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.sql.Timestamp;
@@ -952,8 +956,6 @@ public class BinaryUtils {
             return BinaryWriteMode.BINARY_OBJ;
         else if (Binarylizable.class.isAssignableFrom(cls))
             return BinaryWriteMode.BINARY;
-        else if (Externalizable.class.isAssignableFrom(cls))
-            return BinaryWriteMode.EXTERNALIZABLE;
         else if (isSpecialCollection(cls))
             return BinaryWriteMode.COL;
         else if (isSpecialMap(cls))
@@ -1838,6 +1840,49 @@ public class BinaryUtils {
     }
 
     /**
+     * Check if class is binarylizable.
+     *
+     * @param cls Class.
+     * @return {@code True} if binarylizable.
+     */
+    public static boolean isBinarylizable(Class cls) {
+        for (Class c = cls; c != null && !c.equals(Object.class); c = c.getSuperclass()) {
+            if (Binarylizable.class.isAssignableFrom(c))
+                return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Determines whether class contains custom Java serialization logic.
+     *
+     * @param cls Class.
+     * @return {@code true} if custom Java serialization logic exists, {@code false} otherwise.
+     */
+    @SuppressWarnings("unchecked")
+    public static boolean isCustomJavaSerialization(Class cls) {
+        for (Class c = cls; c != null && !c.equals(Object.class); c = c.getSuperclass()) {
+            if (Externalizable.class.isAssignableFrom(c))
+                return true;
+
+            try {
+                Method writeObj = c.getDeclaredMethod("writeObject", ObjectOutputStream.class);
+                Method readObj = c.getDeclaredMethod("readObject", ObjectInputStream.class);
+
+                if (!Modifier.isStatic(writeObj.getModifiers()) && !Modifier.isStatic(readObj.getModifiers()) &&
+                    writeObj.getReturnType() == void.class && readObj.getReturnType() == void.class)
+                    return true;
+            }
+            catch (NoSuchMethodException ignored) {
+                // No-op.
+            }
+        }
+
+        return false;
+    }
+
+    /**
      * Enum type.
      */
     private static class EnumType {

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteMode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteMode.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteMode.java
index 90127f4..30e0458 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteMode.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriteMode.java
@@ -151,9 +151,6 @@ public enum BinaryWriteMode {
     BINARY(GridBinaryMarshaller.BINARY_OBJ),
 
     /** */
-    EXTERNALIZABLE(GridBinaryMarshaller.OBJ),
-
-    /** */
     OBJECT(GridBinaryMarshaller.OBJ),
 
     /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataKey.java
index 1f2678b..32ab2a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataKey.java
@@ -27,7 +27,7 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 /**
  * Key for binary meta data.
  */
-class BinaryMetadataKey extends GridCacheUtilityKey<BinaryMetadataKey> implements Externalizable {
+public class BinaryMetadataKey extends GridCacheUtilityKey<BinaryMetadataKey> implements Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index e77b85a..6aee7a9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@ -206,7 +206,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
             BinaryMarshaller pMarh0 = (BinaryMarshaller)marsh;
 
-            binaryCtx = new BinaryContext(metaHnd, ctx.config());
+            binaryCtx = new BinaryContext(metaHnd, ctx.config(), ctx.log(BinaryContext.class));
 
             IgniteUtils.invoke(BinaryMarshaller.class, pMarh0, "setBinaryContext", binaryCtx,
                 ctx.config());

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
index 4ac3536..c068d59 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
@@ -36,6 +36,7 @@ import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStrea
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lifecycle.LifecycleBean;
+import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
@@ -241,7 +242,8 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur
     @SuppressWarnings("deprecation")
     private static GridBinaryMarshaller marshaller() {
         try {
-            BinaryContext ctx = new BinaryContext(BinaryNoopMetadataHandler.instance(), new IgniteConfiguration());
+            BinaryContext ctx =
+                new BinaryContext(BinaryNoopMetadataHandler.instance(), new IgniteConfiguration(), new NullLogger());
 
             BinaryMarshaller marsh = new BinaryMarshaller();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/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 0412b4c..28b854c 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,35 +17,13 @@
 
 package org.apache.ignite.internal.processors.query;
 
-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;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-import javax.cache.Cache;
-import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 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.BinaryType;
+import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.cache.CacheTypeMetadata;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.QueryIndex;
@@ -55,15 +33,16 @@ import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.SqlQuery;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.events.CacheQueryExecutedEvent;
-import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
+import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
 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.GridCacheTwoStepQuery;
@@ -87,6 +66,31 @@ import org.apache.ignite.spi.indexing.IndexingQueryFilter;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
+import javax.cache.Cache;
+import javax.cache.CacheException;
+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;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+
 import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED;
 import static org.apache.ignite.internal.IgniteComponentType.INDEXING;
 import static org.apache.ignite.internal.processors.query.GridQueryIndexType.FULLTEXT;
@@ -193,6 +197,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         idx.registerCache(ccfg);
 
         try {
+            List<Class<?>> mustDeserializeClss = null;
+
+            boolean binaryEnabled = ctx.cacheObjects().isBinaryEnabled(ccfg);
+
+            CacheObjectContext coCtx = binaryEnabled ? ctx.cacheObjects().contextForCache(ccfg) : null;
+
             if (!F.isEmpty(ccfg.getQueryEntities())) {
                 for (QueryEntity qryEntity : ccfg.getQueryEntities()) {
                     if (F.isEmpty(qryEntity.getValueType()))
@@ -202,14 +212,23 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                     // Key and value classes still can be available if they are primitive or JDK part.
                     // We need that to set correct types for _key and _val columns.
-                    Class<?> keyCls = U.classForName(qryEntity.getKeyType(), Object.class);
+                    Class<?> keyCls = U.classForName(qryEntity.getKeyType(), null);
                     Class<?> valCls = U.classForName(qryEntity.getValueType(), null);
 
+                    // If local node has the classes and they are externalizable, we must use reflection properties.
+                    boolean keyMustDeserialize = mustDeserializeBinary(keyCls);
+                    boolean valMustDeserialize = mustDeserializeBinary(valCls);
+
+                    boolean keyOrValMustDeserialize = keyMustDeserialize || valMustDeserialize;
+
+                    if (keyCls == null)
+                        keyCls = Object.class;
+
                     String simpleValType = valCls == null ? typeName(qryEntity.getValueType()) : typeName(valCls);
 
                     desc.name(simpleValType);
 
-                    if (ctx.cacheObjects().isBinaryEnabled(ccfg)) {
+                    if (binaryEnabled && !keyOrValMustDeserialize) {
                         // Safe to check null.
                         if (SQL_TYPES.contains(valCls))
                             desc.valueClass(valCls);
@@ -234,10 +253,21 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         desc.keyClass(keyCls);
                     }
 
+                    if (binaryEnabled && keyOrValMustDeserialize) {
+                        if (mustDeserializeClss == null)
+                            mustDeserializeClss = new ArrayList<>();
+
+                        if (keyMustDeserialize)
+                            mustDeserializeClss.add(keyCls);
+
+                        if (valMustDeserialize)
+                            mustDeserializeClss.add(valCls);
+                    }
+
                     TypeId typeId;
                     TypeId altTypeId = null;
 
-                    if (valCls == null || ctx.cacheObjects().isBinaryEnabled(ccfg)) {
+                    if (valCls == null || (binaryEnabled && !keyOrValMustDeserialize)) {
                         processBinaryMeta(qryEntity, desc);
 
                         typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(qryEntity.getValueType()));
@@ -246,7 +276,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                             altTypeId = new TypeId(ccfg.getName(), valCls);
                     }
                     else {
-                        processClassMeta(qryEntity, desc);
+                        processClassMeta(qryEntity, desc, coCtx);
 
                         typeId = new TypeId(ccfg.getName(), valCls);
                         altTypeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(qryEntity.getValueType()));
@@ -276,12 +306,21 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                     // Key and value classes still can be available if they are primitive or JDK part.
                     // We need that to set correct types for _key and _val columns.
-                    Class<?> keyCls = U.classForName(meta.getKeyType(), Object.class);
+                    Class<?> keyCls = U.classForName(meta.getKeyType(), null);
                     Class<?> valCls = U.classForName(meta.getValueType(), null);
 
+                    // If local node has the classes and they are externalizable, we must use reflection properties.
+                    boolean keyMustDeserialize = mustDeserializeBinary(keyCls);
+                    boolean valMustDeserialize = mustDeserializeBinary(valCls);
+
+                    boolean keyOrValMustDeserialize = keyMustDeserialize || valMustDeserialize;
+
+                    if (keyCls == null)
+                        keyCls = Object.class;
+
                     desc.name(meta.getSimpleValueType());
 
-                    if (ctx.cacheObjects().isBinaryEnabled(ccfg)) {
+                    if (binaryEnabled && !keyOrValMustDeserialize) {
                         // Safe to check null.
                         if (SQL_TYPES.contains(valCls))
                             desc.valueClass(valCls);
@@ -298,10 +337,21 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         desc.keyClass(keyCls);
                     }
 
+                    if (binaryEnabled && keyOrValMustDeserialize) {
+                        if (mustDeserializeClss == null)
+                            mustDeserializeClss = new ArrayList<>();
+
+                        if (keyMustDeserialize)
+                            mustDeserializeClss.add(keyCls);
+
+                        if (valMustDeserialize)
+                            mustDeserializeClss.add(valCls);
+                    }
+
                     TypeId typeId;
                     TypeId altTypeId = null;
 
-                    if (valCls == null || ctx.cacheObjects().isBinaryEnabled(ccfg)) {
+                    if (valCls == null || (binaryEnabled && !keyOrValMustDeserialize)) {
                         processBinaryMeta(meta, desc);
 
                         typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType()));
@@ -310,7 +360,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                             altTypeId = new TypeId(ccfg.getName(), valCls);
                     }
                     else {
-                        processClassMeta(meta, desc);
+                        processClassMeta(meta, desc, coCtx);
 
                         typeId = new TypeId(ccfg.getName(), valCls);
                         altTypeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType()));
@@ -327,6 +377,15 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             }
 
             // Indexed types must be translated to CacheTypeMetadata in CacheConfiguration.
+
+            if (mustDeserializeClss != null) {
+                U.quietAndWarn(log, "Some classes in query configuration cannot be written in binary format " +
+                    "because they either implement Externalizable interface or have writeObject/readObject methods. " +
+                    "Instances of these classes will be deserialized in order to build indexes. Please ensure that " +
+                    "all nodes have these classes in classpath. To enable binary serialization either implement " +
+                    Binarylizable.class.getSimpleName() + " interface or set explicit serializer using " +
+                    "BinaryTypeConfiguration.setSerializer() method: " + mustDeserializeClss);
+            }
         }
         catch (IgniteCheckedException | RuntimeException e) {
             idx.unregisterCache(ccfg);
@@ -336,6 +395,22 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Check whether type still must be deserialized when binary marshaller is set.
+     *
+     * @param cls Class.
+     * @return {@code True} if will be deserialized.
+     */
+    private boolean mustDeserializeBinary(Class cls) {
+        if (cls != null && ctx.config().getMarshaller() instanceof BinaryMarshaller) {
+            CacheObjectBinaryProcessorImpl proc0 = (CacheObjectBinaryProcessorImpl)ctx.cacheObjects();
+
+            return proc0.binaryContext().mustDeserialize(cls);
+        }
+        else
+            return false;
+    }
+
+    /**
      * @param ccfg Cache configuration.
      * @param desc Type descriptor.
      * @throws IgniteCheckedException If failed.
@@ -1185,9 +1260,10 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      *
      * @param meta Type metadata.
      * @param d Type descriptor.
+     * @param coCtx Cache object context.
      * @throws IgniteCheckedException If failed.
      */
-    private void processClassMeta(CacheTypeMetadata meta, TypeDescriptor d)
+    private void processClassMeta(CacheTypeMetadata meta, TypeDescriptor d, CacheObjectContext coCtx)
         throws IgniteCheckedException {
         Map<String,String> aliases = meta.getAliases();
 
@@ -1201,13 +1277,13 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         assert valCls != null;
 
         for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet())
-            addToIndex(d, keyCls, valCls, entry.getKey(), entry.getValue(), 0, IndexType.ASC, null, aliases);
+            addToIndex(d, keyCls, valCls, entry.getKey(), entry.getValue(), 0, IndexType.ASC, null, aliases, coCtx);
 
         for (Map.Entry<String, Class<?>> entry : meta.getDescendingFields().entrySet())
-            addToIndex(d, keyCls, valCls, entry.getKey(), entry.getValue(), 0, IndexType.DESC, null, aliases);
+            addToIndex(d, keyCls, valCls, entry.getKey(), entry.getValue(), 0, IndexType.DESC, null, aliases, coCtx);
 
         for (String txtField : meta.getTextFields())
-            addToIndex(d, keyCls, valCls, txtField, String.class, 0, IndexType.TEXT, null, aliases);
+            addToIndex(d, keyCls, valCls, txtField, String.class, 0, IndexType.TEXT, null, aliases, coCtx);
 
         Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> grps = meta.getGroups();
 
@@ -1226,7 +1302,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         descending = false;
 
                     addToIndex(d, keyCls, valCls, idxField.getKey(), idxField.getValue().get1(), order,
-                        descending ? IndexType.DESC : IndexType.ASC, idxName, aliases);
+                        descending ? IndexType.DESC : IndexType.ASC, idxName, aliases, coCtx);
 
                     order++;
                 }
@@ -1239,7 +1315,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 valCls,
                 entry.getKey(),
                 entry.getValue(),
-                aliases);
+                aliases,
+                coCtx);
 
             d.addProperty(prop, false);
         }
@@ -1266,7 +1343,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         int idxOrder,
         IndexType idxType,
         String idxName,
-        Map<String,String> aliases
+        Map<String,String> aliases,
+        CacheObjectContext coCtx
     ) throws IgniteCheckedException {
         String propName;
         Class<?> propCls;
@@ -1281,7 +1359,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 valCls,
                 pathStr,
                 resType,
-                aliases);
+                aliases,
+                coCtx);
 
             d.addProperty(prop, false);
 
@@ -1410,7 +1489,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param d Type descriptor.
      * @throws IgniteCheckedException If failed.
      */
-    private void processClassMeta(QueryEntity qryEntity, TypeDescriptor d) throws IgniteCheckedException {
+    private void processClassMeta(
+        QueryEntity qryEntity,
+        TypeDescriptor d,
+        CacheObjectContext coCtx
+    ) throws IgniteCheckedException {
         Map<String,String> aliases = qryEntity.getAliases();
 
         if (aliases == null)
@@ -1422,7 +1505,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 d.valueClass(),
                 entry.getKey(),
                 U.classForName(entry.getValue(), Object.class),
-                aliases);
+                aliases,
+                coCtx);
 
 
             d.addProperty(prop, false);
@@ -1524,16 +1608,17 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     private static ClassProperty buildClassProperty(Class<?> keyCls, Class<?> valCls, String pathStr, Class<?> resType,
-        Map<String,String> aliases) throws IgniteCheckedException {
+        Map<String,String> aliases, CacheObjectContext coCtx) throws IgniteCheckedException {
         ClassProperty res = buildClassProperty(
             true,
             keyCls,
             pathStr,
             resType,
-            aliases);
+            aliases,
+            coCtx);
 
         if (res == null) // We check key before value consistently with BinaryProperty.
-            res = buildClassProperty(false, valCls, pathStr, resType, aliases);
+            res = buildClassProperty(false, valCls, pathStr, resType, aliases, coCtx);
 
         if (res == null)
             throw new IgniteCheckedException("Failed to initialize property '" + pathStr + "' for " +
@@ -1552,7 +1637,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @return Property instance corresponding to the given path.
      */
     static ClassProperty buildClassProperty(boolean key, Class<?> cls, String pathStr, Class<?> resType,
-        Map<String,String> aliases) {
+        Map<String,String> aliases, CacheObjectContext coCtx) {
         String[] path = pathStr.split("\\.");
 
         ClassProperty res = null;
@@ -1576,7 +1661,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             ClassProperty tmp = null;
 
             try {
-                tmp = new ClassProperty(cls.getMethod(bld.toString()), key, alias);
+                tmp = new ClassProperty(cls.getMethod(bld.toString()), key, alias, coCtx);
             }
             catch (NoSuchMethodException ignore) {
                 // No-op.
@@ -1584,7 +1669,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             if (tmp == null) {
                 try {
-                    tmp = new ClassProperty(cls.getDeclaredField(prop), key, alias);
+                    tmp = new ClassProperty(cls.getDeclaredField(prop), key, alias, coCtx);
                 }
                 catch (NoSuchFieldException ignored) {
                     // No-op.
@@ -1593,7 +1678,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             if (tmp == null) {
                 try {
-                    tmp = new ClassProperty(cls.getMethod(prop), key, alias);
+                    tmp = new ClassProperty(cls.getMethod(prop), key, alias, coCtx);
                 }
                 catch (NoSuchMethodException ignored) {
                     // No-op.
@@ -1733,12 +1818,15 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         /** */
         private boolean key;
 
+        /** */
+        private CacheObjectContext coCtx;
+
         /**
          * Constructor.
          *
          * @param member Element.
          */
-        ClassProperty(Member member, boolean key, String name) {
+        ClassProperty(Member member, boolean key, String name, @Nullable CacheObjectContext coCtx) {
             this.member = member;
             this.key = key;
 
@@ -1749,11 +1837,13 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             ((AccessibleObject) member).setAccessible(true);
 
             field = member instanceof Field;
+
+            this.coCtx = coCtx;
         }
 
         /** {@inheritDoc} */
         @Override public Object value(Object key, Object val) throws IgniteCheckedException {
-            Object x = this.key ? key : val;
+            Object x = unwrap(this.key ? key : val);
 
             if (parent != null)
                 x = parent.value(key, val);
@@ -1778,6 +1868,16 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             }
         }
 
+        /**
+         * 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;

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldsAbstractSelfTest.java
index 943c5aa..fd095e9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFieldsAbstractSelfTest.java
@@ -24,6 +24,7 @@ import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
@@ -47,7 +48,8 @@ public abstract class BinaryFieldsAbstractSelfTest extends GridCommonAbstractTes
      * @throws Exception If failed.
      */
     protected BinaryMarshaller createMarshaller() throws Exception {
-        BinaryContext ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration());
+        BinaryContext ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration(),
+            new NullLogger());
 
         BinaryMarshaller marsh = new BinaryMarshaller();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFooterOffsetsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFooterOffsetsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFooterOffsetsAbstractSelfTest.java
index 0e3749c..265d283 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFooterOffsetsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryFooterOffsetsAbstractSelfTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
@@ -46,7 +47,7 @@ public abstract class BinaryFooterOffsetsAbstractSelfTest extends GridCommonAbst
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();
 
-        ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration());
+        ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration(), new NullLogger());
 
         marsh = new BinaryMarshaller();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
index b83bbad..fcd511b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
@@ -76,6 +76,7 @@ import org.apache.ignite.internal.util.lang.GridMapEntry;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -435,23 +436,6 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void testExternalizableHashCode() throws Exception {
-        SimpleExternalizable sim1 = new SimpleExternalizable("Simple");
-        SimpleExternalizable sim2 = new SimpleExternalizable("Simple");
-
-        BinaryMarshaller marsh = binaryMarshaller();
-
-        BinaryObjectImpl sim1Binary = marshal(sim1, marsh);
-        BinaryObjectImpl sim2Binary = marshal(sim2, marsh);
-
-        assertEquals(sim1.hashCode(), sim2.hashCode());
-        assertEquals(sim1.hashCode(), sim1Binary.hashCode());
-        assertEquals(sim2.hashCode(), sim2Binary.hashCode());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testExternalizableInEnclosing() throws Exception {
         SimpleEnclosingObject obj = new SimpleEnclosingObject();
         obj.simpl = new SimpleExternalizable("field");
@@ -797,6 +781,13 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
         private String name;
 
         /**
+         * {@link Externalizable} support.
+         */
+        public TestQueue() {
+            // No-op.
+        }
+
+        /**
          * @param name Name.
          */
         public TestQueue(String name) {
@@ -2690,7 +2681,7 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
         iCfg.setBinaryConfiguration(bCfg);
 
-        BinaryContext ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), iCfg);
+        BinaryContext ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), iCfg, new NullLogger());
 
         BinaryMarshaller marsh = new BinaryMarshaller();
 
@@ -4067,6 +4058,13 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
         private String field;
 
         /**
+         * {@link Externalizable} support.
+         */
+        public SimpleExternalizable() {
+            // No-op.
+        }
+
+        /**
          * @param field Field.
          */
         public SimpleExternalizable(String field) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMarshallerCtxDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMarshallerCtxDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMarshallerCtxDisabledSelfTest.java
index e433ec0..2b0051e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMarshallerCtxDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMarshallerCtxDisabledSelfTest.java
@@ -25,6 +25,7 @@ import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.MarshallerContextAdapter;
 import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import java.io.Externalizable;
@@ -46,7 +47,7 @@ public class GridBinaryMarshallerCtxDisabledSelfTest extends GridCommonAbstractT
 
         IgniteConfiguration cfg = new IgniteConfiguration();
 
-        BinaryContext context = new BinaryContext(BinaryCachingMetadataHandler.create(), cfg);
+        BinaryContext context = new BinaryContext(BinaryCachingMetadataHandler.create(), cfg, new NullLogger());
 
         IgniteUtils.invoke(BinaryMarshaller.class, marsh, "setBinaryContext", context, cfg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryWildcardsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryWildcardsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryWildcardsSelfTest.java
index 2887afa..ae0cfa0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryWildcardsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryWildcardsSelfTest.java
@@ -26,6 +26,7 @@ import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
@@ -451,7 +452,7 @@ public class GridBinaryWildcardsSelfTest extends GridCommonAbstractTest {
 
         iCfg.setBinaryConfiguration(bCfg);
 
-        BinaryContext ctx = new BinaryContext(BinaryNoopMetadataHandler.instance(), iCfg);
+        BinaryContext ctx = new BinaryContext(BinaryNoopMetadataHandler.instance(), iCfg, new NullLogger());
 
         BinaryMarshaller marsh = new BinaryMarshaller();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java
index d29440e..6f73e0f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java
@@ -21,6 +21,7 @@ import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
+import java.io.Serializable;
 import java.util.HashMap;
 import java.util.Map;
 import javax.cache.Cache;
@@ -414,7 +415,10 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCacheAbstr
     /**
      *
      */
-    public static class TestKey implements Externalizable {
+    public static class TestKey implements Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
         /** */
         private int key;
 
@@ -471,18 +475,6 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCacheAbstr
         }
 
         /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeInt(key);
-            out.writeInt(field);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            key = in.readInt();
-            field = in.readInt();
-        }
-
-        /** {@inheritDoc} */
         @Override public String toString() {
             return "TestKey [field=" + field + ", key=" + key + ']';
         }
@@ -491,7 +483,10 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCacheAbstr
     /**
      *
      */
-    public static class TestValue implements Externalizable {
+    public static class TestValue implements Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
         /** */
         private int val;
 
@@ -541,16 +536,6 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCacheAbstr
         @Override public int hashCode() {
             return val;
         }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeInt(val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            val = in.readInt();
-        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryCacheEntryMemorySizeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryCacheEntryMemorySizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryCacheEntryMemorySizeSelfTest.java
index d1f7826..c7a6a55 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryCacheEntryMemorySizeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryCacheEntryMemorySizeSelfTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.internal.binary.BinaryNoopMetadataHandler;
 import org.apache.ignite.internal.binary.BinaryContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryMemorySizeSelfTest;
 import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
@@ -39,7 +40,7 @@ public class GridBinaryCacheEntryMemorySizeSelfTest extends GridCacheEntryMemory
 
         IgniteConfiguration iCfg = new IgniteConfiguration();
 
-        BinaryContext pCtx = new BinaryContext(BinaryNoopMetadataHandler.instance(), iCfg);
+        BinaryContext pCtx = new BinaryContext(BinaryNoopMetadataHandler.instance(), iCfg, new NullLogger());
 
         IgniteUtils.invoke(BinaryMarshaller.class, marsh, "setBinaryContext", pCtx, iCfg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/test/java/org/apache/ignite/session/GridSessionCheckpointSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/session/GridSessionCheckpointSelfTest.java b/modules/core/src/test/java/org/apache/ignite/session/GridSessionCheckpointSelfTest.java
index 3038cba..a3df301 100644
--- a/modules/core/src/test/java/org/apache/ignite/session/GridSessionCheckpointSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/session/GridSessionCheckpointSelfTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.internal.binary.BinaryCachingMetadataHandler;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.binary.BinaryContext;
 import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.spi.checkpoint.cache.CacheCheckpointSpi;
@@ -96,7 +97,7 @@ public class GridSessionCheckpointSelfTest extends GridSessionCheckpointAbstract
         cfg.setCheckpointSpi(spi);
 
         if (cfg.getMarshaller() instanceof BinaryMarshaller) {
-            BinaryContext ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), cfg);
+            BinaryContext ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), cfg, new NullLogger());
 
             Marshaller marsh = cfg.getMarshaller();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
index 2367688..c49c730 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
@@ -55,6 +55,7 @@ import org.apache.ignite.internal.binary.BinaryContext;
 import org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl;
 import org.apache.ignite.internal.processors.cacheobject.NoOpBinary;
 import org.apache.ignite.lang.IgniteProductVersion;
+import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.plugin.IgnitePlugin;
 import org.apache.ignite.plugin.PluginNotFoundException;
@@ -295,7 +296,7 @@ public class IgniteMock implements Ignite {
 
         if (ctx == null) {
             /** {@inheritDoc} */
-            ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), configuration()) {
+            ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), configuration(), new NullLogger()) {
                 @Override public int typeId(String typeName) {
                     return typeName.hashCode();
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java
index 4e3f326..13d5fbd 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.binary.BinaryContext;
 import org.apache.ignite.internal.processors.resource.GridResourceProcessor;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
@@ -264,7 +265,8 @@ public class IgniteTestResources {
         marsh.setContext(new MarshallerContextTestImpl());
 
         if (marsh instanceof BinaryMarshaller) {
-            BinaryContext ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration());
+            BinaryContext ctx =
+                new BinaryContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration(), new NullLogger());
 
             IgniteUtils.invoke(BinaryMarshaller.class, marsh, "setBinaryContext", ctx, new IgniteConfiguration());
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/057ad5bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/BinarySerializationQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/BinarySerializationQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/BinarySerializationQuerySelfTest.java
new file mode 100644
index 0000000..1eba7d1
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/BinarySerializationQuerySelfTest.java
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryReflectiveSerializer;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.cache.CacheTypeMetadata;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.SqlQuery;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import javax.cache.Cache;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Test for query with BinaryMarshaller and different serialization modes.
+ */
+public class BinarySerializationQuerySelfTest extends GridCommonAbstractTest {
+    /** Ignite instance. */
+    private Ignite ignite;
+
+    /** Cache. */
+    private IgniteCache<Integer, Object> cache;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        IgniteConfiguration cfg = new IgniteConfiguration();
+
+        cfg.setLocalHost("127.0.0.1");
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+        cfg.setDiscoverySpi(discoSpi);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        if (useReflectiveSerializer()) {
+            BinaryTypeConfiguration binTypCfg1 = new BinaryTypeConfiguration(EntityPlain.class.getName());
+            BinaryTypeConfiguration binTypCfg2 = new BinaryTypeConfiguration(EntitySerializable.class.getName());
+            BinaryTypeConfiguration binTypCfg3 = new BinaryTypeConfiguration(EntityExternalizable.class.getName());
+            BinaryTypeConfiguration binTypCfg4 = new BinaryTypeConfiguration(EntityBinarylizable.class.getName());
+            BinaryTypeConfiguration binTypCfg5 = new BinaryTypeConfiguration(EntityWriteReadObject.class.getName());
+
+            binTypCfg1.setSerializer(new BinaryReflectiveSerializer());
+            binTypCfg2.setSerializer(new BinaryReflectiveSerializer());
+            binTypCfg3.setSerializer(new BinaryReflectiveSerializer());
+            binTypCfg4.setSerializer(new BinaryReflectiveSerializer());
+            binTypCfg5.setSerializer(new BinaryReflectiveSerializer());
+
+            List<BinaryTypeConfiguration> binTypCfgs = new ArrayList<>();
+
+            binTypCfgs.add(binTypCfg1);
+            binTypCfgs.add(binTypCfg2);
+            binTypCfgs.add(binTypCfg3);
+            binTypCfgs.add(binTypCfg4);
+            binTypCfgs.add(binTypCfg5);
+
+            BinaryConfiguration binCfg = new BinaryConfiguration();
+
+            binCfg.setTypeConfigurations(binTypCfgs);
+
+            cfg.setBinaryConfiguration(binCfg);
+        }
+
+        CacheConfiguration cacheCfg = new CacheConfiguration();
+
+        cacheCfg.setName(null);
+        cacheCfg.setCacheMode(CacheMode.PARTITIONED);
+        cacheCfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cacheCfg.setRebalanceMode(CacheRebalanceMode.SYNC);
+
+        List<CacheTypeMetadata> metas = new ArrayList<>();
+
+        metas.add(metaForClass(EntityPlain.class));
+        metas.add(metaForClass(EntitySerializable.class));
+        metas.add(metaForClass(EntityExternalizable.class));
+        metas.add(metaForClass(EntityBinarylizable.class));
+        metas.add(metaForClass(EntityWriteReadObject.class));
+
+        cacheCfg.setTypeMetadata(metas);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        ignite = Ignition.start(cfg);
+
+        cache = ignite.cache(null);
+    }
+
+    /**
+     * Create type metadata for class.
+     *
+     * @param cls Class.
+     * @return Type metadata.
+     */
+    private static CacheTypeMetadata metaForClass(Class cls) {
+        CacheTypeMetadata meta = new CacheTypeMetadata();
+
+        meta.setKeyType(Integer.class);
+        meta.setValueType(cls);
+        meta.setAscendingFields(Collections.<String, Class<?>>singletonMap("val", Integer.class));
+
+        return meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        G.stopAll(true);
+
+        ignite = null;
+        cache = null;
+    }
+
+    /**
+     * Test plain type.
+     *
+     * @throws Exception If failed.
+     */
+    public void testPlain() throws Exception {
+        check(EntityPlain.class);
+    }
+
+    /**
+     * Test Serializable type.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSerializable() throws Exception {
+        check(EntitySerializable.class);
+    }
+
+    /**
+     * Test Externalizable type.
+     *
+     * @throws Exception If failed.
+     */
+    public void testExternalizable() throws Exception {
+        check(EntityExternalizable.class);
+    }
+
+    /**
+     * Test Binarylizable type.
+     *
+     * @throws Exception If failed.
+     */
+    public void testBinarylizable() throws Exception {
+        check(EntityBinarylizable.class);
+    }
+
+    /**
+     * Test type with readObject/writeObject methods.
+     *
+     * @throws Exception If failed.
+     */
+    public void testWriteReadObject() throws Exception {
+        check(EntityWriteReadObject.class);
+    }
+
+    /**
+     * Internal check routine.
+     *
+     * @param cls Entity class.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    private void check(Class cls) throws Exception {
+        cache.put(1, createInstance(cls, 10));
+        cache.put(2, createInstance(cls, 20));
+        cache.put(3, createInstance(cls, 30));
+
+        Iterator iter = cache.query(new SqlQuery(cls, "val=20")).iterator();
+
+        assert iter.hasNext();
+
+        Cache.Entry res = (Cache.Entry)iter.next();
+
+        assertEquals(2, res.getKey());
+        assertEquals(20, U.field(res.getValue(), "val"));
+
+        assert !iter.hasNext();
+
+        iter = cache.query(
+            new SqlFieldsQuery("SELECT p.val FROM " + cls.getSimpleName() + " p WHERE p.val=20")).iterator();
+
+        assert iter.hasNext();
+
+        List<Object> fieldsRes = (List<Object>)iter.next();
+
+        assertEquals(20, fieldsRes.get(0));
+
+        assert !iter.hasNext();
+    }
+
+    /**
+     * Create object instance.
+     *
+     * @param cls Class.
+     * @param val Value.
+     * @return Instance.
+     */
+    private static Object createInstance(Class cls, int val) {
+        if (cls.equals(EntityPlain.class))
+            return new EntityPlain(val);
+        else if (cls.equals(EntitySerializable.class))
+            return new EntitySerializable(val);
+        else if (cls.equals(EntityExternalizable.class))
+            return new EntityExternalizable(val);
+        else if (cls.equals(EntityBinarylizable.class))
+            return new EntityBinarylizable(val);
+        else
+            return new EntityWriteReadObject(val);
+    }
+
+    /**
+     * @return Whether reflective serializer should be used.
+     */
+    protected boolean useReflectiveSerializer() {
+        return false;
+    }
+
+    /**
+     * Plain entry.
+     */
+    private static class EntityPlain {
+        /** Value. */
+        public int val;
+
+        /**
+         * Default constructor.
+         */
+        public EntityPlain() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param val Value.
+         */
+        public EntityPlain(int val) {
+            this.val = val;
+        }
+    }
+
+    /**
+     * Serializable entity.
+     */
+    private static class EntitySerializable implements Serializable {
+        /** Value. */
+        public int val;
+
+        /**
+         * Default constructor.
+         */
+        public EntitySerializable() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param val Value.
+         */
+        public EntitySerializable(int val) {
+            this.val = val;
+        }
+    }
+
+    /**
+     * Serializable entity.
+     */
+    private static class EntityExternalizable implements Externalizable {
+        /** Value. */
+        public int val;
+
+        /**
+         * Default constructor.
+         */
+        public EntityExternalizable() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param val Value.
+         */
+        public EntityExternalizable(int val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeInt(val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            val = in.readInt();
+        }
+    }
+
+    /**
+     * Serializable entity.
+     */
+    private static class EntityBinarylizable implements Binarylizable {
+        /** Value. */
+        public int val;
+
+        /**
+         * Default constructor.
+         */
+        public EntityBinarylizable() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param val Value.
+         */
+        public EntityBinarylizable(int val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+            writer.writeInt("val", val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+            val = reader.readInt("val");
+        }
+    }
+
+    /**
+     * Serializable entity.
+     */
+    private static class EntityWriteReadObject implements Serializable {
+        /** Value. */
+        public int val;
+
+        /**
+         * Default constructor.
+         */
+        public EntityWriteReadObject() {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param val Value.
+         */
+        public EntityWriteReadObject(int val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        private void writeObject(ObjectOutputStream s) throws IOException{
+            s.writeInt(val);
+        }
+
+        /** {@inheritDoc} */
+        private void readObject(ObjectInputStream s) throws IOException, ClassNotFoundException {
+            val = s.readInt();
+        }
+    }
+}


[4/6] ignite git commit: Fixed cache store examples

Posted by ak...@apache.org.
Fixed cache store examples


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

Branch: refs/heads/ignite-843-rc2
Commit: e1b2cf2840b1451dbfc0bbef4540e5a9edaf6c35
Parents: 889b268
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Thu Dec 17 16:05:09 2015 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Thu Dec 17 16:05:09 2015 -0800

----------------------------------------------------------------------
 .../hibernate/CacheHibernateStoreExample.java   |   8 +
 .../datagrid/store/hibernate/Person.hbm.xml     |   6 +-
 .../datagrid/store/hibernate/hibernate.cfg.xml  |   7 +-
 .../store/spring/CacheSpringPersonStore.java    | 118 ++++++++++++++
 .../store/spring/CacheSpringStoreExample.java   | 155 +++++++++++++++++++
 .../datagrid/store/spring/package-info.java     |  22 +++
 6 files changed, 311 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e1b2cf28/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java b/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java
index 71c4f68..db34eaf 100644
--- a/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java
+++ b/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java
@@ -30,6 +30,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.examples.ExampleNodeStartup;
 import org.apache.ignite.examples.ExamplesUtils;
 import org.apache.ignite.examples.model.Person;
+import org.apache.ignite.examples.util.DbH2ServerStartup;
 import org.apache.ignite.transactions.Transaction;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -39,6 +40,13 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
  * <p>
  * This example uses {@link CacheHibernatePersonStore} as a persistent store.
  * <p>
+ * To start the example, you should:
+ * <ul>
+ *     <li>Start H2 database TCP server using {@link DbH2ServerStartup}.</li>
+ *     <li>Start a few nodes using {@link ExampleNodeStartup}.</li>
+ *     <li>Start example using {@link CacheHibernateStoreExample}.</li>
+ * </ul>
+ * <p>
  * Remote nodes can be started with {@link ExampleNodeStartup} in another JVM which will
  * start node with {@code examples/config/example-ignite.xml} configuration.
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1b2cf28/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/store/hibernate/Person.hbm.xml
----------------------------------------------------------------------
diff --git a/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/store/hibernate/Person.hbm.xml b/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/store/hibernate/Person.hbm.xml
index 1314b71..0728a03 100644
--- a/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/store/hibernate/Person.hbm.xml
+++ b/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/store/hibernate/Person.hbm.xml
@@ -23,12 +23,12 @@
         "http://www.hibernate.org/dtd/hibernate-mapping-3.0.dtd">
 
 <hibernate-mapping default-access="field">
-    <class name="org.apache.ignite.examples.model.Person" table="PERSONS">
+    <class name="org.apache.ignite.examples.model.Person" table="PERSON">
         <!-- ID. -->
         <id name="id"/>
 
         <!-- We only map data we are interested in. -->
-        <property name="firstName"/>
-        <property name="lastName"/>
+        <property name="firstName" column="first_name"/>
+        <property name="lastName" column="last_name"/>
     </class>
 </hibernate-mapping>

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1b2cf28/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/store/hibernate/hibernate.cfg.xml
----------------------------------------------------------------------
diff --git a/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/store/hibernate/hibernate.cfg.xml b/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/store/hibernate/hibernate.cfg.xml
index 80a43e7..7bf70af 100644
--- a/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/store/hibernate/hibernate.cfg.xml
+++ b/examples/src/main/java-lgpl/org/apache/ignite/examples/datagrid/store/hibernate/hibernate.cfg.xml
@@ -26,8 +26,11 @@
 -->
 <hibernate-configuration>
     <session-factory>
-        <!-- Database connection settings (private in-memory database). -->
-        <property name="connection.url">jdbc:h2:mem:example;DB_CLOSE_DELAY=-1</property>
+        <!-- Database connection URL. -->
+        <property name="connection.url">jdbc:h2:tcp://localhost/mem:ExampleDb</property>
+
+        <!-- Database connection username. -->
+        <property name="connection.username">sa</property>
 
         <!-- Only validate the database schema on startup in production mode. -->
         <property name="hbm2ddl.auto">update</property>

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1b2cf28/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringPersonStore.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringPersonStore.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringPersonStore.java
new file mode 100644
index 0000000..c4b52bb
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringPersonStore.java
@@ -0,0 +1,118 @@
+/*
+ * 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.examples.datagrid.store.spring;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.Cache;
+import javax.cache.integration.CacheLoaderException;
+import javax.sql.DataSource;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.cache.store.CacheStoreAdapter;
+import org.apache.ignite.examples.model.Person;
+import org.apache.ignite.lang.IgniteBiInClosure;
+import org.h2.jdbcx.JdbcConnectionPool;
+import org.springframework.dao.EmptyResultDataAccessException;
+import org.springframework.jdbc.core.JdbcTemplate;
+import org.springframework.jdbc.core.RowCallbackHandler;
+import org.springframework.jdbc.core.RowMapper;
+
+/**
+ * Example of {@link CacheStore} implementation that uses JDBC
+ * transaction with cache transactions and maps {@link Long} to {@link Person}.
+ */
+public class CacheSpringPersonStore extends CacheStoreAdapter<Long, Person> {
+    /** Data source. */
+    public static final DataSource DATA_SRC =
+        JdbcConnectionPool.create("jdbc:h2:tcp://localhost/mem:ExampleDb", "sa", "");
+
+    /** Spring JDBC template. */
+    private JdbcTemplate jdbcTemplate;
+
+    /**
+     * Constructor.
+     *
+     * @throws IgniteException If failed.
+     */
+    public CacheSpringPersonStore() throws IgniteException {
+        jdbcTemplate = new JdbcTemplate(DATA_SRC);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Person load(Long key) {
+        System.out.println(">>> Store load [key=" + key + ']');
+
+        try {
+            return jdbcTemplate.queryForObject("select * from PERSON where id = ?", new RowMapper<Person>() {
+                @Override public Person mapRow(ResultSet rs, int rowNum) throws SQLException {
+                    return new Person(rs.getLong(1), rs.getString(2), rs.getString(3));
+                }
+            }, key);
+        }
+        catch (EmptyResultDataAccessException ignored) {
+            return null;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(Cache.Entry<? extends Long, ? extends Person> entry) {
+        Long key = entry.getKey();
+        Person val = entry.getValue();
+
+        System.out.println(">>> Store write [key=" + key + ", val=" + val + ']');
+
+        int updated = jdbcTemplate.update("update PERSON set first_name = ?, last_name = ? where id = ?",
+            val.firstName, val.lastName, val.id);
+
+        if (updated == 0) {
+            jdbcTemplate.update("insert into PERSON (id, first_name, last_name) values (?, ?, ?)",
+                val.id, val.firstName, val.lastName);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void delete(Object key) {
+        System.out.println(">>> Store delete [key=" + key + ']');
+
+        jdbcTemplate.update("delete from PERSON where id = ?", key);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void loadCache(final IgniteBiInClosure<Long, Person> clo, Object... args) {
+        if (args == null || args.length == 0 || args[0] == null)
+            throw new CacheLoaderException("Expected entry count parameter is not provided.");
+
+        int entryCnt = (Integer)args[0];
+
+        final AtomicInteger cnt = new AtomicInteger();
+
+        jdbcTemplate.query("select * from PERSON limit ?", new RowCallbackHandler() {
+            @Override public void processRow(ResultSet rs) throws SQLException {
+                Person person = new Person(rs.getLong(1), rs.getString(2), rs.getString(3));
+
+                clo.apply(person.id, person);
+
+                cnt.incrementAndGet();
+            }
+        }, entryCnt);
+
+        System.out.println(">>> Loaded " + cnt + " values into cache.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1b2cf28/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringStoreExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringStoreExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringStoreExample.java
new file mode 100644
index 0000000..5380fce
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringStoreExample.java
@@ -0,0 +1,155 @@
+/*
+ * 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.examples.datagrid.store.spring;
+
+import java.util.UUID;
+import javax.cache.configuration.Factory;
+import javax.cache.configuration.FactoryBuilder;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.store.CacheStoreSessionListener;
+import org.apache.ignite.cache.store.spring.CacheSpringStoreSessionListener;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.examples.ExampleNodeStartup;
+import org.apache.ignite.examples.ExamplesUtils;
+import org.apache.ignite.examples.model.Person;
+import org.apache.ignite.examples.util.DbH2ServerStartup;
+import org.apache.ignite.transactions.Transaction;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+
+/**
+ * Demonstrates usage of cache with underlying persistent store configured.
+ * <p>
+ * This example uses {@link CacheSpringPersonStore} as a persistent store.
+ * <p>
+ * To start the example, you should:
+ * <ul>
+ *     <li>Start H2 database TCP server using {@link DbH2ServerStartup}.</li>
+ *     <li>Start a few nodes using {@link ExampleNodeStartup}.</li>
+ *     <li>Start example using {@link CacheSpringStoreExample}.</li>
+ * </ul>
+ * <p>
+ * Remote nodes can be started with {@link ExampleNodeStartup} in another JVM which will
+ * start node with {@code examples/config/example-ignite.xml} configuration.
+ */
+public class CacheSpringStoreExample {
+    /** Cache name. */
+    private static final String CACHE_NAME = CacheSpringStoreExample.class.getSimpleName();
+
+    /** Heap size required to run this example. */
+    public static final int MIN_MEMORY = 1024 * 1024 * 1024;
+
+    /** Number of entries to load. */
+    private static final int ENTRY_COUNT = 100_000;
+
+    /** Global person ID to use across entire example. */
+    private static final Long id = Math.abs(UUID.randomUUID().getLeastSignificantBits());
+
+    /**
+     * Executes example.
+     *
+     * @param args Command line arguments, none required.
+     * @throws IgniteException If example execution failed.
+     */
+    public static void main(String[] args) throws IgniteException {
+        ExamplesUtils.checkMinMemory(MIN_MEMORY);
+
+        // To start ignite with desired configuration uncomment the appropriate line.
+        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
+            System.out.println();
+            System.out.println(">>> Cache store example started.");
+
+            CacheConfiguration<Long, Person> cacheCfg = new CacheConfiguration<>(CACHE_NAME);
+
+            // Set atomicity as transaction, since we are showing transactions in example.
+            cacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+            // Configure Spring store.
+            cacheCfg.setCacheStoreFactory(FactoryBuilder.factoryOf(CacheSpringPersonStore.class));
+
+            // Configure Spring session listener.
+            cacheCfg.setCacheStoreSessionListenerFactories(new Factory<CacheStoreSessionListener>() {
+                @Override public CacheStoreSessionListener create() {
+                    CacheSpringStoreSessionListener lsnr = new CacheSpringStoreSessionListener();
+
+                    lsnr.setDataSource(CacheSpringPersonStore.DATA_SRC);
+
+                    return lsnr;
+                }
+            });
+
+            cacheCfg.setReadThrough(true);
+            cacheCfg.setWriteThrough(true);
+
+            try (IgniteCache<Long, Person> cache = ignite.getOrCreateCache(cacheCfg)) {
+                // Make initial cache loading from persistent store. This is a
+                // distributed operation and will call CacheStore.loadCache(...)
+                // method on all nodes in topology.
+                loadCache(cache);
+
+                // Start transaction and execute several cache operations with
+                // read/write-through to persistent store.
+                executeTransaction(cache);
+            }
+        }
+    }
+
+    /**
+     * Makes initial cache loading.
+     *
+     * @param cache Cache to load.
+     */
+    private static void loadCache(IgniteCache<Long, Person> cache) {
+        long start = System.currentTimeMillis();
+
+        // Start loading cache from persistent store on all caching nodes.
+        cache.loadCache(null, ENTRY_COUNT);
+
+        long end = System.currentTimeMillis();
+
+        System.out.println(">>> Loaded " + cache.size() + " keys with backups in " + (end - start) + "ms.");
+    }
+
+    /**
+     * Executes transaction with read/write-through to persistent store.
+     *
+     * @param cache Cache to execute transaction on.
+     */
+    private static void executeTransaction(IgniteCache<Long, Person> cache) {
+        try (Transaction tx = Ignition.ignite().transactions().txStart()) {
+            Person val = cache.get(id);
+
+            System.out.println("Read value: " + val);
+
+            val = cache.getAndPut(id, new Person(id, "Isaac", "Newton"));
+
+            System.out.println("Overwrote old value: " + val);
+
+            val = cache.get(id);
+
+            System.out.println("Read value: " + val);
+
+            tx.commit();
+        }
+
+        System.out.println("Read value after commit: " + cache.get(id));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e1b2cf28/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/package-info.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/package-info.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/package-info.java
new file mode 100644
index 0000000..211239f
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 description. -->
+ * Contains Spring-based cache store implementation.
+ */
+package org.apache.ignite.examples.datagrid.store.spring;


[6/6] ignite git commit: Merge branches 'ignite-1.5' and 'ignite-843-rc2' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-843-rc2

Posted by ak...@apache.org.
Merge branches 'ignite-1.5' and 'ignite-843-rc2' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-843-rc2


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

Branch: refs/heads/ignite-843-rc2
Commit: f888e54fcbb30088f1256da81bd0a19e4a0d78ef
Parents: 23c3e54 a4b547b
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Dec 18 10:14:11 2015 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Dec 18 10:14:11 2015 +0700

----------------------------------------------------------------------
 .../hibernate/CacheHibernateStoreExample.java   |   8 +
 .../datagrid/store/hibernate/Person.hbm.xml     |   6 +-
 .../datagrid/store/hibernate/hibernate.cfg.xml  |   7 +-
 .../store/spring/CacheSpringPersonStore.java    | 118 ++++++
 .../store/spring/CacheSpringStoreExample.java   | 155 +++++++
 .../datagrid/store/spring/package-info.java     |  22 +
 .../binary/BinaryReflectiveSerializer.java      |  33 ++
 .../ignite/binary/BinaryTypeConfiguration.java  |  20 +-
 .../ignite/cache/affinity/AffinityKey.java      |   2 +-
 .../internal/binary/BinaryClassDescriptor.java  | 124 ++----
 .../ignite/internal/binary/BinaryContext.java   | 109 ++++-
 .../internal/binary/BinaryFieldAccessor.java    |   3 -
 .../ignite/internal/binary/BinaryUtils.java     |  49 ++-
 .../ignite/internal/binary/BinaryWriteMode.java |   3 -
 .../processors/cache/CacheEntryImpl.java        |   6 +-
 .../processors/cache/CacheEntryImplEx.java      |  17 +-
 .../cache/binary/BinaryMetadataKey.java         |   2 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |   2 +-
 .../PlatformDotNetConfigurationClosure.java     |   4 +-
 .../processors/query/GridQueryProcessor.java    | 204 ++++++---
 .../binary/BinaryFieldsAbstractSelfTest.java    |   4 +-
 .../BinaryFooterOffsetsAbstractSelfTest.java    |   3 +-
 .../binary/BinaryMarshallerSelfTest.java        |  34 +-
 ...GridBinaryMarshallerCtxDisabledSelfTest.java |   3 +-
 .../binary/GridBinaryWildcardsSelfTest.java     |   3 +-
 .../GridCacheOnCopyFlagAbstractSelfTest.java    |  33 +-
 .../GridBinaryCacheEntryMemorySizeSelfTest.java |   3 +-
 .../session/GridSessionCheckpointSelfTest.java  |   3 +-
 .../ignite/testframework/junits/IgniteMock.java |   3 +-
 .../junits/IgniteTestResources.java             |   4 +-
 .../cache/BinarySerializationQuerySelfTest.java | 416 +++++++++++++++++++
 ...onQueryWithReflectiveSerializerSelfTest.java |  28 ++
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |   4 +-
 .../IgniteBinaryCacheQueryTestSuite.java        | 119 +++++-
 .../Services/ServicesTest.cs                    |  23 +-
 35 files changed, 1340 insertions(+), 237 deletions(-)
----------------------------------------------------------------------



[3/6] ignite git commit: IGNITE-2193: .NET: Fixed SerivceTest.cs.

Posted by ak...@apache.org.
IGNITE-2193: .NET: Fixed SerivceTest.cs.


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

Branch: refs/heads/ignite-843-rc2
Commit: 889b2689816a7185029eb7c992bf73f14600844c
Parents: 057ad5b
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Thu Dec 17 17:31:31 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Dec 17 17:31:31 2015 +0300

----------------------------------------------------------------------
 .../Services/ServicesTest.cs                    | 23 +++++++++++++++-----
 1 file changed, 17 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/889b2689/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
index 33f255e..ffcdea8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
@@ -218,10 +218,10 @@ namespace Apache.Ignite.Core.Tests.Services
             }
 
             Services.Cancel(SvcName + 0);
-            Services.Cancel(SvcName + 1);
+            AssertNoService(SvcName + 0);
 
-            Assert.IsNull(Services.GetService<ITestIgniteService>(SvcName + 0));
-            Assert.IsNull(Services.GetService<ITestIgniteService>(SvcName + 1));
+            Services.Cancel(SvcName + 1);
+            AssertNoService(SvcName + 1);
 
             for (var i = 2; i < 10; i++)
                 Assert.IsNotNull(Services.GetService<ITestIgniteService>(SvcName + i));
@@ -229,7 +229,7 @@ namespace Apache.Ignite.Core.Tests.Services
             Services.CancelAll();
 
             for (var i = 0; i < 10; i++)
-                Assert.IsNull(Services.GetService<ITestIgniteService>(SvcName + i));
+                AssertNoService(SvcName + i);
         }
 
         /// <summary>
@@ -477,8 +477,7 @@ namespace Apache.Ignite.Core.Tests.Services
             Services.CancelAll();
 
             // Cancellation failed, but service is removed.
-            foreach (var grid in Grids)
-                Assert.IsNull(grid.GetServices().GetService<ITestIgniteService>(SvcName));
+            AssertNoService();
         }
 
         [Test]
@@ -579,6 +578,18 @@ namespace Apache.Ignite.Core.Tests.Services
         }
 
         /// <summary>
+        /// Asserts that there is no service on any grid with given name.
+        /// </summary>
+        /// <param name="name">The name.</param>
+        private void AssertNoService(string name = SvcName)
+        {
+            foreach (var grid in Grids)
+                Assert.IsTrue(
+                    TestUtils.WaitForCondition(() => grid.GetServices()
+                        .GetService<ITestIgniteService>(name) == null, 5000));
+        }
+
+        /// <summary>
         /// Gets the services.
         /// </summary>
         protected virtual IServices Services


[5/6] ignite git commit: Fixed CacheEntry toString()

Posted by ak...@apache.org.
Fixed CacheEntry toString()


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

Branch: refs/heads/ignite-843-rc2
Commit: a4b547bd11ca3bf4a5cdfaac3ad01936cf6e995e
Parents: e1b2cf2
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Thu Dec 17 17:07:26 2015 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Thu Dec 17 17:07:26 2015 -0800

----------------------------------------------------------------------
 .../internal/processors/cache/CacheEntryImpl.java  |  6 +++---
 .../processors/cache/CacheEntryImplEx.java         | 17 ++++++++++++++---
 2 files changed, 17 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a4b547bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImpl.java
index 71c684a..48649d2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImpl.java
@@ -81,10 +81,10 @@ public class CacheEntryImpl<K, V> implements Cache.Entry<K, V>, Externalizable {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public <T> T unwrap(Class<T> cls) {
-        if(cls.isAssignableFrom(getClass()))
+        if (cls.isAssignableFrom(getClass()))
             return cls.cast(this);
 
-        if (ver != null && cls.isAssignableFrom(CacheEntry.class))
+        if (cls.isAssignableFrom(CacheEntry.class))
             return (T)new CacheEntryImplEx<>(key, val, ver);
 
         throw new IllegalArgumentException("Unwrapping to class is not supported: " + cls);
@@ -106,4 +106,4 @@ public class CacheEntryImpl<K, V> implements Cache.Entry<K, V>, Externalizable {
     public String toString() {
         return "Entry [key=" + key + ", val=" + val + ']';
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4b547bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImplEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImplEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImplEx.java
index f3e6c18..1c7111a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImplEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryImplEx.java
@@ -78,7 +78,18 @@ public class CacheEntryImplEx<K, V> extends CacheEntryImpl<K, V> implements Cach
 
     /** {@inheritDoc} */
     public String toString() {
-        return "VersionedEntry [key=" + getKey() + ", val=" + getValue() + ", topVer=" + ver.topologyVersion() +
-            ", nodeOrder=" + ver.nodeOrder() + ", order=" + ver.order() + ", updateTime=" + ver.globalTime() + ']';
+        String res = "CacheEntry [key=" + getKey() +
+            ", val=" + getValue();
+
+        if (ver != null) {
+            res += ", topVer=" + ver.topologyVersion() +
+                ", nodeOrder=" + ver.nodeOrder() +
+                ", order=" + ver.order() +
+                ", updateTime=" + ver.globalTime();
+        }
+        else
+            res += ", ver=n/a";
+
+        return res + ']';
     }
-}
\ No newline at end of file
+}